From ce2c35a686e0f86d7570eda96829624f01875d0b Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 6 Aug 2024 23:34:04 +0200 Subject: [PATCH 001/333] Add support for (a) indicating to downstream Service impls that a certain number of bytes should be reserved (to make space for post-processing adding EDNS options to the response), and (b) strongly typed passing of arbitrary metadata between middleware that produces a type and middleware that consumes the type (e.g. a TSIG key name). --- Cargo.lock | 1 + Cargo.toml | 1 + examples/serve-zone.rs | 6 +- examples/server-transports.rs | 26 ++- src/base/message_builder.rs | 43 +++++ src/net/server/connection.rs | 221 +++++++++++++++---------- src/net/server/dgram.rs | 106 +++++++----- src/net/server/message.rs | 59 ++++++- src/net/server/middleware/cookies.rs | 156 ++++++++--------- src/net/server/middleware/edns.rs | 145 ++++++++++++---- src/net/server/middleware/mandatory.rs | 81 +++++---- src/net/server/middleware/stream.rs | 71 +++++--- src/net/server/service.rs | 50 +++--- src/net/server/tests/integration.rs | 111 ++++++------- src/net/server/util.rs | 133 +++++++-------- 15 files changed, 746 insertions(+), 464 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index fcc1f145f..af9939129 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -245,6 +245,7 @@ dependencies = [ "time", "tokio", "tokio-rustls", + "tokio-stream", "tokio-test", "tokio-tfo", "tracing", diff --git a/Cargo.toml b/Cargo.toml index 995864729..554aae9ff 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -40,6 +40,7 @@ siphasher = { version = "1", optional = true } smallvec = { version = "1.3", optional = true } tokio = { version = "1.33", optional = true, features = ["io-util", "macros", "net", "time", "sync", "rt-multi-thread" ] } tokio-rustls = { version = "0.26", optional = true, default-features = false } +tokio-stream = { version = "0.1.1", optional = true } tracing = { version = "0.1.40", optional = true } tracing-subscriber = { version = "0.3.18", optional = true, features = ["env-filter"] } diff --git a/examples/serve-zone.rs b/examples/serve-zone.rs index 5f9ae4151..db9c986e0 100644 --- a/examples/serve-zone.rs +++ b/examples/serve-zone.rs @@ -80,9 +80,9 @@ async fn main() { let svc = service_fn(my_service, zones); #[cfg(feature = "siphasher")] - let svc = CookiesMiddlewareSvc::, _>::with_random_secret(svc); - let svc = EdnsMiddlewareSvc::, _>::new(svc); - let svc = MandatoryMiddlewareSvc::, _>::new(svc); + let svc = CookiesMiddlewareSvc::, _, _>::with_random_secret(svc); + let svc = EdnsMiddlewareSvc::, _, _>::new(svc); + let svc = MandatoryMiddlewareSvc::, _, _>::new(svc); let svc = Arc::new(svc); let sock = UdpSocket::bind(addr).await.unwrap(); diff --git a/examples/server-transports.rs b/examples/server-transports.rs index 59c48d3fe..ece257ce6 100644 --- a/examples/server-transports.rs +++ b/examples/server-transports.rs @@ -208,7 +208,7 @@ impl Service> for MyAsyncStreamingService { /// The function signature is slightly more complex than when using /// [`service_fn`] (see the [`query`] example below). #[allow(clippy::type_complexity)] -fn name_to_ip(request: Request>) -> ServiceResult> { +fn name_to_ip(request: Request>, _: ()) -> ServiceResult> { let mut out_answer = None; if let Ok(question) = request.message().sole_question() { let qname = question.qname(); @@ -542,6 +542,7 @@ where RequestOctets, Svc::Future, Svc::Stream, + (), Arc>, >, Empty>, @@ -571,13 +572,18 @@ fn build_middleware_chain( ) -> StatsMiddlewareSvc< MandatoryMiddlewareSvc< Vec, - EdnsMiddlewareSvc, CookiesMiddlewareSvc, Svc>>, + EdnsMiddlewareSvc< + Vec, + CookiesMiddlewareSvc, Svc, ()>, + (), + >, + (), >, > { #[cfg(feature = "siphasher")] - let svc = CookiesMiddlewareSvc::, _>::with_random_secret(svc); - let svc = EdnsMiddlewareSvc::, _>::new(svc); - let svc = MandatoryMiddlewareSvc::, _>::new(svc); + let svc = CookiesMiddlewareSvc::, _, _>::with_random_secret(svc); + let svc = EdnsMiddlewareSvc::, _, _>::new(svc); + let svc = MandatoryMiddlewareSvc::, _, _>::new(svc); StatsMiddlewareSvc::new(svc, stats.clone()) } @@ -633,8 +639,10 @@ async fn main() { // 2. name_to_ip: a service impl defined as a function compatible with the // `Service` trait. - let name_into_ip_svc = - Arc::new(build_middleware_chain(name_to_ip, stats.clone())); + let name_into_ip_svc = Arc::new(build_middleware_chain( + service_fn(name_to_ip, ()), + stats.clone(), + )); // 3. query: a service impl defined as a function converted to a `Service` // impl via the `service_fn()` helper function. @@ -642,11 +650,11 @@ async fn main() { // creating a separate middleware chain for use just by this server. let count = Arc::new(AtomicU8::new(5)); let svc = service_fn(query, count); - let svc = MandatoryMiddlewareSvc::, _>::new(svc); + let svc = MandatoryMiddlewareSvc::, _, _>::new(svc); #[cfg(feature = "siphasher")] let svc = { let server_secret = "server12secret34".as_bytes().try_into().unwrap(); - CookiesMiddlewareSvc::, _>::new(svc, server_secret) + CookiesMiddlewareSvc::, _, _>::new(svc, server_secret) }; let svc = StatsMiddlewareSvc::new(svc, stats.clone()); let query_svc = Arc::new(svc); diff --git a/src/base/message_builder.rs b/src/base/message_builder.rs index fa10ecd40..893183550 100644 --- a/src/base/message_builder.rs +++ b/src/base/message_builder.rs @@ -262,6 +262,31 @@ impl MessageBuilder { Ok(builder.answer()) } + /// Starts creating an error for the given message. + /// + /// Like [`start_answer()`] but infallible. Questions will be pushed if possible. + pub fn start_error( + mut self, + msg: &Message, + rcode: Rcode, + ) -> AnswerBuilder { + { + let header = self.header_mut(); + header.set_id(msg.header().id()); + header.set_qr(true); + header.set_opcode(msg.header().opcode()); + header.set_rd(msg.header().rd()); + } + let mut builder = self.question(); + for item in msg.question().flatten() { + if builder.push(item).is_err() { + break; + } + } + builder.header_mut().set_rcode(rcode); + builder.answer() + } + /// Creates an AXFR request for the given domain. /// /// Sets a random ID, pushes the domain and the AXFR record type into @@ -849,6 +874,16 @@ impl AnswerBuilder { |counts| counts.inc_ancount(), ) } + + pub fn push_ref( + &mut self, + record: &impl ComposeRecord, + ) -> Result<(), PushError> { + self.builder.push( + |target| record.compose_record(target).map_err(Into::into), + |counts| counts.inc_ancount(), + ) + } } /// # Conversions @@ -1923,6 +1958,14 @@ where } } +impl FreezeBuilder for StreamTarget { + type Octets = Target; + + fn freeze(self) -> Self::Octets { + self.into_target() + } +} + //------------ StaticCompressor ---------------------------------------------- /// A domain name compressor that doesn’t require an allocator. diff --git a/src/net/server/connection.rs b/src/net/server/connection.rs index 6ae98dfb5..24b192906 100644 --- a/src/net/server/connection.rs +++ b/src/net/server/connection.rs @@ -1,5 +1,6 @@ //! Support for stream based connections. use core::ops::{ControlFlow, Deref}; +use core::sync::atomic::{AtomicBool, Ordering}; use core::time::Duration; use std::fmt::Display; @@ -75,6 +76,14 @@ const RESPONSE_WRITE_TIMEOUT: DefMinMax = DefMinMax::new( Duration::from_secs(60 * 60), ); +/// Limit on the number of attempts that will be made to overcome +/// non-fatal write errors. +/// +/// The value has to be between 0 and 255 with a default of 2. A value of +/// zero means that one try will be attempted. +const RESPONSE_WRITE_RETRIES: DefMinMax = + DefMinMax::new(2, u8::MIN, u8::MAX); + /// Limit on the number of DNS responses queued for writing to the client. /// /// The value has to be between zero and 1,024. The default value is 10. These @@ -106,13 +115,12 @@ pub struct Config { /// Limit on the amount of time to wait for writing a response to /// complete. - /// - /// The value has to be between 1 millisecond and 1 hour with a default of - /// 30 seconds. These values are guesses at something reasonable. The - /// default is based on the Unbound 1.19.2 default value for its - /// `tcp-idle-timeout` setting. response_write_timeout: Duration, + /// Limit on the number of attempts that will be made to overcome + /// non-fatal write errors. + response_write_retries: u8, + /// Limit on the number of DNS responses queued for wriing to the client. max_queued_responses: usize, } @@ -174,6 +182,16 @@ impl Config { self.response_write_timeout = value; } + /// Limit on the number of attempts that will be made to overcome + /// non-fatal write errors. + /// + /// The value has to be between 0 and 255 with a default of 2. A value of + /// zero means that one try will be attempted. + #[allow(dead_code)] + pub fn set_response_write_retries(&mut self, value: u8) { + self.response_write_retries = value; + } + /// Set the limit on the number of DNS responses queued for writing to the /// client. /// @@ -203,6 +221,7 @@ impl Default for Config { Self { idle_timeout: IDLE_TIMEOUT.default(), response_write_timeout: RESPONSE_WRITE_TIMEOUT.default(), + response_write_retries: RESPONSE_WRITE_RETRIES.default(), max_queued_responses: MAX_QUEUED_RESPONSES.default(), } } @@ -264,6 +283,9 @@ where /// DNS protocol idle time out tracking. idle_timer: IdleTimer, + /// Is a transaction in progress? + in_transaction: Arc, + /// [`ServerMetrics`] describing the status of the server. metrics: Arc, } @@ -312,6 +334,7 @@ where mpsc::channel(config.max_queued_responses); let config = Arc::new(ArcSwap::from_pointee(config)); let idle_timer = IdleTimer::new(); + let in_transaction = Arc::new(AtomicBool::new(false)); // Place the ReadHalf of the stream into an Option so that we can take // it out (as we can't clone it and we can't place it into an Arc @@ -333,6 +356,7 @@ where result_q_tx, service, idle_timer, + in_transaction, metrics, } } @@ -423,7 +447,7 @@ where } _ = sleep_until(self.idle_timer.idle_timeout_deadline(self.config.load().idle_timeout)) => { - self.process_dns_idle_timeout() + self.process_dns_idle_timeout(self.config.load().idle_timeout) } res = &mut msg_recv => { @@ -440,9 +464,11 @@ where if let Err(err) = res { match err { ConnectionEvent::DisconnectWithoutFlush => { + trace!("Disconnect without flush"); break 'outer; } ConnectionEvent::DisconnectWithFlush => { + trace!("Disconnect with flush"); self.flush_write_queue().await; break 'outer; } @@ -536,10 +562,7 @@ where } /// Stop queueing new responses and process those already in the queue. - async fn flush_write_queue(&mut self) - // where - // Target: Composer, - { + async fn flush_write_queue(&mut self) { debug!("Flushing connection write queue."); // Stop accepting new response messages (should we check for in-flight // messages that haven't generated a response yet but should be @@ -564,10 +587,7 @@ where async fn process_queued_result( &mut self, response: Option>>, - ) -> Result<(), ConnectionEvent> -// where - // Target: Composer, - { + ) -> Result<(), ConnectionEvent> { // If we failed to read the results of requests processed by the // service because the queue holding those results is empty and can no // longer be read from, then there is no point continuing to read from @@ -583,63 +603,82 @@ where "Writing queued response with id {} to stream", response.header().id() ); - self.write_response_to_stream(response.finish()).await; - - Ok(()) + self.write_response_to_stream(response.finish()).await } /// Write a response back to the caller over the network stream. async fn write_response_to_stream( &mut self, msg: StreamTarget, - ) - // where - // Target: AsRef<[u8]>, - { + ) -> Result<(), ConnectionEvent> { if enabled!(Level::TRACE) { let bytes = msg.as_dgram_slice(); - let pcap_text = to_pcap_text(bytes, bytes.len()); - trace!(addr = %self.addr, pcap_text, "Sending response"); + let pcap_text = to_pcap_text(bytes, bytes.len().min(128)); + trace!(addr = %self.addr, pcap_text, "Sending response (dumping max 128 bytes)"); } - match timeout( - self.config.load().response_write_timeout, - self.stream_tx.write_all(msg.as_stream_slice()), - ) - .await - { - Err(_) => { - error!( - "Write timed out (>{:?})", - self.config.load().response_write_timeout - ); - // TODO: Push it to the back of the queue to retry it? - } - Ok(Err(err)) => { - error!("Write error: {err}"); - } - Ok(Ok(_)) => { - self.metrics.inc_num_sent_responses(); + let max_tries = self.config.load().response_write_retries + 1; + let mut tries_left = max_tries; + while tries_left > 0 { + debug!(addr = %self.addr, "Sending response: attempts left={tries_left}"); + tries_left -= 1; + + match timeout( + self.config.load().response_write_timeout, + self.stream_tx.write_all(msg.as_stream_slice()), + ) + .await + { + Err(_) => { + error!(addr = %self.addr, + "Write timed out (>{:?})", + self.config.load().response_write_timeout + ); + // Retry + continue; + } + Ok(Err(err)) => { + if let ControlFlow::Break(err) = process_io_error(err) { + // Fatal error, abort. + return Err(err); + } else { + // Retry + continue; + } + } + Ok(Ok(_)) => { + // Success. + self.metrics.inc_num_sent_responses(); + self.metrics.dec_num_pending_writes(); + + if self.result_q_tx.capacity() + == self.result_q_tx.max_capacity() + { + self.idle_timer.response_queue_emptied(); + } + + return Ok(()); + } } } - self.metrics.dec_num_pending_writes(); - - if self.result_q_tx.capacity() == self.result_q_tx.max_capacity() { - self.idle_timer.response_queue_emptied(); - } + error!(addr = %self.addr, "Failed to send response after {max_tries}. The connection will be closed."); + Err(ConnectionEvent::DisconnectWithoutFlush) } - /// Implemnt DNS rules regarding timing out of idle connections. + /// Implement DNS rules regarding timing out of idle connections. /// /// Disconnects the current connection of the timer is expired, flushing /// pending responses first. - fn process_dns_idle_timeout(&self) -> Result<(), ConnectionEvent> { + fn process_dns_idle_timeout( + &self, + timeout: Duration, + ) -> Result<(), ConnectionEvent> { // DNS idle timeout elapsed, or was it reset? - if self - .idle_timer - .idle_timeout_expired(self.config.load().idle_timeout) + if self.idle_timer.idle_timeout_expired(timeout) + && !self.in_transaction.load(Ordering::SeqCst) { + trace!("Timing out idle connection"); Err(ConnectionEvent::DisconnectWithoutFlush) } else { Ok(()) @@ -654,13 +693,16 @@ where where Svc::Stream: Send, { + let in_transaction = self.in_transaction.clone(); + match res { Ok(buf) => { let received_at = Instant::now(); if enabled!(Level::TRACE) { - let pcap_text = to_pcap_text(&buf, buf.as_ref().len()); - trace!(addr = %self.addr, pcap_text, "Received message"); + let pcap_text = + to_pcap_text(&buf, buf.as_ref().len().min(128)); + trace!(addr = %self.addr, pcap_text, "Received message (dumping max 128 bytes)"); } self.metrics.inc_num_received_requests(); @@ -679,12 +721,18 @@ where } Ok(msg) => { + trace!(addr = %self.addr, ?msg, "Parsed first question: {:?}", msg.first_question()); let ctx = NonUdpTransportContext::new(Some( self.config.load().idle_timeout, )); let ctx = TransportSpecificContext::NonUdp(ctx); - let request = - Request::new(self.addr, received_at, msg, ctx); + let request = Request::new( + self.addr, + received_at, + msg, + ctx, + (), + ); let svc = self.service.clone(); let result_q_tx = self.result_q_tx.clone(); @@ -701,7 +749,6 @@ where "Calling service for request id {request_id}" ); let mut stream = svc.call(request).await; - let mut in_transaction = false; trace!("Awaiting service call results for request id {request_id}"); while let Some(Ok(call_result)) = @@ -733,11 +780,17 @@ where } ServiceFeedback::BeginTransaction => { - in_transaction = true; + in_transaction.store( + true, + Ordering::SeqCst, + ); } ServiceFeedback::EndTransaction => { - in_transaction = false; + in_transaction.store( + false, + Ordering::SeqCst, + ); } } } @@ -760,14 +813,16 @@ where } Err(TrySendError::Closed(_)) => { - error!("Unable to queue message for sending: server is shutting down."); - break; + error!("Unable to queue message for sending: connection is shutting down."); + return; } Err(TrySendError::Full( unused_response, )) => { - if in_transaction { + if in_transaction + .load(Ordering::SeqCst) + { // Wait until there is space in the message queue. tokio::task::yield_now() .await; @@ -923,35 +978,35 @@ where // buffer. Ok(_size) => return Ok(()), - Err(err) => match Self::process_io_error(err) { + Err(err) => match process_io_error(err) { ControlFlow::Continue(_) => continue, ControlFlow::Break(err) => return Err(err), }, } } } +} - /// Handle I/O errors by deciding whether to log them, and whethr to - /// continue or abort. - #[must_use] - fn process_io_error(err: io::Error) -> ControlFlow { - match err.kind() { - io::ErrorKind::UnexpectedEof => { - // The client disconnected. Per RFC 7766 6.2.4 pending - // responses MUST NOT be sent to the client. - ControlFlow::Break(ConnectionEvent::DisconnectWithoutFlush) - } - io::ErrorKind::TimedOut | io::ErrorKind::Interrupted => { - // These errors might be recoverable, try again. - ControlFlow::Continue(()) - } - _ => { - // Everything else is either unrecoverable or unknown to us at - // the time of writing and so we can't guess how to handle it, - // so abort. - error!("I/O error: {}", err); - ControlFlow::Break(ConnectionEvent::DisconnectWithoutFlush) - } +/// Handle I/O errors by deciding whether to log them, and whether to continue +/// or abort. +#[must_use] +fn process_io_error(err: io::Error) -> ControlFlow { + match err.kind() { + io::ErrorKind::UnexpectedEof => { + // The client disconnected. Per RFC 7766 6.2.4 pending responses + // MUST NOT be sent to the client. + ControlFlow::Break(ConnectionEvent::DisconnectWithoutFlush) + } + io::ErrorKind::TimedOut | io::ErrorKind::Interrupted => { + // These errors might be recoverable, try again. + ControlFlow::Continue(()) + } + _ => { + // Everything else is either unrecoverable or unknown to us at the + // time of writing and so we can't guess how to handle it, so + // abort. + error!("I/O error: {}", err); + ControlFlow::Break(ConnectionEvent::DisconnectWithoutFlush) } } } diff --git a/src/net/server/dgram.rs b/src/net/server/dgram.rs index 814e958a6..c455484d5 100644 --- a/src/net/server/dgram.rs +++ b/src/net/server/dgram.rs @@ -20,6 +20,8 @@ use std::string::String; use std::string::ToString; use std::sync::{Arc, Mutex}; +use arc_swap::ArcSwap; +use futures::prelude::stream::StreamExt; use octseq::Octets; use tokio::io::ReadBuf; use tokio::net::UdpSocket; @@ -32,6 +34,7 @@ use tracing::warn; use tracing::Level; use tracing::{enabled, error, trace}; +use crate::base::wire::Composer; use crate::base::Message; use crate::net::server::buf::BufSource; use crate::net::server::error::Error; @@ -45,9 +48,6 @@ use crate::utils::config::DefMinMax; use super::buf::VecBufSource; use super::message::{TransportSpecificContext, UdpTransportContext}; use super::ServerCommand; -use crate::base::wire::Composer; -use arc_swap::ArcSwap; -use futures::prelude::stream::StreamExt; /// A UDP transport based DNS server transport. /// @@ -118,7 +118,7 @@ impl Config { /// [RFC 6891]: /// https://datatracker.ietf.org/doc/html/rfc6891#section-6.2.5 pub fn set_max_response_size(&mut self, value: Option) { - self.max_response_size = value; + self.max_response_size = value.map(|v| MAX_RESPONSE_SIZE.limit(v)); } /// Sets the time to wait for a complete message to be written to the @@ -251,12 +251,16 @@ type CommandReceiver = watch::Receiver; pub struct DgramServer where Sock: AsyncDgramSock + Send + Sync + 'static, - Buf: BufSource + Send + Sync + 'static, - Buf::Output: Octets + Send + Sync + Unpin, - Svc: Service + Send + Sync + 'static + Clone, - Svc::Target: Composer + Send, - Svc::Stream: Send, - Svc::Future: Send, + Buf: BufSource + Send + Sync, + ::Output: Octets + Send + Sync + Unpin + 'static, + Svc: Clone + + Service<::Output, ()> + + Send + + Sync + + 'static, + ::Output, ()>>::Future: Send, + ::Output, ()>>::Stream: Send, + ::Output, ()>>::Target: Composer + Send, { /// The configuration of the server. config: Arc>, @@ -292,11 +296,11 @@ impl DgramServer where Sock: AsyncDgramSock + Send + Sync, Buf: BufSource + Send + Sync, - Buf::Output: Octets + Send + Sync + Unpin, - Svc: Service + Send + Sync + Clone, - Svc::Target: Composer + Send, - Svc::Stream: Send, - Svc::Future: Send, + ::Output: Octets + Send + Sync + Unpin, + Svc: Clone + Service<::Output, ()> + Send + Sync, + ::Output, ()>>::Future: Send, + ::Output, ()>>::Stream: Send, + ::Output, ()>>::Target: Composer + Send, { /// Constructs a new [`DgramServer`] with default configuration. /// @@ -310,7 +314,7 @@ where /// /// Takes: /// - A socket which must implement [`AsyncDgramSock`] and is responsible - /// receiving new messages and send responses back to the client. + /// receiving new messages and send responses back to the client. /// - A [`BufSource`] for creating buffers on demand. /// - A [`Service`] for handling received requests and generating /// responses. @@ -347,13 +351,13 @@ where /// impl DgramServer where - Sock: AsyncDgramSock + Send + Sync + 'static, - Buf: BufSource + Send + Sync + 'static, - Buf::Output: Octets + Send + Sync + 'static + Debug + Unpin, - Svc: Service + Send + Sync + 'static + Clone, - Svc::Target: Composer + Send, - Svc::Stream: Send, - Svc::Future: Send, + Sock: AsyncDgramSock + Send + Sync, + Buf: BufSource + Send + Sync, + ::Output: Octets + Send + Sync + Unpin, + Svc: Clone + Service<::Output, ()> + Send + Sync, + ::Output, ()>>::Future: Send, + ::Output, ()>>::Stream: Send, + ::Output, ()>>::Target: Composer + Send, { /// Get a reference to the network source being used to receive messages. #[must_use] @@ -374,11 +378,15 @@ impl DgramServer where Sock: AsyncDgramSock + Send + Sync + 'static, Buf: BufSource + Send + Sync, - Buf::Output: Octets + Send + Sync + 'static + Unpin, - Svc: Service + Send + Sync + 'static + Clone, - Svc::Target: Composer + Send, - Svc::Stream: Send, - Svc::Future: Send, + ::Output: Octets + Send + Sync + 'static + Unpin, + Svc: Clone + + Service<::Output, ()> + + Send + + Sync + + 'static, + ::Output, ()>>::Future: Send, + ::Output, ()>>::Stream: Send, + ::Output, ()>>::Target: Composer + Send, { /// Start the server. /// @@ -456,13 +464,13 @@ where impl DgramServer where - Sock: AsyncDgramSock + Send + Sync + 'static, + Sock: AsyncDgramSock + Send + Sync, Buf: BufSource + Send + Sync, - Buf::Output: Octets + Send + Sync + 'static + Unpin, - Svc: Service + Send + Sync + 'static + Clone, - Svc::Target: Composer + Send, - Svc::Stream: Send, - Svc::Future: Send, + ::Output: Octets + Send + Sync + Unpin, + Svc: Clone + Service<::Output, ()> + Send + Sync, + ::Output, ()>>::Future: Send, + ::Output, ()>>::Stream: Send, + ::Output, ()>>::Target: Composer + Send, { /// Receive incoming messages until shutdown or fatal error. async fn run_until_error(&self) -> Result<(), String> { @@ -490,8 +498,8 @@ where self.metrics.inc_num_received_requests(); if enabled!(Level::TRACE) { - let pcap_text = to_pcap_text(&buf, bytes_read); - trace!(%addr, pcap_text, "Received message"); + let pcap_text = to_pcap_text(&buf, bytes_read.min(128)); + trace!(%addr, pcap_text, "Received message (dumping max 128 bytes)"); } let svc = self.service.clone(); @@ -507,9 +515,10 @@ where } Ok(msg) => { + trace!(addr = %addr, ?msg, "Parsed first question: {:?}", msg.first_question()); let ctx = UdpTransportContext::new(cfg.load().max_response_size); let ctx = TransportSpecificContext::Udp(ctx); - let request = Request::new(addr, received_at, msg, ctx); + let request = Request::new(addr, received_at, msg, ctx, ()); let mut stream = svc.call(request).await; while let Some(Ok(call_result)) = stream.next().await { let (response, feedback) = call_result.into_inner(); @@ -536,8 +545,8 @@ where // Logging if enabled!(Level::TRACE) { - let pcap_text = to_pcap_text(bytes, bytes.len()); - trace!(%addr, pcap_text, "Sending response"); + let pcap_text = to_pcap_text(bytes, bytes.len().min(128)); + trace!(%addr, pcap_text, "Sending response (dumping max 128 bytes)"); } metrics.inc_num_pending_writes(); @@ -621,7 +630,8 @@ where /// Receive a single datagram using the user supplied network socket. fn recv_from( &self, - ) -> Result<(Buf::Output, SocketAddr, usize), io::Error> { + ) -> Result<(::Output, SocketAddr, usize), io::Error> + { let mut msg = self.buf.create_buf(); let mut buf = ReadBuf::new(msg.as_mut()); self.sock @@ -659,12 +669,16 @@ where impl Drop for DgramServer where Sock: AsyncDgramSock + Send + Sync + 'static, - Buf: BufSource + Send + Sync + 'static, - Buf::Output: Octets + Send + Sync + Unpin, - Svc: Service + Send + Sync + 'static + Clone, - Svc::Target: Composer + Send, - Svc::Stream: Send, - Svc::Future: Send, + Buf: BufSource + Send + Sync, + ::Output: Octets + Send + Sync + Unpin + 'static, + Svc: Clone + + Service<::Output, ()> + + Send + + Sync + + 'static, + ::Output, ()>>::Future: Send, + ::Output, ()>>::Stream: Send, + ::Output, ()>>::Target: Composer + Send, { fn drop(&mut self) { // Shutdown the DgramServer. Don't handle the failure case here as diff --git a/src/net/server/message.rs b/src/net/server/message.rs index 0eb8369cd..b9da1947d 100644 --- a/src/net/server/message.rs +++ b/src/net/server/message.rs @@ -155,7 +155,10 @@ impl From for TransportSpecificContext { /// message itself but also on the circumstances surrounding its creation and /// delivery. #[derive(Debug)] -pub struct Request + Send + Sync + Unpin> { +pub struct Request +where + Octs: AsRef<[u8]> + Send + Sync + Unpin, +{ /// The network address of the connected client. client_addr: std::net::SocketAddr, @@ -168,21 +171,34 @@ pub struct Request + Send + Sync + Unpin> { /// Properties of the request specific to the server and transport /// protocol via which it was received. transport_specific: TransportSpecificContext, + + /// The number of bytes to be reserved when generating a response + /// to this request so that needed additional data can be added to + /// to the generated response. + num_reserved_bytes: u16, + + metadata: Metadata, // TODO: Make middleware take an impl SomeInterface? } -impl + Send + Sync + Unpin> Request { +impl Request +where + Octs: AsRef<[u8]> + Send + Sync + Unpin, +{ /// Creates a new request wrapper around a message along with its context. pub fn new( client_addr: std::net::SocketAddr, received_at: Instant, message: Message, transport_specific: TransportSpecificContext, + metadata: Metadata, ) -> Self { Self { client_addr, received_at, message: Arc::new(message), transport_specific, + num_reserved_bytes: 0, + metadata, } } @@ -205,17 +221,54 @@ impl + Send + Sync + Unpin> Request { pub fn message(&self) -> &Arc> { &self.message } + + /// Request that an additional number of bytes be reserved in the response + /// to this message. + pub fn reserve_bytes(&mut self, len: u16) { + self.num_reserved_bytes += len; + tracing::trace!( + "Reserved {len} bytes: total now = {}", + self.num_reserved_bytes + ); + } + + /// The number of bytes to reserve when generating a response to this + /// message. + pub fn num_reserved_bytes(&self) -> u16 { + self.num_reserved_bytes + } + + pub fn with_new_metadata(self, new_metadata: T) -> Request { + Request:: { + client_addr: self.client_addr, + received_at: self.received_at, + message: self.message, + transport_specific: self.transport_specific, + num_reserved_bytes: self.num_reserved_bytes, + metadata: new_metadata, + } + } + + pub fn metadata(&self) -> &Metadata { + &self.metadata + } } //--- Clone -impl + Send + Sync + Unpin> Clone for Request { +impl Clone for Request +where + Octs: AsRef<[u8]> + Send + Sync + Unpin, + Metadata: Clone, +{ fn clone(&self) -> Self { Self { client_addr: self.client_addr, received_at: self.received_at, message: Arc::clone(&self.message), transport_specific: self.transport_specific.clone(), + num_reserved_bytes: self.num_reserved_bytes, + metadata: self.metadata.clone(), } } } diff --git a/src/net/server/middleware/cookies.rs b/src/net/server/middleware/cookies.rs index f7dbcb623..4fe5414ec 100644 --- a/src/net/server/middleware/cookies.rs +++ b/src/net/server/middleware/cookies.rs @@ -5,10 +5,10 @@ use core::ops::ControlFlow; use std::vec::Vec; -use futures::stream::{once, Once}; +use futures::stream::{once, Once, Stream}; use octseq::Octets; use rand::RngCore; -use tracing::{debug, trace, warn}; +use tracing::{debug, error, trace, warn}; use crate::base::iana::{OptRcode, Rcode}; use crate::base::message_builder::AdditionalBuilder; @@ -19,8 +19,8 @@ use crate::base::{Serial, StreamTarget}; use crate::net::server::message::Request; use crate::net::server::middleware::stream::MiddlewareStream; use crate::net::server::service::{CallResult, Service}; -use crate::net::server::util::add_edns_options; -use crate::net::server::util::{mk_builder_for_target, start_reply}; +use crate::net::server::util::mk_builder_for_target; +use crate::net::server::util::{add_edns_options, mk_error_response}; //----------- Constants ------------------------------------------------------- @@ -34,7 +34,7 @@ const ONE_HOUR_AS_SECS: u32 = 60 * 60; //----------- CookiesMiddlewareProcessor -------------------------------------- -/// A DNS Cookies middleware service. +/// A middleware service for enforcing the use of DNS Cookies. /// /// Standards covered by ths implementation: /// @@ -46,8 +46,8 @@ const ONE_HOUR_AS_SECS: u32 = 60 * 60; /// [7873]: https://datatracker.ietf.org/doc/html/rfc7873 /// [9018]: https://datatracker.ietf.org/doc/html/rfc7873 #[derive(Clone, Debug)] -pub struct CookiesMiddlewareSvc { - svc: Svc, +pub struct CookiesMiddlewareSvc { + next_svc: NextSvc, /// A user supplied secret used in making the cookie value. server_secret: [u8; 16], @@ -57,25 +57,30 @@ pub struct CookiesMiddlewareSvc { /// to reconnect with TCP in order to "authenticate" themselves. ip_deny_list: Vec, - _phantom: PhantomData, + _phantom: PhantomData<(RequestOctets, RequestMeta)>, + + enabled: bool, } -impl CookiesMiddlewareSvc { +impl + CookiesMiddlewareSvc +{ /// Creates an instance of this processor. #[must_use] - pub fn new(svc: Svc, server_secret: [u8; 16]) -> Self { + pub fn new(next_svc: NextSvc, server_secret: [u8; 16]) -> Self { Self { - svc, + next_svc, server_secret, ip_deny_list: vec![], _phantom: PhantomData, + enabled: true, } } - pub fn with_random_secret(svc: Svc) -> Self { + pub fn with_random_secret(next_svc: NextSvc) -> Self { let mut server_secret = [0u8; 16]; rand::thread_rng().fill_bytes(&mut server_secret); - Self::new(svc, server_secret) + Self::new(next_svc, server_secret) } /// Define IP addresses required to supply DNS cookies if using UDP. @@ -87,13 +92,20 @@ impl CookiesMiddlewareSvc { self.ip_deny_list = ip_deny_list.into(); self } + + pub fn enable(mut self, enabled: bool) -> Self { + self.enabled = enabled; + self + } } -impl CookiesMiddlewareSvc +impl + CookiesMiddlewareSvc where RequestOctets: Octets + Send + Sync + Unpin, - Svc: Service, - Svc::Target: Composer + Default, + RequestMeta: Clone + Default, + NextSvc: Service, + NextSvc::Target: Composer + Default, { /// Get the DNS COOKIE, if any, for the given message. /// @@ -111,7 +123,7 @@ where /// parsed. #[must_use] fn cookie( - request: &Request, + request: &Request, ) -> Option> { // Note: We don't use `opt::Opt::first()` because that will silently // ignore an unparseable COOKIE option but we need to detect and @@ -156,10 +168,22 @@ where /// Create a DNS response message for the given request, including cookie. fn response_with_cookie( &self, - request: &Request, + request: &Request, rcode: OptRcode, - ) -> AdditionalBuilder> { - let mut additional = start_reply(request.message()).additional(); + ) -> AdditionalBuilder> { + let res = mk_builder_for_target() + .start_answer(request.message(), rcode.rcode()); + + let mut additional = match res { + Ok(answer) => answer.additional(), + Err(err) => { + error!("Failed to create response: {err}"); + return mk_error_response( + request.message(), + OptRcode::SERVFAIL, + ); + } + }; if let Some(Ok(client_cookie)) = Self::cookie(request) { let response_cookie = client_cookie.create_response( @@ -193,8 +217,8 @@ where #[must_use] fn bad_cookie_response( &self, - request: &Request, - ) -> AdditionalBuilder> { + request: &Request, + ) -> AdditionalBuilder> { // https://datatracker.ietf.org/doc/html/rfc7873#section-5.2.3 // "If the server responds [ed: by sending a BADCOOKIE error // response], it SHALL generate its own COOKIE option containing @@ -209,8 +233,8 @@ where #[must_use] fn prefetch_cookie_response( &self, - request: &Request, - ) -> AdditionalBuilder> { + request: &Request, + ) -> AdditionalBuilder> { // https://datatracker.ietf.org/doc/html/rfc7873#section-5.4 // Querying for a Server Cookie: // "For servers with DNS Cookies enabled, the @@ -228,8 +252,8 @@ where #[tracing::instrument(skip_all, fields(request_ip = %request.client_addr().ip()))] fn preprocess( &self, - request: &Request, - ) -> ControlFlow>> { + request: &Request, + ) -> ControlFlow>> { match Self::cookie(request) { None => { trace!("Request does not contain a DNS cookie"); @@ -420,66 +444,41 @@ where ControlFlow::Continue(()) } - - // fn postprocess( - // _request: &Request, - // _response: &mut AdditionalBuilder>, - // _server_secret: [u8; 16], - // ) where - // RequestOctets: Octets, - // { - // // https://datatracker.ietf.org/doc/html/rfc7873#section-5.2.1 - // // No OPT RR or No COOKIE Option: - // // If the request lacked a client cookie we don't need to do - // // anything. - // // - // // https://datatracker.ietf.org/doc/html/rfc7873#section-5.2.2 - // // Malformed COOKIE Option: - // // If the request COOKIE option was malformed we would have already - // // rejected it during pre-processing so again nothing to do here. - // // - // // https://datatracker.ietf.org/doc/html/rfc7873#section-5.2.3 - // // Only a Client Cookie: - // // If the request had a client cookie but no server cookie and - // // we didn't already reject the request during pre-processing. - // // - // // https://datatracker.ietf.org/doc/html/rfc7873#section-5.2.4 - // // A Client Cookie and an Invalid Server Cookie: - // // Per RFC 7873 this is handled the same way as the "Only a Client - // // Cookie" case. - // // - // // https://datatracker.ietf.org/doc/html/rfc7873#section-5.2.5 - // // A Client Cookie and a Valid Server Cookie - // // Any server cookie will already have been validated during - // // pre-processing, we don't need to check it again here. - // } } //--- Service -impl Service - for CookiesMiddlewareSvc +impl Service + for CookiesMiddlewareSvc where RequestOctets: Octets + Send + Sync + 'static + Unpin, - Svc: Service, - Svc::Future: core::future::Future + Unpin, - ::Output: Unpin, - Svc::Target: Composer + Default, + RequestMeta: Clone + Default, + NextSvc: Service, + NextSvc::Target: Composer + Default, + NextSvc::Future: Unpin, { - type Target = Svc::Target; + type Target = NextSvc::Target; type Stream = MiddlewareStream< - Svc::Future, - Svc::Stream, - Svc::Stream, - Once::Item>>, - ::Item, + NextSvc::Future, + NextSvc::Stream, + NextSvc::Stream, + Once::Item>>, + ::Item, >; type Future = core::future::Ready; - fn call(&self, request: Request) -> Self::Future { + fn call( + &self, + request: Request, + ) -> Self::Future { + if !self.enabled { + let svc_call_fut = self.next_svc.call(request.clone()); + return ready(MiddlewareStream::IdentityFuture(svc_call_fut)); + } + match self.preprocess(&request) { ControlFlow::Continue(()) => { - let svc_call_fut = self.svc.call(request.clone()); + let svc_call_fut = self.next_svc.call(request.clone()); ready(MiddlewareStream::IdentityFuture(svc_call_fut)) } ControlFlow::Break(response) => ready(MiddlewareStream::Result( @@ -494,6 +493,7 @@ mod tests { use bytes::Bytes; use std::vec::Vec; use tokio::time::Instant; + use tokio_stream::StreamExt; use crate::base::opt::cookie::ClientCookie; use crate::base::opt::Cookie; @@ -502,7 +502,6 @@ mod tests { use crate::net::server::middleware::cookies::CookiesMiddlewareSvc; use crate::net::server::service::{CallResult, Service, ServiceResult}; use crate::net::server::util::service_fn; - use futures::prelude::stream::StreamExt; #[tokio::test] async fn dont_add_cookie_twice() { @@ -520,8 +519,13 @@ mod tests { // as if it came from a UDP server. let ctx = UdpTransportContext::default(); let client_addr = "127.0.0.1:12345".parse().unwrap(); - let request = - Request::new(client_addr, Instant::now(), message, ctx.into()); + let request = Request::new( + client_addr, + Instant::now(), + message, + ctx.into(), + (), + ); fn my_service( _req: Request>, diff --git a/src/net/server/middleware/edns.rs b/src/net/server/middleware/edns.rs index 5dcfa6ec7..530f1d6f7 100644 --- a/src/net/server/middleware/edns.rs +++ b/src/net/server/middleware/edns.rs @@ -3,16 +3,17 @@ use core::future::{ready, Ready}; use core::marker::PhantomData; use core::ops::ControlFlow; -use futures::stream::{once, Once}; +use futures::stream::{once, Once, Stream}; use octseq::Octets; use tracing::{debug, enabled, error, trace, warn, Level}; use crate::base::iana::OptRcode; use crate::base::message_builder::AdditionalBuilder; +use crate::base::name::ToLabelIter; use crate::base::opt::keepalive::IdleTimeout; -use crate::base::opt::{Opt, OptRecord, TcpKeepalive}; +use crate::base::opt::{ComposeOptData, Opt, OptRecord, TcpKeepalive}; use crate::base::wire::Composer; -use crate::base::StreamTarget; +use crate::base::{Name, StreamTarget}; use crate::net::server::message::{Request, TransportSpecificContext}; use crate::net::server::middleware::stream::MiddlewareStream; use crate::net::server::service::{CallResult, Service, ServiceResult}; @@ -45,33 +46,45 @@ const EDNS_VERSION_ZERO: u8 = 0; /// [7828]: https://datatracker.ietf.org/doc/html/rfc7828 /// [9210]: https://datatracker.ietf.org/doc/html/rfc9210 #[derive(Clone, Debug, Default)] -pub struct EdnsMiddlewareSvc { - svc: Svc, +pub struct EdnsMiddlewareSvc { + next_svc: NextSvc, - _phantom: PhantomData, + _phantom: PhantomData<(RequestOctets, RequestMeta)>, + + enabled: bool, } -impl EdnsMiddlewareSvc { +impl + EdnsMiddlewareSvc +{ /// Creates an instance of this processor. #[must_use] - pub fn new(svc: Svc) -> Self { + pub fn new(next_svc: NextSvc) -> Self { Self { - svc, + next_svc, _phantom: PhantomData, + enabled: true, } } + + pub fn enable(mut self, enabled: bool) -> Self { + self.enabled = enabled; + self + } } -impl EdnsMiddlewareSvc +impl + EdnsMiddlewareSvc where RequestOctets: Octets + Send + Sync + Unpin, - Svc: Service, - Svc::Target: Composer + Default, + NextSvc: Service, + NextSvc::Target: Composer + Default, + RequestMeta: Clone + Default, { fn preprocess( &self, - request: &Request, - ) -> ControlFlow>> { + request: &mut Request, + ) -> ControlFlow>> { // https://www.rfc-editor.org/rfc/rfc6891.html#section-6.1.1 // 6.1.1: Basic Elements // ... @@ -188,7 +201,7 @@ where )); } - TransportSpecificContext::NonUdp(_) => { + TransportSpecificContext::NonUdp(ctx) => { // https://datatracker.ietf.org/doc/html/rfc7828#section-3.2.1 // 3.2.1. Sending Queries // "Clients MUST specify an OPTION-LENGTH of 0 @@ -206,6 +219,16 @@ where ); } } + + if let Some(keep_alive) = ctx.idle_timeout() { + if let Ok(timeout) = + IdleTimeout::try_from(keep_alive) + { + Self::reserve_space_for_keep_alive_opt( + request, timeout, + ); + } + } } } } @@ -216,8 +239,8 @@ where } fn postprocess( - request: &Request, - response: &mut AdditionalBuilder>, + request: &Request, + response: &mut AdditionalBuilder>, ) { // https://www.rfc-editor.org/rfc/rfc6891.html#section-6.1.1 // 6.1.1: Basic Elements @@ -274,6 +297,12 @@ where // timeout is known: "Signal the timeout value // using the edns-tcp-keepalive EDNS(0) option // [RFC7828]". + + // Remove the limit we should have imposed + // during pre-processing so that we can use + // the space we reserved for the OPT RR. + response.clear_push_limit(); + if let Err(err) = // TODO: Don't add the option if it // already exists? @@ -304,11 +333,39 @@ where // field to some value? } + fn reserve_space_for_keep_alive_opt( + request: &mut Request, + timeout: IdleTimeout, + ) { + // TODO: Calculate this once as a const value, not on every request. + + let keep_alive_opt = TcpKeepalive::new(Some(timeout)); + let root_name_len = Name::root_ref().compose_len(); + + // See: + // - https://datatracker.ietf.org/doc/html/rfc1035#section-3.2.1 + // - https://datatracker.ietf.org/doc/html/rfc6891#autoid-12 + // - https://datatracker.ietf.org/doc/html/rfc7828#section-3.1 + + // Calculate the size of the DNS OPTION RR that will be added to the + // response during post-processing. + let wire_opt_len = root_name_len // "0" root domain name per RFC 6891 + + 2 // TYPE + + 2 // CLASS + + 4 // TTL + + 2 // RDLEN + + 2 // OPTION-CODE + + 2 // OPTION-LENGTH + + keep_alive_opt.compose_len(); // OPTION-DATA + + request.reserve_bytes(wire_opt_len); + } + fn map_stream_item( - request: Request, - mut stream_item: ServiceResult, - _metadata: (), - ) -> ServiceResult { + request: Request, + mut stream_item: ServiceResult, + _pp_meta: (), + ) -> ServiceResult { if let Ok(cr) = &mut stream_item { if let Some(response) = cr.response_mut() { Self::postprocess(&request, response); @@ -320,28 +377,43 @@ where //--- Service -impl Service - for EdnsMiddlewareSvc +impl Service + for EdnsMiddlewareSvc where RequestOctets: Octets + Send + Sync + 'static + Unpin, - Svc: Service, - Svc::Target: Composer + Default, - Svc::Future: Unpin, + RequestMeta: Clone + Default + Unpin, + NextSvc: Service, + NextSvc::Target: Composer + Default, + NextSvc::Future: Unpin, { - type Target = Svc::Target; + type Target = NextSvc::Target; type Stream = MiddlewareStream< - Svc::Future, - Svc::Stream, - PostprocessingStream, - Once::Item>>, - ::Item, + NextSvc::Future, + NextSvc::Stream, + PostprocessingStream< + RequestOctets, + NextSvc::Future, + NextSvc::Stream, + RequestMeta, + (), + >, + Once::Item>>, + ::Item, >; type Future = core::future::Ready; - fn call(&self, request: Request) -> Self::Future { - match self.preprocess(&request) { + fn call( + &self, + mut request: Request, + ) -> Self::Future { + if !self.enabled { + let svc_call_fut = self.next_svc.call(request.clone()); + return ready(MiddlewareStream::IdentityFuture(svc_call_fut)); + } + + match self.preprocess(&mut request) { ControlFlow::Continue(()) => { - let svc_call_fut = self.svc.call(request.clone()); + let svc_call_fut = self.next_svc.call(request.clone()); let map = PostprocessingStream::new( svc_call_fut, request, @@ -462,6 +534,7 @@ mod tests { Instant::now(), message, ctx.into(), + (), ); fn my_service( @@ -478,7 +551,7 @@ mod tests { // Either call the service directly. let my_svc = service_fn(my_service, ()); let mut stream = my_svc.call(request.clone()).await; - let _call_result: CallResult> = + let _call_result: CallResult<_> = stream.next().await.unwrap().unwrap(); // Or pass the query through the middleware processor diff --git a/src/net/server/middleware/mandatory.rs b/src/net/server/middleware/mandatory.rs index b2b74bae1..914d55980 100644 --- a/src/net/server/middleware/mandatory.rs +++ b/src/net/server/middleware/mandatory.rs @@ -5,7 +5,7 @@ use core::ops::ControlFlow; use std::fmt::Display; -use futures::stream::{once, Once}; +use futures::stream::{once, Once, Stream}; use octseq::Octets; use tracing::{debug, error, trace, warn}; @@ -39,25 +39,27 @@ pub const MINIMUM_RESPONSE_BYTE_LEN: u16 = 512; /// [1035]: https://datatracker.ietf.org/doc/html/rfc1035 /// [2181]: https://datatracker.ietf.org/doc/html/rfc2181 #[derive(Clone, Debug)] -pub struct MandatoryMiddlewareSvc { +pub struct MandatoryMiddlewareSvc { /// In strict mode the processor does more checks on requests and /// responses. strict: bool, - svc: Svc, + next_svc: NextSvc, - _phantom: PhantomData, + _phantom: PhantomData<(RequestOctets, RequestMeta)>, } -impl MandatoryMiddlewareSvc { +impl + MandatoryMiddlewareSvc +{ /// Creates a new processor instance. /// /// The processor will operate in strict mode. #[must_use] - pub fn new(svc: Svc) -> Self { + pub fn new(next_svc: NextSvc) -> Self { Self { strict: true, - svc, + next_svc, _phantom: PhantomData, } } @@ -66,20 +68,22 @@ impl MandatoryMiddlewareSvc { /// /// The processor will operate in relaxed mode. #[must_use] - pub fn relaxed(svc: Svc) -> Self { + pub fn relaxed(next_svc: NextSvc) -> Self { Self { strict: false, - svc, + next_svc, _phantom: PhantomData, } } } -impl MandatoryMiddlewareSvc +impl + MandatoryMiddlewareSvc where RequestOctets: Octets + Send + Sync + Unpin, - Svc: Service, - Svc::Target: Composer + Default, + NextSvc: Service, + NextSvc::Target: Composer + Default, + RequestMeta: Clone + Default, { /// Truncate the given response message if it is too large. /// @@ -92,8 +96,8 @@ where /// any OPT record present which will be preserved, then truncates to the /// specified byte length. fn truncate( - request: &Request, - response: &mut AdditionalBuilder>, + request: &Request, + response: &mut AdditionalBuilder>, ) -> Result<(), TruncateError> { if let TransportSpecificContext::Udp(ctx) = request.transport_ctx() { // https://datatracker.ietf.org/doc/html/rfc1035#section-4.2.1 @@ -190,7 +194,7 @@ where fn preprocess( &self, msg: &Message, - ) -> ControlFlow>> { + ) -> ControlFlow>> { // https://www.rfc-editor.org/rfc/rfc3425.html // 3 - Effect on RFC 1035 // .. @@ -210,8 +214,8 @@ where } fn postprocess( - request: &Request, - response: &mut AdditionalBuilder>, + request: &Request, + response: &mut AdditionalBuilder>, strict: bool, ) { if let Err(err) = Self::truncate(request, response) { @@ -260,10 +264,10 @@ where } fn map_stream_item( - request: Request, - mut stream_item: ServiceResult, + request: Request, + mut stream_item: ServiceResult, strict: bool, - ) -> ServiceResult { + ) -> ServiceResult { if let Ok(cr) = &mut stream_item { if let Some(response) = cr.response_mut() { Self::postprocess(&request, response, strict); @@ -275,28 +279,38 @@ where //--- Service -impl Service - for MandatoryMiddlewareSvc +impl Service + for MandatoryMiddlewareSvc where RequestOctets: Octets + Send + Sync + 'static + Unpin, - Svc: Service, - Svc::Future: Unpin, - Svc::Target: Composer + Default, + NextSvc: Service, + NextSvc::Future: Unpin, + NextSvc::Target: Composer + Default, + RequestMeta: Clone + Default + Unpin, { - type Target = Svc::Target; + type Target = NextSvc::Target; type Stream = MiddlewareStream< - Svc::Future, - Svc::Stream, - PostprocessingStream, - Once::Item>>, - ::Item, + NextSvc::Future, + NextSvc::Stream, + PostprocessingStream< + RequestOctets, + NextSvc::Future, + NextSvc::Stream, + RequestMeta, + bool, + >, + Once::Item>>, + ::Item, >; type Future = Ready; - fn call(&self, request: Request) -> Self::Future { + fn call( + &self, + request: Request, + ) -> Self::Future { match self.preprocess(request.message()) { ControlFlow::Continue(()) => { - let svc_call_fut = self.svc.call(request.clone()); + let svc_call_fut = self.next_svc.call(request.clone()); let map = PostprocessingStream::new( svc_call_fut, request, @@ -418,6 +432,7 @@ mod tests { Instant::now(), message, ctx.into(), + (), ); fn my_service( diff --git a/src/net/server/middleware/stream.rs b/src/net/server/middleware/stream.rs index 99c74f4d7..6d4c7fb2a 100644 --- a/src/net/server/middleware/stream.rs +++ b/src/net/server/middleware/stream.rs @@ -102,25 +102,49 @@ where //------------ PostprocessingStreamCallback ---------------------------------- -type PostprocessingStreamCallback = - fn(Request, StreamItem, Metadata) -> StreamItem; +type PostprocessingStreamCallback< + RequestOctets, + StreamItem, + RequestMeta, + PostProcessingMeta, +> = fn( + Request, + StreamItem, + PostProcessingMeta, +) -> StreamItem; //------------ PostprocessingStream ------------------------------------------ -pub struct PostprocessingStream -where +pub struct PostprocessingStream< + RequestOctets, + Future, + Stream, + RequestMeta, + PostProcessingMeta, +> where RequestOctets: Octets + Send + Sync + Unpin, Future: core::future::Future, Stream: futures::stream::Stream, { - request: Request, + request: Request, state: PostprocessingStreamState, - cb: PostprocessingStreamCallback, - metadata: Metadata, + cb: PostprocessingStreamCallback< + RequestOctets, + Stream::Item, + RequestMeta, + PostProcessingMeta, + >, + pp_meta: PostProcessingMeta, } -impl - PostprocessingStream +impl + PostprocessingStream< + RequestOctets, + Future, + Stream, + RequestMeta, + PostProcessingMeta, + > where RequestOctets: Octets + Send + Sync + Unpin, Future: core::future::Future, @@ -128,33 +152,42 @@ where { pub fn new( svc_call_fut: Future, - request: Request, - metadata: Metadata, + request: Request, + metadata: PostProcessingMeta, cb: PostprocessingStreamCallback< RequestOctets, Stream::Item, - Metadata, + RequestMeta, + PostProcessingMeta, >, ) -> Self { Self { state: PostprocessingStreamState::Pending(svc_call_fut), request, cb, - metadata, + pp_meta: metadata, } } } //--- impl Stream -impl futures::stream::Stream - for PostprocessingStream +impl + futures::stream::Stream + for PostprocessingStream< + RequestOctets, + Future, + Stream, + RequestMeta, + PostProcessingMeta, + > where RequestOctets: Octets + Send + Sync + Unpin, Future: core::future::Future + Unpin, Stream: futures::stream::Stream + Unpin, Self: Unpin, - Metadata: Clone, + RequestMeta: Clone, + PostProcessingMeta: Clone, { type Item = Stream::Item; @@ -173,9 +206,9 @@ where let stream_item = ready!(stream.poll_next_unpin(cx)); trace!("Stream item retrieved, mapping to downstream type"); let request = self.request.clone(); - let metadata = self.metadata.clone(); - let map = stream_item - .map(|item| (self.cb)(request, item, metadata)); + let pp_meta = self.pp_meta.clone(); + let map = + stream_item.map(|item| (self.cb)(request, item, pp_meta)); Poll::Ready(map) } } diff --git a/src/net/server/service.rs b/src/net/server/service.rs index c78e26051..b10a81cd1 100644 --- a/src/net/server/service.rs +++ b/src/net/server/service.rs @@ -6,7 +6,6 @@ use core::fmt::Display; use core::ops::Deref; -use std::sync::Arc; use std::time::Duration; use std::vec::Vec; @@ -16,8 +15,6 @@ use crate::base::wire::ParseError; use crate::base::StreamTarget; use super::message::Request; -use core::future::ready; -use futures::stream::once; //------------ Service ------------------------------------------------------- @@ -178,7 +175,10 @@ pub type ServiceResult = Result, ServiceError>; /// [`net::server`]: crate::net::server /// [`call`]: Self::call() /// [`service_fn`]: crate::net::server::util::service_fn() -pub trait Service + Send + Sync + Unpin = Vec> +pub trait Service< + RequestOctets: AsRef<[u8]> + Send + Sync + Unpin = Vec, + RequestMeta: Clone + Default = (), +> { /// The underlying byte storage type used to hold generated responses. type Target; @@ -191,42 +191,32 @@ pub trait Service + Send + Sync + Unpin = Vec> type Future: core::future::Future; /// Generate a response to a fully pre-processed request. - fn call(&self, request: Request) -> Self::Future; + fn call( + &self, + request: Request, + ) -> Self::Future; } -//--- impl Service for Arc +//--- impl Service for Deref -/// Helper trait impl to treat an [`Arc`] as a `Service`. -impl Service for Arc +/// Helper trait impl to treat a [`Deref`] as a [`Service`]. +impl Service + for U where RequestOctets: Unpin + Send + Sync + AsRef<[u8]>, - T: ?Sized + Service, + T: ?Sized + Service, + U: Deref + Clone, + RequestMeta: Clone + Default, { type Target = T::Target; type Stream = T::Stream; type Future = T::Future; - fn call(&self, request: Request) -> Self::Future { - Arc::deref(self).call(request) - } -} - -//--- impl Service for functions with matching signature - -/// Helper trait impl to treat a function as a `Service`. -impl Service for F -where - RequestOctets: AsRef<[u8]> + Send + Sync + Unpin, - F: Fn(Request) -> ServiceResult, -{ - type Target = Target; - type Stream = futures::stream::Once< - core::future::Ready>, - >; - type Future = core::future::Ready; - - fn call(&self, request: Request) -> Self::Future { - ready(once(ready((*self)(request)))) + fn call( + &self, + request: Request, + ) -> Self::Future { + (**self).call(request) } } diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index 5e9378503..bf6af2a96 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -8,7 +8,6 @@ use std::sync::Arc; use std::time::Duration; use std::vec::Vec; -use octseq::Octets; use rstest::rstest; use tracing::instrument; use tracing::{trace, warn}; @@ -73,61 +72,53 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { let with_cookies = server_config.cookies.enabled && server_config.cookies.secret.is_some(); - async fn finish_svc<'a, RequestOctets, Svc>( - svc: Svc, - server_config: ServerConfig<'a>, - stelline: &parse_stelline::Stelline, - ) where - RequestOctets: Octets + Send + Sync + Unpin, - Svc: Service + Send + Sync + 'static, - // TODO: Why are the following bounds needed to persuade the compiler - // that the `svc` value created _within the function_ (not the one - // passed in as an argument) is actually an impl of the Service trait? - MandatoryMiddlewareSvc, Svc>: Service + Send + Sync, - , Svc> as Service>::Target: - Composer + Default + Send + Sync, - , Svc> as Service>::Stream: - Send + Sync, - , Svc> as Service>::Future: - Send + Sync, - { - let svc = MandatoryMiddlewareSvc::, _>::new(svc); - let svc = Arc::new(svc); - - // Create dgram and stream servers for answering requests - let (dgram_srv, dgram_conn, stream_srv, stream_conn) = - mk_servers(svc, &server_config); - - // Create a client factory for sending requests - let client_factory = mk_client_factory(dgram_conn, stream_conn); - - // Run the Stelline test! - let step_value = Arc::new(CurrStepValue::new()); - do_client(stelline, &step_value, client_factory).await; - - // Await shutdown - if !dgram_srv.await_shutdown(Duration::from_secs(5)).await { - warn!("Datagram server did not shutdown on time."); - } - - if !stream_srv.await_shutdown(Duration::from_secs(5)).await { - warn!("Stream server did not shutdown on time."); - } - } - - let svc = service_fn(test_service, zonefile); - if with_cookies { + let secret = if with_cookies { let secret = server_config.cookies.secret.unwrap(); let secret = base16::decode_vec(secret).unwrap(); - let secret = <[u8; 16]>::try_from(secret).unwrap(); - let svc = CookiesMiddlewareSvc::new(svc, secret) - .with_denied_ips(server_config.cookies.ip_deny_list.clone()); - finish_svc(svc, server_config, &stelline).await; - } else if server_config.edns_tcp_keepalive { - let svc = EdnsMiddlewareSvc::new(svc); - finish_svc(svc, server_config, &stelline).await; + <[u8; 16]>::try_from(secret).unwrap() } else { - finish_svc(svc, server_config, &stelline).await; + Default::default() + }; + + // Create a layered service to respond to received DNS queries. The layers + // are created top to bottom, with the application specific logic service + // on top and generic DNS logic below. Behaviour required by implemented + // DNS RFCs will be applied/enforced before the application logic receives + // it and without it having to know or do anything about it. + + // 1. Application logic service + let svc = service_fn(test_service, zonefile); + + // 2. DNS COOKIES middleware service + let svc = CookiesMiddlewareSvc::new(svc, secret) + .with_denied_ips(server_config.cookies.ip_deny_list.clone()) + .enable(with_cookies); + + // 3. EDNS middleware service + let svc = + EdnsMiddlewareSvc::new(svc).enable(server_config.edns_tcp_keepalive); + + // 4. Mandatory DNS behaviour (e.g. RFC 1034/35 rules). + let svc = MandatoryMiddlewareSvc::new(svc); + + // Create dgram and stream servers for answering requests + let (dgram_srv, dgram_conn, stream_srv, stream_conn) = + mk_servers(svc, &server_config); + + // Create a client factory for sending requests + let client_factory = mk_client_factory(dgram_conn, stream_conn); + + // Run the Stelline test! + let step_value = Arc::new(CurrStepValue::new()); + do_client(&stelline, &step_value, client_factory).await; + + // Await shutdown + if !dgram_srv.await_shutdown(Duration::from_secs(5)).await { + warn!("Datagram server did not shutdown on time."); + } + + if !stream_srv.await_shutdown(Duration::from_secs(5)).await { + warn!("Stream server did not shutdown on time."); } } @@ -135,19 +126,19 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { #[allow(clippy::type_complexity)] fn mk_servers( - service: Arc, + service: Svc, server_config: &ServerConfig, ) -> ( - Arc>>, + Arc>, ClientServerChannel, - Arc>>, + Arc>, ClientServerChannel, ) where - Svc: Service + Send + Sync + 'static, - Svc::Future: Send, - Svc::Target: Composer + Default + Send + Sync, - Svc::Stream: Send, + Svc: Clone + Service + Send + Sync, + ::Future: Send, + ::Target: Composer + Default + Send + Sync, + ::Stream: Send, { // Prepare middleware to be used by the DNS servers to pre-process // received requests and post-process created responses. diff --git a/src/net/server/util.rs b/src/net/server/util.rs index d9d5ec858..c14f7b775 100644 --- a/src/net/server/util.rs +++ b/src/net/server/util.rs @@ -1,15 +1,16 @@ //! Small utilities for building and working with servers. -use core::future::Ready; +use core::future::{ready, Ready}; +use core::marker::PhantomData; use std::string::{String, ToString}; use futures::stream::Once; use octseq::{Octets, OctetsBuilder}; use tracing::warn; -use crate::base::iana::{OptRcode, Rcode}; +use crate::base::iana::OptRcode; use crate::base::message_builder::{ - AdditionalBuilder, OptBuilder, PushError, QuestionBuilder, + AdditionalBuilder, OptBuilder, PushError, }; use crate::base::wire::Composer; use crate::base::Message; @@ -90,21 +91,63 @@ where /// [`Vec`]: std::vec::Vec /// [`CallResult`]: crate::net::server::service::CallResult /// [`Result::Ok`]: std::result::Result::Ok -pub fn service_fn( +pub fn service_fn( request_handler: T, metadata: Metadata, -) -> impl Service< - RequestOctets, - Target = Target, - Stream = Once>>, - Future = Ready>>>, -> + Clone +) -> ServiceFn where RequestOctets: AsRef<[u8]> + Send + Sync + Unpin, + RequestMeta: Clone + Default, Metadata: Clone, - T: Fn(Request, Metadata) -> ServiceResult + Clone, + Target: Composer + Default, + T: Fn( + Request, + Metadata, + ) -> ServiceResult + + Clone, +{ + ServiceFn { + request_handler, + metadata, + _phantom: PhantomData, + } +} + +//--- ServiceFn + +#[derive(Clone, Debug)] +pub struct ServiceFn { + request_handler: T, + metadata: Metadata, + _phantom: PhantomData, +} + +impl + Service for ServiceFn +where + RequestOctets: AsRef<[u8]> + Send + Sync + Unpin, + RequestMeta: Default + Clone, + Metadata: Clone, + Target: Composer + Default, + T: Fn( + Request, + Metadata, + ) -> ServiceResult + + Clone, { - move |request| request_handler(request, metadata.clone()) + type Target = Target; + type Stream = Once>>; + type Future = Ready; + + fn call( + &self, + request: Request, + ) -> Self::Future { + ready(futures_util::stream::once(ready((self.request_handler)( + request, + self.metadata.clone(), + )))) + } } //----------- to_pcap_text() ------------------------------------------------- @@ -138,53 +181,6 @@ pub(crate) fn to_pcap_text>( formatted } -//----------- start_reply ---------------------------------------------------- - -/// Create a DNS response message that is a reply to a given request message. -/// -/// Copy the request question into a new response and return the builder for -/// further message construction. -/// -/// On internal error this function will attempt to set RCODE ServFail in the -/// returned message. -pub fn start_reply( - msg: &Message, -) -> QuestionBuilder> -where - RequestOctets: Octets, - Target: Composer + Default, -{ - let builder = mk_builder_for_target(); - - // RFC (1035?) compliance - copy question from request to response. - let mut abort = false; - let mut builder = builder.question(); - for rr in msg.question() { - match rr { - Ok(rr) => { - if let Err(err) = builder.push(rr) { - warn!("Internal error while copying question RR to the resposne: {err}"); - abort = true; - break; - } - } - Err(err) => { - warn!( - "Parse error while copying question RR to the resposne: {err} [RR: {rr:?}]" - ); - abort = true; - break; - } - } - } - - if abort { - builder.header_mut().set_rcode(Rcode::SERVFAIL); - } - - builder -} - //------------ mk_error_response --------------------------------------------- pub fn mk_error_response( @@ -195,7 +191,9 @@ where RequestOctets: Octets, Target: Composer + Default, { - let mut additional = start_reply(msg).additional(); + let mut additional = mk_builder_for_target() + .start_error(msg, rcode.rcode()) + .additional(); // Note: if rcode is non-extended this will also correctly handle // setting the rcode in the main message header. @@ -343,13 +341,12 @@ mod tests { use crate::base::{Message, MessageBuilder, Name, Rtype, StreamTarget}; use crate::net::server::message::{Request, UdpTransportContext}; - use super::start_reply; use crate::base::iana::{OptRcode, Rcode}; use crate::base::message_builder::AdditionalBuilder; use crate::base::opt::UnknownOptData; use crate::base::wire::Composer; use crate::net::server::util::{ - add_edns_options, remove_edns_opt_record, + add_edns_options, mk_builder_for_target, remove_edns_opt_record, }; use std::vec::Vec; @@ -365,10 +362,12 @@ mod tests { let client_ip = "127.0.0.1:12345".parse().unwrap(); let sent_at = Instant::now(); let ctx = UdpTransportContext::default(); - let request = Request::new(client_ip, sent_at, msg, ctx.into()); + let request = Request::new(client_ip, sent_at, msg, ctx.into(), ()); // Create a dummy DNS reply which does not yet have an OPT record. - let reply = start_reply::<_, Vec>(request.message()); + let reply = mk_builder_for_target::>() + .start_answer(request.message(), Rcode::NOERROR) + .unwrap(); assert_eq!(reply.counts().arcount(), 0); assert_eq!(reply.header().rcode(), Rcode::NOERROR); @@ -453,10 +452,12 @@ mod tests { let client_ip = "127.0.0.1:12345".parse().unwrap(); let sent_at = Instant::now(); let ctx = UdpTransportContext::default(); - let request = Request::new(client_ip, sent_at, msg, ctx.into()); + let request = Request::new(client_ip, sent_at, msg, ctx.into(), ()); // Create a dummy DNS reply which does not yet have an OPT record. - let reply = start_reply::<_, Vec>(request.message()); + let reply = mk_builder_for_target::>() + .start_answer(request.message(), Rcode::NOERROR) + .unwrap(); assert_eq!(reply.counts().arcount(), 0); // Add an OPT record to the reply. From 5f1a647bc858c19fd13e0de8c22fc85008dfe567 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 6 Aug 2024 23:43:12 +0200 Subject: [PATCH 002/333] Remove fn accidentally included from the xfr branch but not intended to be part of this PR branch. --- src/base/message_builder.rs | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/src/base/message_builder.rs b/src/base/message_builder.rs index 893183550..4a7dd6cca 100644 --- a/src/base/message_builder.rs +++ b/src/base/message_builder.rs @@ -21,9 +21,7 @@ //! something that looks like a [`Record`]. Apart from actual values //! of these types, tuples of the components also work, such as a pair of a //! domain name and a record type for a question or a triple of the owner -//! name, TTL, and record data for a record. If you already have a question -//! or record, you can use the `push_ref` method to add -//! +//! name, TTL, and record data for a record. //! //! The `push` method of the record //! section builders is also available via the [`RecordSectionBuilder`] @@ -874,16 +872,6 @@ impl AnswerBuilder { |counts| counts.inc_ancount(), ) } - - pub fn push_ref( - &mut self, - record: &impl ComposeRecord, - ) -> Result<(), PushError> { - self.builder.push( - |target| record.compose_record(target).map_err(Into::into), - |counts| counts.inc_ancount(), - ) - } } /// # Conversions From b5e33a7011ea522fb45128be8d21daad2565a6c3 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 6 Aug 2024 23:44:07 +0200 Subject: [PATCH 003/333] Remove fn accidentally included from the xfr branch but not intended to be part of this PR branch. --- src/base/message_builder.rs | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/base/message_builder.rs b/src/base/message_builder.rs index 4a7dd6cca..f5dccd323 100644 --- a/src/base/message_builder.rs +++ b/src/base/message_builder.rs @@ -1946,14 +1946,6 @@ where } } -impl FreezeBuilder for StreamTarget { - type Octets = Target; - - fn freeze(self) -> Self::Octets { - self.into_target() - } -} - //------------ StaticCompressor ---------------------------------------------- /// A domain name compressor that doesn’t require an allocator. From 6701131377e18d46711088dad3487d64fba8831b Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 6 Aug 2024 23:49:18 +0200 Subject: [PATCH 004/333] Remove changes accidentally included from the xfr branch but not intended to be part of this PR branch. --- src/net/server/connection.rs | 212 +++++++++++++---------------------- 1 file changed, 81 insertions(+), 131 deletions(-) diff --git a/src/net/server/connection.rs b/src/net/server/connection.rs index 24b192906..a805b1b17 100644 --- a/src/net/server/connection.rs +++ b/src/net/server/connection.rs @@ -1,6 +1,5 @@ //! Support for stream based connections. use core::ops::{ControlFlow, Deref}; -use core::sync::atomic::{AtomicBool, Ordering}; use core::time::Duration; use std::fmt::Display; @@ -76,14 +75,6 @@ const RESPONSE_WRITE_TIMEOUT: DefMinMax = DefMinMax::new( Duration::from_secs(60 * 60), ); -/// Limit on the number of attempts that will be made to overcome -/// non-fatal write errors. -/// -/// The value has to be between 0 and 255 with a default of 2. A value of -/// zero means that one try will be attempted. -const RESPONSE_WRITE_RETRIES: DefMinMax = - DefMinMax::new(2, u8::MIN, u8::MAX); - /// Limit on the number of DNS responses queued for writing to the client. /// /// The value has to be between zero and 1,024. The default value is 10. These @@ -115,12 +106,13 @@ pub struct Config { /// Limit on the amount of time to wait for writing a response to /// complete. + /// + /// The value has to be between 1 millisecond and 1 hour with a default of + /// 30 seconds. These values are guesses at something reasonable. The + /// default is based on the Unbound 1.19.2 default value for its + /// `tcp-idle-timeout` setting. response_write_timeout: Duration, - /// Limit on the number of attempts that will be made to overcome - /// non-fatal write errors. - response_write_retries: u8, - /// Limit on the number of DNS responses queued for wriing to the client. max_queued_responses: usize, } @@ -182,16 +174,6 @@ impl Config { self.response_write_timeout = value; } - /// Limit on the number of attempts that will be made to overcome - /// non-fatal write errors. - /// - /// The value has to be between 0 and 255 with a default of 2. A value of - /// zero means that one try will be attempted. - #[allow(dead_code)] - pub fn set_response_write_retries(&mut self, value: u8) { - self.response_write_retries = value; - } - /// Set the limit on the number of DNS responses queued for writing to the /// client. /// @@ -221,7 +203,6 @@ impl Default for Config { Self { idle_timeout: IDLE_TIMEOUT.default(), response_write_timeout: RESPONSE_WRITE_TIMEOUT.default(), - response_write_retries: RESPONSE_WRITE_RETRIES.default(), max_queued_responses: MAX_QUEUED_RESPONSES.default(), } } @@ -283,9 +264,6 @@ where /// DNS protocol idle time out tracking. idle_timer: IdleTimer, - /// Is a transaction in progress? - in_transaction: Arc, - /// [`ServerMetrics`] describing the status of the server. metrics: Arc, } @@ -334,7 +312,6 @@ where mpsc::channel(config.max_queued_responses); let config = Arc::new(ArcSwap::from_pointee(config)); let idle_timer = IdleTimer::new(); - let in_transaction = Arc::new(AtomicBool::new(false)); // Place the ReadHalf of the stream into an Option so that we can take // it out (as we can't clone it and we can't place it into an Arc @@ -356,7 +333,6 @@ where result_q_tx, service, idle_timer, - in_transaction, metrics, } } @@ -447,7 +423,7 @@ where } _ = sleep_until(self.idle_timer.idle_timeout_deadline(self.config.load().idle_timeout)) => { - self.process_dns_idle_timeout(self.config.load().idle_timeout) + self.process_dns_idle_timeout() } res = &mut msg_recv => { @@ -464,11 +440,9 @@ where if let Err(err) = res { match err { ConnectionEvent::DisconnectWithoutFlush => { - trace!("Disconnect without flush"); break 'outer; } ConnectionEvent::DisconnectWithFlush => { - trace!("Disconnect with flush"); self.flush_write_queue().await; break 'outer; } @@ -562,7 +536,10 @@ where } /// Stop queueing new responses and process those already in the queue. - async fn flush_write_queue(&mut self) { + async fn flush_write_queue(&mut self) + // where + // Target: Composer, + { debug!("Flushing connection write queue."); // Stop accepting new response messages (should we check for in-flight // messages that haven't generated a response yet but should be @@ -587,7 +564,10 @@ where async fn process_queued_result( &mut self, response: Option>>, - ) -> Result<(), ConnectionEvent> { + ) -> Result<(), ConnectionEvent> +// where + // Target: Composer, + { // If we failed to read the results of requests processed by the // service because the queue holding those results is empty and can no // longer be read from, then there is no point continuing to read from @@ -603,82 +583,63 @@ where "Writing queued response with id {} to stream", response.header().id() ); - self.write_response_to_stream(response.finish()).await + self.write_response_to_stream(response.finish()).await; + + Ok(()) } /// Write a response back to the caller over the network stream. async fn write_response_to_stream( &mut self, msg: StreamTarget, - ) -> Result<(), ConnectionEvent> { + ) + // where + // Target: AsRef<[u8]>, + { if enabled!(Level::TRACE) { let bytes = msg.as_dgram_slice(); - let pcap_text = to_pcap_text(bytes, bytes.len().min(128)); - trace!(addr = %self.addr, pcap_text, "Sending response (dumping max 128 bytes)"); + let pcap_text = to_pcap_text(bytes, bytes.len()); + trace!(addr = %self.addr, pcap_text, "Sending response"); } - let max_tries = self.config.load().response_write_retries + 1; - let mut tries_left = max_tries; - while tries_left > 0 { - debug!(addr = %self.addr, "Sending response: attempts left={tries_left}"); - tries_left -= 1; - - match timeout( - self.config.load().response_write_timeout, - self.stream_tx.write_all(msg.as_stream_slice()), - ) - .await - { - Err(_) => { - error!(addr = %self.addr, - "Write timed out (>{:?})", - self.config.load().response_write_timeout - ); - // Retry - continue; - } - Ok(Err(err)) => { - if let ControlFlow::Break(err) = process_io_error(err) { - // Fatal error, abort. - return Err(err); - } else { - // Retry - continue; - } - } - Ok(Ok(_)) => { - // Success. - self.metrics.inc_num_sent_responses(); - self.metrics.dec_num_pending_writes(); - - if self.result_q_tx.capacity() - == self.result_q_tx.max_capacity() - { - self.idle_timer.response_queue_emptied(); - } - - return Ok(()); - } + match timeout( + self.config.load().response_write_timeout, + self.stream_tx.write_all(msg.as_stream_slice()), + ) + .await + { + Err(_) => { + error!( + "Write timed out (>{:?})", + self.config.load().response_write_timeout + ); + // TODO: Push it to the back of the queue to retry it? + } + Ok(Err(err)) => { + error!("Write error: {err}"); + } + Ok(Ok(_)) => { + self.metrics.inc_num_sent_responses(); } } - error!(addr = %self.addr, "Failed to send response after {max_tries}. The connection will be closed."); - Err(ConnectionEvent::DisconnectWithoutFlush) + self.metrics.dec_num_pending_writes(); + + if self.result_q_tx.capacity() == self.result_q_tx.max_capacity() { + self.idle_timer.response_queue_emptied(); + } } - /// Implement DNS rules regarding timing out of idle connections. + /// Implemnt DNS rules regarding timing out of idle connections. /// /// Disconnects the current connection of the timer is expired, flushing /// pending responses first. - fn process_dns_idle_timeout( - &self, - timeout: Duration, - ) -> Result<(), ConnectionEvent> { + fn process_dns_idle_timeout(&self) -> Result<(), ConnectionEvent> { // DNS idle timeout elapsed, or was it reset? - if self.idle_timer.idle_timeout_expired(timeout) - && !self.in_transaction.load(Ordering::SeqCst) + if self + .idle_timer + .idle_timeout_expired(self.config.load().idle_timeout) { - trace!("Timing out idle connection"); Err(ConnectionEvent::DisconnectWithoutFlush) } else { Ok(()) @@ -693,16 +654,13 @@ where where Svc::Stream: Send, { - let in_transaction = self.in_transaction.clone(); - match res { Ok(buf) => { let received_at = Instant::now(); if enabled!(Level::TRACE) { - let pcap_text = - to_pcap_text(&buf, buf.as_ref().len().min(128)); - trace!(addr = %self.addr, pcap_text, "Received message (dumping max 128 bytes)"); + let pcap_text = to_pcap_text(&buf, buf.as_ref().len()); + trace!(addr = %self.addr, pcap_text, "Received message"); } self.metrics.inc_num_received_requests(); @@ -721,7 +679,6 @@ where } Ok(msg) => { - trace!(addr = %self.addr, ?msg, "Parsed first question: {:?}", msg.first_question()); let ctx = NonUdpTransportContext::new(Some( self.config.load().idle_timeout, )); @@ -749,6 +706,7 @@ where "Calling service for request id {request_id}" ); let mut stream = svc.call(request).await; + let mut in_transaction = false; trace!("Awaiting service call results for request id {request_id}"); while let Some(Ok(call_result)) = @@ -780,17 +738,11 @@ where } ServiceFeedback::BeginTransaction => { - in_transaction.store( - true, - Ordering::SeqCst, - ); + in_transaction = true; } ServiceFeedback::EndTransaction => { - in_transaction.store( - false, - Ordering::SeqCst, - ); + in_transaction = false; } } } @@ -813,16 +765,14 @@ where } Err(TrySendError::Closed(_)) => { - error!("Unable to queue message for sending: connection is shutting down."); - return; + error!("Unable to queue message for sending: server is shutting down."); + break; } Err(TrySendError::Full( unused_response, )) => { - if in_transaction - .load(Ordering::SeqCst) - { + if in_transaction { // Wait until there is space in the message queue. tokio::task::yield_now() .await; @@ -978,35 +928,35 @@ where // buffer. Ok(_size) => return Ok(()), - Err(err) => match process_io_error(err) { + Err(err) => match Self::process_io_error(err) { ControlFlow::Continue(_) => continue, ControlFlow::Break(err) => return Err(err), }, } } } -} -/// Handle I/O errors by deciding whether to log them, and whether to continue -/// or abort. -#[must_use] -fn process_io_error(err: io::Error) -> ControlFlow { - match err.kind() { - io::ErrorKind::UnexpectedEof => { - // The client disconnected. Per RFC 7766 6.2.4 pending responses - // MUST NOT be sent to the client. - ControlFlow::Break(ConnectionEvent::DisconnectWithoutFlush) - } - io::ErrorKind::TimedOut | io::ErrorKind::Interrupted => { - // These errors might be recoverable, try again. - ControlFlow::Continue(()) - } - _ => { - // Everything else is either unrecoverable or unknown to us at the - // time of writing and so we can't guess how to handle it, so - // abort. - error!("I/O error: {}", err); - ControlFlow::Break(ConnectionEvent::DisconnectWithoutFlush) + /// Handle I/O errors by deciding whether to log them, and whethr to + /// continue or abort. + #[must_use] + fn process_io_error(err: io::Error) -> ControlFlow { + match err.kind() { + io::ErrorKind::UnexpectedEof => { + // The client disconnected. Per RFC 7766 6.2.4 pending + // responses MUST NOT be sent to the client. + ControlFlow::Break(ConnectionEvent::DisconnectWithoutFlush) + } + io::ErrorKind::TimedOut | io::ErrorKind::Interrupted => { + // These errors might be recoverable, try again. + ControlFlow::Continue(()) + } + _ => { + // Everything else is either unrecoverable or unknown to us at + // the time of writing and so we can't guess how to handle it, + // so abort. + error!("I/O error: {}", err); + ControlFlow::Break(ConnectionEvent::DisconnectWithoutFlush) + } } } } From a0e1a1d84b46caee00ec5035b5b6d8b8bf7761d4 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 6 Aug 2024 23:50:40 +0200 Subject: [PATCH 005/333] Remove changes accidentally included from the xfr branch but not intended to be part of this PR branch. --- src/net/server/dgram.rs | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/src/net/server/dgram.rs b/src/net/server/dgram.rs index c455484d5..752b8f4ca 100644 --- a/src/net/server/dgram.rs +++ b/src/net/server/dgram.rs @@ -118,7 +118,7 @@ impl Config { /// [RFC 6891]: /// https://datatracker.ietf.org/doc/html/rfc6891#section-6.2.5 pub fn set_max_response_size(&mut self, value: Option) { - self.max_response_size = value.map(|v| MAX_RESPONSE_SIZE.limit(v)); + self.max_response_size = value; } /// Sets the time to wait for a complete message to be written to the @@ -314,7 +314,7 @@ where /// /// Takes: /// - A socket which must implement [`AsyncDgramSock`] and is responsible - /// receiving new messages and send responses back to the client. + /// receiving new messages and send responses back to the client. /// - A [`BufSource`] for creating buffers on demand. /// - A [`Service`] for handling received requests and generating /// responses. @@ -498,8 +498,8 @@ where self.metrics.inc_num_received_requests(); if enabled!(Level::TRACE) { - let pcap_text = to_pcap_text(&buf, bytes_read.min(128)); - trace!(%addr, pcap_text, "Received message (dumping max 128 bytes)"); + let pcap_text = to_pcap_text(&buf, bytes_read); + trace!(%addr, pcap_text, "Received message"); } let svc = self.service.clone(); @@ -515,7 +515,6 @@ where } Ok(msg) => { - trace!(addr = %addr, ?msg, "Parsed first question: {:?}", msg.first_question()); let ctx = UdpTransportContext::new(cfg.load().max_response_size); let ctx = TransportSpecificContext::Udp(ctx); let request = Request::new(addr, received_at, msg, ctx, ()); @@ -545,8 +544,8 @@ where // Logging if enabled!(Level::TRACE) { - let pcap_text = to_pcap_text(bytes, bytes.len().min(128)); - trace!(%addr, pcap_text, "Sending response (dumping max 128 bytes)"); + let pcap_text = to_pcap_text(bytes, bytes.len()); + trace!(%addr, pcap_text, "Sending response"); } metrics.inc_num_pending_writes(); @@ -630,8 +629,7 @@ where /// Receive a single datagram using the user supplied network socket. fn recv_from( &self, - ) -> Result<(::Output, SocketAddr, usize), io::Error> - { + ) -> Result<(Buf::Output, SocketAddr, usize), io::Error> { let mut msg = self.buf.create_buf(); let mut buf = ReadBuf::new(msg.as_mut()); self.sock From a5fcce567b75b1af39dc70760aae83069674be4a Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 7 Aug 2024 11:58:49 +0200 Subject: [PATCH 006/333] Update Service RustDocs now that implementing a service impicitly via a function is no longer possible. --- src/net/server/service.rs | 85 ++------------------------------------- src/net/server/util.rs | 24 ++++++----- 2 files changed, 18 insertions(+), 91 deletions(-) diff --git a/src/net/server/service.rs b/src/net/server/service.rs index b10a81cd1..d1362e0bc 100644 --- a/src/net/server/service.rs +++ b/src/net/server/service.rs @@ -21,8 +21,7 @@ use super::message::Request; /// The type of item that `Service` implementations stream as output. pub type ServiceResult = Result, ServiceError>; -/// `Service`s are responsible for determining how to respond to DNS -/// requests. +/// `Service`s are responsible for determining how to respond to DNS requests. /// /// For an overview of how services fit into the total flow of request and /// response handling see the [`net::server`] module documentation. @@ -37,20 +36,8 @@ pub type ServiceResult = Result, ServiceError>; /// /// # Usage /// -/// There are three ways to implement the `Service` trait: -/// -/// 1. Implement the `Service` trait on a struct. -/// 2. Define a function compatible with the `Service` trait. -/// 3. Define a function compatible with [`service_fn`]. -/// -///
-/// -/// Whichever approach you choose it is important to minimize the work done -/// before returning from [`Service::call`], as time spent here blocks the -/// caller. Instead as much work as possible should be delegated to the -/// future returned. -/// -///
+/// You can either implement the [`Service`] trait on a struct or use the +/// helper function [`service_fn`] to turn a function into a [`Service`]. /// /// # Implementing the `Service` trait on a `struct` /// @@ -103,72 +90,6 @@ pub type ServiceResult = Result, ServiceError>; /// } /// ``` /// -/// # Define a function compatible with the `Service` trait -/// -/// ``` -/// use core::fmt::Debug; -/// use core::future::ready; -/// use core::future::Future; -/// -/// use domain::base::{Name, Message}; -/// use domain::base::iana::{Class, Rcode}; -/// use domain::base::name::ToLabelIter; -/// use domain::base::wire::Composer; -/// use domain::dep::octseq::{OctetsBuilder, FreezeBuilder, Octets}; -/// use domain::net::server::message::Request; -/// use domain::net::server::service::{CallResult, ServiceError, ServiceResult}; -/// use domain::net::server::util::mk_builder_for_target; -/// use domain::rdata::A; -/// -/// fn name_to_ip(request: Request>) -> ServiceResult> { -/// let mut out_answer = None; -/// if let Ok(question) = request.message().sole_question() { -/// let qname = question.qname(); -/// let num_labels = qname.label_count(); -/// if num_labels >= 5 { -/// let mut iter = qname.iter_labels(); -/// let a = iter.nth(num_labels - 5).unwrap(); -/// let b = iter.next().unwrap(); -/// let c = iter.next().unwrap(); -/// let d = iter.next().unwrap(); -/// let a_rec: Result = format!("{a}.{b}.{c}.{d}").parse(); -/// if let Ok(a_rec) = a_rec { -/// let builder = mk_builder_for_target(); -/// let mut answer = -/// builder -/// .start_answer(request.message(), Rcode::NOERROR) -/// .unwrap(); -/// answer -/// .push((Name::root_ref(), Class::IN, 86400, a_rec)) -/// .unwrap(); -/// out_answer = Some(answer); -/// } -/// } -/// } -/// -/// if out_answer.is_none() { -/// let builder = mk_builder_for_target(); -/// let answer = builder -/// .start_answer(request.message(), Rcode::REFUSED) -/// .unwrap(); -/// out_answer = Some(answer); -/// } -/// -/// let additional = out_answer.unwrap().additional(); -/// Ok(CallResult::new(additional)) -/// } -/// ``` -/// -/// Now when you want to use the service pass it to the server: -/// -/// ```ignore -/// let srv = DgramServer::new(sock, buf, name_to_ip); -/// ``` -/// -/// # Define a function compatible with [`service_fn`] -/// -/// See [`service_fn`] for an example of how to use it to create a `Service` -/// impl from a function. /// /// [`DgramServer`]: crate::net::server::dgram::DgramServer /// [`StreamServer`]: crate::net::server::stream::StreamServer diff --git a/src/net/server/util.rs b/src/net/server/util.rs index c14f7b775..b5f89e67f 100644 --- a/src/net/server/util.rs +++ b/src/net/server/util.rs @@ -44,9 +44,17 @@ where /// those of its associated types, but this makes implementing it for simple /// cases quite verbose. /// -/// `service_fn()` enables you to write a slightly simpler function definition -/// that implements the [`Service`] trait than implementing [`Service`] -/// directly. +/// [`service_fn()`] enables you to write a slightly simpler function +/// definition that implements the [`Service`] trait than implementing +/// [`Service`] directly. +/// +///
+/// +/// Note that [`service_fn`] does not support async service functions. To +/// use async code in a service you must implement the [`Service`] trait +/// manually on a struct. +/// +///
/// /// # Example /// @@ -56,13 +64,9 @@ where /// /// ``` /// // Import the types we need. -/// use std::boxed::Box; -/// use std::future::Future; -/// use std::pin::Pin; /// use domain::base::iana::Rcode; -/// use domain::base::Message; /// use domain::net::server::message::Request; -/// use domain::net::server::service::{CallResult, ServiceError, ServiceResult}; +/// use domain::net::server::service::{CallResult, ServiceResult}; /// use domain::net::server::util::{mk_builder_for_target, service_fn}; /// /// // Define some types to make the example easier to read. @@ -70,7 +74,9 @@ where /// /// // Implement the application logic of our service. /// // Takes the received DNS request and any additional meta data you wish to -/// // provide, and returns one or more future DNS responses. +/// // provide, and returns one or more DNS responses. +/// // +/// // Note that using `service_fn()` does not permit you to use async code! /// fn my_service(req: Request>, _meta: MyMeta) /// -> ServiceResult> /// { From 53017f5ee2d836e54bb5ec58ac7cf1b0975717e2 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 7 Aug 2024 13:57:09 +0200 Subject: [PATCH 007/333] Cargo fmt. --- src/net/server/util.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/net/server/util.rs b/src/net/server/util.rs index b5f89e67f..adf5f969c 100644 --- a/src/net/server/util.rs +++ b/src/net/server/util.rs @@ -47,13 +47,13 @@ where /// [`service_fn()`] enables you to write a slightly simpler function /// definition that implements the [`Service`] trait than implementing /// [`Service`] directly. -/// +/// ///
-/// +/// /// Note that [`service_fn`] does not support async service functions. To /// use async code in a service you must implement the [`Service`] trait /// manually on a struct. -/// +/// ///
/// /// # Example From 40eea813febe612d96bf9da475e10a64e4479067 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 7 Aug 2024 13:57:41 +0200 Subject: [PATCH 008/333] Show more ways to impl Service on a struct. --- src/net/server/service.rs | 65 +++++++++++++++++++++++++++++++++++---- 1 file changed, 59 insertions(+), 6 deletions(-) diff --git a/src/net/server/service.rs b/src/net/server/service.rs index d1362e0bc..5d47e0007 100644 --- a/src/net/server/service.rs +++ b/src/net/server/service.rs @@ -44,6 +44,9 @@ pub type ServiceResult = Result, ServiceError>; /// ``` /// use core::future::ready; /// use core::future::Ready; +/// use core::pin::Pin; +/// +/// use std::task::{Context, Poll}; /// /// use futures::stream::{once, Once, Stream}; /// @@ -71,25 +74,75 @@ pub type ServiceResult = Result, ServiceError>; /// answer.additional() /// } /// -/// struct MyService; +/// fn mk_response_stream(msg: &Request>) +/// -> Once>>> +/// { +/// let builder = mk_builder_for_target(); +/// let additional = mk_answer(msg, builder); +/// let item = Ok(CallResult::new(additional)); +/// once(ready(item)) +/// } +/// +/// //------------ A synchronous service example ------------------------------ +/// struct MySyncService; /// -/// impl Service> for MyService { +/// impl Service> for MySyncService { /// type Target = Vec; /// type Stream = Once>>; /// type Future = Ready; +/// +/// fn call( +/// &self, +/// msg: Request>, +/// ) -> Self::Future { +/// ready(mk_response_stream(&msg)) +/// } +/// } +/// +/// //------------ An anonymous async block service example ------------------- +/// struct MyAsyncBlockService; +/// +/// impl Service> for MyAsyncBlockService { +/// type Target = Vec; +/// type Stream = Once>>; +/// type Future = Pin>>; /// /// fn call( /// &self, /// msg: Request>, /// ) -> Self::Future { -/// let builder = mk_builder_for_target(); -/// let additional = mk_answer(&msg, builder); -/// let item = Ok(CallResult::new(additional)); -/// ready(once(ready(item))) +/// Box::pin(async move { mk_response_stream(&msg) }) +/// } +/// } +/// +/// //------------ A named Future service example ----------------------------- +/// struct MyFut(Request>); +/// +/// impl std::future::Future for MyFut { +/// type Output = Once>>>; +/// +/// fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { +/// Poll::Ready(mk_response_stream(&self.0)) /// } /// } +/// +/// struct MyNamedFutureService; +/// +/// impl Service> for MyNamedFutureService { +/// type Target = Vec; +/// type Stream = Once>>; +/// type Future = MyFut; +/// +/// fn call(&self, msg: Request>) -> Self::Future { MyFut(msg) } +/// } /// ``` /// +/// The above are minimalist examples to illustrate what you need to do, but +/// lacking any actual useful behaviour. +/// +/// In your own [`Service`] impl you would implement actual business logic +/// using synchronous or asynchronous code and returning single or multiple +/// responses as needed. /// /// [`DgramServer`]: crate::net::server::dgram::DgramServer /// [`StreamServer`]: crate::net::server::stream::StreamServer From 0a289e9a7cd9bca1c4c9d59a357e9035c134e749 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 7 Aug 2024 14:10:05 +0200 Subject: [PATCH 009/333] More Service RustDoc improvements. --- src/net/server/service.rs | 31 +++++++++++++++++++++++++++---- 1 file changed, 27 insertions(+), 4 deletions(-) diff --git a/src/net/server/service.rs b/src/net/server/service.rs index 5d47e0007..403e7b60b 100644 --- a/src/net/server/service.rs +++ b/src/net/server/service.rs @@ -33,7 +33,7 @@ pub type ServiceResult = Result, ServiceError>; /// /// Most DNS requests result in a single response, with the exception of AXFR /// and IXFR requests which can result in a stream of responses. -/// +/// /// # Usage /// /// You can either implement the [`Service`] trait on a struct or use the @@ -138,14 +138,37 @@ pub type ServiceResult = Result, ServiceError>; /// ``` /// /// The above are minimalist examples to illustrate what you need to do, but -/// lacking any actual useful behaviour. +/// lacking any actual useful behaviour. They also only demonstrate returning +/// a response stream containing a single immediately available value via +/// `futures::stream::Once` and `std::future::Ready`. /// /// In your own [`Service`] impl you would implement actual business logic -/// using synchronous or asynchronous code and returning single or multiple -/// responses as needed. +/// returning single or multiple responses synchronously or asynchronously as +/// needed. +/// +/// # Advanced usage +/// +/// The [`Service`] trait takes two generic types which in most cases you +/// don't need to specify as the defaults will be fine. +/// +/// For more advanced cases you may need to override these defaults. +/// +/// - `RequestMeta`: If implementing a [middleware] `Service` you may need to +/// supply your own `RequestMeta` type. `RequestMeta` is intended to enable +/// middleware `Service` impls to express strongly typed support for +/// middleware specific data that can be consumed by upstream middleware, or +/// even by your application service. For example a middleware `Service` may +/// detect that the request is signed using a particular key and communicate +/// the name of the key to any upstream `Service` that needs to know the +/// name of the key used to sign the request. +/// +/// - `RequestOctets`: By specifying your own `RequestOctets` type you can use +/// a type other than `Vec` to transport request bytes through your +/// application. /// /// [`DgramServer`]: crate::net::server::dgram::DgramServer /// [`StreamServer`]: crate::net::server::stream::StreamServer +/// [middleware]: crate::net::server::middleware /// [`net::server`]: crate::net::server /// [`call`]: Self::call() /// [`service_fn`]: crate::net::server::util::service_fn() From 37c35e929e4596cdf3f085a524ada31262302029 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 7 Aug 2024 14:14:34 +0200 Subject: [PATCH 010/333] Minor RustDoc fixes. --- src/net/server/middleware/cookies.rs | 22 +++++++++++++-------- src/net/server/middleware/edns.rs | 27 +++++++++++++++----------- src/net/server/middleware/mandatory.rs | 22 +++++++++++---------- src/net/server/service.rs | 12 ++++++------ 4 files changed, 48 insertions(+), 35 deletions(-) diff --git a/src/net/server/middleware/cookies.rs b/src/net/server/middleware/cookies.rs index 4fe5414ec..152d16e3d 100644 --- a/src/net/server/middleware/cookies.rs +++ b/src/net/server/middleware/cookies.rs @@ -32,7 +32,7 @@ const FIVE_MINUTES_AS_SECS: u32 = 5 * 60; /// https://www.rfc-editor.org/rfc/rfc9018.html#section-4.3. const ONE_HOUR_AS_SECS: u32 = 60 * 60; -//----------- CookiesMiddlewareProcessor -------------------------------------- +//----------- CookiesMiddlewareSvc -------------------------------------------- /// A middleware service for enforcing the use of DNS Cookies. /// @@ -47,6 +47,8 @@ const ONE_HOUR_AS_SECS: u32 = 60 * 60; /// [9018]: https://datatracker.ietf.org/doc/html/rfc7873 #[derive(Clone, Debug)] pub struct CookiesMiddlewareSvc { + /// The upstream [`Service`] to pass requests to and receive responses + /// from. next_svc: NextSvc, /// A user supplied secret used in making the cookie value. @@ -57,15 +59,19 @@ pub struct CookiesMiddlewareSvc { /// to reconnect with TCP in order to "authenticate" themselves. ip_deny_list: Vec, - _phantom: PhantomData<(RequestOctets, RequestMeta)>, - + /// Is the middleware service enabled? + /// + /// Defaults to true. If false, the service will pass requests and + /// responses through unmodified. enabled: bool, + + _phantom: PhantomData<(RequestOctets, RequestMeta)>, } impl CookiesMiddlewareSvc { - /// Creates an instance of this processor. + /// Creates an instance of this middleware service. #[must_use] pub fn new(next_svc: NextSvc, server_secret: [u8; 16]) -> Self { Self { @@ -274,7 +280,7 @@ where // back to using TCP is reasonable." // While not required by RFC 7873, like Unbound the caller can - // configure this middleware processor to require clients + // configure this middleware service to require clients // contacting it from certain IP addresses to authenticate // themselves or be refused with TC=1 to signal that they // should resubmit their request via TCP. @@ -535,14 +541,14 @@ mod tests { todo!() } - // And pass the query through the middleware processor + // And pass the query through the middleware service let my_svc = service_fn(my_service, ()); let server_secret: [u8; 16] = [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15]; - let processor_svc = CookiesMiddlewareSvc::new(my_svc, server_secret) + let middleware_svc = CookiesMiddlewareSvc::new(my_svc, server_secret) .with_denied_ips(["127.0.0.1".parse().unwrap()]); - let mut stream = processor_svc.call(request).await; + let mut stream = middleware_svc.call(request).await; let call_result: CallResult> = stream.next().await.unwrap().unwrap(); let (response, _feedback) = call_result.into_inner(); diff --git a/src/net/server/middleware/edns.rs b/src/net/server/middleware/edns.rs index 530f1d6f7..c39fe1981 100644 --- a/src/net/server/middleware/edns.rs +++ b/src/net/server/middleware/edns.rs @@ -47,17 +47,23 @@ const EDNS_VERSION_ZERO: u8 = 0; /// [9210]: https://datatracker.ietf.org/doc/html/rfc9210 #[derive(Clone, Debug, Default)] pub struct EdnsMiddlewareSvc { + /// The upstream [`Service`] to pass requests to and receive responses + /// from. next_svc: NextSvc, - _phantom: PhantomData<(RequestOctets, RequestMeta)>, - + /// Is the middleware service enabled? + /// + /// Defaults to true. If false, the service will pass requests and + /// responses through unmodified. enabled: bool, + + _phantom: PhantomData<(RequestOctets, RequestMeta)>, } impl EdnsMiddlewareSvc { - /// Creates an instance of this processor. + /// Creates an instance of this middleware service. #[must_use] pub fn new(next_svc: NextSvc) -> Self { Self { @@ -468,17 +474,16 @@ mod tests { // --- Only server specified max UDP response sizes // - // The EdnsMiddlewareProcessor should leave these untouched as no EDNS + // The EdnsMiddlewareSvc should leave these untouched as no EDNS // option was present in the request, only the server hint exists, and - // EdnsMiddlewareProcessor only acts if the client EDNS option is - // present. + // EdnsMiddlewareSvc only acts if the client EDNS option is present. assert_eq!(process(None, TOO_SMALL).await, TOO_SMALL); assert_eq!(process(None, JUST_RIGHT).await, JUST_RIGHT); assert_eq!(process(None, HUGE).await, HUGE); // --- Only client specified max UDP response sizes // - // The EdnsMiddlewareProcessor should adopt these, after clamping + // The EdnsMiddlewareSvc should adopt these, after clamping // them. assert_eq!(process(TOO_SMALL, None).await, JUST_RIGHT); assert_eq!(process(JUST_RIGHT, None).await, JUST_RIGHT); @@ -486,7 +491,7 @@ mod tests { // --- Both client and server specified max UDP response sizes // - // The EdnsMiddlewareProcessor should negotiate the largest size + // The EdnsMiddlewareSvc should negotiate the largest size // acceptable to both sides. assert_eq!(process(TOO_SMALL, TOO_SMALL).await, MIN_ALLOWED); assert_eq!(process(TOO_SMALL, JUST_RIGHT).await, JUST_RIGHT); @@ -554,9 +559,9 @@ mod tests { let _call_result: CallResult<_> = stream.next().await.unwrap().unwrap(); - // Or pass the query through the middleware processor - let processor_svc = EdnsMiddlewareSvc::new(my_svc); - let mut stream = processor_svc.call(request.clone()).await; + // Or pass the query through the middleware service + let middleware_svc = EdnsMiddlewareSvc::new(my_svc); + let mut stream = middleware_svc.call(request.clone()).await; let call_result: CallResult> = stream.next().await.unwrap().unwrap(); let (_response, _feedback) = call_result.into_inner(); diff --git a/src/net/server/middleware/mandatory.rs b/src/net/server/middleware/mandatory.rs index 914d55980..9d1a9f90b 100644 --- a/src/net/server/middleware/mandatory.rs +++ b/src/net/server/middleware/mandatory.rs @@ -40,21 +40,23 @@ pub const MINIMUM_RESPONSE_BYTE_LEN: u16 = 512; /// [2181]: https://datatracker.ietf.org/doc/html/rfc2181 #[derive(Clone, Debug)] pub struct MandatoryMiddlewareSvc { - /// In strict mode the processor does more checks on requests and + /// The upstream [`Service`] to pass requests to and receive responses + /// from. + next_svc: NextSvc, + + /// In strict mode the service does more checks on requests and /// responses. strict: bool, - next_svc: NextSvc, - _phantom: PhantomData<(RequestOctets, RequestMeta)>, } impl MandatoryMiddlewareSvc { - /// Creates a new processor instance. + /// Creates an instance of this middleware service. /// - /// The processor will operate in strict mode. + /// The service will operate in strict mode. #[must_use] pub fn new(next_svc: NextSvc) -> Self { Self { @@ -64,9 +66,9 @@ impl } } - /// Creates a new processor instance. + /// Creates an instance of this middleware service. /// - /// The processor will operate in relaxed mode. + /// The service will operate in relaxed mode. #[must_use] pub fn relaxed(next_svc: NextSvc) -> Self { Self { @@ -452,9 +454,9 @@ mod tests { let _call_result: CallResult> = stream.next().await.unwrap().unwrap(); - // Or pass the query through the middleware processor - let processor_svc = MandatoryMiddlewareSvc::new(my_svc); - let mut stream = processor_svc.call(request).await; + // Or pass the query through the middleware service + let middleware_svc = MandatoryMiddlewareSvc::new(my_svc); + let mut stream = middleware_svc.call(request).await; let call_result: CallResult> = stream.next().await.unwrap().unwrap(); let (response, _feedback) = call_result.into_inner(); diff --git a/src/net/server/service.rs b/src/net/server/service.rs index 403e7b60b..cbca6a4b2 100644 --- a/src/net/server/service.rs +++ b/src/net/server/service.rs @@ -33,7 +33,7 @@ pub type ServiceResult = Result, ServiceError>; /// /// Most DNS requests result in a single response, with the exception of AXFR /// and IXFR requests which can result in a stream of responses. -/// +/// /// # Usage /// /// You can either implement the [`Service`] trait on a struct or use the @@ -145,14 +145,14 @@ pub type ServiceResult = Result, ServiceError>; /// In your own [`Service`] impl you would implement actual business logic /// returning single or multiple responses synchronously or asynchronously as /// needed. -/// +/// /// # Advanced usage -/// +/// /// The [`Service`] trait takes two generic types which in most cases you /// don't need to specify as the defaults will be fine. -/// +/// /// For more advanced cases you may need to override these defaults. -/// +/// /// - `RequestMeta`: If implementing a [middleware] `Service` you may need to /// supply your own `RequestMeta` type. `RequestMeta` is intended to enable /// middleware `Service` impls to express strongly typed support for @@ -161,7 +161,7 @@ pub type ServiceResult = Result, ServiceError>; /// detect that the request is signed using a particular key and communicate /// the name of the key to any upstream `Service` that needs to know the /// name of the key used to sign the request. -/// +/// /// - `RequestOctets`: By specifying your own `RequestOctets` type you can use /// a type other than `Vec` to transport request bytes through your /// application. From ace9498993459ee1db2b6fba3e650252622cdc74 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 7 Aug 2024 14:22:49 +0200 Subject: [PATCH 011/333] More RustDoc about request metadata and response byte reservation. --- src/net/server/message.rs | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/src/net/server/message.rs b/src/net/server/message.rs index b9da1947d..46f9f0e60 100644 --- a/src/net/server/message.rs +++ b/src/net/server/message.rs @@ -172,12 +172,22 @@ where /// protocol via which it was received. transport_specific: TransportSpecificContext, - /// The number of bytes to be reserved when generating a response - /// to this request so that needed additional data can be added to - /// to the generated response. + /// The number of bytes to be reserved when generating a response to this + /// request so that needed additional data can be added to to the + /// generated response. + /// + /// Note: This is only a hint to code that considers this value, it is + /// still possible to generate responses that ignore this value. num_reserved_bytes: u16, - metadata: Metadata, // TODO: Make middleware take an impl SomeInterface? + /// user defined metadata to associate with the request. + /// + /// For example this could be used to pass data from one [middleware] + /// [`Service`] impl to another. + /// + /// [middleware]: crate::net::server::middleware + /// [`Service`]: crate::net::server::service::Service + metadata: Metadata, } impl Request @@ -238,6 +248,7 @@ where self.num_reserved_bytes } + /// Set user defined metadata to associate with this request. pub fn with_new_metadata(self, new_metadata: T) -> Request { Request:: { client_addr: self.client_addr, @@ -249,6 +260,7 @@ where } } + /// Get the user defined metadata associated with this request. pub fn metadata(&self) -> &Metadata { &self.metadata } From cb87682b24dad7b756e3e177545d652470101805 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 8 Aug 2024 10:47:04 +0200 Subject: [PATCH 012/333] Reorder field setting order to match struct field order. --- src/net/server/middleware/cookies.rs | 2 +- src/net/server/middleware/edns.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/net/server/middleware/cookies.rs b/src/net/server/middleware/cookies.rs index 152d16e3d..58708e504 100644 --- a/src/net/server/middleware/cookies.rs +++ b/src/net/server/middleware/cookies.rs @@ -78,8 +78,8 @@ impl next_svc, server_secret, ip_deny_list: vec![], - _phantom: PhantomData, enabled: true, + _phantom: PhantomData, } } diff --git a/src/net/server/middleware/edns.rs b/src/net/server/middleware/edns.rs index c39fe1981..94235a846 100644 --- a/src/net/server/middleware/edns.rs +++ b/src/net/server/middleware/edns.rs @@ -68,8 +68,8 @@ impl pub fn new(next_svc: NextSvc) -> Self { Self { next_svc, - _phantom: PhantomData, enabled: true, + _phantom: PhantomData, } } From 6e7a7540b08ec59b1814555d2d4b17ae4ff3d900 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 8 Aug 2024 10:47:48 +0200 Subject: [PATCH 013/333] Remove EDNS fixes that are also dealt with by separate work in branch fix-edns-middleware / PR #355. --- src/net/server/middleware/edns.rs | 46 +------------------------------ 1 file changed, 1 insertion(+), 45 deletions(-) diff --git a/src/net/server/middleware/edns.rs b/src/net/server/middleware/edns.rs index 94235a846..b1b07396c 100644 --- a/src/net/server/middleware/edns.rs +++ b/src/net/server/middleware/edns.rs @@ -207,7 +207,7 @@ where )); } - TransportSpecificContext::NonUdp(ctx) => { + TransportSpecificContext::NonUdp(_) => { // https://datatracker.ietf.org/doc/html/rfc7828#section-3.2.1 // 3.2.1. Sending Queries // "Clients MUST specify an OPTION-LENGTH of 0 @@ -225,16 +225,6 @@ where ); } } - - if let Some(keep_alive) = ctx.idle_timeout() { - if let Ok(timeout) = - IdleTimeout::try_from(keep_alive) - { - Self::reserve_space_for_keep_alive_opt( - request, timeout, - ); - } - } } } } @@ -303,12 +293,6 @@ where // timeout is known: "Signal the timeout value // using the edns-tcp-keepalive EDNS(0) option // [RFC7828]". - - // Remove the limit we should have imposed - // during pre-processing so that we can use - // the space we reserved for the OPT RR. - response.clear_push_limit(); - if let Err(err) = // TODO: Don't add the option if it // already exists? @@ -339,34 +323,6 @@ where // field to some value? } - fn reserve_space_for_keep_alive_opt( - request: &mut Request, - timeout: IdleTimeout, - ) { - // TODO: Calculate this once as a const value, not on every request. - - let keep_alive_opt = TcpKeepalive::new(Some(timeout)); - let root_name_len = Name::root_ref().compose_len(); - - // See: - // - https://datatracker.ietf.org/doc/html/rfc1035#section-3.2.1 - // - https://datatracker.ietf.org/doc/html/rfc6891#autoid-12 - // - https://datatracker.ietf.org/doc/html/rfc7828#section-3.1 - - // Calculate the size of the DNS OPTION RR that will be added to the - // response during post-processing. - let wire_opt_len = root_name_len // "0" root domain name per RFC 6891 - + 2 // TYPE - + 2 // CLASS - + 4 // TTL - + 2 // RDLEN - + 2 // OPTION-CODE - + 2 // OPTION-LENGTH - + keep_alive_opt.compose_len(); // OPTION-DATA - - request.reserve_bytes(wire_opt_len); - } - fn map_stream_item( request: Request, mut stream_item: ServiceResult, From ef0bfd253886d844e36e7f760fec9969b7b0d715 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 8 Aug 2024 11:06:50 +0200 Subject: [PATCH 014/333] Clippy. --- src/net/server/middleware/edns.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/net/server/middleware/edns.rs b/src/net/server/middleware/edns.rs index b1b07396c..affe96362 100644 --- a/src/net/server/middleware/edns.rs +++ b/src/net/server/middleware/edns.rs @@ -9,11 +9,10 @@ use tracing::{debug, enabled, error, trace, warn, Level}; use crate::base::iana::OptRcode; use crate::base::message_builder::AdditionalBuilder; -use crate::base::name::ToLabelIter; use crate::base::opt::keepalive::IdleTimeout; -use crate::base::opt::{ComposeOptData, Opt, OptRecord, TcpKeepalive}; +use crate::base::opt::{Opt, OptRecord, TcpKeepalive}; use crate::base::wire::Composer; -use crate::base::{Name, StreamTarget}; +use crate::base::StreamTarget; use crate::net::server::message::{Request, TransportSpecificContext}; use crate::net::server::middleware::stream::MiddlewareStream; use crate::net::server::service::{CallResult, Service, ServiceResult}; From c2e9e9a5fb1abc7eacc9dddb817bf77ceca29afc Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 8 Aug 2024 13:19:05 +0200 Subject: [PATCH 015/333] Support streaming responses with the stream client. --- src/base/message.rs | 13 ++ src/net/client/multi_stream.rs | 3 + src/net/client/request.rs | 69 +++++++-- src/net/client/stream.rs | 259 +++++++++++++++++++++++++++++---- 4 files changed, 305 insertions(+), 39 deletions(-) diff --git a/src/base/message.rs b/src/base/message.rs index b1272e68c..144833276 100644 --- a/src/base/message.rs +++ b/src/base/message.rs @@ -439,6 +439,19 @@ impl Message { } } + /// Could this message result in a stream of responses? + /// + /// Most DNS queries result in a single response, but some (only AXFR and + /// IXFR at the time of writing) can result in a stream of responses. + /// + /// Returns true if the first question is of a type that might result in a + /// stream of responses. + pub fn is_streaming(&self) -> bool { + self.first_question() + .map(|q| matches!(q.qtype(), Rtype::AXFR | Rtype::IXFR)) + .unwrap_or_default() + } + /// Returns the first question, if there is any. /// /// The method will return `None` both if there are no questions or if diff --git a/src/net/client/multi_stream.rs b/src/net/client/multi_stream.rs index 6016178aa..7ba9082be 100644 --- a/src/net/client/multi_stream.rs +++ b/src/net/client/multi_stream.rs @@ -355,6 +355,9 @@ impl GetResponse for Request { > { Box::pin(Self::get_response(self)) } + + // TODO: Override stream_complete() and is_stream_complete() like + // net::client::stream does? } //------------ Transport ------------------------------------------------ diff --git a/src/net/client/request.rs b/src/net/client/request.rs index dd4f5ddcb..6062eed96 100644 --- a/src/net/client/request.rs +++ b/src/net/client/request.rs @@ -1,16 +1,4 @@ //! Constructing and sending requests. - -use crate::base::iana::Rcode; -use crate::base::message::{CopyRecordsError, ShortMessage}; -use crate::base::message_builder::{ - AdditionalBuilder, MessageBuilder, PushError, StaticCompressor, -}; -use crate::base::opt::{ComposeOptData, LongOptData, OptRecord}; -use crate::base::wire::{Composer, ParseError}; -use crate::base::{Header, Message, ParsedName, Rtype}; -use crate::rdata::AllRecordData; -use bytes::Bytes; -use octseq::Octets; use std::boxed::Box; use std::fmt::Debug; use std::future::Future; @@ -18,8 +6,21 @@ use std::pin::Pin; use std::sync::Arc; use std::vec::Vec; use std::{error, fmt}; + +use bytes::Bytes; +use octseq::Octets; use tracing::trace; +use crate::base::iana::Rcode; +use crate::base::message::{CopyRecordsError, ShortMessage}; +use crate::base::message_builder::{ + AdditionalBuilder, MessageBuilder, PushError, +}; +use crate::base::opt::{ComposeOptData, LongOptData, OptRecord}; +use crate::base::wire::{Composer, ParseError}; +use crate::base::{Header, Message, ParsedName, Rtype, StaticCompressor}; +use crate::rdata::AllRecordData; + //------------ ComposeRequest ------------------------------------------------ /// A trait that allows composing a request as a series. @@ -58,6 +59,9 @@ pub trait ComposeRequest: Debug + Send + Sync { /// Returns whether a message is an answer to the request. fn is_answer(&self, answer: &Message<[u8]>) -> bool; + /// Returns whether a message results in a response stream or not. + fn is_streaming(&self) -> bool; + /// Return the status of the DNSSEC OK flag. fn dnssec_ok(&self) -> bool; } @@ -86,6 +90,10 @@ pub trait GetResponse: Debug { /// Get the result of a DNS request. /// /// This function is intended to be cancel safe. + /// + /// If [`is_stream_complete()`] returns false you can call this function + /// again to receive the next response when dealing with a stream of + /// responses. fn get_response( &mut self, ) -> Pin< @@ -96,6 +104,39 @@ pub trait GetResponse: Debug { + '_, >, >; + + /// Signal that no more responses are expected. + /// + /// The DNS protocol does not provide a standardized way to detect the end + /// of a stream of responses. At the time of writing the only query types + /// that can result in a stream of responses are AXFR and IXFR. In both + /// cases the end of the response data is detected by examining the + /// content of the DNS responses, there is no actual END signal per se. So + /// we rely on the caller to inspect the response messages and telling us + /// by calling this function that it has detected the end of the stream. + + fn stream_complete(&mut self) -> Result<(), Error> { + // Nothing to do. + Ok(()) + } + + /// Has the last response been received or are more expected? + /// + /// Call this after each call to [`get_response`] to check if more + /// responses are expected. + /// + /// Returns false if more responses are expected, true otherwise. + fn is_stream_complete(&self) -> bool { + // DNS response streams only exist at the time of writing for + // AXFR/IXFR over TCP, not over UDP, so in most cases there will not + // be a subsequent response. Implementations that know whether or not + // there will be a subsequent response can override this default + // implementaiton. We return true because the caller should always + // check for at least one response and then if they call this function + // to find out if there will be more we say no, the stream is + // complete. + true + } } //------------ RequestMessage ------------------------------------------------ @@ -290,6 +331,10 @@ impl + Debug + Octets + Send + Sync> ComposeRequest } } + fn is_streaming(&self) -> bool { + self.msg.is_streaming() + } + fn dnssec_ok(&self) -> bool { match &self.opt { None => false, diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index 2c82fecc0..7641c641b 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -14,16 +14,9 @@ // - request timeout // - create new connection after end/failure of previous one -use crate::base::message::Message; -use crate::base::message_builder::StreamTarget; -use crate::base::opt::{AllOptData, OptRecord, TcpKeepalive}; -use crate::net::client::request::{ - ComposeRequest, Error, GetResponse, SendRequest, -}; -use crate::utils::config::DefMinMax; -use bytes::{Bytes, BytesMut}; use core::cmp; -use octseq::Octets; +use core::future::ready; + use std::boxed::Box; use std::fmt::Debug; use std::future::Future; @@ -31,9 +24,22 @@ use std::pin::Pin; use std::sync::Arc; use std::time::{Duration, Instant}; use std::vec::Vec; + +use bytes::{Bytes, BytesMut}; +use octseq::Octets; use tokio::io::{AsyncRead, AsyncReadExt, AsyncWrite, AsyncWriteExt}; +use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; use tokio::sync::{mpsc, oneshot}; use tokio::time::sleep; +use tracing::trace; + +use crate::base::message::Message; +use crate::base::message_builder::StreamTarget; +use crate::base::opt::{AllOptData, OptRecord, TcpKeepalive}; +use crate::net::client::request::{ + ComposeRequest, Error, GetResponse, SendRequest, +}; +use crate::utils::config::DefMinMax; //------------ Configuration Constants ---------------------------------------- @@ -77,9 +83,15 @@ const READ_REPLY_CHAN_CAP: usize = 8; /// Configuration for a stream transport connection. #[derive(Clone, Debug)] pub struct Config { - /// Response timeout. + /// Response timeout currently in effect. response_timeout: Duration, + /// Single response timeout. + single_response_timeout: Duration, + + /// Streaming response timeout. + streaming_response_timeout: Duration, + /// Default idle timeout. /// /// This value is used if the other side does not send a TcpKeepalive @@ -102,11 +114,53 @@ impl Config { } /// Sets the response timeout. + /// + /// For requests where ComposeRequest::is_streaming() returns true see + /// set_streaming_response_timeout() instead. + /// + /// Excessive values are quietly trimmed. + // + // XXX Maybe that’s wrong and we should rather return an error? pub fn set_response_timeout(&mut self, timeout: Duration) { self.response_timeout = RESPONSE_TIMEOUT.limit(timeout); + self.streaming_response_timeout = self.response_timeout; + } + + /// Returns the streaming response timeout. + pub fn streaming_response_timeout(&self) -> Duration { + self.streaming_response_timeout } - /// Sets the idle timeout. + /// Sets the streaming response timeout. + /// + /// Only used for requests where ComposeRequest::is_streaming() returns + /// true as it is typically desirable that such response streams be + /// allowed to complete even if the individual responses arrive very + /// slowly. + /// + /// Excessive values are quietly trimmed. + pub fn set_streaming_response_timeout(&mut self, timeout: Duration) { + self.streaming_response_timeout = RESPONSE_TIMEOUT.limit(timeout); + } + + /// Returns the initial idle timeout, if set. + pub fn idle_timeout(&self) -> Duration { + self.idle_timeout + } + + /// Sets the initial idle timeout. + /// + /// By default the stream is immediately closed if there are no pending + /// requests or responses. + /// + /// Set this to allow requests to be sent in sequence with delays between + /// such as a SOA query followed by AXFR for more efficient use of the + /// stream per RFC 9103. + /// + /// Note: May be overridden by an RFC 7828 edns-tcp-keepalive timeout + /// received from a server. + /// + /// Excessive values are quietly trimmed. pub fn set_idle_timeout(&mut self, timeout: Duration) { self.idle_timeout = IDLE_TIMEOUT.limit(timeout) } @@ -116,6 +170,8 @@ impl Default for Config { fn default() -> Self { Self { response_timeout: RESPONSE_TIMEOUT.default(), + single_response_timeout: RESPONSE_TIMEOUT.default(), + streaming_response_timeout: RESPONSE_TIMEOUT.default(), idle_timeout: IDLE_TIMEOUT.default(), } } @@ -160,12 +216,13 @@ impl Connection { /// Start a DNS request. /// /// This function takes a precomposed message as a parameter and - /// returns a [`Message`] object wrapped in a [`Result`]. + /// returns a response [`Message`] object wrapped in a [`Result`]. async fn handle_request_impl( self, msg: Req, ) -> Result, Error> { let (sender, receiver) = oneshot::channel(); + let sender = ReplySender::Single(Some(sender)); let req = ChanReq { sender, msg }; self.sender.send(req).await.map_err(|_| { // Send error. The receiver is gone, this means that the @@ -175,10 +232,49 @@ impl Connection { receiver.await.map_err(|_| Error::StreamReceiveError)? } + /// Start a DNS request that may result in multiple responses. + /// + /// This function takes a precomposed message as a parameter and a stream + /// sender which should be used to send responses back to the caller as + /// responses are received. + /// + /// Note: The return type is and must be compatible with that of + /// [`handle_request_impl`] but has no meaning and should not be checked. + async fn handle_streaming_request_impl( + self, + msg: Req, + sender: UnboundedSender, Error>>, + ) -> Result, Error> { + let reply_sender = ReplySender::Stream(sender); + let req = ChanReq { + sender: reply_sender, + msg, + }; + let _ = self.sender.send(req).await; + + // TODO: It would be nicer if we could return Ok(()) here. + Err(Error::ConnectionClosed) + } + /// Returns a request handler for this connection. pub fn get_request(&self, request_msg: Req) -> Request { Request { + stream: None, fut: Box::pin(self.clone().handle_request_impl(request_msg)), + stream_complete: false, + } + } + + /// TODO + pub fn get_streaming_request(&self, request_msg: Req) -> Request { + let (sender, receiver) = mpsc::unbounded_channel(); + Request { + stream: Some(receiver), + fut: Box::pin( + self.clone() + .handle_streaming_request_impl(request_msg, sender), + ), + stream_complete: false, } } } @@ -191,14 +287,16 @@ impl Clone for Connection { } } -impl SendRequest - for Connection -{ +impl SendRequest for Connection { fn send_request( &self, request_msg: Req, ) -> Box { - Box::new(self.get_request(request_msg)) + if request_msg.is_streaming() { + Box::new(self.get_streaming_request(request_msg)) + } else { + Box::new(self.get_request(request_msg)) + } } } @@ -206,16 +304,59 @@ impl SendRequest /// An active request. pub struct Request { + /// The stream of responses to await when [`get_streaming_request()`] was + /// called, None otherwise. + stream: Option, Error>>>, + /// The underlying future. fut: Pin< Box, Error>> + Send + Sync>, >, + + /// True if the caller has signalled that the last data in the stream has + /// been received. + /// + /// The DNS protocol does not provide a standardized way to detect the end + /// of a stream of responses. At the time of writing the only query types that + /// can result in a stream of responses are AXFR and IXFR. In both cases the + /// end of the response data is detected by examining the content of the DNS + /// responses, there is no actual END signal per se. So we rely on the caller + /// inspecting the response messages and telling us that it has detected the + /// end of the stream by calling [`stream_complete()`] at which point this + /// flag will be set to true. By default this flag is set to false. + stream_complete: bool, } impl Request { /// Async function that waits for the future stored in Request to complete. async fn get_response_impl(&mut self) -> Result, Error> { - (&mut self.fut).await + // In most cases the caller will have called [`send_request()`] and + // only a single response is expected which will result from resolving + // this future to a successful result. However, if + // [`send_streaming_request()`] was called instead this future will + // always resolve to Error::ConnectionClosed as the response is not + // delivered immediately but instead via the separate response stream. + // In both cases no response will be received if the future is not + // first resolved to completion, so we must await it in either case. + let mut res = (&mut self.fut).await; + + // Do we have a response stream that we should consume from? If not + // the result is already available and can be returned immediately. + let stream = self.stream else { + return res; + }; + + // Fetch from the stream + res = stream + .recv() + .await + .ok_or(Error::ConnectionClosed) + .map_err(|_| Error::ConnectionClosed)?; + + // Setup the next future + self.fut = Box::pin(ready(Err(Error::ConnectionClosed))); + + res } } @@ -232,6 +373,21 @@ impl GetResponse for Request { > { Box::pin(self.get_response_impl()) } + + fn stream_complete(&mut self) -> Result<(), Error> { + if let Some(mut stream) = self.stream.take() { + trace!("Closing response stream"); + stream.close(); + } + + self.stream_complete = true; + + Ok(()) + } + + fn is_stream_complete(&self) -> bool { + self.stream_complete + } } impl Debug for Request { @@ -247,7 +403,7 @@ impl Debug for Request { /// The underlying machinery of a stream transport. #[derive(Debug)] pub struct Transport { - /// The stream socket towards the remove end. + /// The stream socket towards the remote end. stream: Stream, /// Transport configuration. @@ -257,6 +413,44 @@ pub struct Transport { receiver: mpsc::Receiver>, } +/// A sender used to communicate a received response back to the caller. +#[derive(Debug)] +pub enum ReplySender { + /// A single immediate response. + /// + /// For most DNS query types this is the appropriate sender to use because + /// most DNS requests result in a single response. + Single(Option>), + + /// For DNS query types that can result in a stream of responses use this + /// sender to send an unknown number of responses back to the caller. + Stream(mpsc::UnboundedSender), +} + +impl ReplySender { + /// Send a response back to the caller. + /// + /// If this ReplySender is of type Single, attempts to call this function + /// more than once will return an error containing the response value + /// supplied by the caller. + pub fn send(&mut self, resp: ChanResp) -> Result<(), ChanResp> { + match self { + ReplySender::Single(sender) => match sender.take() { + Some(sender) => sender.send(resp), + None => Err(resp), + }, + ReplySender::Stream(sender) => { + sender.send(resp).map_err(|err| err.0) + } + } + } + + /// Is this ReplySender of type Stream? + pub fn is_stream(&self) -> bool { + matches!(self, Self::Stream(_)) + } +} + /// A message from a [`Request`] to start a new request. #[derive(Debug)] struct ChanReq { @@ -267,9 +461,6 @@ struct ChanReq { sender: ReplySender, } -/// This is the type of sender in [ChanReq]. -type ReplySender = oneshot::Sender; - /// A message back to [`Request`] returning a response. type ChanResp = Result, Error>; @@ -479,6 +670,15 @@ where res = recv_fut, if !do_write => { match res { Some(req) => { + // Wait longer for response streams than for + // single responses. + if req.sender.is_stream() { + self.config.response_timeout = + self.config.streaming_response_timeout; + } else { + self.config.response_timeout = + self.config.single_response_timeout; + } Self::insert_req( req, &mut status, &mut reqmsg, &mut query_vec ) @@ -586,7 +786,7 @@ where fn error(error: Error, query_vec: &mut Queries>) { // Update all requests that are in progress. Don't wait for // any reply that may be on its way. - for item in query_vec.drain() { + for mut item in query_vec.drain() { _ = item.sender.send(Err(error.clone())); } } @@ -621,7 +821,7 @@ where status.state = ConnState::Active(Some(Instant::now())); // Get the correct query and send it the reply. - let req = match query_vec.try_remove(answer.header().id()) { + let mut req = match query_vec.try_remove(answer.header().id()) { Some(req) => req, None => { // No query with this ID. We should @@ -636,6 +836,11 @@ where }; _ = req.sender.send(answer); + // TODO: Discard streaming requests once the stream is complete. + if req.sender.is_stream() { + query_vec.insert(req).unwrap(); + } + if query_vec.is_empty() { // Clear the activity timer. There is no need to do // this because state will be set to either IdleTimeout @@ -660,7 +865,7 @@ where /// idle. Addend a edns-tcp-keepalive option if needed. // Note: maybe reqmsg should be a return value. fn insert_req( - req: ChanReq, + mut req: ChanReq, status: &mut Status, reqmsg: &mut Option>, query_vec: &mut Queries>, @@ -696,11 +901,11 @@ where } // Note that insert may fail if there are too many - // outstanding queires. First call insert before checking + // outstanding queries. First call insert before checking // send_keepalive. let (index, req) = match query_vec.insert(req) { Ok(res) => res, - Err(req) => { + Err(mut req) => { // Send an appropriate error and return. _ = req .sender @@ -732,7 +937,7 @@ where } Err(err) => { // Take the sender out again and return the error. - if let Some(req) = query_vec.try_remove(index) { + if let Some(mut req) = query_vec.try_remove(index) { _ = req.sender.send(Err(err)); } } From d8c8b14740613505fc761f659f6c3f062381aaa6 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 8 Aug 2024 13:22:25 +0200 Subject: [PATCH 016/333] Support adding to a message generated by a client before sending it so that a wrapper client (such as a future TSIG client) can augment the message before it gets sent. --- src/net/client/request.rs | 31 +++++++++++++++---------------- src/net/client/stream.rs | 7 ++++--- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/src/net/client/request.rs b/src/net/client/request.rs index 6062eed96..970f79a00 100644 --- a/src/net/client/request.rs +++ b/src/net/client/request.rs @@ -25,11 +25,14 @@ use crate::rdata::AllRecordData; /// A trait that allows composing a request as a series. pub trait ComposeRequest: Debug + Send + Sync { - /// Appends the final message to a provided composer. - fn append_message( + /// Writes the message to a provided composer. + /// + /// Returns the builder used to allow the caller to make further changes + /// if needed. + fn to_message_builder( &self, - target: &mut Target, - ) -> Result<(), CopyRecordsError>; + target: Target, + ) -> Result, CopyRecordsError>; /// Create a message that captures the recorded changes. fn to_message(&self) -> Result>, Error>; @@ -173,7 +176,7 @@ impl + Debug + Octets> RequestMessage { self.opt.get_or_insert_with(Default::default) } - /// Appends the message to a composer. + /// Appends the message to a builder. fn append_message_impl( &self, mut target: MessageBuilder, @@ -232,13 +235,9 @@ impl + Debug + Octets> RequestMessage { MessageBuilder::from_target(StaticCompressor::new(Vec::new())) .expect("Vec is expected to have enough space"); - let target = self.append_message_impl(target)?; + let builder = self.append_message_impl(target)?; - // It would be nice to use .builder() here. But that one deletes all - // sections. We have to resort to .as_builder() which gives a - // reference and then .clone() - let result = target.as_builder().clone(); - let msg = Message::from_octets(result.finish().into_target()).expect( + let msg = Message::from_octets(builder.finish().into_target()).expect( "Message should be able to parse output from MessageBuilder", ); Ok(msg) @@ -248,14 +247,14 @@ impl + Debug + Octets> RequestMessage { impl + Debug + Octets + Send + Sync> ComposeRequest for RequestMessage { - fn append_message( + fn to_message_builder( &self, - target: &mut Target, - ) -> Result<(), CopyRecordsError> { + target: Target, + ) -> Result, CopyRecordsError> { let target = MessageBuilder::from_target(target) .map_err(|_| CopyRecordsError::Push(PushError::ShortBuf))?; - self.append_message_impl(target)?; - Ok(()) + let builder = self.append_message_impl(target)?; + Ok(builder) } fn to_vec(&self) -> Result, Error> { diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index 7641c641b..60c561e46 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -954,10 +954,11 @@ where /// Convert the query message to a vector. fn convert_query(msg: &Req) -> Result, Error> { - let mut target = StreamTarget::new_vec(); - msg.append_message(&mut target) + let target = StreamTarget::new_vec(); + let target = msg + .to_message_builder(target) .map_err(|_| Error::StreamLongMessage)?; - Ok(target.into_target()) + Ok(target.finish().into_target()) } } From 1376041e586d67b77cd343108e0cad6f9eb62a70 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 8 Aug 2024 13:24:46 +0200 Subject: [PATCH 017/333] Fix compilation error caused by the recent streaming support commit. --- src/net/client/stream.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index 60c561e46..91b9b140f 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -342,7 +342,7 @@ impl Request { // Do we have a response stream that we should consume from? If not // the result is already available and can be returned immediately. - let stream = self.stream else { + let Some(stream) = self.stream.as_mut() else { return res; }; From 1447bfcf83d83e3764a158e11f42c1ad477f861d Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 8 Aug 2024 13:30:16 +0200 Subject: [PATCH 018/333] Cargo fmt. --- src/base/message.rs | 4 ++-- src/net/client/multi_stream.rs | 2 +- src/net/client/request.rs | 19 ++++++++++--------- 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/src/base/message.rs b/src/base/message.rs index 144833276..0dfac4d9c 100644 --- a/src/base/message.rs +++ b/src/base/message.rs @@ -440,10 +440,10 @@ impl Message { } /// Could this message result in a stream of responses? - /// + /// /// Most DNS queries result in a single response, but some (only AXFR and /// IXFR at the time of writing) can result in a stream of responses. - /// + /// /// Returns true if the first question is of a type that might result in a /// stream of responses. pub fn is_streaming(&self) -> bool { diff --git a/src/net/client/multi_stream.rs b/src/net/client/multi_stream.rs index 7ba9082be..06b6c5f6b 100644 --- a/src/net/client/multi_stream.rs +++ b/src/net/client/multi_stream.rs @@ -356,7 +356,7 @@ impl GetResponse for Request { Box::pin(Self::get_response(self)) } - // TODO: Override stream_complete() and is_stream_complete() like + // TODO: Override stream_complete() and is_stream_complete() like // net::client::stream does? } diff --git a/src/net/client/request.rs b/src/net/client/request.rs index 970f79a00..d15b6948a 100644 --- a/src/net/client/request.rs +++ b/src/net/client/request.rs @@ -26,7 +26,7 @@ use crate::rdata::AllRecordData; /// A trait that allows composing a request as a series. pub trait ComposeRequest: Debug + Send + Sync { /// Writes the message to a provided composer. - /// + /// /// Returns the builder used to allow the caller to make further changes /// if needed. fn to_message_builder( @@ -93,7 +93,7 @@ pub trait GetResponse: Debug { /// Get the result of a DNS request. /// /// This function is intended to be cancel safe. - /// + /// /// If [`is_stream_complete()`] returns false you can call this function /// again to receive the next response when dealing with a stream of /// responses. @@ -109,7 +109,7 @@ pub trait GetResponse: Debug { >; /// Signal that no more responses are expected. - /// + /// /// The DNS protocol does not provide a standardized way to detect the end /// of a stream of responses. At the time of writing the only query types /// that can result in a stream of responses are AXFR and IXFR. In both @@ -124,10 +124,10 @@ pub trait GetResponse: Debug { } /// Has the last response been received or are more expected? - /// + /// /// Call this after each call to [`get_response`] to check if more /// responses are expected. - /// + /// /// Returns false if more responses are expected, true otherwise. fn is_stream_complete(&self) -> bool { // DNS response streams only exist at the time of writing for @@ -137,7 +137,7 @@ pub trait GetResponse: Debug { // implementaiton. We return true because the caller should always // check for at least one response and then if they call this function // to find out if there will be more we say no, the stream is - // complete. + // complete. true } } @@ -237,9 +237,10 @@ impl + Debug + Octets> RequestMessage { let builder = self.append_message_impl(target)?; - let msg = Message::from_octets(builder.finish().into_target()).expect( - "Message should be able to parse output from MessageBuilder", - ); + let msg = Message::from_octets(builder.finish().into_target()) + .expect( + "Message should be able to parse output from MessageBuilder", + ); Ok(msg) } } From 5e36a5c5f84c65af4b28fbcb4bd483de14a40ca2 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 8 Aug 2024 15:38:13 +0200 Subject: [PATCH 019/333] Stelline server testing changes: - Added a simple UDP client that doesn't interfere with requests before sending (for TSIG testing). - Added support for receiving multiple responses (for XFR testing). - Added support for connection timeout errors - Added support for connection termination errors (for EDNS testing). - Added support for specifying the TSIG key to use (for TSIG testing). - Added support for $ORIGIN in zone file fragments. - Simplified rcode checking and use it instead of the yxrrset BADCOOKIE hack. - In memory channel changes: - Fixed a trace message that incorrectly referred to client instead of server. - Fixed a too-tight connection read loop that was preventing Tokio task switching. - Added connection shutdown detection. - Fixed incorrect setting of TCP mode to true when UDP mode was requested. --- src/net/server/tests/integration.rs | 48 ++-- src/stelline/channel.rs | 47 +++- src/stelline/client.rs | 206 ++++++++++++++--- src/stelline/matches.rs | 141 +++++------- src/stelline/mod.rs | 4 +- src/stelline/parse_stelline.rs | 111 +++++++--- src/stelline/server.rs | 50 +++-- src/stelline/simple_dgram_client.rs | 221 +++++++++++++++++++ test-data/server/edns_downstream_cookies.rpl | 6 +- 9 files changed, 627 insertions(+), 207 deletions(-) create mode 100644 src/stelline/simple_dgram_client.rs diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index bf6af2a96..982f0580c 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -33,12 +33,14 @@ use crate::stelline::client::{ QueryTailoredClientFactory, }; use crate::stelline::parse_stelline::{self, parse_file, Config, Matches}; +use crate::stelline::simple_dgram_client; use crate::utils::base16; use crate::zonefile::inplace::{Entry, ScannedRecord, Zonefile}; //----------- Tests ---------------------------------------------------------- -/// Stelline test cases for which the .rpl file defines a server: config block. +/// Stelline test cases for which the .rpl file defines a server: config +/// block. /// /// Note: Adding or removing .rpl files on disk won't be detected until the /// test is re-compiled. @@ -48,9 +50,9 @@ use crate::zonefile::inplace::{Entry, ScannedRecord, Zonefile}; #[rstest] #[tokio::test(start_paused = true)] async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { - // Load the test .rpl file that determines which queries will be sent - // and which responses will be expected, and how the server that - // answers them should be configured. + // Load the test .rpl file that determines which queries will be sent and + // which responses will be expected, and how the server that answers them + // should be configured. // Initialize tracing based logging. Override with env var RUST_LOG, e.g. // RUST_LOG=trace. DEBUG level will show the .rpl file name, Stelline step @@ -182,15 +184,15 @@ fn mk_client_factory( stream_server_conn: ClientServerChannel, ) -> impl ClientFactory { // Create a TCP client factory that only creates a client if (a) no - // existing TCP client exists for the source address of the Stelline query, - // and (b) if the query specifies "MATCHES TCP". Clients created by this - // factory connect to the TCP server created above. + // existing TCP client exists for the source address of the Stelline + // query, and (b) if the query specifies "MATCHES TCP". Clients created by + // this factory connect to the TCP server created above. let only_for_tcp_queries = |entry: &parse_stelline::Entry| { matches!(entry.matches, Some(Matches { tcp: true, .. })) }; let tcp_client_factory = PerClientAddressClientFactory::new( - move |source_addr| { + move |source_addr, _entry| { let stream = stream_server_conn .connect(Some(SocketAddr::new(*source_addr, 0))); let (conn, transport) = stream::Connection::new(stream); @@ -201,15 +203,21 @@ fn mk_client_factory( ); // Create a UDP client factory that only creates a client if (a) no - // existing UDP client exists for the source address of the Stelline query. + // existing UDP client exists for the source address of the Stelline + // query. let for_all_other_queries = |_: &_| true; let udp_client_factory = PerClientAddressClientFactory::new( - move |source_addr| { - Box::new(dgram::Connection::new( - dgram_server_conn - .new_client(Some(SocketAddr::new(*source_addr, 0))), - )) + move |source_addr, entry| { + let connect = dgram_server_conn + .new_client(Some(SocketAddr::new(*source_addr, 0))); + + match entry.matches.as_ref().map(|v| v.mock_client) { + Some(true) => { + Box::new(simple_dgram_client::Connection::new(connect)) + } + _ => Box::new(dgram::Connection::new(connect)), + } }, for_all_other_queries, ); @@ -243,14 +251,14 @@ fn mk_server_configs( // This function can be used with `service_fn()` to create a `Service` // instance designed to respond to test queries. // -// The functionality provided is the mininum common set of behaviour needed -// by the tests that use it. +// The functionality provided is the mininum common set of behaviour needed by +// the tests that use it. // // It's behaviour should be influenced to match the conditions under test by: // - Using different `MiddlewareChain` setups with the server(s) to which // the `Service` will be passed. -// - Controlling the content of the `Zonefile` passed to instances of -// this `Service` impl. +// - Controlling the content of the `Zonefile` passed to instances of this +// `Service` impl. #[allow(clippy::type_complexity)] fn test_service( request: Request>, @@ -362,8 +370,8 @@ fn parse_server_config(config: &Config) -> ServerConfig { // TODO: Strictly speaking the "ip" is a netblock // "given as an IPv4 or IPv6 address /size appended // for a classless network block", but we only handle - // an IP address here for now. - // See: https://unbound.docs.nlnetlabs.nl/en/latest/manpages/unbound.conf.html?highlight=edns-tcp-keepalive#unbound-conf-access-control + // an IP address here for now. See: + // https://unbound.docs.nlnetlabs.nl/en/latest/manpages/unbound.conf.html?highlight=edns-tcp-keepalive#unbound-conf-access-control if let Some((ip, action)) = v.split_once(|c: char| c.is_whitespace()) { diff --git a/src/stelline/channel.rs b/src/stelline/channel.rs index 66f79c085..ee1efea99 100644 --- a/src/stelline/channel.rs +++ b/src/stelline/channel.rs @@ -1,6 +1,9 @@ // Using tokio::io::duplex() seems appealing but it can only create a channel // between two ends, it isn't possible to create additional client ends for a // single server end for example. +use core::sync::atomic::{AtomicBool, AtomicU16, Ordering}; +use core::time::Duration; + use std::boxed::Box; use std::collections::HashMap; use std::future::ready; @@ -22,7 +25,6 @@ use crate::net::client::protocol::{ AsyncConnect, AsyncDgramRecv, AsyncDgramSend, }; use crate::net::server::sock::{AsyncAccept, AsyncDgramSock}; -use core::sync::atomic::{AtomicU16, Ordering}; // If MSRV gets bumped to 1.69.0 we can replace these with a const SocketAddr. pub const DEF_CLIENT_ADDR: IpAddr = IpAddr::V4(Ipv4Addr::LOCALHOST); @@ -72,7 +74,7 @@ struct ClientSocket { /// Sender for sender requests to the server. tx: mpsc::Sender, - /// Receiver for receving responses from the server. + /// Receiver for receiving responses from the server. /// /// Wrapped in a mutex so that it can be mutated even from a &self fn. rx: Mutex>>, @@ -168,6 +170,9 @@ pub struct ClientServerChannel { /// Type of connection. is_stream: bool, + + /// Is connection terminated? + is_shutdown: Arc, } impl Default for ClientServerChannel { @@ -180,6 +185,7 @@ impl Default for ClientServerChannel { client_addr, next_client_port: Arc::new(AtomicU16::new(1)), is_stream: Default::default(), + is_shutdown: Default::default(), } } } @@ -194,6 +200,7 @@ impl Clone for ClientServerChannel { client_addr: self.client_addr, next_client_port: self.next_client_port.clone(), is_stream: self.is_stream, + is_shutdown: self.is_shutdown.clone(), } } } @@ -230,6 +237,7 @@ impl ClientServerChannel { client_addr, next_client_port: self.next_client_port.clone(), is_stream: self.is_stream, + is_shutdown: self.is_shutdown.clone(), } } @@ -269,6 +277,7 @@ impl ClientServerChannel { client_addr: self.client_addr, next_client_port: self.next_client_port.clone(), is_stream: false, + is_shutdown: Arc::new(AtomicBool::new(false)), } } @@ -282,9 +291,10 @@ impl ClientServerChannel { let channel = Self { server: Arc::new(Mutex::new(server_socket)), client: Some(client), - client_addr: self.client_addr, + client_addr, next_client_port: self.next_client_port.clone(), is_stream: true, + is_shutdown: Arc::new(AtomicBool::new(false)), }; // Tell the server how to receive from and respond to the client @@ -497,9 +507,10 @@ impl Future for ClientServerChannelReadableFut { trace!("Server socket is now readable"); Poll::Ready(Ok(())) } else { - trace!("Server socket is not yet readable"); let waker = cx.waker().clone(); tokio::task::spawn(async move { + // Give other tasks a chance to run. + tokio::time::sleep(Duration::from_nanos(1)).await; waker.wake(); }); Poll::Pending @@ -527,10 +538,8 @@ impl AsyncAccept for ClientServerChannel { // sends the details of a new client connection to us. Poll::Ready(Some(Data::StreamAccept(channel))) => { trace!("Accepted connection in stream channel",); - Poll::Ready(Ok(( - ready(Ok(channel)), - SocketAddr::new("::".parse().unwrap(), 0), - ))) + let client_addr = channel.client_addr; + Poll::Ready(Ok((ready(Ok(channel)), client_addr))) } Poll::Ready(Some(Data::StreamRequest(..))) => unreachable!(), Poll::Ready(Some(Data::DgramRequest(..))) => unreachable!(), @@ -557,12 +566,21 @@ impl AsyncRead for ClientServerChannel { cx: &mut Context<'_>, buf: &mut ReadBuf<'_>, ) -> Poll> { + if self.is_shutdown.load(Ordering::SeqCst) { + trace!( + "Stream shutdown while reading in client-server stream channel" + ); + return Poll::Ready(Err(io::Error::from( + io::ErrorKind::BrokenPipe, + ))); + } + match &mut self.client { Some(client) => { let rx = &mut client.rx.lock().unwrap(); match rx.poll_recv(cx) { Poll::Ready(Some(data)) => { - trace!("Reading {} bytes into internal buffer in server stream channel", data.len()); + trace!("Reading {} bytes into internal buffer in client stream channel", data.len()); client.unread_buf.extend(data).fill(buf); Poll::Ready(Ok(())) } @@ -629,6 +647,15 @@ impl AsyncWrite for ClientServerChannel { cx: &mut Context<'_>, data: &[u8], ) -> Poll> { + if self.is_shutdown.load(Ordering::SeqCst) { + trace!( + "Stream shutdown while writing in client-server stream channel" + ); + return Poll::Ready(Err(io::Error::from( + io::ErrorKind::BrokenPipe, + ))); + } + match &self.client { Some(client) => { let mut fut = Box::pin( @@ -700,7 +727,7 @@ impl AsyncWrite for ClientServerChannel { self: Pin<&mut Self>, _cx: &mut Context<'_>, ) -> Poll> { - // Unsupported? + self.is_shutdown.store(true, Ordering::SeqCst); Poll::Ready(Ok(())) } } diff --git a/src/stelline/client.rs b/src/stelline/client.rs index 6d8f85ba2..20f8a2447 100644 --- a/src/stelline/client.rs +++ b/src/stelline/client.rs @@ -19,8 +19,10 @@ use crate::base::iana::{Opcode, OptionCode}; use crate::base::opt::{ComposeOptData, OptData}; use crate::base::{Message, MessageBuilder}; use crate::net::client::request::{ - ComposeRequest, Error, RequestMessage, SendRequest, + ComposeRequest, Error, GetResponse, RequestMessage, SendRequest, }; +use crate::stelline::matches::match_multi_msg; +use crate::zonefile::inplace::Entry::Record; use super::matches::match_msg; use super::parse_stelline::{Entry, Reply, Stelline, StepType}; @@ -68,6 +70,7 @@ pub enum StellineErrorCause { MissingResponse, MissingStepEntry, MissingClient, + AnswerTimedOut, } impl From for StellineErrorCause { @@ -94,6 +97,9 @@ impl std::fmt::Display for StellineErrorCause { StellineErrorCause::MissingStepEntry => { f.write_str("Missing step entry") } + StellineErrorCause::AnswerTimedOut => { + f.write_str("Timed out waiting for answer") + } } } } @@ -203,15 +209,14 @@ impl Dispatcher { Self(None) } - pub async fn dispatch( + pub fn dispatch( &self, entry: &Entry, - ) -> Result>, StellineErrorCause> { + ) -> Result, StellineErrorCause> { if let Some(dispatcher) = &self.0 { let reqmsg = entry2reqmsg(entry); trace!(?reqmsg); - let mut req = dispatcher.send_request(reqmsg); - return Ok(Some(req.get_response().await?)); + return Ok(dispatcher.send_request(reqmsg)); } Err(StellineErrorCause::MissingClient) @@ -264,7 +269,7 @@ impl ClientFactory for SingleClientFactory { pub struct PerClientAddressClientFactory where - F: Fn(&IpAddr) -> Box>>>, + F: Fn(&IpAddr, &Entry) -> Box>>>, S: Fn(&Entry) -> bool, { clients_by_address: @@ -275,7 +280,7 @@ where impl PerClientAddressClientFactory where - F: Fn(&IpAddr) -> Box>>>, + F: Fn(&IpAddr, &Entry) -> Box>>>, S: Fn(&Entry) -> bool, { pub fn new(factory_func: F, is_suitable_func: S) -> Self { @@ -289,7 +294,7 @@ where impl ClientFactory for PerClientAddressClientFactory where - F: Fn(&IpAddr) -> Box>>>, + F: Fn(&IpAddr, &Entry) -> Box>>>, S: Fn(&Entry) -> bool, { fn get( @@ -303,7 +308,9 @@ where let client = self .clients_by_address .entry(client_addr) - .or_insert_with_key(|addr| Rc::new((self.factory_func)(addr))) + .or_insert_with_key(|addr| { + Rc::new((self.factory_func)(addr, entry)) + }) .clone(); Box::pin(ready(Dispatcher::with_rc_boxed_client(client))) @@ -373,7 +380,9 @@ pub async fn do_client<'a, T: ClientFactory>( step_value: &CurrStepValue, mut client_factory: T, ) -> Result<(), StellineErrorCause> { - let mut resp: Option> = None; + let mut last_sent_request: Option< + Box, + > = None; #[cfg(all(feature = "std", test))] { @@ -397,40 +406,153 @@ pub async fn do_client<'a, T: ClientFactory>( .ok_or(StellineErrorCause::MissingStepEntry)?; // Dispatch the request to a suitable client. - let mut res = - client_factory.get(entry).await.dispatch(entry).await; + let mut send_request = + client_factory.get(entry).await.dispatch(entry); // If the client is no longer connected, discard it and // try again with a new client. if let Err(StellineErrorCause::ClientError( Error::ConnectionClosed, - )) = res + )) = send_request { client_factory.discard(entry); - res = client_factory - .get(entry) - .await - .dispatch(entry) - .await; + send_request = + client_factory.get(entry).await.dispatch(entry); } - trace!("Receive result: {res:?}"); - - resp = res?; - - trace!(?resp); + last_sent_request = Some(send_request?); } StepType::CheckAnswer => { - let answer = resp - .take() - .ok_or(StellineErrorCause::MissingResponse)?; let entry = step .entry .as_ref() .ok_or(StellineErrorCause::MissingStepEntry)?; - if !match_msg(entry, &answer, true) { - return Err(StellineErrorCause::MismatchedAnswer); + + let Some(mut send_request) = last_sent_request else { + return Err(StellineErrorCause::MissingResponse); + }; + + if entry + .matches + .as_ref() + .map(|v| v.extra_packets) + .unwrap_or_default() + { + // This assumes that the client used for the test knows + // how to detect the last response in a set of + // responses, e.g. the xfr client knows how to detect + // the last response in an AXFR/IXFR response set. + trace!("Awaiting an unknown number of answers"); + let mut entry = entry.clone(); + loop { + let resp = match tokio::time::timeout( + Duration::from_secs(3), + send_request.get_response(), + ) + .await + .map_err(|_| StellineErrorCause::AnswerTimedOut)? + { + Err( + Error::StreamReceiveError + | Error::ConnectionClosed, + ) if entry + .matches + .as_ref() + .map(|v| v.conn_closed) + == Some(true) => + { + trace!( + "Connection terminated as expected" + ); + break; + } + other => other, + }?; + trace!("Received answer."); + trace!(?resp); + + let mut out_entry = Some(vec![]); + match_multi_msg( + &entry, + 0, + &resp, + true, + &mut out_entry, + ); + let num_rrs_remaining_after = out_entry + .as_ref() + .map(|entries| entries.len()) + .unwrap_or_default(); + if let Some(section) = &mut entry.sections { + section.answer[0] = out_entry.unwrap(); + } + trace!("Answer RRs remaining = {num_rrs_remaining_after}"); + + if send_request.is_stream_complete() { + trace!("Stream complete"); + if !entry.sections.as_ref().unwrap().answer[0] + .is_empty() + { + return Err( + StellineErrorCause::MismatchedAnswer, + ); + } else { + break; + } + } + } + } else { + let num_expected_answers = entry + .sections + .as_ref() + .map(|section| section.answer.len()) + .unwrap_or_default(); + + for idx in 0..num_expected_answers { + trace!( + "Awaiting answer {}/{num_expected_answers}...", + idx + 1 + ); + let resp = match tokio::time::timeout( + Duration::from_secs(3), + send_request.get_response(), + ) + .await + .map_err(|_| StellineErrorCause::AnswerTimedOut)? + { + Err( + Error::StreamReceiveError + | Error::ConnectionClosed, + ) if entry + .matches + .as_ref() + .map(|v| v.conn_closed) + == Some(true) => + { + trace!( + "Connection terminated as expected" + ); + break; + } + other => other, + }?; + trace!("Received answer."); + trace!(?resp); + if !match_multi_msg( + entry, idx, &resp, true, &mut None, + ) { + return Err( + StellineErrorCause::MismatchedAnswer, + ); + } + } + + if num_expected_answers > 1 { + send_request.stream_complete().unwrap(); + } } + + last_sent_request = None; } StepType::TimePasses => { let duration = @@ -492,20 +614,27 @@ fn init_logging() { fn entry2reqmsg(entry: &Entry) -> RequestMessage> { let sections = entry.sections.as_ref().unwrap(); let mut msg = MessageBuilder::new_vec().question(); + if let Some(opcode) = entry.opcode { + msg.header_mut().set_opcode(opcode); + } for q in §ions.question { msg.push(q).unwrap(); } let msg = msg.answer(); - for _a in §ions.answer { + for _a in §ions.answer[0] { todo!(); } - let msg = msg.authority(); - for _a in §ions.authority { - todo!(); + let mut msg = msg.authority(); + for zone_file_entry in §ions.authority { + if let Record(rec) = zone_file_entry { + msg.push(rec).unwrap(); + } } let mut msg = msg.additional(); - for _a in §ions.additional.zone_entries { - todo!(); + for zone_file_entry in §ions.additional.zone_entries { + if let Record(rec) = zone_file_entry { + msg.push(rec).unwrap(); + } } let reply: Reply = match &entry.reply { Some(reply) => reply.clone(), @@ -518,7 +647,14 @@ fn entry2reqmsg(entry: &Entry) -> RequestMessage> { let msg = msg.into_message(); let mut reqmsg = RequestMessage::new(msg); - reqmsg.set_dnssec_ok(reply.fl_do); + if !entry + .matches + .as_ref() + .map(|v| v.mock_client) + .unwrap_or_default() + { + reqmsg.set_dnssec_ok(reply.fl_do); + } if reply.notify { reqmsg.header_mut().set_opcode(Opcode::NOTIFY); } diff --git a/src/stelline/matches.rs b/src/stelline/matches.rs index b5dd57404..3873c8a32 100644 --- a/src/stelline/matches.rs +++ b/src/stelline/matches.rs @@ -12,6 +12,19 @@ pub fn match_msg<'a, Octs: AsRef<[u8]> + Clone + Octets + 'a>( msg: &'a Message, verbose: bool, ) -> bool +where + ::Range<'a>: Clone, +{ + match_multi_msg(entry, 0, msg, verbose, &mut None) +} + +pub fn match_multi_msg<'a, Octs: AsRef<[u8]> + Clone + Octets + 'a>( + entry: &Entry, + idx: usize, + msg: &'a Message, + verbose: bool, + out_answer: &mut Option>, +) -> bool where ::Range<'a>: Clone, { @@ -64,6 +77,8 @@ where arcount, matches.ttl, verbose, + false, + &mut None, ) { if verbose { println!("match_msg: additional section does not match"); @@ -71,20 +86,29 @@ where return false; } } - if matches.answer - && !match_section( - sections.answer.clone(), + if matches.answer { + let Some(answer) = sections.answer.get(idx) else { + if verbose { + println!("match_msg: answer section {idx} missing"); + } + return false; + }; + if !match_section( + answer.clone(), None, msg.answer().unwrap(), msg.header_counts().ancount(), matches.ttl, verbose, - ) - { - if verbose { - println!("match_msg: answer section does not match"); + matches.extra_packets, + out_answer, + ) && !matches.extra_packets + { + if verbose { + println!("match_msg: answer section {idx} does not match"); + } + return false; } - return false; } if matches.authority && !match_section( @@ -94,6 +118,8 @@ where msg.header_counts().nscount(), matches.ttl, verbose, + false, + &mut None, ) { if verbose { @@ -207,6 +233,8 @@ where if matches.opcode { let expected_opcode = if reply.notify { Opcode::NOTIFY + } else if let Some(opcode) = entry.opcode { + opcode } else { Opcode::QUERY }; @@ -238,84 +266,16 @@ where if matches.rcode { let msg_rcode = get_opt_rcode(&Message::from_octets(msg.as_slice()).unwrap()); - if reply.noerror { - if let OptRcode::NOERROR = msg_rcode { - // Okay - } else { - if verbose { - println!( - "Wrong Rcode, expected NOERROR, got {msg_rcode}" - ); - } - return false; - } - } else if reply.formerr { - if let OptRcode::FORMERR = msg_rcode { - // Okay - } else { + match (reply.rcode, msg_rcode) { + (Some(reply_rcode), msg_rcode) if reply_rcode != msg_rcode => { if verbose { println!( - "Wrong Rcode, expected FORMERR, got {msg_rcode}" + "Wrong Rcode, expected {reply_rcode}, got {msg_rcode}" ); } return false; } - } else if reply.notimp { - if let OptRcode::NOTIMP = msg_rcode { - // Okay - } else { - if verbose { - println!("Wrong Rcode, expected NOTIMP, got {msg_rcode}"); - } - return false; - } - } else if reply.nxdomain { - if let OptRcode::NXDOMAIN = msg_rcode { - // Okay - } else { - if verbose { - println!( - "Wrong Rcode, expected NXDOMAIN, got {msg_rcode}" - ); - } - return false; - } - } else if reply.refused { - if let OptRcode::REFUSED = msg_rcode { - // Okay - } else { - if verbose { - println!( - "Wrong Rcode, expected REFUSED, got {msg_rcode}" - ); - } - return false; - } - } else if reply.servfail { - if let OptRcode::SERVFAIL = msg_rcode { - // Okay - } else { - if verbose { - println!( - "Wrong Rcode, expected SERVFAIL, got {msg_rcode}" - ); - } - return false; - } - } else if "BADCOOKIE" == reply.yxrrset.as_str() { - if !matches!(msg_rcode, OptRcode::BADCOOKIE) { - if verbose { - println!( - "Wrong Rcode, expected BADCOOKIE, got {msg_rcode}" - ); - } - return false; - } - } else { - if verbose { - println!("Unexpected Rcode: {msg_rcode}"); - } - return false; + _ => { /* Okay */ } } } if matches.tcp { @@ -334,6 +294,7 @@ where true } +#[allow(clippy::too_many_arguments)] fn match_section< 'a, Octs: Clone + Octets = Octs2> + 'a, @@ -345,20 +306,27 @@ fn match_section< msg_count: u16, match_ttl: bool, verbose: bool, + allow_partial_match: bool, + out_entry: &mut Option>, ) -> bool { let mat_opt = match_edns_bytes.map(|bytes| Opt::from_slice(bytes).unwrap()); - if match_section.len() != >::into(msg_count) { + if !allow_partial_match + && match_section.len() != >::into(msg_count) + { if verbose { println!("match_section: expected section length {} doesn't match message count {}", match_section.len(), msg_count); if !match_section.is_empty() { println!("expected sections:"); - for section in match_section { + for section in &match_section { println!(" {section:?}"); } } } + if let Some(out_entry) = out_entry { + *out_entry = match_section; + } return false; } 'outer: for msg_rr in msg_section { @@ -428,6 +396,9 @@ fn match_section< msg_rr.owner(), msg_rr.class(), msg_rr.rtype(), msg_rr.ttl(), mat_rr.ttl()); } + if let Some(out_entry) = out_entry { + *out_entry = match_section; + } return false; } // Delete this entry @@ -443,10 +414,16 @@ fn match_section< msg_rr.rtype() ); } + if let Some(out_entry) = out_entry { + *out_entry = match_section; + } return false; } // All entries in the reply were matched. + if let Some(out_entry) = out_entry { + *out_entry = match_section; + } true } diff --git a/src/stelline/mod.rs b/src/stelline/mod.rs index 4714b01e8..da5581f33 100644 --- a/src/stelline/mod.rs +++ b/src/stelline/mod.rs @@ -1,9 +1,11 @@ #![cfg(feature = "unstable-stelline")] +mod matches; + pub mod channel; pub mod client; pub mod connect; pub mod connection; pub mod dgram; -mod matches; pub mod parse_stelline; pub mod server; +pub mod simple_dgram_client; diff --git a/src/stelline/parse_stelline.rs b/src/stelline/parse_stelline.rs index ddd4fd67e..09b005108 100644 --- a/src/stelline/parse_stelline.rs +++ b/src/stelline/parse_stelline.rs @@ -9,6 +9,8 @@ use std::vec::Vec; use bytes::Bytes; use crate::base; +use crate::base::iana::{Opcode, OptRcode}; +use crate::tsig::KeyName; use crate::utils::base16; use crate::zonefile::inplace::Entry as ZonefileEntry; use crate::zonefile::inplace::Zonefile; @@ -19,16 +21,19 @@ const SCENARIO_END: &str = "SCENARIO_END"; const RANGE_BEGIN: &str = "RANGE_BEGIN"; const RANGE_END: &str = "RANGE_END"; const ADDRESS: &str = "ADDRESS"; +const KEY: &str = "KEY"; const ENTRY_BEGIN: &str = "ENTRY_BEGIN"; const ENTRY_END: &str = "ENTRY_END"; const MATCH: &str = "MATCH"; const ADJUST: &str = "ADJUST"; const REPLY: &str = "REPLY"; +const OPCODE: &str = "OPCODE"; const SECTION: &str = "SECTION"; const QUESTION: &str = "QUESTION"; const ANSWER: &str = "ANSWER"; const AUTHORITY: &str = "AUTHORITY"; const ADDITIONAL: &str = "ADDITIONAL"; +const EXTRA_PACKET: &str = "EXTRA_PACKET"; const STEP: &str = "STEP"; const STEP_TYPE_QUERY: &str = "QUERY"; const STEP_TYPE_CHECK_ANSWER: &str = "CHECK_ANSWER"; @@ -230,6 +235,7 @@ fn parse_step>>( l: &mut Lines, ) -> Step { let mut step_client_address = None; + let mut step_key_name = None; let step_value = tokens.next().unwrap().parse::().unwrap(); let step_type_str = tokens.next().unwrap(); let step_type = if step_type_str == STEP_TYPE_QUERY { @@ -263,6 +269,10 @@ fn parse_step>>( (Some(ADDRESS), Some(addr)) => { step_client_address = Some(addr.parse().unwrap()); } + (Some(KEY), Some(key_name)) => { + step_key_name = + Some(KeyName::from_str(key_name).unwrap()); + } (Some(param), Some(value)) => { eprintln!("Ignoring unknown query parameter '{param}' with value '{value}'"); } @@ -319,7 +329,9 @@ fn parse_step>>( let token = tokens.next().unwrap(); if token == ENTRY_BEGIN { step.entry = Some(parse_entry(l)); - step.entry.as_mut().unwrap().client_addr = step_client_address; + let entry = step.entry.as_mut().unwrap(); + entry.client_addr = step_client_address; + entry.key_name = step_key_name; //println!("parse_step: {:?}", step); return step; } @@ -330,8 +342,10 @@ fn parse_step>>( #[derive(Clone, Debug, Default)] pub struct Entry { pub client_addr: Option, + pub key_name: Option, pub matches: Option, pub adjust: Option, + pub opcode: Option, pub reply: Option, pub sections: Option, } @@ -339,13 +353,7 @@ pub struct Entry { fn parse_entry>>( l: &mut Lines, ) -> Entry { - let mut entry = Entry { - client_addr: None, - matches: None, - adjust: None, - reply: None, - sections: None, - }; + let mut entry = Entry::default(); loop { let line = l.next().unwrap().unwrap(); let clean_line = get_clean_line(line.as_ref()); @@ -355,6 +363,11 @@ fn parse_entry>>( let clean_line = clean_line.unwrap(); let mut tokens = LineTokens::new(clean_line); let token = tokens.next().unwrap(); + if token == OPCODE { + entry.opcode = + Some(Opcode::from_str(tokens.next().unwrap()).unwrap()); + continue; + } if token == MATCH { entry.matches = Some(parse_match(tokens)); continue; @@ -383,7 +396,7 @@ fn parse_entry>>( if token == ENTRY_END { break; } - todo!(); + todo!("Unsupported token '{token}'"); } entry } @@ -394,14 +407,25 @@ pub struct AdditionalSection { pub edns_bytes: Vec, } -#[derive(Clone, Debug, Default)] +#[derive(Clone, Debug)] pub struct Sections { pub question: Vec, - pub answer: Vec, + pub answer: Vec>, pub authority: Vec, pub additional: AdditionalSection, } +impl Default for Sections { + fn default() -> Self { + Self { + question: Default::default(), + answer: vec![vec![]], + authority: Default::default(), + additional: Default::default(), + } + } +} + pub type Name = base::Name; pub type Question = base::Question; @@ -411,6 +435,8 @@ fn parse_section>>( ) -> (Sections, String) { let mut sections = Sections::default(); let next = tokens.next().unwrap(); + let mut answer_idx = 0; + let mut origin = ".".to_string(); let mut section = if next == QUESTION { Section::Question } else { @@ -439,11 +465,16 @@ fn parse_section>>( } else { panic!("Bad section {next}"); }; + origin = ".".to_string(); continue; } if token == ENTRY_END { return (sections, line); } + if token == EXTRA_PACKET { + answer_idx += 1; + continue; + } match section { Section::Question => { @@ -475,19 +506,39 @@ fn parse_section>>( .edns_bytes .extend(edns_line_bytes); } + } else if clean_line.starts_with("$ORIGIN") { + if let Some((_, new_origin)) = clean_line.split_once(' ') + { + origin = new_origin.to_string(); + } } else { let mut zonefile = Zonefile::new(); - zonefile.extend_from_slice(b"$ORIGIN .\n"); + zonefile.extend_from_slice( + format!("$ORIGIN {origin}\n").as_bytes(), + ); zonefile.extend_from_slice(b"ignore 3600 in ns ignore\n"); zonefile.extend_from_slice(clean_line.as_ref()); zonefile.extend_from_slice(b"\n"); let _e = zonefile.next_entry().unwrap(); - let e = zonefile.next_entry().unwrap(); + let e = zonefile.next_entry().map_err(|err| format!("Failed to parse zone file line '{clean_line}': {err}")).unwrap(); let e = e.unwrap(); match section { Section::Question => unreachable!(), - Section::Answer => sections.answer.push(e), + Section::Answer => { + let answer = + match sections.answer.get_mut(answer_idx) { + Some(answer) => answer, + None => { + sections.answer.push(vec![]); + sections + .answer + .get_mut(answer_idx) + .unwrap() + } + }; + answer.push(e); + } Section::Authority => sections.authority.push(e), Section::Additional => { sections.additional.zone_entries.push(e) @@ -521,6 +572,9 @@ pub struct Matches { pub udp: bool, pub server_cookie: bool, pub edns_data: bool, + pub mock_client: bool, + pub conn_closed: bool, + pub extra_packets: bool, } fn parse_match(mut tokens: LineTokens<'_>) -> Matches { @@ -567,11 +621,17 @@ fn parse_match(mut tokens: LineTokens<'_>) -> Matches { } else if token == "ttl" { matches.ttl = true; } else if token == "UDP" { - matches.tcp = true; + matches.tcp = false; } else if token == "server_cookie" { matches.server_cookie = true; } else if token == "ednsdata" { matches.edns_data = true; + } else if token == "MOCK_CLIENT" { + matches.mock_client = true; + } else if token == "CONNECTION_CLOSED" { + matches.conn_closed = true; + } else if token == "EXTRA_PACKETS" { + matches.extra_packets = true; } else { println!("should handle match {token:?}"); todo!(); @@ -615,14 +675,13 @@ pub struct Reply { pub ra: bool, pub rd: bool, pub tc: bool, - pub formerr: bool, + pub rcode: Option, pub noerror: bool, pub notimp: bool, pub nxdomain: bool, pub refused: bool, pub servfail: bool, pub yxdomain: bool, - pub yxrrset: String, pub notify: bool, } @@ -651,22 +710,8 @@ fn parse_reply(mut tokens: LineTokens<'_>) -> Reply { reply.rd = true; } else if token == "TC" { reply.tc = true; - } else if token == "FORMERR" { - reply.formerr = true; - } else if token == "NOERROR" { - reply.noerror = true; - } else if token == "NOTIMP" { - reply.notimp = true; - } else if token == "NXDOMAIN" { - reply.nxdomain = true; - } else if token == "REFUSED" { - reply.refused = true; - } else if token == "SERVFAIL" { - reply.servfail = true; - } else if token == "YXDOMAIN" { - reply.yxdomain = true; - } else if token.starts_with("YXRRSET=") { - reply.yxrrset = token.split_once('=').unwrap().1.to_string(); + } else if let Ok(rcode) = token.parse() { + reply.rcode = Some(rcode); } else if token == "NOTIFY" { reply.notify = true; } else { diff --git a/src/stelline/server.rs b/src/stelline/server.rs index e6887271c..b9d06da9d 100644 --- a/src/stelline/server.rs +++ b/src/stelline/server.rs @@ -1,8 +1,8 @@ use std::fmt::Debug; use octseq::{OctetsBuilder, Truncate}; +use tracing::trace; -use crate::base::iana::rcode::Rcode; use crate::base::iana::Opcode; use crate::base::message_builder::AdditionalBuilder; use crate::base::wire::Composer; @@ -32,12 +32,23 @@ where // Take the last entry. That works better if the RPL is written with // a recursive resolver in mind. + trace!( + "Looking for matching Stelline range response for opcode {} qtype {}", + msg.header().opcode(), + msg.first_question().unwrap().qtype() + ); for range in ranges { + trace!( + "Checking against range {} <= {}", + range.start_value, + range.end_value + ); if step < range.start_value || step > range.end_value { continue; } for entry in &range.entry { if match_msg(entry, msg, false) { + trace!("Match found"); opt_entry = Some(entry); } } @@ -49,6 +60,7 @@ where Some(reply) } None => { + trace!("No matching reply found"); println!("do_server: no reply at step value {step}"); todo!(); } @@ -82,7 +94,7 @@ where } } let mut msg = msg.answer(); - for a in §ions.answer { + for a in §ions.answer[0] { let rec = if let ZonefileEntry::Record(record) = a { record } else { @@ -116,33 +128,12 @@ where header.set_aa(reply.aa); header.set_ad(reply.ad); header.set_cd(reply.cd); - if reply.formerr { - header.set_rcode(Rcode::FORMERR); - } - if reply.noerror { - header.set_rcode(Rcode::NOERROR); - } - if reply.notimp { - header.set_rcode(Rcode::NOTIMP); - } - if reply.nxdomain { - header.set_rcode(Rcode::NXDOMAIN); - } header.set_qr(reply.qr); header.set_ra(reply.ra); header.set_rd(reply.rd); - if reply.refused { - header.set_rcode(Rcode::REFUSED); - } - if reply.servfail { - header.set_rcode(Rcode::SERVFAIL); - } if reply.tc { todo!() } - if reply.yxdomain { - todo!() - } if reply.notify { header.set_opcode(Opcode::NOTIFY); } @@ -156,9 +147,22 @@ where if reply.fl_do { msg.opt(|o| { o.set_dnssec_ok(reply.fl_do); + if let Some(rcode) = reply.rcode { + o.set_rcode(rcode); + } Ok(()) }) .unwrap() + } else if let Some(rcode) = reply.rcode { + if rcode.is_ext() { + msg.opt(|o| { + o.set_rcode(rcode); + Ok(()) + }) + .unwrap(); + } else { + header.set_rcode(rcode.rcode()); + } } msg } diff --git a/src/stelline/simple_dgram_client.rs b/src/stelline/simple_dgram_client.rs new file mode 100644 index 000000000..405a6b3c3 --- /dev/null +++ b/src/stelline/simple_dgram_client.rs @@ -0,0 +1,221 @@ +//! A client for testing over datagram protocols. +//! +//! Based on net::client::dgram but intended for use by Stelline tests that +//! require complete control over the client request. This client does NOT +//! for example add EDNS OPT records or apply DNS name compression to queries +//! that it is asked to send, nor does it time out or retry (as it is assumed +//! that a mock network connection will be used). + +#![warn(missing_docs)] + +use core::fmt; + +use std::boxed::Box; +use std::future::Future; +use std::pin::Pin; +use std::sync::Arc; + +use bytes::Bytes; +use octseq::OctetsInto; +use tracing::trace; + +use crate::base::Message; +use crate::net::client::protocol::{ + AsyncConnect, AsyncDgramRecv, AsyncDgramRecvEx, AsyncDgramSend, + AsyncDgramSendEx, +}; +use crate::net::client::request::{ + ComposeRequest, Error, GetResponse, SendRequest, +}; + +//------------ Configuration Constants ---------------------------------------- + +/// The default receive buffer size. +const DEF_RECV_SIZE: usize = 2000; + +//------------ Config --------------------------------------------------------- + +/// Configuration of a datagram transport. +#[derive(Clone, Debug)] +pub struct Config { + /// Receive buffer size. + recv_size: usize, +} + +impl Config { + /// Creates a new config with default values. + #[allow(dead_code)] + pub fn new() -> Self { + Default::default() + } + + /// Sets the receive buffer size. + /// + /// This is the amount of memory that is allocated for receiving a + /// response. + #[allow(dead_code)] + pub fn set_recv_size(&mut self, size: usize) { + self.recv_size = size + } + + /// Returns the receive buffer size. + #[allow(dead_code)] + pub fn recv_size(&self) -> usize { + self.recv_size + } +} + +impl Default for Config { + fn default() -> Self { + Self { + recv_size: DEF_RECV_SIZE, + } + } +} + +//------------ Connection ----------------------------------------------------- + +/// A datagram protocol connection. +#[derive(Clone, Debug)] +pub struct Connection { + /// Actual state of the connection. + state: Arc>, +} + +/// Because it owns the connection’s resources, this type is not [`Clone`]. +/// However, it is entirely safe to share it by sticking it into e.g. an arc. +#[derive(Debug)] +struct ConnectionState { + /// User configuration variables. + config: Config, + + /// Connections to datagram sockets. + connect: S, +} + +impl Connection { + /// Create a new datagram transport with default configuration. + pub fn new(connect: S) -> Self { + Self::with_config(connect, Default::default()) + } + + /// Create a new datagram transport with a given configuration. + pub fn with_config(connect: S, config: Config) -> Self { + Self { + state: Arc::new(ConnectionState { config, connect }), + } + } +} + +impl Connection +where + S: AsyncConnect, + S::Connection: AsyncDgramRecv + AsyncDgramSend + Unpin, +{ + /// Performs a request. + /// + /// Sends the provided and returns either a response or an error. + async fn handle_request_impl( + self, + mut request: Req, + ) -> Result, Error> { + // A place to store the receive buffer for reuse. + let mut reuse_buf = None; + + let mut sock = self + .state + .connect + .connect() + .await + .map_err(|_| Error::ConnectionClosed)?; + + // Set predictable ID in header. + request.header_mut().set_id(0); + + // Create the message and send it out. + let request_msg = request.to_message()?; + let dgram = request_msg.as_slice(); + let sent = sock + .send(dgram) + .await + .map_err(|err| Error::StreamWriteError(Arc::new(err)))?; + if sent != dgram.len() { + return Err(Error::ShortMessage); + } + + let mut buf = reuse_buf.take().unwrap_or_else(|| { + // XXX use uninit'ed mem here. + vec![0; self.state.config.recv_size] + }); + let len = sock + .recv(&mut buf) + .await + .map_err(|err| Error::StreamReadError(Arc::new(err)))?; + + trace!("Received {len} bytes of message"); + buf.truncate(len); + + let answer = Message::try_from_octets(buf) + .expect("Response could not be parsed"); + + trace!("Received message is accepted"); + Ok(answer.octets_into()) + } +} + +//--- SendRequest + +impl SendRequest for Connection +where + S: AsyncConnect + Clone + Send + Sync + 'static, + S::Connection: + AsyncDgramRecv + AsyncDgramSend + Send + Sync + Unpin + 'static, + Req: ComposeRequest + Send + Sync + 'static, +{ + fn send_request( + &self, + request_msg: Req, + ) -> Box { + Box::new(Request { + fut: Box::pin(self.clone().handle_request_impl(request_msg)), + }) + } +} + +//------------ Request ------------------------------------------------------ + +/// The state of a DNS request. +pub struct Request { + /// Future that does the actual work of GetResponse. + fut: Pin< + Box, Error>> + Send + Sync>, + >, +} + +impl Request { + /// Async function that waits for the future stored in Request to complete. + async fn get_response_impl(&mut self) -> Result, Error> { + (&mut self.fut).await + } +} + +impl fmt::Debug for Request { + fn fmt(&self, _: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { + todo!() + } +} + +impl GetResponse for Request { + fn get_response( + &mut self, + ) -> Pin< + Box< + dyn Future, Error>> + + Send + + Sync + + '_, + >, + > { + Box::pin(self.get_response_impl()) + } +} diff --git a/test-data/server/edns_downstream_cookies.rpl b/test-data/server/edns_downstream_cookies.rpl index 8130ca77f..a86bdf1e9 100644 --- a/test-data/server/edns_downstream_cookies.rpl +++ b/test-data/server/edns_downstream_cookies.rpl @@ -68,7 +68,7 @@ ENTRY_END STEP 21 CHECK_ANSWER ENTRY_BEGIN MATCH all server_cookie -REPLY QR RD DO YXRRSET=BADCOOKIE ; BADCOOKIE is an extended rcode (NOTE: THIS YXRRSET=XXX SYNTAX IS A CUSTOM DOMAIN EXTENSION TO .RPL FORMAT) +REPLY QR RD DO BADCOOKIE SECTION QUESTION test. IN TXT ENTRY_END @@ -93,7 +93,7 @@ ENTRY_END STEP 31 CHECK_ANSWER ENTRY_BEGIN MATCH all server_cookie -REPLY QR RD DO YXRRSET=BADCOOKIE ; BADCOOKIE is an extended rcode (NOTE: THIS YXRRSET=XXX SYNTAX IS A CUSTOM DOMAIN EXTENSION TO .RPL FORMAT) +REPLY QR RD DO BADCOOKIE SECTION QUESTION test. IN TXT ENTRY_END @@ -202,7 +202,7 @@ ENTRY_END STEP 71 CHECK_ANSWER ENTRY_BEGIN MATCH all server_cookie -REPLY QR RD DO YXRRSET=BADCOOKIE ; BADCOOKIE is an extended rcode (NOTE: THIS YXRRSET=XXX SYNTAX IS A CUSTOM DOMAIN EXTENSION TO .RPL FORMAT) +REPLY QR RD DO BADCOOKIE SECTION QUESTION test. IN TXT ENTRY_END From 6aacae90c42a197b7ccab062da0f8a1993726a95 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 9 Aug 2024 00:19:05 +0200 Subject: [PATCH 020/333] Add net::client::tsig. --- examples/client-transports.rs | 77 +++++- src/net/client/mod.rs | 8 + src/net/client/request.rs | 15 ++ src/net/client/tsig.rs | 446 ++++++++++++++++++++++++++++++++++ src/tsig/mod.rs | 2 +- 5 files changed, 540 insertions(+), 8 deletions(-) create mode 100644 src/net/client/tsig.rs diff --git a/examples/client-transports.rs b/examples/client-transports.rs index 92705bc09..3c5950b68 100644 --- a/examples/client-transports.rs +++ b/examples/client-transports.rs @@ -1,5 +1,9 @@ -use domain::base::MessageBuilder; /// Using the `domain::net::client` module for sending a query. +use std::net::{IpAddr, SocketAddr}; +use std::str::FromStr; +use std::time::Duration; + +use domain::base::MessageBuilder; use domain::base::Name; use domain::base::Rtype; use domain::net::client::cache; @@ -10,9 +14,12 @@ use domain::net::client::protocol::{TcpConnect, TlsConnect, UdpConnect}; use domain::net::client::redundant; use domain::net::client::request::{RequestMessage, SendRequest}; use domain::net::client::stream; -use std::net::{IpAddr, SocketAddr}; -use std::str::FromStr; -use std::time::Duration; + +#[cfg(feature = "tsig")] +use domain::net::client::tsig; +#[cfg(feature = "tsig")] +use domain::tsig::{Algorithm, Key, KeyName}; + use tokio::net::TcpStream; use tokio::time::timeout; use tokio_rustls::rustls::{ClientConfig, RootCertStore}; @@ -36,7 +43,8 @@ async fn main() { let req = RequestMessage::new(msg); // Destination for UDP and TCP - let server_addr = SocketAddr::new(IpAddr::from_str("::1").unwrap(), 53); + let server_addr = + SocketAddr::new(IpAddr::from_str("127.0.0.1").unwrap(), 8055); let mut stream_config = stream::Config::new(); stream_config.set_response_timeout(Duration::from_millis(100)); @@ -222,7 +230,7 @@ async fn main() { let reply = request.get_response().await; println!("Dgram reply: {reply:?}"); - // Create a single TCP transport connection. This is usefull for a + // Create a single TCP transport connection. This is useful for a // single request or a small burst of requests. let tcp_conn = match TcpStream::connect(server_addr).await { Ok(conn) => conn, @@ -241,13 +249,27 @@ async fn main() { }); // Send a request message. - let mut request = tcp.send_request(req); + let mut request = tcp.send_request(req.clone()); // Get the reply let reply = request.get_response().await; println!("TCP reply: {reply:?}"); drop(tcp); + + #[cfg(feature = "tsig")] + { + let tcp_conn = TcpStream::connect(server_addr).await.unwrap(); + let (tcp, transport) = stream::Connection::new(tcp_conn); + tokio::spawn(async move { + transport.run().await; + println!("single TSIG TCP run terminated"); + }); + + do_tsig(tcp.clone(), req).await; + + drop(tcp); + } } #[cfg(feature = "unstable-validator")] @@ -283,3 +305,44 @@ where let reply = request.get_response().await; println!("Validator reply: {:?}", reply); } + +#[cfg(feature = "tsig")] +async fn do_tsig(conn: SR, req: RequestMessage) +where + Octs: AsRef<[u8]> + + Send + + Sync + + std::fmt::Debug + + domain::dep::octseq::Octets + + 'static, + SR: SendRequest< + tsig::AuthenticatedRequestMessage, Key>, + > + Send + + Sync + + 'static, +{ + // Create a signing key. + let key_name = KeyName::from_str("demo-key").unwrap(); + let secret = domain::utils::base64::decode::>( + "zlCZbVJPIhobIs1gJNQfrsS3xCxxsR9pMUrGwG8OgG8=", + ) + .unwrap(); + let key = + Key::new(Algorithm::Sha256, &secret, key_name, None, None).unwrap(); + + // Create a signing transport. This assumes that the server being + // connected to is configured with a key with the same name, algorithm and + // secret and to allow that key to be used for the request we are making. + // I'm not aware of any public server with a publically announced TSIG key + // that can be used for testing so this will fail, but has been tested to + // work locally with an appropriately configured NSD server. + let tsig_conn = tsig::Connection::new(conn, Some(key)); + + // Send a query message. + let mut request = tsig_conn.send_request(req); + + // Get the reply + println!("Wating for signed reply"); + let reply = request.get_response().await; + println!("Signed reply: {:?}", reply); +} diff --git a/src/net/client/mod.rs b/src/net/client/mod.rs index c41b8a6b9..0275f3bef 100644 --- a/src/net/client/mod.rs +++ b/src/net/client/mod.rs @@ -23,6 +23,12 @@ //! as upstream transports. //! * [cache] This is a simple message cache provided as a pass through //! transport. The cache works with any of the other transports. +#![cfg_attr(feature = "tsig", doc = "* [tsig]:")] +#![cfg_attr(not(feature = "tsig",), doc = "* tsig:")] +//! This is a TSIG request signer and response verifier provided as a +//! pass through transport. The tsig transport works with any upstream +//! transports so long as they don't modify the message once signed nor +//! modify the response before it can be verified. #![cfg_attr(feature = "unstable-validator", doc = "* [validator]:")] #![cfg_attr(not(feature = "unstable-validator",), doc = "* validator:")] //! This is a DNSSEC validator provided as a pass through transport. @@ -182,6 +188,8 @@ pub mod protocol; pub mod redundant; pub mod request; pub mod stream; +#[cfg(feature = "tsig")] +pub mod tsig; #[cfg(feature = "unstable-validator")] pub mod validator; pub mod validator_test; diff --git a/src/net/client/request.rs b/src/net/client/request.rs index d15b6948a..0ded57cfc 100644 --- a/src/net/client/request.rs +++ b/src/net/client/request.rs @@ -21,6 +21,9 @@ use crate::base::wire::{Composer, ParseError}; use crate::base::{Header, Message, ParsedName, Rtype, StaticCompressor}; use crate::rdata::AllRecordData; +#[cfg(feature = "tsig")] +use crate::tsig; + //------------ ComposeRequest ------------------------------------------------ /// A trait that allows composing a request as a series. @@ -400,6 +403,10 @@ pub enum Error { /// An error happened in the datagram transport. Dgram(Arc), + #[cfg(feature = "tsig")] + /// TSIG authentication failed. + Authentication(tsig::ValidationError), + #[cfg(feature = "unstable-validator")] /// An error happened during DNSSEC validation. Validation(crate::validator::context::Error), @@ -481,6 +488,10 @@ impl fmt::Display for Error { write!(f, "no transport available") } Error::Dgram(err) => fmt::Display::fmt(err, f), + + #[cfg(feature = "tsig")] + Error::Authentication(err) => fmt::Display::fmt(err, f), + #[cfg(feature = "unstable-validator")] Error::Validation(_) => { write!(f, "error validating response") @@ -518,6 +529,10 @@ impl error::Error for Error { Error::WrongReplyForQuery => None, Error::NoTransportAvailable => None, Error::Dgram(err) => Some(err), + + #[cfg(feature = "tsig")] + Error::Authentication(err) => Some(err), + #[cfg(feature = "unstable-validator")] Error::Validation(err) => Some(err), } diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs new file mode 100644 index 000000000..93ddb34fa --- /dev/null +++ b/src/net/client/tsig.rs @@ -0,0 +1,446 @@ +//! A transport that signs requests and verifies response signatures. +//! +//! This module implements an [RFC 8945] Secret Key Transaction Authentication +//! for DNS (TSIG) client transport. +//! +//! This client cannot be used on its own, instead it must be used with an +//! upstream transport. The upstream transport must build the message then +//! send it without modifying it as that could invalidate the signature. The +//! upstream transport must also not modify the response as that could cause +//! signature verification to fail. +//! +//! [RFC 8945]: https://www.rfc-editor.org/rfc/rfc8945.html +#![warn(missing_docs)] + +use core::convert::AsRef; +use core::ops::DerefMut; + +use std::boxed::Box; +use std::fmt::{Debug, Formatter}; +use std::future::Future; +use std::pin::Pin; +use std::sync::Arc; +use std::vec::Vec; + +use bytes::Bytes; +use octseq::Octets; +use tracing::{debug, trace, warn}; + +use crate::base::message::CopyRecordsError; +use crate::base::message_builder::{AdditionalBuilder, PushError}; +use crate::base::wire::Composer; +use crate::base::{Message, StaticCompressor}; +use crate::net::client::request::{ + ComposeRequest, Error, GetResponse, SendRequest, +}; +use crate::rdata::tsig::Time48; +use crate::tsig::{ClientSequence, ClientTransaction, Key, ValidationError}; + +//------------ Connection ----------------------------------------------------- + +#[derive(Clone)] +/// A connection that TSIG signs requests and verifies upstream responses. +pub struct Connection { + /// Upstream transport to use for requests. + /// + /// This should be the final transport, there should be no further + /// modification to the request before it is sent to the recipient. + upstream: Arc, + + /// The TSIG key to sign with. + /// + /// If None, signing will be skipped. + key: Option, +} + +impl Connection { + /// Create a new TSIG transport with default configuration. + /// + /// Requests will be signed with the given key, if any, then sent via the + /// provided upstream transport. + pub fn new(upstream: Upstream, key: Option) -> Self { + Self { + upstream: Arc::new(upstream), + key, + } + } +} + +//--- SendRequest + +impl SendRequest for Connection +where + CR: ComposeRequest + 'static, + Upstream: SendRequest> + + Send + + Sync + + 'static, + K: Clone + AsRef + Send + Sync + 'static, +{ + fn send_request( + &self, + request_msg: CR, + ) -> Box { + Box::new(Request::::new( + request_msg, + self.key.clone(), + self.upstream.clone(), + )) + } +} + +//------------ Request -------------------------------------------------------- + +/// The state of a request that is executed. +struct Request +where + CR: ComposeRequest, +{ + /// State of the request. + state: RequestState, + + /// The request message. + request_msg: Option, + + /// The key to sign the request with. + /// + /// If None, no signing will be done. + key: Option, + + /// The upstream transport of the connection. + upstream: Arc, +} + +impl Request +where + CR: ComposeRequest, + Upstream: SendRequest> + Send + Sync, + K: Clone + AsRef, + Self: GetResponse, +{ + /// Create a new Request object. + fn new(request_msg: CR, key: Option, upstream: Arc) -> Self { + Self { + state: RequestState::Init, + request_msg: Some(request_msg), + key, + upstream, + } + } + + /// This is the implementation of the get_response method. + /// + /// This function is cancel safe. + async fn get_response_impl(&mut self) -> Result, Error> { + let mut mark_as_complete = false; + + let res = loop { + match &mut self.state { + RequestState::Init => { + let tsig_client = Arc::new(std::sync::Mutex::new(None)); + + let msg = AuthenticatedRequestMessage { + request: self.request_msg.take().unwrap(), + key: self.key.clone(), + signer: tsig_client.clone(), + }; + + trace!("Sending request upstream..."); + let request = self.upstream.send_request(msg); + self.state = + RequestState::GetResponse(request, tsig_client); + continue; + } + + RequestState::GetResponse(request, tsig_client) => { + trace!("Receiving response"); + let response = request.get_response().await; + if self.key.is_some() { + assert!(tsig_client.lock().unwrap().is_some()); + } + + // TSIG validation + match response { + Ok(msg) => { + let mut modifiable_msg = Message::from_octets( + msg.as_slice().to_vec(), + )?; + + let mut client = tsig_client.lock().unwrap(); + if let Some(client) = client.deref_mut() { + client + .answer( + &mut modifiable_msg, + Time48::now(), + ) + .map_err(Error::Authentication)?; + } + + if request.is_stream_complete() { + mark_as_complete = true; + } + + let out_vec = modifiable_msg.into_octets(); + let out_bytes = Bytes::from(out_vec); + let out_msg = + Message::::from_octets(out_bytes)?; + + break Ok(out_msg); + } + + Err(err) => break Err(err), + } + } + + RequestState::Complete => { + break Err(Error::StreamReceiveError); + } + } + }; + + if mark_as_complete { + self.stream_complete()?; + } + + res + } +} + +impl Debug for Request +where + CR: ComposeRequest, + Upstream: SendRequest>, +{ + fn fmt(&self, f: &mut Formatter<'_>) -> Result<(), core::fmt::Error> { + f.debug_struct("Request").finish() + } +} + +impl GetResponse for Request +where + CR: ComposeRequest, + Upstream: SendRequest> + Send + Sync, + K: Clone + AsRef + Send + Sync, +{ + fn get_response( + &mut self, + ) -> Pin< + Box< + dyn Future, Error>> + + Send + + Sync + + '_, + >, + > { + Box::pin(self.get_response_impl()) + } + + fn stream_complete(&mut self) -> Result<(), Error> { + match &mut self.state { + RequestState::Init => { + debug!("Ignoring attempt to complete TSIG stream that hasn't been read from yet."); + } + + RequestState::GetResponse(ref mut request, tsig_client) => { + if let Some(client) = tsig_client.lock().unwrap().take() { + trace!("Completing TSIG sequence"); + client.done().map_err(Error::Authentication)?; + request.stream_complete()?; + } + + self.state = RequestState::Complete; + } + + RequestState::Complete => { + debug!("Ignoring attempt to complete TSIG stream that is already complete."); + } + } + + Ok(()) + } + + fn is_stream_complete(&self) -> bool { + matches!(self.state, RequestState::Complete) + } +} + +//------------ RequestState --------------------------------------------------- + +/// States of the state machine in get_response_impl +enum RequestState { + /// Initial state, prepare the request for signing. + Init, + + /// Wait for a response and verify it. + GetResponse( + Box, + Arc>>>, + ), + + /// The response has been received. + Complete, +} + +//------------ AuthenticatedRequestMessage ------------------------------------ + +/// A wrapper around a [`ComposeRequest`] impl that signs the request. +#[derive(Debug)] +pub struct AuthenticatedRequestMessage +where + CR: Send + Sync, +{ + /// The request to sign. + request: CR, + + /// The key to sign the request with. + /// + /// If None, signing will be skipped. + key: Option, + + /// The TSIG signing client. + /// + /// Used to sign the request and verify the response. + /// + /// If None, signing was skipped because no key was supplied. + signer: Arc>>>, +} + +impl ComposeRequest for AuthenticatedRequestMessage +where + CR: ComposeRequest, + K: Clone + Debug + Send + Sync + AsRef, +{ + /// Writes the message to a provided composer. + /// + /// Use [`to_message()`] instead if you can. This function should only be + /// used to supply the target to write to. This client MUST be the final + /// modifier of the message before it is finished. Modifying the built + /// message using the returned builder could invalidate the TSIG message + /// signature. + /// + /// [`to_message()`]: Self::to_message + fn to_message_builder( + &self, + target: Target, + ) -> Result, CopyRecordsError> { + let mut target = self.request.to_message_builder(target)?; + + if let Some(key) = &self.key { + let client = TsigClient::request( + key.clone(), + &mut target, + Time48::now(), + self.request.is_streaming(), + ) + .unwrap(); + + *self.signer.lock().unwrap() = Some(client); + } else { + trace!("No signing key was configured for this request, nothing to do"); + } + + Ok(target) + } + + fn to_vec(&self) -> Result, Error> { + let msg = self.to_message()?; + Ok(msg.into_octets()) + } + + fn to_message(&self) -> Result>, Error> { + let target = StaticCompressor::new(Vec::new()); + + let builder = self.to_message_builder(target)?; + + let msg = Message::from_octets(builder.finish().into_target()) + .expect( + "Message should be able to parse output from MessageBuilder", + ); + Ok(msg) + } + + fn header(&self) -> &crate::base::Header { + self.request.header() + } + + fn header_mut(&mut self) -> &mut crate::base::Header { + self.request.header_mut() + } + + fn set_udp_payload_size(&mut self, value: u16) { + self.request.set_udp_payload_size(value) + } + + fn set_dnssec_ok(&mut self, value: bool) { + self.request.set_dnssec_ok(value) + } + + fn add_opt( + &mut self, + opt: &impl crate::base::opt::ComposeOptData, + ) -> Result<(), crate::base::opt::LongOptData> { + self.request.add_opt(opt) + } + + fn is_answer(&self, answer: &Message<[u8]>) -> bool { + self.request.is_answer(answer) + } + + fn is_streaming(&self) -> bool { + self.request.is_streaming() + } + + fn dnssec_ok(&self) -> bool { + self.request.dnssec_ok() + } +} + +//------------ TsigClient ----------------------------------------------------- + +/// An asbtraction layer over [`ClientTransaction`] and [`ClientSequence`]. +#[derive(Clone, Debug)] +enum TsigClient { + /// TSIG Client transaction state. + Transaction(ClientTransaction), + + /// TSIG client sequence state. + Sequence(ClientSequence), +} + +impl> TsigClient { + /// Creates a TSIG client for a request. + pub fn request( + key: K, + msg: &mut AdditionalBuilder, + now: Time48, + streaming: bool, + ) -> Result { + let client = if streaming { + Self::Sequence(ClientSequence::request(key, msg, now)?) + } else { + Self::Transaction(ClientTransaction::request(key, msg, now)?) + }; + + Ok(client) + } + + /// Validates an answer. + pub fn answer + ?Sized>( + &mut self, + message: &mut Message, + now: Time48, + ) -> Result<(), ValidationError> { + match self { + TsigClient::Transaction(c) => c.answer(message, now), + TsigClient::Sequence(c) => c.answer(message, now), + } + } + + /// Validates the end of the sequence. + pub fn done(self) -> Result<(), ValidationError> { + match self { + TsigClient::Transaction(_) => Ok(()), + TsigClient::Sequence(c) => c.done(), + } + } +} diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index 89e7dd8c4..4fec901ad 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -103,7 +103,7 @@ pub type KeyName = Name>; /// [`new`]: #method.new /// [`min_mac_len`]: #method.min_mac_len /// [`signing_len`]: #method.signing_len -#[derive(Debug)] +#[derive(Clone, Debug)] pub struct Key { /// The key’s bits and algorithm. key: hmac::Key, From 67f0f33858f4cbe49559c8d83e30d45fb2327793 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 9 Aug 2024 00:24:06 +0200 Subject: [PATCH 021/333] Remove the added Clone derive on tsig::Key, use Arc in the example instead. --- examples/client-transports.rs | 8 +++++--- src/tsig/mod.rs | 2 +- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/examples/client-transports.rs b/examples/client-transports.rs index 3c5950b68..8135aa53b 100644 --- a/examples/client-transports.rs +++ b/examples/client-transports.rs @@ -1,6 +1,7 @@ /// Using the `domain::net::client` module for sending a query. use std::net::{IpAddr, SocketAddr}; use std::str::FromStr; +use std::sync::Arc; use std::time::Duration; use domain::base::MessageBuilder; @@ -316,7 +317,7 @@ where + domain::dep::octseq::Octets + 'static, SR: SendRequest< - tsig::AuthenticatedRequestMessage, Key>, + tsig::AuthenticatedRequestMessage, Arc>, > + Send + Sync + 'static, @@ -327,8 +328,9 @@ where "zlCZbVJPIhobIs1gJNQfrsS3xCxxsR9pMUrGwG8OgG8=", ) .unwrap(); - let key = - Key::new(Algorithm::Sha256, &secret, key_name, None, None).unwrap(); + let key = Arc::new( + Key::new(Algorithm::Sha256, &secret, key_name, None, None).unwrap(), + ); // Create a signing transport. This assumes that the server being // connected to is configured with a key with the same name, algorithm and diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index 4fec901ad..89e7dd8c4 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -103,7 +103,7 @@ pub type KeyName = Name>; /// [`new`]: #method.new /// [`min_mac_len`]: #method.min_mac_len /// [`signing_len`]: #method.signing_len -#[derive(Clone, Debug)] +#[derive(Debug)] pub struct Key { /// The key’s bits and algorithm. key: hmac::Key, From 6e72597ebcc1ec548ba27ca0a9b44fb1b214eef0 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 9 Aug 2024 00:33:36 +0200 Subject: [PATCH 022/333] More coding standards compliance. --- src/net/client/tsig.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index 93ddb34fa..c5be431c4 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -206,6 +206,8 @@ where } } +//--- Debug + impl Debug for Request where CR: ComposeRequest, @@ -216,6 +218,8 @@ where } } +//--- GetResponse + impl GetResponse for Request where CR: ComposeRequest, @@ -305,6 +309,8 @@ where signer: Arc>>>, } +//--- ComposeRequest + impl ComposeRequest for AuthenticatedRequestMessage where CR: ComposeRequest, From a7ca10905ee4005bfb8860c33bd182f57fafcd53 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 19 Aug 2024 13:16:50 +0200 Subject: [PATCH 023/333] Add end-of-stream detection to stream.rs. --- src/net/client/stream.rs | 63 ++++++++++++++++++++++++++++++++++++++-- 1 file changed, 61 insertions(+), 2 deletions(-) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index 91b9b140f..a6b7f0ecb 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -35,10 +35,13 @@ use tracing::trace; use crate::base::message::Message; use crate::base::message_builder::StreamTarget; +use crate::base::name::FlattenInto; use crate::base::opt::{AllOptData, OptRecord, TcpKeepalive}; +use crate::base::{Name, ParsedName, Record, Rtype}; use crate::net::client::request::{ ComposeRequest, Error, GetResponse, SendRequest, }; +use crate::rdata::Soa; use crate::utils::config::DefMinMax; //------------ Configuration Constants ---------------------------------------- @@ -223,7 +226,12 @@ impl Connection { ) -> Result, Error> { let (sender, receiver) = oneshot::channel(); let sender = ReplySender::Single(Some(sender)); - let req = ChanReq { sender, msg }; + let req = ChanReq { + sender, + msg, + eos: None, + eos_seen_count: 0, + }; self.sender.send(req).await.map_err(|_| { // Send error. The receiver is gone, this means that the // connection is closed. @@ -249,6 +257,8 @@ impl Connection { let req = ChanReq { sender: reply_sender, msg, + eos: None, + eos_seen_count: 0, }; let _ = self.sender.send(req).await; @@ -459,6 +469,54 @@ struct ChanReq { /// Sender to send result back to [`Request`] sender: ReplySender, + + /// End of stream marker to check for, when streaming + eos: Option>>, + + /// Number of times eos has been seen + eos_seen_count: usize, +} + +impl ChanReq { + fn is_eos(&mut self, msg: &Message) -> bool { + if self.sender.is_stream() { + if let Ok(mut answer) = msg.answer() { + if let Some(Ok(record)) = answer.next() { + if record.rtype() == Rtype::SOA { + if let Ok(Some(record)) = record.into_record() { + let soa_rec: Record<_, Soa>> = + record; + let soa = soa_rec.data().clone().flatten_into(); + match &self.eos { + Some(expected_soa) => { + let found = &soa == expected_soa; + if found { + self.eos_seen_count += 1; + } + let res = match msg.qtype() { + Some(Rtype::AXFR) => { + self.eos_seen_count == 1 + } + Some(Rtype::IXFR) => { + self.eos_seen_count == 2 + } + _ => false, + }; + return res; + } + None => { + self.eos = Some(soa); + } + } + } + } + } + } + false + } else { + true + } + } } /// A message back to [`Request`] returning a response. @@ -829,6 +887,7 @@ where return; } }; + let is_eos = req.is_eos(&answer); let answer = if req.msg.is_answer(answer.for_slice()) { Ok(answer) } else { @@ -837,7 +896,7 @@ where _ = req.sender.send(answer); // TODO: Discard streaming requests once the stream is complete. - if req.sender.is_stream() { + if req.sender.is_stream() && !is_eos { query_vec.insert(req).unwrap(); } From 46cd1bd1dc4d08137007256197c6bb6fba39cfc6 Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Tue, 20 Aug 2024 14:12:00 +0200 Subject: [PATCH 024/333] Client transport support for AXFR and IXFR. --- examples/client-transports.rs | 9 +- src/base/message.rs | 6 + src/net/client/mod.rs | 14 +- src/net/client/multi_stream.rs | 9 +- src/net/client/request.rs | 377 ++++++++++++- src/net/client/stream.rs | 819 ++++++++++++++++++++++++---- src/net/client/validator.rs | 2 +- src/net/server/tests/integration.rs | 3 +- src/resolv/stub/mod.rs | 4 +- src/stelline/client.rs | 2 +- src/validator/context.rs | 2 +- src/validator/mod.rs | 2 +- tests/net-client-cache.rs | 2 +- tests/net-client.rs | 9 +- 14 files changed, 1104 insertions(+), 156 deletions(-) diff --git a/examples/client-transports.rs b/examples/client-transports.rs index 92705bc09..cea3bce17 100644 --- a/examples/client-transports.rs +++ b/examples/client-transports.rs @@ -8,7 +8,9 @@ use domain::net::client::dgram_stream; use domain::net::client::multi_stream; use domain::net::client::protocol::{TcpConnect, TlsConnect, UdpConnect}; use domain::net::client::redundant; -use domain::net::client::request::{RequestMessage, SendRequest}; +use domain::net::client::request::{ + RequestMessage, RequestMessageMulti, SendRequest, +}; use domain::net::client::stream; use std::net::{IpAddr, SocketAddr}; use std::str::FromStr; @@ -33,7 +35,7 @@ async fn main() { let mut msg = msg.question(); msg.push((Name::vec_from_str("example.com").unwrap(), Rtype::AAAA)) .unwrap(); - let req = RequestMessage::new(msg); + let req = RequestMessage::new(msg).unwrap(); // Destination for UDP and TCP let server_addr = SocketAddr::new(IpAddr::from_str("::1").unwrap(), 53); @@ -234,7 +236,8 @@ async fn main() { } }; - let (tcp, transport) = stream::Connection::new(tcp_conn); + let (tcp, transport) = + stream::Connection::<_, RequestMessageMulti>>::new(tcp_conn); tokio::spawn(async move { transport.run().await; println!("single TCP run terminated"); diff --git a/src/base/message.rs b/src/base/message.rs index b1272e68c..8b13c4ed4 100644 --- a/src/base/message.rs +++ b/src/base/message.rs @@ -439,6 +439,12 @@ impl Message { } } + pub fn is_streaming(&self) -> bool { + self.first_question() + .map(|q| matches!(q.qtype(), Rtype::AXFR | Rtype::IXFR)) + .unwrap_or_default() + } + /// Returns the first question, if there is any. /// /// The method will return `None` both if there are no questions or if diff --git a/src/net/client/mod.rs b/src/net/client/mod.rs index c41b8a6b9..fa6144b67 100644 --- a/src/net/client/mod.rs +++ b/src/net/client/mod.rs @@ -87,7 +87,7 @@ //! tokio::spawn(transport.run()); //! # let req = domain::net::client::request::RequestMessage::new( //! # domain::base::MessageBuilder::new_vec() -//! # ); +//! # ).unwrap(); //! # let mut request = tcp_conn.send_request(req); //! # } //! ``` @@ -100,18 +100,18 @@ //! //! For example: //! ```no_run -//! # use domain::net::client::request::SendRequest; +//! # use domain::net::client::request::{RequestMessageMulti, SendRequest}; //! # use std::net::{IpAddr, SocketAddr}; //! # use std::str::FromStr; //! # async fn _test() { -//! # let (tls_conn, _) = domain::net::client::stream::Connection::new( +//! # let (tls_conn, _) = domain::net::client::stream::Connection::<_, RequestMessageMulti>>::new( //! # domain::net::client::protocol::TcpConnect::new( //! # SocketAddr::new(IpAddr::from_str("::1").unwrap(), 53) //! # ) //! # ); //! # let req = domain::net::client::request::RequestMessage::new( //! # domain::base::MessageBuilder::new_vec() -//! # ); +//! # ).unwrap(); //! let mut request = tls_conn.send_request(req); //! # } //! ``` @@ -128,18 +128,18 @@ //! //! For example: //! ```no_run -//! # use crate::domain::net::client::request::SendRequest; +//! # use crate::domain::net::client::request::{RequestMessageMulti, SendRequest}; //! # use std::net::{IpAddr, SocketAddr}; //! # use std::str::FromStr; //! # async fn _test() { -//! # let (tls_conn, _) = domain::net::client::stream::Connection::new( +//! # let (tls_conn, _) = domain::net::client::stream::Connection::<_, RequestMessageMulti>>::new( //! # domain::net::client::protocol::TcpConnect::new( //! # SocketAddr::new(IpAddr::from_str("::1").unwrap(), 53) //! # ) //! # ); //! # let req = domain::net::client::request::RequestMessage::new( //! # domain::base::MessageBuilder::new_vec() -//! # ); +//! # ).unwrap(); //! # let mut request = tls_conn.send_request(req); //! let reply = request.get_response().await; //! # } diff --git a/src/net/client/multi_stream.rs b/src/net/client/multi_stream.rs index 6016178aa..d0c65c753 100644 --- a/src/net/client/multi_stream.rs +++ b/src/net/client/multi_stream.rs @@ -6,7 +6,7 @@ use crate::base::Message; use crate::net::client::protocol::AsyncConnect; use crate::net::client::request::{ - ComposeRequest, Error, GetResponse, SendRequest, + ComposeRequest, Error, GetResponse, RequestMessageMulti, SendRequest, }; use crate::net::client::stream; use bytes::Bytes; @@ -19,6 +19,7 @@ use std::future::Future; use std::pin::Pin; use std::sync::Arc; use std::time::Duration; +use std::vec::Vec; use tokio::io; use tokio::io::{AsyncRead, AsyncWrite}; use tokio::sync::{mpsc, oneshot}; @@ -197,7 +198,7 @@ enum QueryState { ReceiveConn(oneshot::Receiver>), /// Start a query using the given stream transport. - StartQuery(Arc>), + StartQuery(Arc>>>), /// Get the result of the query. GetResult(stream::Request), @@ -222,7 +223,7 @@ struct ChanRespOk { id: u64, /// The new stream transport to use for sending a request. - conn: Arc>, + conn: Arc>>>, } impl Request { @@ -409,7 +410,7 @@ enum SingleConnState3 { None, /// Current stream transport. - Some(Arc>), + Some(Arc>>>), /// State that deals with an error getting a new octet stream from /// a connection stream. diff --git a/src/net/client/request.rs b/src/net/client/request.rs index dd4f5ddcb..2f79b2dbd 100644 --- a/src/net/client/request.rs +++ b/src/net/client/request.rs @@ -1,13 +1,12 @@ //! Constructing and sending requests. - -use crate::base::iana::Rcode; +use crate::base::iana::{Opcode, Rcode}; use crate::base::message::{CopyRecordsError, ShortMessage}; use crate::base::message_builder::{ - AdditionalBuilder, MessageBuilder, PushError, StaticCompressor, + AdditionalBuilder, MessageBuilder, PushError, }; use crate::base::opt::{ComposeOptData, LongOptData, OptRecord}; use crate::base::wire::{Composer, ParseError}; -use crate::base::{Header, Message, ParsedName, Rtype}; +use crate::base::{Header, Message, ParsedName, Rtype, StaticCompressor}; use crate::rdata::AllRecordData; use bytes::Bytes; use octseq::Octets; @@ -20,6 +19,9 @@ use std::vec::Vec; use std::{error, fmt}; use tracing::trace; +#[cfg(feature = "tsig")] +use crate::tsig; + //------------ ComposeRequest ------------------------------------------------ /// A trait that allows composing a request as a series. @@ -27,8 +29,8 @@ pub trait ComposeRequest: Debug + Send + Sync { /// Appends the final message to a provided composer. fn append_message( &self, - target: &mut Target, - ) -> Result<(), CopyRecordsError>; + target: Target, + ) -> Result, CopyRecordsError>; /// Create a message that captures the recorded changes. fn to_message(&self) -> Result>, Error>; @@ -58,6 +60,53 @@ pub trait ComposeRequest: Debug + Send + Sync { /// Returns whether a message is an answer to the request. fn is_answer(&self, answer: &Message<[u8]>) -> bool; + /// Returns whether a message results in a response stream or not. + fn is_streaming(&self) -> bool; + + /// Return the status of the DNSSEC OK flag. + fn dnssec_ok(&self) -> bool; +} + +//------------ ComposeRequestMulti -------------------------------------------- + +/// A trait that allows composing a request as a series. +pub trait ComposeRequestMulti: Debug + Send + Sync { + /// Appends the final message to a provided composer. + fn append_message( + &self, + target: Target, + ) -> Result, CopyRecordsError>; + + /// Create a message that captures the recorded changes. + fn to_message(&self) -> Result>, Error>; + + /// Create a message that captures the recorded changes and convert to + /// a Vec. + + /// Return a reference to the current Header. + fn header(&self) -> &Header; + + /// Return a reference to a mutable Header to record changes to the header. + fn header_mut(&mut self) -> &mut Header; + + /// Set the UDP payload size. + fn set_udp_payload_size(&mut self, value: u16); + + /// Set the DNSSEC OK flag. + fn set_dnssec_ok(&mut self, value: bool); + + /// Add an EDNS option. + fn add_opt( + &mut self, + opt: &impl ComposeOptData, + ) -> Result<(), LongOptData>; + + /// Returns whether a message is an answer to the request. + fn is_answer(&self, answer: &Message<[u8]>) -> bool; + + /// Returns whether a message results in a response stream or not. + fn is_streaming(&self) -> bool; + /// Return the status of the DNSSEC OK flag. fn dnssec_ok(&self) -> bool; } @@ -76,6 +125,20 @@ pub trait SendRequest { ) -> Box; } +//------------ SendRequestMulti ----------------------------------------------- + +/// Trait for starting a DNS request based on a request composer. +/// +/// In the future, the return type of request should become an associated type. +/// However, the use of 'dyn Request' in redundant currently prevents that. +pub trait SendRequestMulti { + /// Request function that takes a ComposeRequestMulti type. + fn send_request( + &self, + request_msg: CR, + ) -> Box; +} + //------------ GetResponse --------------------------------------------------- /// Trait for getting the result of a DNS query. @@ -98,6 +161,28 @@ pub trait GetResponse: Debug { >; } +//------------ GetResponseMulti ---------------------------------------------- +/// Trait for getting a stream of result of a DNS query. +/// +/// In the future, the return type of get_response should become an associated +/// type. However, too many uses of 'dyn GetResponse' currently prevent that. +#[allow(clippy::type_complexity)] +pub trait GetResponseMulti: Debug { + /// Get the result of a DNS request. + /// + /// This function is intended to be cancel safe. + fn get_response( + &mut self, + ) -> Pin< + Box< + dyn Future>, Error>> + + Send + + Sync + + '_, + >, + >; +} + //------------ RequestMessage ------------------------------------------------ /// Object that implements the ComposeRequest trait for a Message object. @@ -114,15 +199,26 @@ pub struct RequestMessage> { } impl + Debug + Octets> RequestMessage { - /// Create a new BMB object. - pub fn new(msg: impl Into>) -> Self { + /// Create a new RequestMessage object. + pub fn new(msg: impl Into>) -> Result { let msg = msg.into(); + + // On UDP, IXFR results in a single responses, so we need to accept it. + // We can reject AXFR because it always requires support for multiple + // responses. + if msg.header().opcode() == Opcode::QUERY + && msg.first_question().ok_or(Error::FormError)?.qtype() + == Rtype::AXFR + { + return Err(Error::FormError); + } + let header = msg.header(); - Self { + Ok(Self { msg, header, opt: None, - } + }) } /// Returns a mutable reference to the OPT record. @@ -209,12 +305,12 @@ impl + Debug + Octets + Send + Sync> ComposeRequest { fn append_message( &self, - target: &mut Target, - ) -> Result<(), CopyRecordsError> { + target: Target, + ) -> Result, CopyRecordsError> { let target = MessageBuilder::from_target(target) .map_err(|_| CopyRecordsError::Push(PushError::ShortBuf))?; - self.append_message_impl(target)?; - Ok(()) + let builder = self.append_message_impl(target)?; + Ok(builder) } fn to_vec(&self) -> Result, Error> { @@ -290,6 +386,230 @@ impl + Debug + Octets + Send + Sync> ComposeRequest } } + fn is_streaming(&self) -> bool { + self.msg.is_streaming() + } + + fn dnssec_ok(&self) -> bool { + match &self.opt { + None => false, + Some(opt) => opt.dnssec_ok(), + } + } +} + +//------------ RequestMessageMulti -------------------------------------------- + +/// Object that implements the ComposeRequestMulti trait for a Message object. +#[derive(Clone, Debug)] +pub struct RequestMessageMulti +where + Octs: AsRef<[u8]>, +{ + /// Base message. + msg: Message, + + /// New header. + header: Header, + + /// The OPT record to add if required. + opt: Option>>, +} + +impl + Debug + Octets> RequestMessageMulti { + /// Create a new BMB object. + pub fn new(msg: impl Into>) -> Result { + let msg = msg.into(); + + // Only accept the streaming types (IXFR and AXFR). + if !msg.is_streaming() { + return Err(Error::FormError); + } + let header = msg.header(); + Ok(Self { + msg, + header, + opt: None, + }) + } + + /// Returns a mutable reference to the OPT record. + /// + /// Adds one if necessary. + fn opt_mut(&mut self) -> &mut OptRecord> { + self.opt.get_or_insert_with(Default::default) + } + + /// Appends the message to a composer. + fn append_message_impl( + &self, + mut target: MessageBuilder, + ) -> Result, CopyRecordsError> { + let source = &self.msg; + + *target.header_mut() = self.header; + + let source = source.question(); + let mut target = target.question(); + for rr in source { + target.push(rr?)?; + } + let mut source = source.answer()?; + let mut target = target.answer(); + for rr in &mut source { + let rr = rr? + .into_record::>>()? + .expect("record expected"); + target.push(rr)?; + } + + let mut source = + source.next_section()?.expect("section should be present"); + let mut target = target.authority(); + for rr in &mut source { + let rr = rr? + .into_record::>>()? + .expect("record expected"); + target.push(rr)?; + } + + let source = + source.next_section()?.expect("section should be present"); + let mut target = target.additional(); + for rr in source { + let rr = rr?; + if rr.rtype() != Rtype::OPT { + let rr = rr + .into_record::>>()? + .expect("record expected"); + target.push(rr)?; + } + } + + if let Some(opt) = self.opt.as_ref() { + target.push(opt.as_record())?; + } + + Ok(target) + } + + /// Create new message based on the changes to the base message. + fn to_message_impl(&self) -> Result>, Error> { + let target = + MessageBuilder::from_target(StaticCompressor::new(Vec::new())) + .expect("Vec is expected to have enough space"); + + let target = self.append_message_impl(target)?; + + // It would be nice to use .builder() here. But that one deletes all + // sections. We have to resort to .as_builder() which gives a + // reference and then .clone() + let result = target.as_builder().clone(); + let msg = Message::from_octets(result.finish().into_target()).expect( + "Message should be able to parse output from MessageBuilder", + ); + Ok(msg) + } +} + +impl + Debug + Octets + Send + Sync> ComposeRequestMulti + for RequestMessageMulti +{ + fn append_message( + &self, + target: Target, + ) -> Result, CopyRecordsError> { + let target = MessageBuilder::from_target(target) + .map_err(|_| CopyRecordsError::Push(PushError::ShortBuf))?; + let builder = self.append_message_impl(target)?; + Ok(builder) + } + + fn to_message(&self) -> Result>, Error> { + self.to_message_impl() + } + + fn header(&self) -> &Header { + &self.header + } + + fn header_mut(&mut self) -> &mut Header { + &mut self.header + } + + fn set_udp_payload_size(&mut self, value: u16) { + self.opt_mut().set_udp_payload_size(value); + } + + fn set_dnssec_ok(&mut self, value: bool) { + self.opt_mut().set_dnssec_ok(value); + } + + fn add_opt( + &mut self, + opt: &impl ComposeOptData, + ) -> Result<(), LongOptData> { + self.opt_mut().push(opt).map_err(|e| e.unlimited_buf()) + } + + fn is_answer(&self, answer: &Message<[u8]>) -> bool { + let answer_header = answer.header(); + let answer_hcounts = answer.header_counts(); + + // First check qr is set and IDs match. + if !answer_header.qr() || answer_header.id() != self.header.id() { + trace!( + "Wrong QR or ID: QR={}, answer ID={}, self ID={}", + answer_header.qr(), + answer_header.id(), + self.header.id() + ); + return false; + } + + // If the result is an error, then the question section can be empty. + // In that case we require all other sections to be empty as well. + if answer_header.rcode() != Rcode::NOERROR + && answer_hcounts.qdcount() == 0 + && answer_hcounts.ancount() == 0 + && answer_hcounts.nscount() == 0 + && answer_hcounts.arcount() == 0 + { + // We can accept this as a valid reply. + return true; + } + + // Now the question section in the reply has to be the same as in the + // query, except in the case of an AXFR subsequent response: + // + // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2 + // 2.2. AXFR Response + // "The AXFR server MUST copy the Question section from the + // corresponding AXFR query message into the first response + // message's Question section. For subsequent messages, it MAY do + // the same or leave the Question section empty." + if self.msg.qtype() == Some(Rtype::AXFR) + && answer_hcounts.qdcount() == 0 + { + true + } else if answer_hcounts.qdcount() + != self.msg.header_counts().qdcount() + { + trace!("Wrong QD count"); + false + } else { + let res = answer.question() == self.msg.for_slice().question(); + if !res { + trace!("Wrong question"); + } + res + } + } + + fn is_streaming(&self) -> bool { + self.msg.is_streaming() + } + fn dnssec_ok(&self) -> bool { match &self.opt { None => false, @@ -318,6 +638,9 @@ pub enum Error { /// Underlying transport not found in redundant connection RedundantTransportNotFound, + /// The message violated some constraints. + FormError, + /// Octet sequence too short to be a valid DNS message. ShortMessage, @@ -355,6 +678,14 @@ pub enum Error { /// An error happened in the datagram transport. Dgram(Arc), + #[cfg(feature = "unstable-server-transport")] + /// Zone write failed. + ZoneWrite, + + #[cfg(feature = "tsig")] + /// TSIG authentication failed. + Authentication(tsig::ValidationError), + #[cfg(feature = "unstable-validator")] /// An error happened during DNSSEC validation. Validation(crate::validator::context::Error), @@ -407,6 +738,9 @@ impl fmt::Display for Error { Error::ShortMessage => { write!(f, "octet sequence to short to be a valid message") } + Error::FormError => { + write!(f, "message violates a constraint") + } Error::StreamLongMessage => { write!(f, "message too long for stream transport") } @@ -436,6 +770,13 @@ impl fmt::Display for Error { write!(f, "no transport available") } Error::Dgram(err) => fmt::Display::fmt(err, f), + + #[cfg(feature = "unstable-server-transport")] + Error::ZoneWrite => write!(f, "zone write error"), + + #[cfg(feature = "tsig")] + Error::Authentication(err) => fmt::Display::fmt(err, f), + #[cfg(feature = "unstable-validator")] Error::Validation(_) => { write!(f, "error validating response") @@ -462,6 +803,7 @@ impl error::Error for Error { Error::MessageParseError => None, Error::RedundantTransportNotFound => None, Error::ShortMessage => None, + Error::FormError => None, Error::StreamLongMessage => None, Error::StreamIdleTimeout => None, Error::StreamReceiveError => None, @@ -473,6 +815,13 @@ impl error::Error for Error { Error::WrongReplyForQuery => None, Error::NoTransportAvailable => None, Error::Dgram(err) => Some(err), + + #[cfg(feature = "unstable-server-transport")] + Error::ZoneWrite => None, + + #[cfg(feature = "tsig")] + Error::Authentication(err) => Some(err), + #[cfg(feature = "unstable-validator")] Error::Validation(err) => Some(err), } diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index 2c82fecc0..e04153e49 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -14,18 +14,23 @@ // - request timeout // - create new connection after end/failure of previous one +use super::request::{ + ComposeRequest, ComposeRequestMulti, Error, GetResponse, + GetResponseMulti, SendRequest, SendRequestMulti, +}; +use crate::base::iana::{Rcode, Rtype}; use crate::base::message::Message; use crate::base::message_builder::StreamTarget; use crate::base::opt::{AllOptData, OptRecord, TcpKeepalive}; -use crate::net::client::request::{ - ComposeRequest, Error, GetResponse, SendRequest, -}; +use crate::base::{ParsedName, Serial}; +use crate::rdata::AllRecordData; use crate::utils::config::DefMinMax; use bytes::{Bytes, BytesMut}; use core::cmp; use octseq::Octets; use std::boxed::Box; use std::fmt::Debug; +use std::future::ready; use std::future::Future; use std::pin::Pin; use std::sync::Arc; @@ -34,6 +39,7 @@ use std::vec::Vec; use tokio::io::{AsyncRead, AsyncReadExt, AsyncWrite, AsyncWriteExt}; use tokio::sync::{mpsc, oneshot}; use tokio::time::sleep; +use tracing::trace; //------------ Configuration Constants ---------------------------------------- @@ -77,9 +83,15 @@ const READ_REPLY_CHAN_CAP: usize = 8; /// Configuration for a stream transport connection. #[derive(Clone, Debug)] pub struct Config { - /// Response timeout. + /// Response timeout currently in effect. response_timeout: Duration, + /// Single response timeout. + single_response_timeout: Duration, + + /// Streaming response timeout. + streaming_response_timeout: Duration, + /// Default idle timeout. /// /// This value is used if the other side does not send a TcpKeepalive @@ -102,11 +114,53 @@ impl Config { } /// Sets the response timeout. + /// + /// For requests where ComposeRequest::is_streaming() returns true see + /// set_streaming_response_timeout() instead. + /// + /// Excessive values are quietly trimmed. + // + // XXX Maybe that’s wrong and we should rather return an error? pub fn set_response_timeout(&mut self, timeout: Duration) { self.response_timeout = RESPONSE_TIMEOUT.limit(timeout); + self.streaming_response_timeout = self.response_timeout; + } + + /// Returns the streaming response timeout. + pub fn streaming_response_timeout(&self) -> Duration { + self.streaming_response_timeout + } + + /// Sets the streaming response timeout. + /// + /// Only used for requests where ComposeRequest::is_streaming() returns + /// true as it is typically desirable that such response streams be + /// allowed to complete even if the individual responses arrive very + /// slowly. + /// + /// Excessive values are quietly trimmed. + pub fn set_streaming_response_timeout(&mut self, timeout: Duration) { + self.streaming_response_timeout = RESPONSE_TIMEOUT.limit(timeout); + } + + /// Returns the initial idle timeout, if set. + pub fn idle_timeout(&self) -> Duration { + self.idle_timeout } - /// Sets the idle timeout. + /// Sets the initial idle timeout. + /// + /// By default the stream is immediately closed if there are no pending + /// requests or responses. + /// + /// Set this to allow requests to be sent in sequence with delays between + /// such as a SOA query followed by AXFR for more efficient use of the + /// stream per RFC 9103. + /// + /// Note: May be overridden by an RFC 7828 edns-tcp-keepalive timeout + /// received from a server. + /// + /// Excessive values are quietly trimmed. pub fn set_idle_timeout(&mut self, timeout: Duration) { self.idle_timeout = IDLE_TIMEOUT.limit(timeout) } @@ -116,6 +170,8 @@ impl Default for Config { fn default() -> Self { Self { response_timeout: RESPONSE_TIMEOUT.default(), + single_response_timeout: RESPONSE_TIMEOUT.default(), + streaming_response_timeout: RESPONSE_TIMEOUT.default(), idle_timeout: IDLE_TIMEOUT.default(), } } @@ -125,19 +181,21 @@ impl Default for Config { /// A connection to a single stream transport. #[derive(Debug)] -pub struct Connection { +pub struct Connection { /// The sender half of the request channel. - sender: mpsc::Sender>, + sender: mpsc::Sender>, } -impl Connection { +impl Connection { /// Creates a new stream transport with default configuration. /// /// Returns a connection and a future that drives the transport using /// the provided stream. This future needs to be run while any queries /// are active. This is most easly achieved by spawning it into a runtime. /// It terminates when the last connection is dropped. - pub fn new(stream: Stream) -> (Self, Transport) { + pub fn new( + stream: Stream, + ) -> (Self, Transport) { Self::with_config(stream, Default::default()) } @@ -150,13 +208,17 @@ impl Connection { pub fn with_config( stream: Stream, config: Config, - ) -> (Self, Transport) { + ) -> (Self, Transport) { let (sender, transport) = Transport::new(stream, config); (Self { sender }, transport) } } -impl Connection { +impl Connection +where + Req: ComposeRequest + 'static, + ReqMulti: ComposeRequestMulti + 'static, +{ /// Start a DNS request. /// /// This function takes a precomposed message as a parameter and @@ -166,24 +228,77 @@ impl Connection { msg: Req, ) -> Result, Error> { let (sender, receiver) = oneshot::channel(); + let sender = ReplySender::Single(Some(sender)); + let msg = ReqSingleMulti::Single(msg); let req = ChanReq { sender, msg }; self.sender.send(req).await.map_err(|_| { // Send error. The receiver is gone, this means that the // connection is closed. + println!("handle_request_impl: returning ConnectionClosed"); Error::ConnectionClosed })?; receiver.await.map_err(|_| Error::StreamReceiveError)? } + /// TODO: Document me. + async fn handle_streaming_request_impl( + self, + msg: ReqMulti, + sender: mpsc::Sender>, Error>>, + ) -> Result<(), Error> { + let reply_sender = ReplySender::Stream(sender); + let msg = ReqSingleMulti::Multi(msg); + let req = ChanReq { + sender: reply_sender, + msg, + }; + self.sender.send(req).await.map_err(|e| { + // Send error. The receiver is gone, this means that the + // connection is closed. +println!("handle_streaming_request_impl: send error {e:?}"); +println!("handle_streaming_request_impl: returning ConnectionClosed (1)"); + Error::ConnectionClosed + })?; + Ok(()) + } + /// Returns a request handler for this connection. pub fn get_request(&self, request_msg: Req) -> Request { - Request { - fut: Box::pin(self.clone().handle_request_impl(request_msg)), + if request_msg.is_streaming() { + Request { + fut: Box::pin(ready(Err(Error::FormError))), + } + } else { + Request { + fut: Box::pin(self.clone().handle_request_impl(request_msg)), + } + } + } + + /// TODO + pub fn get_streaming_request( + &self, + request_msg: ReqMulti, + ) -> RequestMulti { + let (sender, receiver) = mpsc::channel(DEF_CHAN_CAP); + if !request_msg.is_streaming() { + RequestMulti { + stream: receiver, + fut: Some(Box::pin(ready(Err(Error::FormError)))), + } + } else { + RequestMulti { + stream: receiver, + fut: Some(Box::pin( + self.clone() + .handle_streaming_request_impl(request_msg, sender), + )), + } } } } -impl Clone for Connection { +impl Clone for Connection { fn clone(&self) -> Self { Self { sender: self.sender.clone(), @@ -191,8 +306,10 @@ impl Clone for Connection { } } -impl SendRequest - for Connection +impl SendRequest for Connection +where + Req: ComposeRequest + 'static, + ReqMulti: ComposeRequestMulti + Debug + Send + Sync + 'static, { fn send_request( &self, @@ -202,6 +319,19 @@ impl SendRequest } } +impl SendRequestMulti for Connection +where + Req: ComposeRequest + Debug + Send + Sync + 'static, + ReqMulti: ComposeRequestMulti + 'static, +{ + fn send_request( + &self, + request_msg: ReqMulti, + ) -> Box { + Box::new(self.get_streaming_request(request_msg)) + } +} + //------------ Request ------------------------------------------------------- /// An active request. @@ -215,7 +345,12 @@ pub struct Request { impl Request { /// Async function that waits for the future stored in Request to complete. async fn get_response_impl(&mut self) -> Result, Error> { - (&mut self.fut).await + println!("in get_response_impl"); + let res = (&mut self.fut).await; + + println!("get_response_impl: got res {res:?}"); + + res } } @@ -242,34 +377,147 @@ impl Debug for Request { } } +//------------ RequestMulti -------------------------------------------------- + +/// An active request. +pub struct RequestMulti { + /// TODO + stream: mpsc::Receiver>, Error>>, + + /// The underlying future. + #[allow(clippy::type_complexity)] + fut: Option< + Pin> + Send + Sync>>, + >, +} + +impl RequestMulti { + /// Async function that waits for the future stored in Request to complete. + async fn get_response_impl( + &mut self, + ) -> Result>, Error> { + println!("in get_response_impl(RequestMulti)"); + if self.fut.is_some() { + let fut = self.fut.take().expect("Some expected"); + fut.await?; + } + + // Fetch from the stream + println!("get_response_impl(RequestMulti): before stream.recv"); + let res = self + .stream + .recv() + .await + .ok_or(Error::ConnectionClosed) + .map_err(|_| Error::ConnectionClosed)?; + + println!("get_response_impl(RequestMulti): res {res:?}"); + + res + } +} + +impl GetResponseMulti for RequestMulti { + fn get_response( + &mut self, + ) -> Pin< + Box< + dyn Future>, Error>> + + Send + + Sync + + '_, + >, + > { + let fut = self.get_response_impl(); + Box::pin(fut) + } +} + +impl Debug for RequestMulti { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + f.debug_struct("Request") + .field("fut", &format_args!("_")) + .finish() + } +} + //------------ Transport ----------------------------------------------------- /// The underlying machinery of a stream transport. #[derive(Debug)] -pub struct Transport { - /// The stream socket towards the remove end. +pub struct Transport { + /// The stream socket towards the remote end. stream: Stream, /// Transport configuration. config: Config, /// The receiver half of request channel. - receiver: mpsc::Receiver>, + receiver: mpsc::Receiver>, +} + +/// This is the type of sender in [ChanReq]. +#[derive(Debug)] +enum ReplySender { + /// TODO + Single(Option>), + + /// TODO + Stream(mpsc::Sender>, Error>>), +} + +impl ReplySender { + /// TODO + async fn send(&mut self, resp: ChanResp) -> Result<(), ()> { + match self { + ReplySender::Single(sender) => match sender.take() { + Some(sender) => sender.send(resp).map_err(|_| ()), + None => Err(()), + }, + ReplySender::Stream(sender) => { + sender.send(resp.map(Some)).await.map_err(|_| ()) + } + } + } + + /// Send EOF on a response stream. + async fn send_eof(&mut self) -> Result<(), ()> { + match self { + ReplySender::Single(_) => { + panic!("cannot send EOF for Single"); + } + ReplySender::Stream(sender) => { + sender.send(Ok(None)).await.map_err(|_| ()) + } + } + } + + /// TODO + pub fn is_stream(&self) -> bool { + matches!(self, Self::Stream(_)) + } +} + +#[derive(Debug)] +/// Enum that can either store a request for a single response or one for +/// multiple responses. +enum ReqSingleMulti { + /// Single response request. + Single(Req), + /// Multi-response request. + Multi(ReqMulti), } /// A message from a [`Request`] to start a new request. #[derive(Debug)] -struct ChanReq { +struct ChanReq { /// DNS request message - msg: Req, + msg: ReqSingleMulti, /// Sender to send result back to [`Request`] sender: ReplySender, } -/// This is the type of sender in [ChanReq]. -type ReplySender = oneshot::Sender; - /// A message back to [`Request`] returning a response. type ChanResp = Result, Error>; @@ -323,12 +571,71 @@ enum ConnState { WriteError(Error), } -impl Transport { +//--- Display +impl std::fmt::Display for ConnState { + fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { + match self { + ConnState::Active(instant) => f.write_fmt(format_args!( + "Active (since {}s ago)", + instant + .map(|v| Instant::now().duration_since(v).as_secs()) + .unwrap_or_default() + )), + ConnState::Idle(instant) => f.write_fmt(format_args!( + "Idle (since {}s ago)", + Instant::now().duration_since(*instant).as_secs() + )), + ConnState::IdleTimeout => f.write_str("IdleTimeout"), + ConnState::ReadError(err) => { + f.write_fmt(format_args!("ReadError: {err}")) + } + ConnState::ReadTimeout => f.write_str("ReadTimeout"), + ConnState::WriteError(err) => { + f.write_fmt(format_args!("WriteError: {err}")) + } + } + } +} + +#[derive(Debug)] +/// State of an AXFR or IXFR responses stream for detecting the end of the +/// stream. +enum XFRState { + /// Start of AXFR. + AXFRInit, + /// After the first SOA record has been encountered. + AXFRFirstSoa(Serial), + /// Start of IXFR. + IXFRInit, + /// After the first SOA record has been encountered. + IXFRFirstSoa(Serial), + /// After the first SOA record in a diff section has been encountered. + IXFRFirstDiffSoa(Serial), + /// After the second SOA record in a diff section has been encountered. + IXFRSecondDiffSoa(Serial), + /// End of the stream has been found. + Done, + /// An error has occured. + Error, +} + +/* +#[derive(Debug)] +struct XFRData { + /// State needed for AXFR and IXFR. + state: XFRState, + + /// + serial: Serial, +} +*/ + +impl Transport { /// Creates a new transport. fn new( stream: Stream, config: Config, - ) -> (mpsc::Sender>, Self) { + ) -> (mpsc::Sender>, Self) { let (sender, receiver) = mpsc::channel(DEF_CHAN_CAP); ( sender, @@ -341,13 +648,15 @@ impl Transport { } } -impl Transport +impl Transport where Stream: AsyncRead + AsyncWrite, Req: ComposeRequest, + ReqMulti: ComposeRequestMulti, { /// Run the transport machinery. pub async fn run(mut self) { + println!("in run"); let (reply_sender, mut reply_receiver) = mpsc::channel::>(READ_REPLY_CHAN_CAP); @@ -361,7 +670,8 @@ where idle_timeout: self.config.idle_timeout, send_keepalive: true, }; - let mut query_vec = Queries::new(); + let mut query_vec = + Queries::<(ChanReq, Option)>::new(); let mut reqmsg: Option> = None; let mut reqmsg_offset = 0; @@ -427,75 +737,83 @@ where }; tokio::select! { - biased; - res = &mut reader_fut => { - match res { - Ok(_) => - // The reader should not - // terminate without - // error. - panic!("reader terminated"), + biased; + res = &mut reader_fut => { + match res { + Ok(_) => + // The reader should not + // terminate without + // error. + panic!("reader terminated"), + Err(error) => { + Self::error(error.clone(), &mut query_vec); + status.state = ConnState::ReadError(error); + // Reader failed. Break + // out of loop and + // shut down + break + } + } + } + opt_answer = reply_receiver.recv() => { + let answer = opt_answer.expect("reader died?"); + // Check for a edns-tcp-keepalive option + let opt_record = answer.opt(); + if let Some(ref opts) = opt_record { + Self::handle_opts(opts, + &mut status); + }; + drop(opt_record); + Self::demux_reply(answer, &mut status, &mut query_vec).await; + } + res = write_stream.write(&msg[reqmsg_offset..]), + if do_write => { + match res { Err(error) => { + let error = + Error::StreamWriteError(Arc::new(error)); Self::error(error.clone(), &mut query_vec); - status.state = ConnState::ReadError(error); - // Reader failed. Break - // out of loop and - // shut down - break + status.state = + ConnState::WriteError(error); + break; } - } - } - opt_answer = reply_receiver.recv() => { - let answer = opt_answer.expect("reader died?"); - // Check for a edns-tcp-keepalive option - let opt_record = answer.opt(); - if let Some(ref opts) = opt_record { - Self::handle_opts(opts, - &mut status); - }; - drop(opt_record); - Self::demux_reply(answer, &mut status, &mut query_vec); - } - res = write_stream.write(&msg[reqmsg_offset..]), - if do_write => { - match res { - Err(error) => { - let error = - Error::StreamWriteError(Arc::new(error)); - Self::error(error.clone(), &mut query_vec); - status.state = - ConnState::WriteError(error); - break; - } - Ok(len) => { - reqmsg_offset += len; - if reqmsg_offset >= msg.len() { - reqmsg = None; - reqmsg_offset = 0; - } - } - } - } - res = recv_fut, if !do_write => { - match res { - Some(req) => { - Self::insert_req( - req, &mut status, &mut reqmsg, &mut query_vec - ) + Ok(len) => { + reqmsg_offset += len; + if reqmsg_offset >= msg.len() { + reqmsg = None; + reqmsg_offset = 0; + } } - None => { - // All references to the connection object have - // been dropped. Shutdown. - break; } - } - } - _ = sleep_fut => { - // Timeout expired, just - // continue with the loop - } + } + res = recv_fut, if !do_write => { + match res { + Some(req) => { + if req.sender.is_stream() { + self.config.response_timeout = + self.config.streaming_response_timeout; + } else { + self.config.response_timeout = + self.config.single_response_timeout; + } + Self::insert_req( + req, &mut status, &mut reqmsg, &mut query_vec + ); + } + None => { + // All references to the connection object have + // been dropped. Shutdown. + println!("None from recv_fut"); + break; + } + } + } + _ = sleep_fut => { + // Timeout expired, just + // continue with the loop + } - } + } // Check if the connection is idle match status.state { @@ -511,7 +829,10 @@ where } } + trace!("Closing TCP connecting in state: {}", status.state); + // Send FIN + println!("shutdown stream"); _ = write_stream.shutdown().await; } @@ -583,11 +904,14 @@ where } /// Reports an error to all outstanding queries. - fn error(error: Error, query_vec: &mut Queries>) { + fn error( + error: Error, + query_vec: &mut Queries<(ChanReq, Option)>, + ) { // Update all requests that are in progress. Don't wait for // any reply that may be on its way. - for item in query_vec.drain() { - _ = item.sender.send(Err(error.clone())); + for (mut req, _) in query_vec.drain() { + _ = req.sender.send(Err(error.clone())); } } @@ -612,29 +936,51 @@ where /// /// In addition, the status is updated to IdleTimeout or Idle if there /// are no remaining pending requests. - fn demux_reply( + async fn demux_reply( answer: Message, status: &mut Status, - query_vec: &mut Queries>, + query_vec: &mut Queries<(ChanReq, Option)>, ) { // We got an answer, reset the timer status.state = ConnState::Active(Some(Instant::now())); // Get the correct query and send it the reply. - let req = match query_vec.try_remove(answer.header().id()) { - Some(req) => req, - None => { - // No query with this ID. We should - // mark the connection as broken - return; + let (mut req, mut opt_xfr_data) = + match query_vec.try_remove(answer.header().id()) { + Some(req) => req, + None => { + // No query with this ID. We should + // mark the connection as broken + return; + } + }; + let mut send_eof = false; + let answer = if match &req.msg { + ReqSingleMulti::Single(msg) => msg.is_answer(answer.for_slice()), + ReqSingleMulti::Multi(msg) => { + let xfr_data = + opt_xfr_data.expect("xfr_data should be present"); + let (eof, xfr_data, is_answer) = + check_stream(msg, xfr_data, &answer); + send_eof = eof; + opt_xfr_data = Some(xfr_data); + is_answer } - }; - let answer = if req.msg.is_answer(answer.for_slice()) { + } { Ok(answer) } else { Err(Error::WrongReplyForQuery) }; - _ = req.sender.send(answer); + _ = req.sender.send(answer).await; + + // TODO: Discard streaming requests once the stream is complete. + if req.sender.is_stream() { + if send_eof { + _ = req.sender.send_eof().await; + } else { + query_vec.insert((req, opt_xfr_data)).unwrap(); + } + } if query_vec.is_empty() { // Clear the activity timer. There is no need to do @@ -660,10 +1006,10 @@ where /// idle. Addend a edns-tcp-keepalive option if needed. // Note: maybe reqmsg should be a return value. fn insert_req( - req: ChanReq, + mut req: ChanReq, status: &mut Status, reqmsg: &mut Option>, - query_vec: &mut Queries>, + query_vec: &mut Queries<(ChanReq, Option)>, ) { match &status.state { ConnState::Active(timer) => { @@ -695,12 +1041,38 @@ where } } + let xfr_data = match &req.msg { + ReqSingleMulti::Single(_) => None, + ReqSingleMulti::Multi(msg) => { + let qtype = match msg.to_message().and_then(|m| { + m.sole_question() + .map_err(|_| Error::MessageParseError) + .map(|q| q.qtype()) + }) { + Ok(msg) => msg, + Err(e) => { + _ = req.sender.send(Err(e)); + return; + } + }; + if qtype == Rtype::AXFR { + Some(XFRState::AXFRInit) + } else if qtype == Rtype::IXFR { + Some(XFRState::IXFRInit) + } else { + // Stream requests should be either AXFR or IXFR. + _ = req.sender.send(Err(Error::FormError)); + return; + } + } + }; + // Note that insert may fail if there are too many - // outstanding queires. First call insert before checking + // outstanding queries. First call insert before checking // send_keepalive. - let (index, req) = match query_vec.insert(req) { + let (index, (req, _)) = match query_vec.insert((req, xfr_data)) { Ok(res) => res, - Err(req) => { + Err((mut req, _)) => { // Send an appropriate error and return. _ = req .sender @@ -717,11 +1089,22 @@ where // nature of its use of sequence numbers, is far more // resilient against forgery by third parties." - let hdr = req.msg.header_mut(); + let hdr = match &mut req.msg { + ReqSingleMulti::Single(msg) => msg.header_mut(), + ReqSingleMulti::Multi(msg) => msg.header_mut(), + }; hdr.set_id(index); if status.send_keepalive - && req.msg.add_opt(&TcpKeepalive::new(None)).is_ok() + && match &mut req.msg { + ReqSingleMulti::Single(msg) => { + msg.add_opt(&TcpKeepalive::new(None)).is_ok() + } + // Do we need to set TcpKeepalive for XFR? + ReqSingleMulti::Multi(msg) => { + msg.add_opt(&TcpKeepalive::new(None)).is_ok() + } + } { status.send_keepalive = false; } @@ -732,7 +1115,7 @@ where } Err(err) => { // Take the sender out again and return the error. - if let Some(req) = query_vec.try_remove(index) { + if let Some((mut req, _)) = query_vec.try_remove(index) { _ = req.sender.send(Err(err)); } } @@ -748,12 +1131,210 @@ where } /// Convert the query message to a vector. - fn convert_query(msg: &Req) -> Result, Error> { - let mut target = StreamTarget::new_vec(); - msg.append_message(&mut target) - .map_err(|_| Error::StreamLongMessage)?; - Ok(target.into_target()) + fn convert_query( + msg: &ReqSingleMulti, + ) -> Result, Error> { + match msg { + ReqSingleMulti::Single(msg) => { + let mut target = StreamTarget::new_vec(); + msg.append_message(&mut target) + .map_err(|_| Error::StreamLongMessage)?; + Ok(target.into_target()) + } + ReqSingleMulti::Multi(msg) => { + let target = StreamTarget::new_vec(); + let target = msg + .append_message(target) + .map_err(|_| Error::StreamLongMessage)?; + Ok(target.finish().into_target()) + } + } + } +} + +/// Upstate the response stream state based on a response message. +fn check_stream( + msg: &CRM, + mut xfr_state: XFRState, + answer: &Message, +) -> (bool, XFRState, bool) +where + CRM: ComposeRequestMulti, +{ + // First check if the reply matches the request. + // RFC 5936, Section 2.2.2: + // "In the first response message, this section MUST be copied from the + // query. In subsequent messages, this section MAY be copied from the + // query, or it MAY be empty. However, in an error response message + // (see Section 2.2), this section MUST be copied as well." + match xfr_state { + XFRState::AXFRInit | XFRState::IXFRInit => { + if !msg.is_answer(answer.for_slice()) { + xfr_state = XFRState::Error; + // If we detect an error, then keep the stream open. We are + // likely out of sync with respect to the sender. + println!("check_stream: line {}", line!()); + return (false, xfr_state, false); + } + } + XFRState::AXFRFirstSoa(_) + | XFRState::IXFRFirstSoa(_) + | XFRState::IXFRFirstDiffSoa(_) + | XFRState::IXFRSecondDiffSoa(_) => + // No need to check anything. + {} + XFRState::Done => { + // We should not be here. Switch to error state. + xfr_state = XFRState::Error; + return (false, xfr_state, false); + } + XFRState::Error => + // Keep the stream open. + { + return (false, xfr_state, false) + } + } + + // Then check if the reply status an error. + if answer.header().rcode() != Rcode::NOERROR { + // Also check if this answers the question. + if !msg.is_answer(answer.for_slice()) { + xfr_state = XFRState::Error; + // If we detect an error, then keep the stream open. We are + // likely out of sync with respect to the sender. + return (false, xfr_state, false); + } + return (true, xfr_state, true); + } + + let ans_sec = answer.answer().unwrap(); + for rr in + ans_sec.into_records::>>() + { + println!("found rr {rr:?}"); + let rr = rr.unwrap(); + match xfr_state { + XFRState::AXFRInit => { + // The first record has to be a SOA record. + if let AllRecordData::Soa(soa) = rr.data() { + xfr_state = XFRState::AXFRFirstSoa(soa.serial()); + continue; + } + // Bad data. Switch to error status. + xfr_state = XFRState::Error; + return (false, xfr_state, false); + } + XFRState::AXFRFirstSoa(serial) => { + // Find the SOA at the end. + if let AllRecordData::Soa(soa) = rr.data() { + if serial == soa.serial() { + // We found a match. + xfr_state = XFRState::Done; + continue; + } + + // Serial does not match. Move to error state. + xfr_state = XFRState::Error; + return (false, xfr_state, false); + } + + // Any other record, just continue. + } + XFRState::IXFRInit => { + // The first record has to be a SOA record. + if let AllRecordData::Soa(soa) = rr.data() { + xfr_state = XFRState::IXFRFirstSoa(soa.serial()); + continue; + } + // Bad data. Switch to error status. + xfr_state = XFRState::Error; + return (false, xfr_state, false); + } + XFRState::IXFRFirstSoa(serial) => { + // We have three possibilities: + // 1) The record is not a SOA. In that case the format is AXFR. + // 2) The record is a SOA and the serial is not the current + // serial. That is expected for an IXFR format. Move to + // IXFRFirstDiffSoa. + // 3) The record is a SOA and the serial is equal to the + // current serial. Treat this as a strange empty AXFR. + if let AllRecordData::Soa(soa) = rr.data() { + if serial == soa.serial() { + // We found a match. + xfr_state = XFRState::Done; + continue; + } + + xfr_state = XFRState::IXFRFirstDiffSoa(serial); + continue; + } + + // Any other record, move to AXFRFirstSoa. + xfr_state = XFRState::AXFRFirstSoa(serial); + } + XFRState::IXFRFirstDiffSoa(serial) => { + // Move to IXFRSecondDiffSoa if the record is a SOA record, + // otherwise stay in the current state. + if let AllRecordData::Soa(_) = rr.data() { + xfr_state = XFRState::IXFRSecondDiffSoa(serial); + continue; + } + + // Any other record, just continue. + } + XFRState::IXFRSecondDiffSoa(serial) => { + // Move to Done if the record is a SOA record and the + // serial is the one from the first SOA record, move to + // IXFRFirstDiffSoa for any other SOA record and + // otherwise stay in the current state. + if let AllRecordData::Soa(soa) = rr.data() { + if serial == soa.serial() { + // We found a match. + xfr_state = XFRState::Done; + continue; + } + + xfr_state = XFRState::IXFRFirstDiffSoa(serial); + continue; + } + + // Any other record, just continue. + } + XFRState::Done => { + // We got a record after we are done. Switch to error state. + xfr_state = XFRState::Error; + return (false, xfr_state, false); + } + XFRState::Error => panic!("should not be here"), + } + } + + // Check the final state. + match xfr_state { + XFRState::AXFRInit | XFRState::IXFRInit => { + // Still in one of the init state. So the data section was empty. + // Switch to error state. + xfr_state = XFRState::Error; + return (false, xfr_state, false); + } + XFRState::AXFRFirstSoa(_) + | XFRState::IXFRFirstDiffSoa(_) + | XFRState::IXFRSecondDiffSoa(_) => + // Just continue. + {} + XFRState::IXFRFirstSoa(_) => { + // We are still in IXFRFirstSoa. Assume the other side doesn't + // have anything more to say. We could check the SOA serial in + // the request. Just assume that we are done. + xfr_state = XFRState::Done; + return (true, xfr_state, true); + } + XFRState::Done => return (true, xfr_state, true), + XFRState::Error => panic!("should not be here"), } + + // (eof, xfr_data, is_answer) + (false, xfr_state, true) } //------------ Queries ------------------------------------------------------- diff --git a/src/net/client/validator.rs b/src/net/client/validator.rs index 1c22e19fa..838da3f09 100644 --- a/src/net/client/validator.rs +++ b/src/net/client/validator.rs @@ -70,7 +70,7 @@ //! # let mut msg = msg.question(); //! # msg.push((Name::vec_from_str("example.com").unwrap(), Rtype::AAAA)) //! # .unwrap(); -//! let req = RequestMessage::new(msg); +//! let req = RequestMessage::new(msg).unwrap(); //! //! let ta = TrustAnchors::from_u8(b". 172800 IN DNSKEY 257 3 8 AwEAAaz/tAm8yTn4Mfeh5eyI96WSVexTBAvkMgJzkKTOiW1vkIbzxeF3+/4RgWOq7HrxRixHlFlExOLAJr5emLvN7SWXgnLh4+B5xQlNVz8Og8kvArMtNROxVQuCaSnIDdD5LKyWbRd2n9WGe2R8PzgCmr3EgVLrjyBxWezF0jLHwVN8efS3rCj/EWgvIWgb9tarpVUDK/b58Da+sqqls3eNbuv7pr+eoZG+SrDK6nWeL3c6H5Apxz7LjVc1uTIdsIXxuOLYA4/ilBmSVIzuDWfdRUfhHdY6+cn8HFRm+2hM8AnXGXws9555KrUB5qihylGa8subX2Nn6UwNR1AkUTV74bU= ;{id = 20326 (ksk), size = 2048b} ;;state=2 [ VALID ] ;;count=0 ;;lastchange=1683463064 ;;Sun May 7 12:37:44 2023").unwrap(); //! let vc = ValidationContext::new(ta, udptcp_conn.clone()); diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index bf6af2a96..9f19a8e26 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -17,6 +17,7 @@ use crate::base::name::{Name, ToName}; use crate::base::net::IpAddr; use crate::base::wire::Composer; use crate::net::client::{dgram, stream}; +use crate::net::client::request::RequestMessageMulti; use crate::net::server; use crate::net::server::buf::VecBufSource; use crate::net::server::dgram::DgramServer; @@ -193,7 +194,7 @@ fn mk_client_factory( move |source_addr| { let stream = stream_server_conn .connect(Some(SocketAddr::new(*source_addr, 0))); - let (conn, transport) = stream::Connection::new(stream); + let (conn, transport) = stream::Connection::<_, RequestMessageMulti>>::new(stream); tokio::spawn(transport.run()); Box::new(conn) }, diff --git a/src/resolv/stub/mod.rs b/src/resolv/stub/mod.rs index af66f1702..f80ee0691 100644 --- a/src/resolv/stub/mod.rs +++ b/src/resolv/stub/mod.rs @@ -395,7 +395,9 @@ impl<'a> Query<'a> { let msg = Message::from_octets(message.as_target().to_vec()) .expect("Message::from_octets should not fail"); - let request_msg = RequestMessage::new(msg); + let request_msg = RequestMessage::new(msg).map_err(|e| { + io::Error::new(io::ErrorKind::Other, e.to_string()) + })?; let transport = self.resolver.get_transport().await.map_err(|e| { io::Error::new(io::ErrorKind::Other, e.to_string()) diff --git a/src/stelline/client.rs b/src/stelline/client.rs index 6d8f85ba2..57dfbeb27 100644 --- a/src/stelline/client.rs +++ b/src/stelline/client.rs @@ -517,7 +517,7 @@ fn entry2reqmsg(entry: &Entry) -> RequestMessage> { header.set_cd(reply.cd); let msg = msg.into_message(); - let mut reqmsg = RequestMessage::new(msg); + let mut reqmsg = RequestMessage::new(msg).expect("should not fail unless the request is AXFR"); reqmsg.set_dnssec_ok(reply.fl_do); if reply.notify { reqmsg.header_mut().set_opcode(Opcode::NOTIFY); diff --git a/src/validator/context.rs b/src/validator/context.rs index 9052179e7..d9e44ba1a 100644 --- a/src/validator/context.rs +++ b/src/validator/context.rs @@ -2221,7 +2221,7 @@ where } }; let msg = Message::from_octets(octs).expect("should not fail"); - let mut req = RequestMessage::new(msg); + let mut req = RequestMessage::new(msg).expect("should not fail"); req.set_dnssec_ok(true); let mut request = upstream.send_request(req); diff --git a/src/validator/mod.rs b/src/validator/mod.rs index e2d223293..af70e18f9 100644 --- a/src/validator/mod.rs +++ b/src/validator/mod.rs @@ -80,7 +80,7 @@ //! # let mut msg = msg.question(); //! # msg.push((Name::vec_from_str("example.com").unwrap(), Rtype::AAAA)) //! # .unwrap(); -//! let mut req = RequestMessage::new(msg); +//! let mut req = RequestMessage::new(msg).unwrap(); //! req.set_dnssec_ok(true); //! //! // Send a query message. diff --git a/tests/net-client-cache.rs b/tests/net-client-cache.rs index 01f97bb69..5f3f2b5e8 100644 --- a/tests/net-client-cache.rs +++ b/tests/net-client-cache.rs @@ -84,7 +84,7 @@ async fn test_transport_error() { let mut msg = msg.question(); msg.push((Name::vec_from_str("example.com").unwrap(), Rtype::AAAA)) .unwrap(); - let req = RequestMessage::new(msg); + let req = RequestMessage::new(msg).unwrap(); let mut request = cached.send_request(req.clone()); let reply = request.get_response().await; diff --git a/tests/net-client.rs b/tests/net-client.rs index a10371725..0996b8220 100644 --- a/tests/net-client.rs +++ b/tests/net-client.rs @@ -11,6 +11,7 @@ use domain::net::client::dgram; use domain::net::client::dgram_stream; use domain::net::client::multi_stream; use domain::net::client::redundant; +use domain::net::client::request::RequestMessageMulti; use domain::net::client::stream; use std::fs::File; use std::net::IpAddr; @@ -43,7 +44,8 @@ fn single() { let step_value = Arc::new(CurrStepValue::new()); let conn = Connection::new(stelline.clone(), step_value.clone()); - let (octstr, transport) = stream::Connection::new(conn); + let (octstr, transport) = + stream::Connection::<_, RequestMessageMulti>>::new(conn); tokio::spawn(async move { transport.run().await; }); @@ -138,7 +140,10 @@ fn tcp() { } }; - let (tcp, transport) = stream::Connection::new(tcp_conn); + let (tcp, transport) = stream::Connection::< + _, + RequestMessageMulti>, + >::new(tcp_conn); tokio::spawn(async move { transport.run().await; println!("single TCP run terminated"); From 461c3426494f7d5197668c5d0904029a7a9c947c Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Tue, 20 Aug 2024 14:13:20 +0200 Subject: [PATCH 025/333] Fmt --- src/net/server/tests/integration.rs | 7 +++++-- src/stelline/client.rs | 3 ++- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index 9f19a8e26..ff18b6339 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -16,8 +16,8 @@ use crate::base::iana::Rcode; use crate::base::name::{Name, ToName}; use crate::base::net::IpAddr; use crate::base::wire::Composer; -use crate::net::client::{dgram, stream}; use crate::net::client::request::RequestMessageMulti; +use crate::net::client::{dgram, stream}; use crate::net::server; use crate::net::server::buf::VecBufSource; use crate::net::server::dgram::DgramServer; @@ -194,7 +194,10 @@ fn mk_client_factory( move |source_addr| { let stream = stream_server_conn .connect(Some(SocketAddr::new(*source_addr, 0))); - let (conn, transport) = stream::Connection::<_, RequestMessageMulti>>::new(stream); + let (conn, transport) = stream::Connection::< + _, + RequestMessageMulti>, + >::new(stream); tokio::spawn(transport.run()); Box::new(conn) }, diff --git a/src/stelline/client.rs b/src/stelline/client.rs index 57dfbeb27..3aea49b1a 100644 --- a/src/stelline/client.rs +++ b/src/stelline/client.rs @@ -517,7 +517,8 @@ fn entry2reqmsg(entry: &Entry) -> RequestMessage> { header.set_cd(reply.cd); let msg = msg.into_message(); - let mut reqmsg = RequestMessage::new(msg).expect("should not fail unless the request is AXFR"); + let mut reqmsg = RequestMessage::new(msg) + .expect("should not fail unless the request is AXFR"); reqmsg.set_dnssec_ok(reply.fl_do); if reply.notify { reqmsg.header_mut().set_opcode(Opcode::NOTIFY); From 911365e31a0b19bf415d93e704c245463907abdf Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Wed, 21 Aug 2024 15:56:09 +0200 Subject: [PATCH 026/333] A bit of cleanup. --- src/net/client/stream.rs | 208 +++++++++++++++++++++------------------ 1 file changed, 110 insertions(+), 98 deletions(-) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index e04153e49..c8b88488c 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -234,7 +234,6 @@ where self.sender.send(req).await.map_err(|_| { // Send error. The receiver is gone, this means that the // connection is closed. - println!("handle_request_impl: returning ConnectionClosed"); Error::ConnectionClosed })?; receiver.await.map_err(|_| Error::StreamReceiveError)? @@ -252,11 +251,9 @@ where sender: reply_sender, msg, }; - self.sender.send(req).await.map_err(|e| { + self.sender.send(req).await.map_err(|_| { // Send error. The receiver is gone, this means that the // connection is closed. -println!("handle_streaming_request_impl: send error {e:?}"); -println!("handle_streaming_request_impl: returning ConnectionClosed (1)"); Error::ConnectionClosed })?; Ok(()) @@ -345,11 +342,8 @@ pub struct Request { impl Request { /// Async function that waits for the future stored in Request to complete. async fn get_response_impl(&mut self) -> Result, Error> { - println!("in get_response_impl"); let res = (&mut self.fut).await; - println!("get_response_impl: got res {res:?}"); - res } } @@ -396,14 +390,12 @@ impl RequestMulti { async fn get_response_impl( &mut self, ) -> Result>, Error> { - println!("in get_response_impl(RequestMulti)"); if self.fut.is_some() { let fut = self.fut.take().expect("Some expected"); fut.await?; } // Fetch from the stream - println!("get_response_impl(RequestMulti): before stream.recv"); let res = self .stream .recv() @@ -411,8 +403,6 @@ impl RequestMulti { .ok_or(Error::ConnectionClosed) .map_err(|_| Error::ConnectionClosed)?; - println!("get_response_impl(RequestMulti): res {res:?}"); - res } } @@ -656,7 +646,6 @@ where { /// Run the transport machinery. pub async fn run(mut self) { - println!("in run"); let (reply_sender, mut reply_receiver) = mpsc::channel::>(READ_REPLY_CHAN_CAP); @@ -737,83 +726,82 @@ where }; tokio::select! { - biased; - res = &mut reader_fut => { - match res { - Ok(_) => - // The reader should not - // terminate without - // error. - panic!("reader terminated"), - Err(error) => { - Self::error(error.clone(), &mut query_vec); - status.state = ConnState::ReadError(error); - // Reader failed. Break - // out of loop and - // shut down - break - } - } - } - opt_answer = reply_receiver.recv() => { - let answer = opt_answer.expect("reader died?"); - // Check for a edns-tcp-keepalive option - let opt_record = answer.opt(); - if let Some(ref opts) = opt_record { - Self::handle_opts(opts, - &mut status); - }; - drop(opt_record); - Self::demux_reply(answer, &mut status, &mut query_vec).await; - } - res = write_stream.write(&msg[reqmsg_offset..]), - if do_write => { - match res { + biased; + res = &mut reader_fut => { + match res { + Ok(_) => + // The reader should not + // terminate without + // error. + panic!("reader terminated"), Err(error) => { - let error = - Error::StreamWriteError(Arc::new(error)); Self::error(error.clone(), &mut query_vec); - status.state = - ConnState::WriteError(error); - break; + status.state = ConnState::ReadError(error); + // Reader failed. Break + // out of loop and + // shut down + break } - Ok(len) => { - reqmsg_offset += len; - if reqmsg_offset >= msg.len() { - reqmsg = None; - reqmsg_offset = 0; + } + } + opt_answer = reply_receiver.recv() => { + let answer = opt_answer.expect("reader died?"); + // Check for a edns-tcp-keepalive option + let opt_record = answer.opt(); + if let Some(ref opts) = opt_record { + Self::handle_opts(opts, + &mut status); + }; + drop(opt_record); + Self::demux_reply(answer, &mut status, &mut query_vec).await; + } + res = write_stream.write(&msg[reqmsg_offset..]), + if do_write => { + match res { + Err(error) => { + let error = + Error::StreamWriteError(Arc::new(error)); + Self::error(error.clone(), &mut query_vec); + status.state = + ConnState::WriteError(error); + break; + } + Ok(len) => { + reqmsg_offset += len; + if reqmsg_offset >= msg.len() { + reqmsg = None; + reqmsg_offset = 0; + } + } + } + } + res = recv_fut, if !do_write => { + match res { + Some(req) => { + if req.sender.is_stream() { + self.config.response_timeout = + self.config.streaming_response_timeout; + } else { + self.config.response_timeout = + self.config.single_response_timeout; } + Self::insert_req( + req, &mut status, &mut reqmsg, &mut query_vec + ); } + None => { + // All references to the connection object have + // been dropped. Shutdown. + break; } - } - res = recv_fut, if !do_write => { - match res { - Some(req) => { - if req.sender.is_stream() { - self.config.response_timeout = - self.config.streaming_response_timeout; - } else { - self.config.response_timeout = - self.config.single_response_timeout; - } - Self::insert_req( - req, &mut status, &mut reqmsg, &mut query_vec - ); - } - None => { - // All references to the connection object have - // been dropped. Shutdown. - println!("None from recv_fut"); - break; - } - } - } - _ = sleep_fut => { - // Timeout expired, just - // continue with the loop - } + } + } + _ = sleep_fut => { + // Timeout expired, just + // continue with the loop + } - } + } // Check if the connection is idle match status.state { @@ -832,7 +820,6 @@ where trace!("Closing TCP connecting in state: {}", status.state); // Send FIN - println!("shutdown stream"); _ = write_stream.shutdown().await; } @@ -944,16 +931,17 @@ where // We got an answer, reset the timer status.state = ConnState::Active(Some(Instant::now())); + let id = answer.header().id(); + // Get the correct query and send it the reply. - let (mut req, mut opt_xfr_data) = - match query_vec.try_remove(answer.header().id()) { - Some(req) => req, - None => { - // No query with this ID. We should - // mark the connection as broken - return; - } - }; + let (mut req, mut opt_xfr_data) = match query_vec.try_remove(id) { + Some(req) => req, + None => { + // No query with this ID. We should + // mark the connection as broken + return; + } + }; let mut send_eof = false; let answer = if match &req.msg { ReqSingleMulti::Single(msg) => msg.is_answer(answer.for_slice()), @@ -973,12 +961,11 @@ where }; _ = req.sender.send(answer).await; - // TODO: Discard streaming requests once the stream is complete. if req.sender.is_stream() { if send_eof { _ = req.sender.send_eof().await; } else { - query_vec.insert((req, opt_xfr_data)).unwrap(); + query_vec.insert_at(id, (req, opt_xfr_data)); } } @@ -1173,7 +1160,6 @@ where xfr_state = XFRState::Error; // If we detect an error, then keep the stream open. We are // likely out of sync with respect to the sender. - println!("check_stream: line {}", line!()); return (false, xfr_state, false); } } @@ -1207,12 +1193,26 @@ where return (true, xfr_state, true); } - let ans_sec = answer.answer().unwrap(); + let ans_sec = match answer.answer() { + Ok(ans) => ans, + Err(_) => { + // Bad message, switch to error state. + xfr_state = XFRState::Error; + // If we detect an error, then keep the stream open. + return (true, xfr_state, false); + } + }; for rr in ans_sec.into_records::>>() { - println!("found rr {rr:?}"); - let rr = rr.unwrap(); + let rr = match rr { + Ok(rr) => rr, + Err(_) => { + // Bad message, switch to error state. + xfr_state = XFRState::Error; + return (true, xfr_state, false); + } + }; match xfr_state { XFRState::AXFRInit => { // The first record has to be a SOA record. @@ -1424,6 +1424,18 @@ impl Queries { Ok((idx, req)) } + /// Inserts the given query at a specified position. The slot has to be + /// empty. + fn insert_at(&mut self, id: u16, req: T) { + let id = id as usize; + self.vec[id] = Some(req); + + self.count += 1; + if id == self.curr { + self.curr += 1; + } + } + /// Tries to remove and return the query at the given index. /// /// Returns `None` if there was no query there. From 1a3d9a37e96e6a9249f629117d8318623c09f8b3 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 21 Aug 2024 15:56:15 +0200 Subject: [PATCH 027/333] Add support for high level XFR response sequence processing. --- src/base/message.rs | 17 +- src/zonetree/mod.rs | 1 + src/zonetree/xfr.rs | 1123 +++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 1139 insertions(+), 2 deletions(-) create mode 100644 src/zonetree/xfr.rs diff --git a/src/base/message.rs b/src/base/message.rs index 0dfac4d9c..398e11b0a 100644 --- a/src/base/message.rs +++ b/src/base/message.rs @@ -428,8 +428,7 @@ impl Message { &self, query: &Message, ) -> bool { - if !self.header().qr() - || self.header().id() != query.header().id() + if !self.is_answer_header(query) || self.header_counts().qdcount() != query.header_counts().qdcount() { @@ -439,6 +438,20 @@ impl Message { } } + /// Returns whether this is the answer to some other message. + /// + /// The method checks whether the ID fields of the headers are the same, + /// whether the QR flag is set in this message. It does NOT check whether + /// the questions are the same which is useful in case of responses such + /// as subsequent RFC 5936 AXFR responses which are not required to copy + /// the question from the request to the response. + pub fn is_answer_header( + &self, + query: &Message, + ) -> bool { + self.header().qr() && self.header().id() == query.header().id() + } + /// Could this message result in a stream of responses? /// /// Most DNS queries result in a single response, but some (only AXFR and diff --git a/src/zonetree/mod.rs b/src/zonetree/mod.rs index cc7b6f7d2..10d091f8c 100644 --- a/src/zonetree/mod.rs +++ b/src/zonetree/mod.rs @@ -88,6 +88,7 @@ mod traits; mod tree; pub mod types; mod walk; +pub mod xfr; mod zone; pub use self::answer::{Answer, AnswerAuthority, AnswerContent}; diff --git a/src/zonetree/xfr.rs b/src/zonetree/xfr.rs new file mode 100644 index 000000000..0ced33de8 --- /dev/null +++ b/src/zonetree/xfr.rs @@ -0,0 +1,1123 @@ +//! Parsing of AXFR/IXFR response messages for higher level processing. +//! +//! This module provides [`XfrResponseProcessor`] which enables you to process +//! one or more AXFR/IXFR response messages in terms of the high level +//! [`XfrEvent`]s that they represent without having to deal with the +//! AXFR/IXFR protocol details. +use std::fmt::Debug; + +use bytes::Bytes; +use tracing::trace; + +use crate::base::iana::Opcode; +use crate::base::wire::ParseError; +use crate::base::{ + Message, ParsedName, Record, RecordSection, Rtype, Serial, +}; +use crate::rdata::{Soa, ZoneRecordData}; + +//------------ XfrRecord ------------------------------------------------------ + +/// The type of record processed by [`XfrResponseProcessor`]. +pub type XfrRecord = + Record, ZoneRecordData>>; + +//------------ XfrResponseProcessor ------------------------------------------- + +/// An AXFR/IXFR response processor. +/// +/// [`XfrResponseProcessor`] can be invoked on one ore more sequentially +/// AXFR/IXFR received response messages to verify them and during processing +/// emit events which an implementor of [`XfrEventHandler`] can handle. +pub struct XfrResponseProcessor { + evt_handler: T, + + mode: Mode, +} + +impl XfrResponseProcessor { + /// Create a new XFR response processor. + /// + /// Events will be emitted to the given [`XfrEventHandler`] implementation. + pub fn new(evt_handler: T) -> Self { + Self { + evt_handler, + mode: Mode::default(), + } + } + + /// Process a single AXFR/IXFR response message. + /// + /// During processing events will be emitted to the registered + /// [`XfrEventHandler`] for handling. + /// + /// Returns Ok(true) if the XFR response was the last in the seqence, + /// Ok(false) if more XFR response messages are needed to complete the + /// sequence, or Err on error. + pub async fn answer( + &mut self, + req: &Message, + resp: Message, + ) -> Result { + // Check that the given message is a DNS XFR response. + let res = self.check_is_xfr_answer(req, &resp).await; + + // Unpack the XFR type and answer object. We cannot do this in the + // line above using `map_err()` and `?` as the Rust compiler complains + // about attempting to return `resp` while a reference to it still + // exists. + let (xfr_type, answer) = match res { + Ok(values) => values, + Err(err) => return Err(Error::from_check_error(resp, err)), + }; + + // Walk over all records that are of the type that can exist in a zone + // file. TODO: Should this actually walk over all possible record + // types irrespective of whether they are intended to exist in a zone + // file or not? + let mut records = answer.limit_to::>(); + + match self.mode { + // When given the first response in a sequence, do some initial + // setup. + Mode::AwaitingFirstAnswer => { + let Some(Ok(record)) = records.next() else { + return Err(Error::Malformed); + }; + + if let Err(err) = + self.initialize(xfr_type, req.header().id(), record).await + { + return Err(Error::from_check_error(resp, err)); + } + } + + // For subsequent messages make sure that the XFR + Mode::AwaitingNextAnswer { + initial_xfr_type, + initial_query_id, + .. + } => { + if xfr_type != initial_xfr_type + || req.header().id() != initial_query_id + { + // The XFR type is extracted from the request. If we were + // given a different request with a different question and + // qtype on a subsequent invocation of answer() that would + // be unexpected. + return Err(Error::NotValidXfrQuery); + } + } + + Mode::TransferComplete => { + // We already finished processing an XFR response sequence. We + // don't expect there to be any more messages to process!. + return Err(Error::Malformed); + } + + Mode::TransferFailed => { + // We had ot terminate processing of the XFR response sequence + // due to a problem with the received data, so we don't expect + // to be invoked again with another response message! + return Err(Error::Terminated); + } + }; + + let Mode::AwaitingNextAnswer { read, .. } = &mut self.mode else { + unreachable!(); + }; + + for record in records.flatten() { + trace!("XFR record {}: {record:?}", read.rr_count); + + if let Some(event) = read.record(record).await? { + match event { + XfrEvent::EndOfTransfer => { + self.mode = Mode::TransferComplete; + self.evt_handler.handle_event(event).await?; + return Ok(true); + } + + XfrEvent::ProcessingFailed => { + self.mode = Mode::TransferFailed; + let _ = self.evt_handler.handle_event(event).await; + return Err(Error::Malformed); + } + + _ => { + self.evt_handler.handle_event(event).await?; + } + } + } + } + + // Finished processing this message but did not yet reach the end of + // the transfer, more responses are expected. + Ok(false) + } + + async fn check_is_xfr_answer<'a>( + &mut self, + req: &Message, + resp: &'a Message, + ) -> Result<(XfrType, RecordSection<'a, Bytes>), CheckError> { + // Enforce the rules defined in 2. AXFR Messages of RFC 5936. + // See: https://www.rfc-editor.org/rfc/rfc5936.html#section-2 + + // Check the request. + let req_header = req.header(); + let req_counts = req.header_counts(); + + if req.is_error() + || req_header.qr() + || req_counts.qdcount() != 1 + || req_counts.ancount() != 0 + || req_header.opcode() != Opcode::QUERY + { + return Err(CheckError::NotValidXfrQuery); + } + + let Some(qtype) = req.qtype() else { + return Err(CheckError::NotValidXfrResponse); + }; + + let xfr_type = match qtype { + Rtype::AXFR => XfrType::Axfr, + Rtype::IXFR => XfrType::Ixfr, + _ => return Err(CheckError::NotValidXfrResponse), + }; + + // https://datatracker.ietf.org/doc/html/rfc1995#section-3 + // 3. Query Format + // "The IXFR query packet format is the same as that of a normal DNS + // query, but with the query type being IXFR and the authority + // section containing the SOA record of client's version of the + // zone." + if matches!(xfr_type, XfrType::Ixfr) && req_counts.nscount() != 1 { + return Err(CheckError::NotValidXfrResponse); + } + + // Check the response. + let resp_header = resp.header(); + let resp_counts = resp.header_counts(); + + if resp.is_error() + || !resp.is_answer_header(req) + || resp_header.opcode() != Opcode::QUERY + || resp_header.tc() + || resp_counts.ancount() == 0 + || resp_counts.nscount() != 0 + { + return Err(CheckError::NotValidXfrResponse); + } + + // + // 2.2.1. Header Values + // "QDCOUNT MUST be 1 in the first message; + // MUST be 0 or 1 in all following messages;" + if matches!(self.mode, Mode::AwaitingFirstAnswer) + && (resp_counts.qdcount() != 1 + || resp.sole_question() != req.sole_question()) + { + return Err(CheckError::NotValidXfrResponse); + } + + let answer = resp.answer().map_err(CheckError::ParseError)?; + + Ok((xfr_type, answer)) + } + + async fn initialize( + &mut self, + initial_xfr_type: XfrType, + initial_query_id: u16, + record: XfrRecord, + ) -> Result<(), CheckError> { + // The initial record should be a SOA record. + let data = record.into_data(); + + let ZoneRecordData::Soa(soa) = data else { + return Err(CheckError::NotValidXfrResponse); + }; + + let read = ReadState::new(initial_xfr_type, soa); + + self.mode = Mode::AwaitingNextAnswer { + initial_xfr_type, + initial_query_id, + read, + }; + + Ok(()) + } +} + +//------------ Mode ----------------------------------------------------------- + +#[derive(Default)] +enum Mode { + #[default] + AwaitingFirstAnswer, + + AwaitingNextAnswer { + initial_xfr_type: XfrType, + initial_query_id: u16, + read: ReadState, + }, + + TransferComplete, + + TransferFailed, +} + +//------------ ReadState ------------------------------------------------------ + +#[derive(Debug)] +struct ReadState { + actual_xfr_type: XfrType, + initial_soa: Soa>, + current_soa: Soa>, + ixfr_update_mode: IxfrUpdateMode, + rr_count: usize, +} + +impl ReadState { + fn new( + initial_xfr_type: XfrType, + initial_soa: Soa>, + ) -> Self { + Self { + actual_xfr_type: initial_xfr_type, + initial_soa: initial_soa.clone(), + current_soa: initial_soa, + rr_count: 1, + ixfr_update_mode: Default::default(), + } + } + + async fn record( + &mut self, + rec: XfrRecord, + ) -> Result>, Error> { + self.rr_count += 1; + + // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2 + // 2.2. AXFR Response + // "..clients MUST accept any ordering and grouping of the non-SOA + // RRs. Each RR SHOULD be transmitted only once, and AXFR clients + // MUST ignore any duplicate RRs received." + // + // Note: We do NOT implement this MUST here because it would be very + // inefficient to actually check that any received non-SOA RR has not + // been seen before during the in-progress transfer. Clients of + // XfrResponseProcessor are better placed to enforce this rule if + // needed, e.g. at the moment of insertion into a zone tree checking + // that the record is not already present or insertion of a duplicate + // having no effect as it is already present. + + let soa = match rec.data() { + ZoneRecordData::Soa(soa) => Some(soa), + _ => None, + }; + + let record_matches_initial_soa = soa == Some(&self.initial_soa); + + match self.actual_xfr_type { + XfrType::Axfr if record_matches_initial_soa => { + // https://www.rfc-editor.org/rfc/rfc5936.html#section-2.2 + // 2.2. AXFR Response + // ... + // "In such a series, the first message MUST begin with the + // SOA resource record of the zone, and the last message + // MUST conclude with the same SOA resource record. + // Intermediate messages MUST NOT contain the SOA resource + // record." + Ok(Some(XfrEvent::EndOfTransfer)) + } + + XfrType::Axfr => { + Ok(Some(XfrEvent::AddRecord(self.current_soa.serial(), rec))) + } + + XfrType::Ixfr if self.rr_count < 2 => unreachable!(), + + XfrType::Ixfr if self.rr_count == 2 => { + if record_matches_initial_soa { + // IXFR not available, AXFR of empty zone detected. + Ok(Some(XfrEvent::EndOfTransfer)) + } else if let Some(soa) = soa { + // This SOA record is the start of an IXFR diff sequence. + self.current_soa = soa.clone(); + + // We don't need to set the IXFR update more here as it + // should already be set to Deleting. + debug_assert_eq!( + self.ixfr_update_mode, + IxfrUpdateMode::Deleting + ); + + Ok(Some(XfrEvent::BeginBatchDelete(soa.serial()))) + } else { + // https://datatracker.ietf.org/doc/html/rfc1995#section-4 + // 4. Response Format + // "If incremental zone transfer is not available, the + // entire zone is returned. The first and the last RR + // of the response is the SOA record of the zone. I.e. + // the behavior is the same as an AXFR response except + // the query type is IXFR. + // + // If incremental zone transfer is available, one or + // more difference sequences is returned. The list of + // difference sequences is preceded and followed by a + // copy of the server's current version of the SOA. + // + // Each difference sequence represents one update to + // the zone (one SOA serial change) consisting of + // deleted RRs and added RRs. The first RR of the + // deleted RRs is the older SOA RR and the first RR of + // the added RRs is the newer SOA RR. + // + // Modification of an RR is performed first by removing + // the original RR and then adding the modified one." + + // As this is IXFR and this is the second record, it should + // be the "first RR of the deleted RRs" which should be + // "the older SOA RR". However, it isn't a SOA RR. As such + // assume that "incremental zone transfer is not available" + // and so "the behaviour is the same as an AXFR response", + self.actual_xfr_type = XfrType::Axfr; + Ok(Some(XfrEvent::AddRecord( + self.current_soa.serial(), + rec, + ))) + } + } + + XfrType::Ixfr => { + if let Some(soa) = soa { + self.ixfr_update_mode.toggle(); + self.current_soa = soa.clone(); + + match self.ixfr_update_mode { + IxfrUpdateMode::Deleting => { + // We just finished a (Delete, Add) diff sequence. + // Is this the end of the transfer, or the start + // of a new diff sequence? + if record_matches_initial_soa { + Ok(Some(XfrEvent::EndOfTransfer)) + } else { + Ok(Some(XfrEvent::BeginBatchDelete( + self.current_soa.serial(), + ))) + } + } + IxfrUpdateMode::Adding => { + // We just switched from the Delete phase of a + // diff sequence to the add phase of the diff + // sequence. + Ok(Some(XfrEvent::BeginBatchAdd( + self.current_soa.serial(), + ))) + } + } + } else { + match self.ixfr_update_mode { + IxfrUpdateMode::Deleting => { + Ok(Some(XfrEvent::DeleteRecord( + self.current_soa.serial(), + rec, + ))) + } + IxfrUpdateMode::Adding => { + Ok(Some(XfrEvent::AddRecord( + self.current_soa.serial(), + rec, + ))) + } + } + } + } + } + } +} + +//------------ RecordResult --------------------------------------------------- + +/// An event emitted by [`XfrResponseProcessor`] during transfer processing.` +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +pub enum XfrEvent { + /// Delete record R in zone serial S. + /// + /// The transfer signalled that the given record should be deleted from + /// the zone version with the given serial number. + /// + /// Note: If the transfer contains N deletions of fhe same record then + /// this event will occur N times. + DeleteRecord(Serial, R), + + /// Add record R in zone serial S. + /// + /// The transfer signalled that the given record should be added to the + /// zone version with the given serial number. + /// + /// Note: If the transfer contains N additions of fhe same record then + /// this event will occur N times. + AddRecord(Serial, R), + + /// Prepare to delete records in zone serial S. + /// + /// The transfer signalled that zero or more record deletions will follow, + /// all for the zone version with the given serial number. + BeginBatchDelete(Serial), + + /// Prepare to add records in zone serial S. + /// + /// The transfer signalled that zero or more record additions will follow, + /// all for the zone version with the given serial number. + BeginBatchAdd(Serial), + + /// Transfer completed successfully. + /// + /// Note: This event is not emitted until the final record of the final + /// response in a set of one or more transfer responss has been seen. + EndOfTransfer, + + /// Transfer processing failed. + /// + /// This event indicates that there is a problem with the transfer data + /// and that transfer processing cannot continue. + ProcessingFailed, +} + +//--- Display + +impl std::fmt::Display for XfrEvent { + fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { + match self { + XfrEvent::DeleteRecord(_, _) => f.write_str("DeleteRecord"), + XfrEvent::AddRecord(_, _) => f.write_str("AddRecord"), + XfrEvent::BeginBatchDelete(_) => f.write_str("BeginBatchDelete"), + XfrEvent::BeginBatchAdd(_) => f.write_str("BeginBatchAdd"), + XfrEvent::EndOfTransfer => f.write_str("EndOfTransfer"), + XfrEvent::ProcessingFailed => f.write_str("ProcessingFailed"), + } + } +} + +//------------ XfrEventHandler --------------------------------------------------- + +/// A trait for implementing handlers of [`XfrEvent`]s. +pub trait XfrEventHandler { + /// Handle the given [`XfrEvent`]. + /// + /// Returning an Err will cause transfer processsing to be aborted and the + /// error to be returned to the client of [`XfrResponseProcessor`], except in + /// the case of [`XfrEvent::ProcessingFailed`] for which the return value of + /// this handler will be ignored by [`XfrResponseProcessor`]. + fn handle_event( + &self, + evt: XfrEvent, + ) -> impl std::future::Future> + Send; +} + +//------------ IxfrUpdateMode ------------------------------------------------- + +#[derive(Copy, Clone, Debug, Default, PartialEq, Eq)] +enum IxfrUpdateMode { + #[default] + Deleting, + + Adding, +} +impl IxfrUpdateMode { + fn toggle(&mut self) { + match self { + IxfrUpdateMode::Deleting => *self = IxfrUpdateMode::Adding, + IxfrUpdateMode::Adding => *self = IxfrUpdateMode::Deleting, + } + } +} + +//------------ Error ---------------------------------------------------------- + +/// An error reported by [`XfrResponseProcessor`]. +#[derive(Debug)] +pub enum Error { + /// The message could not be parsed. + ParseError(ParseError, Message), + + /// The request message is not an XFR query/ + NotValidXfrQuery, + + /// The response message is not an XFR response. + NotValidXfrResponse(Message), + + /// At least one record in the XFR response sequence is incorrect. + Malformed, + + /// Processing was already terminated for this XFR response sequence. + Terminated, +} + +impl Error { + fn from_check_error( + msg: Message, + prepare_err: CheckError, + ) -> Self { + match prepare_err { + CheckError::ParseError(err) => Self::ParseError(err, msg), + CheckError::NotValidXfrQuery => Self::NotValidXfrQuery, + CheckError::NotValidXfrResponse => Self::NotValidXfrResponse(msg), + } + } +} + +//------------ PrepareError --------------------------------------------------- + +#[derive(Debug)] +enum CheckError { + ParseError(ParseError), + + NotValidXfrQuery, + + NotValidXfrResponse, +} + +//--- From + +impl From for CheckError { + fn from(err: ParseError) -> Self { + Self::ParseError(err) + } +} + +//------------ XfrType -------------------------------------------------------- + +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +enum XfrType { + Axfr, + Ixfr, +} + +//--- From + +impl TryFrom for XfrType { + type Error = (); + + fn try_from(rtype: Rtype) -> Result { + match rtype { + Rtype::AXFR => Ok(XfrType::Axfr), + Rtype::IXFR => Ok(XfrType::Ixfr), + _ => Err(()), + } + } +} + +#[cfg(test)] +mod tests { + use core::net::Ipv4Addr; + use core::str::FromStr; + + use std::string::String; + use std::sync::Arc; + use std::vec::Vec; + + use bytes::BytesMut; + use octseq::Octets; + use tokio::sync::Mutex; + + use crate::base::iana::Rcode; + use crate::base::message_builder::{ + AnswerBuilder, AuthorityBuilder, QuestionBuilder, + }; + use crate::base::rdata::ComposeRecordData; + use crate::base::{MessageBuilder, Ttl}; + use crate::base::{Name, ToName}; + use crate::rdata::A; + + use super::*; + + #[tokio::test] + async fn request_message_is_rejected() { + init_logging(); + + // Create a non-XFR request to reply to. + let req = mk_request("example.com", Rtype::A).into_message(); + + // Process the request and assert that it is rejected as not being + // a valid XFR response and that no XFR processor events were emitted. + assert_xfr_response( + &req.clone(), + req, + |res| matches!(res, Err(Error::NotValidXfrResponse(_))), + &[], + ) + .await; + } + + #[tokio::test] + async fn non_xfr_response_is_rejected() { + init_logging(); + + // Create a non-XFR request to reply to. + let req = mk_request("example.com", Rtype::A).into_message(); + + // Create a non-XFR response. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + + // Process the response and assert that it is rejected as not being + // a valid XFR response and that no XFR processor events were emitted. + assert_xfr_response( + &req, + answer.into_message(), + |res| matches!(res, Err(Error::NotValidXfrResponse(_))), + &[], + ) + .await; + } + + #[tokio::test] + async fn axfr_response_with_no_answers_is_rejected() { + init_logging(); + + // Create an AXFR request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create a response that lacks answers. + let answer = mk_empty_answer(&req, Rcode::NOERROR); + + // Process the response and assert that it is rejected as not being + // a valid XFR response and that no XFR processor events were emitted. + assert_xfr_response( + &req, + answer.into_message(), + |res| matches!(res, Err(Error::NotValidXfrResponse(_))), + &[], + ) + .await; + } + + #[tokio::test] + async fn error_axfr_response_is_rejected() { + init_logging(); + + // Create an AXFR request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create a minimal valid AXFR response, just something that should + // not be rejected by the XFR processor due to its content. It should + // however be rejected due to the non-NOERROR rcode. + let mut answer = mk_empty_answer(&req, Rcode::SERVFAIL); + add_answer_record(&req, &mut answer, mk_soa(Serial::now())); + + // Process the response and assert that it is rejected as not being + // a valid XFR response and that no XFR processor events were emitted. + assert_xfr_response( + &req, + answer.into_message(), + |res| matches!(res, Err(Error::NotValidXfrResponse(_))), + &[], + ) + .await; + } + + #[tokio::test] + async fn incomplete_axfr_response_is_accepted() { + init_logging(); + + // Create an AXFR request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create an incomplete AXFR response. A proper AXFR response has at + // least two identical SOA records, one at the start and one at the + // end, but this response contains only a single SOA record. This is + // not considered invalid however because a subsequent response could + // still provide the missing SOA record. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + add_answer_record(&req, &mut answer, mk_soa(Serial::now())); + + // Process the response and assert that Ok(false) is returned by the + // XFR processor indicating that the XFR response was incomplete. Also + // verify the events emitted by the XFR processor. + assert_xfr_response( + &req, + answer.into_message(), + |res| matches!(res, Ok(false)), + &[], + ) + .await; + } + + #[tokio::test] + async fn axfr_response_with_only_soas_is_accepted() { + init_logging(); + + // Create an AXFR request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create a complete but minimal AXFR response. A proper AXFR response + // has at least two identical SOA records, one at the start and one at + // the end, with actual zone records in between. This response has only + // the start and end SOA and no content in between. RFC 5936 doesn't + // seem to disallow this. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + let soa = mk_soa(Serial::now()); + add_answer_record(&req, &mut answer, soa.clone()); + add_answer_record(&req, &mut answer, soa); + + // Process the response and assert that Ok(true) is returned by the + // XFR processor indicating that the XFR response was complete. Also + // verify the events emitted by the XFR processor. + assert_xfr_response( + &req, + answer.into_message(), + |res| matches!(res, Ok(true)), + &["EndOfTransfer"], + ) + .await; + } + + #[tokio::test] + async fn axfr_multi_response_with_only_soas_is_accepted() { + init_logging(); + + // Create an AXFR request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create a complete but minimal AXFR response. A proper AXFR response + // has at least two identical SOA records, one at the start and one at + // the end, with actual zone records in between. This response has only + // the start and end SOA and no content in between. RFC 5936 doesn't + // seem to disallow this. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + let soa = mk_soa(Serial::now()); + add_answer_record(&req, &mut answer, soa.clone()); + + // Process the response and assert that Ok(true) is returned by the + // XFR processor indicating that the XFR response was complete. Also + // verify the events emitted by the XFR processor. + let (evt_handler, mut processor) = assert_xfr_response( + &req, + answer.into_message(), + |res| matches!(res, Ok(false)), + &[], + ) + .await; + + // Create another AXFR response to complete the transfer. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + add_answer_record(&req, &mut answer, soa); + + // Process the response and assert that Ok(true) is returned by the + // XFR processor indicating that the XFR response was complete. Also + // verify the events emitted by the XFR processor. + assert_xfr_response_with_processor( + &req, + answer.into_message(), + |res| matches!(res, Ok(true)), + &["EndOfTransfer"], + evt_handler, + &mut processor, + ) + .await; + } + + #[tokio::test] + async fn axfr_response_generates_expected_events() { + init_logging(); + + // Create an AXFR request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create an AXFR response. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + let soa = mk_soa(Serial::now()); + add_answer_record(&req, &mut answer, soa.clone()); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); + add_answer_record(&req, &mut answer, soa); + + // Process the response and assert that Ok(true) is returned by the + // XFR processor indicating that the XFR response was complete. Also + // verify the events emitted by the XFR processor. + assert_xfr_response( + &req, + answer.into_message(), + |res| matches!(res, Ok(true)), + &["AddRecord", "AddRecord", "EndOfTransfer"], + ) + .await; + } + + #[tokio::test] + async fn ixfr_response_generates_expected_events() { + init_logging(); + + // Create an IXFR request to reply to. + let req = mk_request("example.com", Rtype::IXFR); + let mut authority = req.authority(); + let client_serial = Serial::now(); + let soa = mk_soa(client_serial); + add_authority_record(&mut authority, soa); + let req = authority.into_message(); + + // Prepare some serial numbers and SOA records to use in the IXFR response. + let old_serial = client_serial; + let new_serial = client_serial.add(1); + let old_soa = mk_soa(old_serial); + let new_soa = mk_soa(new_serial); + + // Create an IXFR response. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + // Outer SOA with servers current SOA + add_answer_record(&req, &mut answer, new_soa.clone()); + // Start of diff sequence: SOA of the servers' previous zone version + // (which matches that of the client) followed by records to be + // deleted as they were in that version of the zone but are not in the + // new version of the zone. + add_answer_record(&req, &mut answer, old_soa); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); + // SOA of the servers` new zone version (which is ahead of that of the + // client) followed by records to be added as they were added in this + // new version of the zone.` + add_answer_record(&req, &mut answer, new_soa.clone()); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + // Closing SOA with servers current SOA + add_answer_record(&req, &mut answer, new_soa); + + // Process the response and assert that Ok(true) is returned by the + // XFR processor indicating that the XFR response was complete. Also + // verify the events emitted by the XFR processor. + assert_xfr_response( + &req, + answer.into_message(), + |res| matches!(res, Ok(true)), + &[ + "BeginBatchDelete", + "DeleteRecord", + "DeleteRecord", + "BeginBatchAdd", + "AddRecord", + "AddRecord", + "EndOfTransfer", + ], + ) + .await; + } + + #[tokio::test] + async fn multi_ixfr_response_generates_expected_events() { + init_logging(); + + // Create an IXFR request to reply to. + let req = mk_request("example.com", Rtype::IXFR); + let mut authority = req.authority(); + let client_serial = Serial::now(); + let soa = mk_soa(client_serial); + add_authority_record(&mut authority, soa); + let req = authority.into_message(); + + // Prepare some serial numbers and SOA records to use in the IXFR response. + let old_serial = client_serial; + let new_serial = client_serial.add(1); + let old_soa = mk_soa(old_serial); + let new_soa = mk_soa(new_serial); + + // Create a partial IXFR response. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + // Outer SOA with servers current SOA + add_answer_record(&req, &mut answer, new_soa.clone()); + // Start of diff sequence: SOA of the servers' previous zone version + // (which matches that of the client) followed by records to be + // deleted as they were in that version of the zone but are not in the + // new version of the zone. + add_answer_record(&req, &mut answer, old_soa); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + + // Process the response and assert that Ok(true) is returned by the + // XFR processor indicating that the XFR response was complete. Also + // verify the events emitted by the XFR processor. + let (evt_handler, mut processor) = assert_xfr_response( + &req, + answer.into_message(), + |res| matches!(res, Ok(false)), + &["BeginBatchDelete", "DeleteRecord"], + ) + .await; + + // Craete a second IXFR response that completes the transfer + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); + // SOA of the servers` new zone version (which is ahead of that of the + // client) followed by records to be added as they were added in this + // new version of the zone.` + add_answer_record(&req, &mut answer, new_soa.clone()); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + // Closing SOA with servers current SOA + add_answer_record(&req, &mut answer, new_soa); + + assert_xfr_response_with_processor( + &req, + answer.into_message(), + |res| matches!(res, Ok(true)), + &[ + "BeginBatchDelete", // Seen during processing of the 1st answer + "DeleteRecord", // Seen during processing of the 1st answer + "DeleteRecord", // Seen during processing of the 2nd answer + "BeginBatchAdd", // Seen during processing of the 2nd answer + "AddRecord", // Seen during processing of the 2nd answer + "AddRecord", // Seen during processing of the 2nd answer + "EndOfTransfer", // Seen during processing of the 2nd answer + ], + evt_handler, + &mut processor, + ) + .await; + } + + //------------ TestXfrEventHandler ---------------------------------------- + + #[derive(Clone, Default)] + struct TestXfrEventHandler { + events: Arc>>, + } + + impl TestXfrEventHandler { + pub fn new() -> Self { + Self::default() + } + + pub async fn events(self) -> Vec { + self.events.lock().await.clone() + } + } + + impl XfrEventHandler for TestXfrEventHandler { + async fn handle_event( + &self, + evt: XfrEvent, + ) -> Result<(), Error> { + trace!("Received event: {evt}"); + self.events.lock().await.push(format!("{evt}")); + Ok(()) + } + } + + //------------ Helper functions ------------------------------------------- + + fn init_logging() { + // Initialize tracing based logging. Override with env var RUST_LOG, e.g. + // RUST_LOG=trace. DEBUG level will show the .rpl file name, Stelline step + // numbers and types as they are being executed. + tracing_subscriber::fmt() + .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) + .with_thread_ids(true) + .without_time() + .try_init() + .ok(); + } + + async fn assert_xfr_response( + req: &Message, + resp: Message, + res_check_cb: fn(&Result) -> bool, + expected_events: &[&str], + ) -> ( + TestXfrEventHandler, + XfrResponseProcessor, + ) { + let evt_handler = TestXfrEventHandler::new(); + let mut processor = XfrResponseProcessor::new(evt_handler.clone()); + + assert_xfr_response_with_processor( + req, + resp, + res_check_cb, + expected_events, + evt_handler.clone(), + &mut processor, + ) + .await; + + (evt_handler, processor) + } + + async fn assert_xfr_response_with_processor( + req: &Message, + resp: Message, + res_check_cb: fn(&Result) -> bool, + expected_events: &[&str], + evt_handler: TestXfrEventHandler, + processor: &mut XfrResponseProcessor, + ) { + let res = processor.answer(req, resp).await; + + // Verify that the XFR processor returns an error. + assert!( + res_check_cb(&res), + "Unexpected result {res:?} from the XFR processor", + ); + + // Verify that no XFR processing events were emitted. + assert_eq!( + &evt_handler.clone().events().await, + expected_events, + "Unexpected events were emitted by the XFR processor" + ); + } + + fn mk_request(qname: &str, qtype: Rtype) -> QuestionBuilder { + let req = MessageBuilder::new_bytes(); + let mut req = req.question(); + req.push((Name::vec_from_str(qname).unwrap(), qtype)) + .unwrap(); + req + } + + fn mk_empty_answer( + req: &Message, + rcode: Rcode, + ) -> AnswerBuilder { + let builder = MessageBuilder::new_bytes(); + builder.start_answer(req, rcode).unwrap() + } + + fn add_answer_record( + req: &Message, + answer: &mut AnswerBuilder, + item: T, + ) { + let question = req.sole_question().unwrap(); + let qname = question.qname(); + let qclass = question.qclass(); + answer + .push((qname, qclass, Ttl::from_secs(0), item)) + .unwrap(); + } + + fn add_authority_record( + authority: &mut AuthorityBuilder, + item: T, + ) { + let (qname, qclass) = { + let question = authority.as_message().sole_question().unwrap(); + let qname = question.qname().to_bytes(); + let qclass = question.qclass(); + (qname, qclass) + }; + authority + .push((qname, qclass, Ttl::from_secs(0), item)) + .unwrap(); + } + + fn mk_soa(serial: Serial) -> Soa> { + let mname = Name::from_str("mname").unwrap(); + let rname = Name::from_str("rname").unwrap(); + let ttl = Ttl::from_secs(0); + Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl) + } +} From 0976abe7baab21efbece56d9f540bda4a04ef126 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 21 Aug 2024 15:57:19 +0200 Subject: [PATCH 028/333] Revert "Add end-of-stream detection to stream.rs." This reverts commit a7ca10905ee4005bfb8860c33bd182f57fafcd53. --- src/net/client/stream.rs | 63 ++-------------------------------------- 1 file changed, 2 insertions(+), 61 deletions(-) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index a6b7f0ecb..91b9b140f 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -35,13 +35,10 @@ use tracing::trace; use crate::base::message::Message; use crate::base::message_builder::StreamTarget; -use crate::base::name::FlattenInto; use crate::base::opt::{AllOptData, OptRecord, TcpKeepalive}; -use crate::base::{Name, ParsedName, Record, Rtype}; use crate::net::client::request::{ ComposeRequest, Error, GetResponse, SendRequest, }; -use crate::rdata::Soa; use crate::utils::config::DefMinMax; //------------ Configuration Constants ---------------------------------------- @@ -226,12 +223,7 @@ impl Connection { ) -> Result, Error> { let (sender, receiver) = oneshot::channel(); let sender = ReplySender::Single(Some(sender)); - let req = ChanReq { - sender, - msg, - eos: None, - eos_seen_count: 0, - }; + let req = ChanReq { sender, msg }; self.sender.send(req).await.map_err(|_| { // Send error. The receiver is gone, this means that the // connection is closed. @@ -257,8 +249,6 @@ impl Connection { let req = ChanReq { sender: reply_sender, msg, - eos: None, - eos_seen_count: 0, }; let _ = self.sender.send(req).await; @@ -469,54 +459,6 @@ struct ChanReq { /// Sender to send result back to [`Request`] sender: ReplySender, - - /// End of stream marker to check for, when streaming - eos: Option>>, - - /// Number of times eos has been seen - eos_seen_count: usize, -} - -impl ChanReq { - fn is_eos(&mut self, msg: &Message) -> bool { - if self.sender.is_stream() { - if let Ok(mut answer) = msg.answer() { - if let Some(Ok(record)) = answer.next() { - if record.rtype() == Rtype::SOA { - if let Ok(Some(record)) = record.into_record() { - let soa_rec: Record<_, Soa>> = - record; - let soa = soa_rec.data().clone().flatten_into(); - match &self.eos { - Some(expected_soa) => { - let found = &soa == expected_soa; - if found { - self.eos_seen_count += 1; - } - let res = match msg.qtype() { - Some(Rtype::AXFR) => { - self.eos_seen_count == 1 - } - Some(Rtype::IXFR) => { - self.eos_seen_count == 2 - } - _ => false, - }; - return res; - } - None => { - self.eos = Some(soa); - } - } - } - } - } - } - false - } else { - true - } - } } /// A message back to [`Request`] returning a response. @@ -887,7 +829,6 @@ where return; } }; - let is_eos = req.is_eos(&answer); let answer = if req.msg.is_answer(answer.for_slice()) { Ok(answer) } else { @@ -896,7 +837,7 @@ where _ = req.sender.send(answer); // TODO: Discard streaming requests once the stream is complete. - if req.sender.is_stream() && !is_eos { + if req.sender.is_stream() { query_vec.insert(req).unwrap(); } From 89418d5c62058d869dd3ebceb94f64bb4871aa13 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 21 Aug 2024 16:00:43 +0200 Subject: [PATCH 029/333] Move high level XFR response processing under src/net/client. --- src/net/client/mod.rs | 1 + src/{zonetree => net/client}/xfr.rs | 0 src/zonetree/mod.rs | 1 - 3 files changed, 1 insertion(+), 1 deletion(-) rename src/{zonetree => net/client}/xfr.rs (100%) diff --git a/src/net/client/mod.rs b/src/net/client/mod.rs index 0275f3bef..cb3efad86 100644 --- a/src/net/client/mod.rs +++ b/src/net/client/mod.rs @@ -190,6 +190,7 @@ pub mod request; pub mod stream; #[cfg(feature = "tsig")] pub mod tsig; +pub mod xfr; #[cfg(feature = "unstable-validator")] pub mod validator; pub mod validator_test; diff --git a/src/zonetree/xfr.rs b/src/net/client/xfr.rs similarity index 100% rename from src/zonetree/xfr.rs rename to src/net/client/xfr.rs diff --git a/src/zonetree/mod.rs b/src/zonetree/mod.rs index 10d091f8c..cc7b6f7d2 100644 --- a/src/zonetree/mod.rs +++ b/src/zonetree/mod.rs @@ -88,7 +88,6 @@ mod traits; mod tree; pub mod types; mod walk; -pub mod xfr; mod zone; pub use self::answer::{Answer, AnswerAuthority, AnswerContent}; From 767fac44e4d491090766bf3e99d45209cae73349 Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Wed, 21 Aug 2024 16:07:03 +0200 Subject: [PATCH 030/333] Clippy --- src/net/client/stream.rs | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index c8b88488c..19714babd 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -342,9 +342,7 @@ pub struct Request { impl Request { /// Async function that waits for the future stored in Request to complete. async fn get_response_impl(&mut self) -> Result, Error> { - let res = (&mut self.fut).await; - - res + (&mut self.fut).await } } @@ -396,14 +394,11 @@ impl RequestMulti { } // Fetch from the stream - let res = self - .stream + self.stream .recv() .await .ok_or(Error::ConnectionClosed) - .map_err(|_| Error::ConnectionClosed)?; - - res + .map_err(|_| Error::ConnectionClosed)? } } From 338409361c074c47679dcd02eb7aeadb4711ef07 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 21 Aug 2024 18:39:23 +0200 Subject: [PATCH 031/333] WIP: Zone updated via add/delete XFR-like events, including diff generation for later IXFR serving. --- Cargo.lock | 7 + Cargo.toml | 3 +- src/base/serial.rs | 23 +- src/net/client/xfr.rs | 17 +- src/zonetree/in_memory/nodes.rs | 32 ++- src/zonetree/in_memory/versioned.rs | 24 +- src/zonetree/in_memory/write.rs | 293 ++++++++++++++++++++++- src/zonetree/mod.rs | 9 +- src/zonetree/traits.rs | 92 +++++-- src/zonetree/types.rs | 34 ++- src/zonetree/xfr_event_handler.rs | 355 ++++++++++++++++++++++++++++ src/zonetree/zone.rs | 29 ++- 12 files changed, 848 insertions(+), 70 deletions(-) create mode 100644 src/zonetree/xfr_event_handler.rs diff --git a/Cargo.lock b/Cargo.lock index af9939129..f6af4fb44 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -235,6 +235,7 @@ dependencies = [ "ring", "rstest", "rustls-pemfile", + "rustversion", "serde", "serde_json", "serde_test", @@ -942,6 +943,12 @@ dependencies = [ "untrusted", ] +[[package]] +name = "rustversion" +version = "1.0.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "955d28af4278de8121b7ebeb796b6a45735dc01436d898801014aced2773a3d6" + [[package]] name = "ryu" version = "1.0.18" diff --git a/Cargo.toml b/Cargo.toml index 554aae9ff..a0f31682b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -35,6 +35,7 @@ parking_lot = { version = "0.12", optional = true } moka = { version = "0.12.3", optional = true, features = ["future"] } proc-macro2 = { version = "1.0.69", optional = true } # Force proc-macro2 to at least 1.0.69 for minimal-version build ring = { version = "0.17", optional = true } +rustversion = { version = "1", optional = true } serde = { version = "1.0.130", optional = true, features = ["derive"] } siphasher = { version = "1", optional = true } smallvec = { version = "1.3", optional = true } @@ -61,7 +62,7 @@ zonefile = ["bytes", "serde", "std"] # Unstable features unstable-client-transport = ["moka", "net", "tracing"] -unstable-server-transport = ["arc-swap", "chrono/clock", "libc", "net", "siphasher", "tracing"] +unstable-server-transport = ["arc-swap", "chrono/clock", "libc", "net", "rustversion", "siphasher", "tracing"] unstable-stelline = ["tokio/test-util", "tracing", "tracing-subscriber", "unstable-server-transport", "zonefile"] unstable-validator = ["validate", "zonefile", "unstable-client-transport"] unstable-zonetree = ["futures", "parking_lot", "serde", "tokio", "tracing"] diff --git a/src/base/serial.rs b/src/base/serial.rs index 9bb91b46e..9569fdc68 100644 --- a/src/base/serial.rs +++ b/src/base/serial.rs @@ -4,19 +4,21 @@ //! viewed as the 32 bit modulus of a larger number space. Because of that, //! special rules apply when processing these values. This module provides //! the type [`Serial`] that implements these rules. +use core::cmp::Ordering; +use core::{cmp, fmt, str}; + +#[cfg(all(feature = "std", not(test)))] +use std::time::{SystemTime, UNIX_EPOCH}; -use super::cmp::CanonicalOrd; -use super::scan::{Scan, Scanner}; -use super::wire::{Compose, Composer, Parse, ParseError}; #[cfg(feature = "chrono")] use chrono::{DateTime, TimeZone}; -use core::cmp::Ordering; -use core::{cmp, fmt, str}; #[cfg(all(feature = "std", test))] use mock_instant::thread_local::{SystemTime, UNIX_EPOCH}; use octseq::parse::Parser; -#[cfg(all(feature = "std", not(test)))] -use std::time::{SystemTime, UNIX_EPOCH}; + +use super::cmp::CanonicalOrd; +use super::scan::{Scan, Scanner}; +use super::wire::{Compose, Composer, Parse, ParseError}; //------------ Serial -------------------------------------------------------- @@ -90,6 +92,13 @@ impl Serial { Serial(self.0.wrapping_add(other)) } + #[allow(clippy::should_implement_trait)] + #[must_use] + pub fn sub(self, other: u32) -> Self { + assert!(other <= 0x7FFF_FFFF); + Serial(self.0.wrapping_sub(other)) + } + pub fn scan(scanner: &mut S) -> Result { u32::scan(scanner).map(Into::into) } diff --git a/src/net/client/xfr.rs b/src/net/client/xfr.rs index 0ced33de8..8ef86aa14 100644 --- a/src/net/client/xfr.rs +++ b/src/net/client/xfr.rs @@ -443,7 +443,14 @@ impl ReadState { //------------ RecordResult --------------------------------------------------- -/// An event emitted by [`XfrResponseProcessor`] during transfer processing.` +/// An event emitted by [`XfrResponseProcessor`] during transfer processing. +/// +/// Notes: +/// - `DeleteRecord` preceeds `AddRecord` for the same record. +/// - `BeginBatchDelete` and `BeginBatchAdd` are optional. If used then +/// `BeginBatchDelete` must be the first event emitted. +/// - `BeginBatchDelete` preceeds `BeginBatchAdd` for the same serial. +/// - `AddRecord` cannot occur following `BeginBatchDelete` #[derive(Copy, Clone, Debug, PartialEq, Eq)] pub enum XfrEvent { /// Delete record R in zone serial S. @@ -515,11 +522,10 @@ pub trait XfrEventHandler { /// the case of [`XfrEvent::ProcessingFailed`] for which the return value of /// this handler will be ignored by [`XfrResponseProcessor`]. fn handle_event( - &self, + &mut self, evt: XfrEvent, ) -> impl std::future::Future> + Send; } - //------------ IxfrUpdateMode ------------------------------------------------- #[derive(Copy, Clone, Debug, Default, PartialEq, Eq)] @@ -555,6 +561,9 @@ pub enum Error { /// At least one record in the XFR response sequence is incorrect. Malformed, + /// The event handler failed to handle an emitted event. + EventHandlerError, + /// Processing was already terminated for this XFR response sequence. Terminated, } @@ -998,7 +1007,7 @@ mod tests { impl XfrEventHandler for TestXfrEventHandler { async fn handle_event( - &self, + &mut self, evt: XfrEvent, ) -> Result<(), Error> { trace!("Received event: {evt}"); diff --git a/src/zonetree/in_memory/nodes.rs b/src/zonetree/in_memory/nodes.rs index c8bb26860..d54e14b33 100644 --- a/src/zonetree/in_memory/nodes.rs +++ b/src/zonetree/in_memory/nodes.rs @@ -1,5 +1,6 @@ //! The nodes in a zone tree. +use core::any::Any; use std::boxed::Box; use std::collections::{hash_map, HashMap}; use std::future::Future; @@ -103,9 +104,9 @@ impl ZoneApex { self.children.rollback(version); } - pub fn clean(&self, version: Version) { - self.rrsets.clean(version); - self.children.clean(version); + pub fn remove_all(&self, version: Version) { + self.rrsets.remove_all(version); + self.children.remove_all(version); } pub fn versions(&self) -> &RwLock { @@ -135,7 +136,8 @@ impl ZoneStore for ZoneApex { fn write( self: Arc, - ) -> Pin>>> { + ) -> Pin> + Send + Sync + 'static)>> + { Box::pin(async move { let lock = self.update_lock.clone().lock_owned().await; let version = self.versions().read().current().0.next(); @@ -144,6 +146,10 @@ impl ZoneStore for ZoneApex { as Box }) } + + fn as_any(&self) -> &dyn Any { + self as &dyn Any + } } //--- impl From<&'a ZoneApex> @@ -213,10 +219,10 @@ impl ZoneNode { self.children.rollback(version); } - pub fn clean(&self, version: Version) { - self.rrsets.clean(version); - self.special.write().clean(version); - self.children.clean(version); + pub fn remove_all(&self, version: Version) { + self.rrsets.remove_all(version); + self.special.write().remove(version); + self.children.remove_all(version); } } @@ -276,11 +282,11 @@ impl NodeRrsets { .for_each(|rrset| rrset.rollback(version)); } - pub fn clean(&self, version: Version) { + pub fn remove_all(&self, version: Version) { self.rrsets .write() .values_mut() - .for_each(|rrset| rrset.clean(version)); + .for_each(|rrset| rrset.remove(version)); } pub(super) fn iter(&self) -> NodeRrsetsIter { @@ -322,7 +328,7 @@ impl NodeRrset { } fn remove(&mut self, version: Version) { - self.rrsets.clean(version) + self.rrsets.remove(version) } pub fn rollback(&mut self, version: Version) { @@ -385,11 +391,11 @@ impl NodeChildren { .for_each(|item| item.rollback(version)) } - fn clean(&self, version: Version) { + fn remove_all(&self, version: Version) { self.children .read() .values() - .for_each(|item| item.clean(version)) + .for_each(|item| item.remove_all(version)) } pub(super) fn walk( diff --git a/src/zonetree/in_memory/versioned.rs b/src/zonetree/in_memory/versioned.rs index ce4ce2b38..85b1d99e7 100644 --- a/src/zonetree/in_memory/versioned.rs +++ b/src/zonetree/in_memory/versioned.rs @@ -21,6 +21,9 @@ impl Version { pub fn next(self) -> Version { Version(self.0.add(1)) } + pub fn prev(self) -> Version { + Version(self.0.sub(1)) + } } impl Default for Version { @@ -68,8 +71,25 @@ impl Versioned { } } - pub fn clean(&mut self, version: Version) { - self.data.retain(|item| item.0 >= version) + pub fn remove(&mut self, version: Version) { + // WARNING: This isn't safe to do while updating a zone, e.g. via an + // AXFR that lacks some records that were in the previous version of + // the zone, as the effects are immediately visible to users of the + // zone! + // + // self.data.retain(|item| item.0 >= version) + // + // When updating a Zone via ZoneStore::write(), the new version of the + // zone that is created will be one higher than the highest version of + // data currently in the zone. + // + // So adding an empty value at the new version will cause current + // clients to continue seeing the old version, but clients of the zone + // after it is committed will see the new version, i.e. the empty + // value which will cause get() to return None. + if self.data.last().map(|item| item.0) != None { + self.data.push((version, None)); + } } } diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index c4a787b5f..4aa6c04f4 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -5,6 +5,7 @@ use std::boxed::Box; use std::future::Future; use std::pin::Pin; use std::sync::Arc; +use std::sync::Mutex; use std::sync::Weak; use std::vec::Vec; use std::{fmt, io}; @@ -12,15 +13,19 @@ use std::{fmt, io}; use futures::future::Either; use parking_lot::RwLock; use tokio::sync::OwnedMutexGuard; +use tracing::trace; use crate::base::iana::Rtype; use crate::base::name::Label; -use crate::zonetree::types::ZoneCut; -use crate::zonetree::SharedRr; +use crate::base::NameBuilder; +use crate::zonetree::types::{ZoneCut, ZoneDiff}; +use crate::zonetree::StoredName; +use crate::zonetree::{Rrset, SharedRr}; use crate::zonetree::{SharedRrset, WritableZone, WritableZoneNode}; use super::nodes::{Special, ZoneApex, ZoneNode}; use super::versioned::{Version, VersionMarker}; +use crate::rdata::ZoneRecordData; //------------ WriteZone ----------------------------------------------------- @@ -30,6 +35,7 @@ pub struct WriteZone { version: Version, dirty: bool, zone_versions: Arc>, + diff: Arc>>>>, } impl WriteZone { @@ -45,6 +51,7 @@ impl WriteZone { version, dirty: false, zone_versions, + diff: Arc::new(Mutex::new(None)), } } } @@ -59,6 +66,7 @@ impl Clone for WriteZone { version: self.version, dirty: self.dirty, zone_versions: self.zone_versions.clone(), + diff: self.diff.clone(), } } } @@ -80,12 +88,23 @@ impl WritableZone for WriteZone { #[allow(clippy::type_complexity)] fn open( &self, + create_diff: bool, ) -> Pin< Box< - dyn Future, io::Error>>, + dyn Future, io::Error>> + + Send + + Sync, >, > { - let res = WriteNode::new_apex(self.clone()) + let new_apex = WriteNode::new_apex(self.clone(), create_diff); + + if let Ok(write_node) = &new_apex { + // Note: the start and end serial of the diff will be filled in + // when commit() is invoked. + *self.diff.lock().unwrap() = write_node.diff(); + } + + let res = new_apex .map(|node| Box::new(node) as Box) .map_err(|err| { io::Error::new( @@ -93,25 +112,134 @@ impl WritableZone for WriteZone { format!("Open error: {err}"), ) }); + Box::pin(ready(res)) } fn commit( &mut self, - ) -> Pin>>> { + bump_soa_serial: bool, + ) -> Pin< + Box< + dyn Future, io::Error>> + + Send + + Sync, + >, + > { + let mut out_diff = None; + + // An empty zone that is being filled by AXFR won't have an existing SOA. + if let Some(old_soa_rr) = self.apex.get_soa(self.version.prev()) { + let ZoneRecordData::Soa(old_soa) = old_soa_rr.data() else { + unreachable!() + }; + trace!("Commit: old_soa={old_soa:#?}"); + + if bump_soa_serial { + // Ensure that the SOA record in the zone is updated. + let mut new_soa_rrset = + Rrset::new(Rtype::SOA, old_soa_rr.ttl()); + let new_soa_serial = old_soa.serial().add(1); + let new_soa_data = crate::rdata::Soa::new( + old_soa.mname().clone(), + old_soa.rname().clone(), + new_soa_serial, + old_soa.refresh(), + old_soa.retry(), + old_soa.expire(), + old_soa.minimum(), + ); + new_soa_rrset.push_data(new_soa_data.into()); + trace!("Commit: new_soa={new_soa_rrset:#?}"); + let new_soa_shared_rrset = SharedRrset::new(new_soa_rrset); + + self.apex + .rrsets() + .update(new_soa_shared_rrset.clone(), self.version); + } + + // Extract the created diff, if any. + if let Some(diff) = self.diff.lock().unwrap().take() { + let new_soa_rr = self.apex.get_soa(self.version).unwrap(); + let ZoneRecordData::Soa(new_soa) = new_soa_rr.data() else { + unreachable!() + }; + + let diff = arc_into_inner(diff).unwrap(); + let mut diff = Mutex::into_inner(diff).unwrap(); + diff.start_serial = Some(old_soa.serial()); + diff.end_serial = Some(new_soa.serial()); + + if bump_soa_serial { + let mut removed_soa_rrset = + Rrset::new(Rtype::SOA, old_soa_rr.ttl()); + removed_soa_rrset.push_data(old_soa_rr.data().clone()); + let removed_soa_rrset = + SharedRrset::new(removed_soa_rrset); + + let mut new_soa_shared_rrset = + Rrset::new(Rtype::SOA, new_soa_rr.ttl()); + new_soa_shared_rrset.push_data(new_soa_rr.data().clone()); + let new_soa_rrset = + SharedRrset::new(new_soa_shared_rrset); + + let k = (self.apex.name().clone(), Rtype::SOA); + trace!("Diff: recording removal of old SOA: {removed_soa_rrset:#?}"); + diff.removed.insert(k.clone(), removed_soa_rrset); + + trace!( + "Diff: recording addition of new SOA: {new_soa_rrset:#?}" + ); + diff.added.insert(k, new_soa_rrset); + } + + out_diff = Some(diff); + } + } + + // Make the new version visible. + trace!("Commit: Making zone version '{:#?}' current", self.version); let marker = self.zone_versions.write().update_current(self.version); self.zone_versions .write() .push_version(self.version, marker); + trace!("Commit: zone versions: {:#?}", self.zone_versions); + trace!("Commit: zone dump:\n{:#?}", self.apex); + // Start the next version. self.version = self.version.next(); self.dirty = false; - Box::pin(ready(Ok(()))) + Box::pin(ready(Ok(out_diff))) } } +#[rustversion::since(1.70.0)] +fn arc_into_inner(this: Arc>) -> Option> { + #[allow(clippy::incompatible_msrv)] + Arc::into_inner(this) +} + +#[rustversion::before(1.70.0)] +fn arc_into_inner(this: Arc>) -> Option> { + // From: https://doc.rust-lang.org/std/sync/struct.Arc.html#method.into_inner + // + // "If Arc::into_inner is called on every clone of this Arc, it is + // guaranteed that exactly one of the calls returns the inner value. This + // means in particular that the inner value is not dropped. + // + // Arc::try_unwrap is conceptually similar to Arc::into_inner, but it is + // meant for different use-cases. If used as a direct replacement for + // Arc::into_inner anyway, such as with the expression + // Arc::try_unwrap(this).ok(), then it does not give the same guarantee as + // described in the previous paragraph. For more information, see the + // examples below and read the documentation of Arc::try_unwrap." + // + // In our case there is no other thread trying to unwrap the value. + Arc::try_unwrap(this).ok() +} + //------------ WriteNode ------------------------------------------------------ pub struct WriteNode { @@ -120,27 +248,56 @@ pub struct WriteNode { /// The node we are updating. node: Either, Arc>, + + /// The diff we are building, if enabled. + diff: Option<(StoredName, Arc>)>, } impl WriteNode { - fn new_apex(zone: WriteZone) -> Result { + fn new_apex( + zone: WriteZone, + create_diff: bool, + ) -> Result { let apex = zone.apex.clone(); + + let diff = if create_diff { + Some(( + zone.apex.name().clone(), + Arc::new(Mutex::new(ZoneDiff::new())), + )) + } else { + None + }; + Ok(WriteNode { zone, node: Either::Left(apex), + diff, }) } + fn update_child(&self, label: &Label) -> Result { let children = match self.node { Either::Left(ref apex) => apex.children(), Either::Right(ref node) => node.children(), }; + let (node, created) = children .with_or_default(label, |node, created| (node.clone(), created)); + + let diff = self.diff.as_ref().map(|(owner, diff)| { + let mut builder = NameBuilder::new_bytes(); + builder.append_label(label.as_slice()).unwrap(); + let new_owner = builder.append_origin(&owner).unwrap(); + (new_owner, diff.clone()) + }); + let node = WriteNode { zone: self.zone.clone(), node: Either::Right(node), + diff, }; + if created { node.make_regular()?; } @@ -153,18 +310,79 @@ impl WriteNode { Either::Right(ref apex) => apex.rrsets(), Either::Left(ref node) => node.rrsets(), }; + + trace!("Updating RRset"); + if let Some((owner, diff)) = &self.diff { + let k = (owner.clone(), rrset.rtype()); + + let changed = if let Some(removed_rrset) = + rrsets.get(rrset.rtype(), self.zone.version.prev()) + { + let changed = rrset != removed_rrset; + + if changed && !removed_rrset.is_empty() { + trace!("Diff detected: update of existing RRSET - recording removal of the current RRSET: {removed_rrset:#?}"); + diff.lock() + .unwrap() + .removed + .insert(k.clone(), removed_rrset.clone()); + } + + changed + } else { + true + }; + + if changed && !rrset.is_empty() { + trace!("Diff detected: update of existing RRSET - recording addition of the new RRSET: {rrset:#?}"); + diff.lock().unwrap().added.insert(k, rrset.clone()); + } + } + + // if rrset.is_empty() { + // rrsets.remove(rrset.rtype(), self.zone.version.prev()); + // } else { rrsets.update(rrset, self.zone.version); + // } self.check_nx_domain()?; Ok(()) } + fn get_rrset( + &self, + rtype: Rtype, + ) -> Result, io::Error> { + let rrsets = match self.node { + Either::Left(ref apex) => apex.rrsets(), + Either::Right(ref node) => node.rrsets(), + }; + + Ok(rrsets.get(rtype, self.zone.version)) + } + fn remove_rrset(&self, rtype: Rtype) -> Result<(), io::Error> { let rrsets = match self.node { Either::Left(ref apex) => apex.rrsets(), Either::Right(ref node) => node.rrsets(), }; + + if let Some((owner, diff)) = &self.diff { + if let Some(removed) = rrsets.get(rtype, self.zone.version.prev()) + { + trace!( + "Diff detected: removal of existing RRSET: {removed:#?}" + ); + let k = (owner.clone(), rtype); + diff.lock() + .unwrap() + .removed + .insert(k.clone(), removed.clone()); + } + } + rrsets.remove(rtype, self.zone.version); self.check_nx_domain()?; + Ok(()) } @@ -214,6 +432,19 @@ impl WriteNode { }) } + fn remove_all(&self) -> Result<(), io::Error> { + match self.node { + Either::Left(ref apex) => { + apex.remove_all(self.zone.version); + } + Either::Right(ref node) => { + node.remove_all(self.zone.version); + } + } + + Ok(()) + } + /// Makes sure a NXDomain special is set or removed as necesssary. fn check_nx_domain(&self) -> Result<(), io::Error> { let node = match self.node { @@ -250,6 +481,10 @@ impl WriteNode { } Ok(()) } + + fn diff(&self) -> Option>> { + self.diff.as_ref().map(|(_, diff)| diff.clone()) + } } //--- impl WritableZoneNode @@ -261,7 +496,9 @@ impl WritableZoneNode for WriteNode { label: &Label, ) -> Pin< Box< - dyn Future, io::Error>>, + dyn Future, io::Error>> + + Send + + Sync, >, > { let node = self @@ -273,36 +510,61 @@ impl WritableZoneNode for WriteNode { fn update_rrset( &self, rrset: SharedRrset, - ) -> Pin>>> { + ) -> Pin> + Send + Sync>> + { Box::pin(ready(self.update_rrset(rrset))) } + fn get_rrset( + &self, + rtype: Rtype, + ) -> Pin< + Box< + dyn Future, io::Error>> + + Send + + Sync, + >, + > { + Box::pin(ready(self.get_rrset(rtype))) + } + fn remove_rrset( &self, rtype: Rtype, - ) -> Pin>>> { + ) -> Pin> + Send + Sync>> + { Box::pin(ready(self.remove_rrset(rtype))) } fn make_regular( &self, - ) -> Pin>>> { + ) -> Pin> + Send + Sync>> + { Box::pin(ready(self.make_regular())) } fn make_zone_cut( &self, cut: ZoneCut, - ) -> Pin>>> { + ) -> Pin> + Send + Sync>> + { Box::pin(ready(self.make_zone_cut(cut))) } fn make_cname( &self, cname: SharedRr, - ) -> Pin>>> { + ) -> Pin> + Send + Sync>> + { Box::pin(ready(self.make_cname(cname))) } + + fn remove_all( + &self, + ) -> Pin> + Send + Sync>> + { + Box::pin(ready(self.remove_all())) + } } //------------ WriteApexError ------------------------------------------------ @@ -357,6 +619,10 @@ pub struct ZoneVersions { impl ZoneVersions { pub fn update_current(&mut self, version: Version) -> Arc { let marker = Arc::new(VersionMarker); + trace!( + "Changing current zone version from {:?} to {version:?}", + self.current + ); self.current = (version, marker.clone()); marker } @@ -366,6 +632,7 @@ impl ZoneVersions { version: Version, marker: Arc, ) { + trace!("Pushing new zone version {version:?}"); self.all.push((version, Arc::downgrade(&marker))) } diff --git a/src/zonetree/mod.rs b/src/zonetree/mod.rs index cc7b6f7d2..206f75b5b 100644 --- a/src/zonetree/mod.rs +++ b/src/zonetree/mod.rs @@ -89,13 +89,16 @@ mod tree; pub mod types; mod walk; mod zone; +pub mod xfr_event_handler; pub use self::answer::{Answer, AnswerAuthority, AnswerContent}; pub use self::in_memory::ZoneBuilder; pub use self::traits::{ ReadableZone, WritableZone, WritableZoneNode, ZoneStore, }; -pub use self::tree::ZoneTree; -pub use self::types::{Rrset, SharedRr, SharedRrset}; +pub use self::tree::{ZoneSetIter, ZoneTree}; +pub use self::types::{ + Rrset, SharedRr, SharedRrset, StoredName, StoredRecord, ZoneDiff, +}; pub use self::walk::WalkOp; -pub use self::zone::Zone; +pub use self::zone::{Zone, ZoneKey}; diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index 762fcc1af..716d318d4 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -7,23 +7,26 @@ //! //! //! [`ZoneTree`]: super::ZoneTree -use bytes::Bytes; +use core::any::Any; use core::future::ready; use core::pin::Pin; + use std::boxed::Box; use std::fmt::Debug; use std::future::Future; use std::io; use std::sync::Arc; +use bytes::Bytes; + use crate::base::iana::Class; use crate::base::name::Label; use crate::base::{Name, Rtype}; use super::answer::Answer; use super::error::OutOfZone; -use super::types::{StoredName, ZoneCut}; -use super::{SharedRr, SharedRrset, WalkOp}; +use super::types::{ZoneCut, ZoneDiff}; +use super::{SharedRr, SharedRrset, StoredName, WalkOp}; //------------ ZoneStore ----------------------------------------------------- @@ -33,7 +36,7 @@ use super::{SharedRr, SharedRrset, WalkOp}; /// a particular backing store implementation. /// /// [`Zone`]: super::Zone -pub trait ZoneStore: Debug + Sync + Send { +pub trait ZoneStore: Debug + Sync + Send + Any { /// Returns the class of the zone. fn class(&self) -> Class; @@ -46,7 +49,17 @@ pub trait ZoneStore: Debug + Sync + Send { /// Get a write interface to this store. fn write( self: Arc, - ) -> Pin>>>; + ) -> Pin< + Box< + (dyn Future> + + Send + + Sync + + 'static), + >, + >; + + /// TODO + fn as_any(&self) -> &dyn Any; } //------------ ReadableZone -------------------------------------------------- @@ -57,7 +70,7 @@ pub trait ZoneStore: Debug + Sync + Send { /// the [`ZoneStore`] backing storage for a [`Zone`]. /// /// [`Zone`]: super::Zone -pub trait ReadableZone: Send { +pub trait ReadableZone: Send + Sync { /// Returns true if ths `_async` variants of the functions offered by this /// trait should be used by callers instead of the non-`_async` /// equivalents. @@ -96,7 +109,8 @@ pub trait ReadableZone: Send { &self, qname: Name, qtype: Rtype, - ) -> Pin> + Send>> { + ) -> Pin> + Send + Sync>> + { Box::pin(ready(self.query(qname, qtype))) } @@ -104,7 +118,7 @@ pub trait ReadableZone: Send { fn walk_async( &self, op: WalkOp, - ) -> Pin + Send>> { + ) -> Pin + Send + Sync>> { self.walk(op); Box::pin(ready(())) } @@ -115,29 +129,38 @@ pub trait ReadableZone: Send { /// An asynchronous write interface to a [`Zone`]. /// /// [`Zone`]: super::Zone -pub trait WritableZone { +pub trait WritableZone: Send + Sync { /// Start a write operation for the zone. #[allow(clippy::type_complexity)] fn open( &self, + create_diff: bool, ) -> Pin< Box< - dyn Future, io::Error>>, + dyn Future, io::Error>> + + Send + + Sync, >, >; /// Complete a write operation for the zone. /// - /// This function commits the changes accumulated since - /// [`WritableZone::open`] was invoked. Clients who obtain a - /// [`ReadableZone`] interface to this zone _before_ this function has - /// been called will not see any of the changes made since the last - /// commit. Only clients who obtain a [`ReadableZone`] _after_ invoking - /// this function will be able to see the changes made since - /// [`WritableZone::open`] was called. called. + /// This function commits the changes accumulated since [`open`] was + /// invoked. Clients who obtain a [`ReadableZone`] interface to this zone + /// _before_ this function has been called will not see any of the changes + /// made since the last commit. Only clients who obtain a [`ReadableZone`] + /// _after_ invoking this function will be able to see the changes made + /// since [`open`] was called. fn commit( &mut self, - ) -> Pin>>>; + bump_soa_serial: bool, + ) -> Pin< + Box< + dyn Future, io::Error>> + + Send + + Sync, + >, + >; } //------------ WritableZoneNode ---------------------------------------------- @@ -145,7 +168,7 @@ pub trait WritableZone { /// An asynchronous write interface to a particular node in a [`ZoneTree`]. /// /// [`ZoneTree`]: super::ZoneTree -pub trait WritableZoneNode { +pub trait WritableZoneNode: Send + Sync { /// Get a write interface to a child node of this node. #[allow(clippy::type_complexity)] fn update_child( @@ -153,7 +176,21 @@ pub trait WritableZoneNode { label: &Label, ) -> Pin< Box< - dyn Future, io::Error>>, + dyn Future, io::Error>> + + Send + + Sync, + >, + >; + + /// Get an RRset of the given type at this node, if any. + fn get_rrset( + &self, + rtype: Rtype, + ) -> Pin< + Box< + dyn Future, io::Error>> + + Send + + Sync, >, >; @@ -161,13 +198,13 @@ pub trait WritableZoneNode { fn update_rrset( &self, rrset: SharedRrset, - ) -> Pin>>>; + ) -> Pin> + Send + Sync>>; /// Remove an RRset of the given type at this node, if any. fn remove_rrset( &self, rtype: Rtype, - ) -> Pin>>>; + ) -> Pin> + Send + Sync>>; /// Mark this node as a regular node. /// @@ -175,7 +212,7 @@ pub trait WritableZoneNode { /// function will erase that data. fn make_regular( &self, - ) -> Pin>>>; + ) -> Pin> + Send + Sync>>; /// Mark this node as a zone cut. /// @@ -183,7 +220,7 @@ pub trait WritableZoneNode { fn make_zone_cut( &self, cut: ZoneCut, - ) -> Pin>>>; + ) -> Pin> + Send + Sync>>; /// Mark this node as a CNAME. /// @@ -191,5 +228,10 @@ pub trait WritableZoneNode { fn make_cname( &self, cname: SharedRr, - ) -> Pin>>>; + ) -> Pin> + Send + Sync>>; + + /// Recursively make all content at and below this point appear to be removed. + fn remove_all( + &self, + ) -> Pin> + Send + Sync>>; } diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index c66a9256d..dae95f3a4 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -1,5 +1,6 @@ //! Zone tree related types. +use std::collections::HashMap; use std::ops; use std::sync::Arc; use std::vec::Vec; @@ -7,9 +8,11 @@ use std::vec::Vec; use bytes::Bytes; use serde::{Deserialize, Serialize}; +use crate::base::name::Name; use crate::base::rdata::RecordData; +use crate::base::record::Record; +use crate::base::Serial; use crate::base::{iana::Rtype, Ttl}; -use crate::base::{Name, Record}; use crate::rdata::ZoneRecordData; //------------ Type Aliases -------------------------------------------------- @@ -247,3 +250,32 @@ pub struct ZoneCut { /// Zero or more glue records at the zone cut. pub glue: Vec, } + +//------------ ZoneDiff ------------------------------------------------------ + +/// The differences between one serial and another for a Zone. +#[derive(Clone, Debug, Default)] +pub struct ZoneDiff { + /// The serial number of the Zone which was modified. + /// + /// For a completed diff this must be Some. + pub start_serial: Option, + + /// The serial number of the Zone that resulted from the modifications. + /// + /// For a completed diff this must be Some. + pub end_serial: Option, + + /// The records added to the Zone. + pub added: HashMap<(StoredName, Rtype), SharedRrset>, + + /// The records removed from the Zone. + pub removed: HashMap<(StoredName, Rtype), SharedRrset>, +} + +impl ZoneDiff { + /// Creates a new empty diff. + pub fn new() -> Self { + Self::default() + } +} diff --git a/src/zonetree/xfr_event_handler.rs b/src/zonetree/xfr_event_handler.rs new file mode 100644 index 000000000..773916b39 --- /dev/null +++ b/src/zonetree/xfr_event_handler.rs @@ -0,0 +1,355 @@ +//! Support for applying XFR changes to a [`Zone`]. +use crate::net::client::xfr::{Error, XfrEvent, XfrEventHandler, XfrRecord}; + +use super::error::OutOfZone; +use super::{WritableZone, WritableZoneNode, Zone}; +use crate::base::name::{FlattenInto, Label, ToLabelIter}; +use crate::base::{Name, Record, Rtype, ToName}; +use crate::rdata::ZoneRecordData; +use crate::zonetree::{Rrset, SharedRrset}; +use bytes::Bytes; +use std::borrow::ToOwned; +use std::boxed::Box; +use tracing::{error, trace}; + +struct ZoneUpdateEventHandler { + zone: Zone, + + write: WriteState, + + batching: bool, + + first_event_seen: bool, +} + +impl ZoneUpdateEventHandler { + async fn new(zone: Zone) -> std::io::Result { + let write = WriteState::new(&zone).await?; + + Ok(Self { + zone, + write, + batching: false, + first_event_seen: false, + }) + } + + async fn init_batch(&mut self) -> Result<(), Error> { + if self.batching { + self.write = WriteState::new(&self.zone) + .await + .map_err(|_| Error::EventHandlerError)?; + } + + Ok(()) + } + + fn mk_relative_name_iterator<'l>( + apex_name: &Name, + qname: &'l impl ToName, + ) -> Result + Clone, OutOfZone> { + let mut qname = qname.iter_labels().rev(); + for apex_label in apex_name.iter_labels().rev() { + let qname_label = qname.next(); + if Some(apex_label) != qname_label { + error!( + "Qname '{qname_label:?}' is not in zone '{apex_name}'" + ); + return Err(OutOfZone); + } + } + Ok(qname) + } + + async fn prep_add_del( + &mut self, + rec: XfrRecord, + ) -> Result< + ( + Rtype, + ZoneRecordData>, + Option>, + Rrset, + ), + Error, + > { + let owner = rec.owner().to_owned(); + let ttl = rec.ttl(); + let rtype = rec.rtype(); + let data: ZoneRecordData> = + rec.into_data().flatten_into(); + + let mut end_node: Option> = None; + + let name = + Self::mk_relative_name_iterator(self.zone.apex_name(), &owner) + .map_err(|_| Error::EventHandlerError)?; + + let writable = self.write.writable.as_ref().unwrap(); + + for label in name { + trace!("Relativised label: {label}"); + end_node = Some( + match end_node { + Some(new_node) => new_node.update_child(label), + None => writable.update_child(label), + } + .await + .map_err(|_| Error::EventHandlerError)?, + ); + } + + let rrset = Rrset::new(rtype, ttl); + Ok((rtype, data, end_node, rrset)) + } +} + +impl XfrEventHandler for ZoneUpdateEventHandler { + async fn handle_event( + &mut self, + evt: XfrEvent, + ) -> Result<(), Error> { + match evt { + XfrEvent::DeleteRecord(_serial, rec) => { + let (rtype, data, end_node, mut rrset) = + self.prep_add_del(rec).await?; + + let writable = self.write.writable.as_ref().unwrap(); + + trace!("Deleting RR for {rtype}"); + + match end_node { + Some(n) => { + trace!("Deleting RR at end_node"); + + if let Some(existing_rrset) = n + .get_rrset(rtype) + .await + .map_err(|_| Error::EventHandlerError)? + { + for existing_data in existing_rrset.data() { + if existing_data != &data { + rrset.push_data(existing_data.clone()); + } + } + + trace!("Removing single RR of {rtype} so updating RRSET"); + n.update_rrset(SharedRrset::new(rrset)) + .await + .map_err(|_| Error::EventHandlerError)?; + } + } + None => { + trace!("Deleting RR at root"); + if let Some(existing_rrset) = writable + .get_rrset(rtype) + .await + .map_err(|_| Error::EventHandlerError)? + { + for existing_data in existing_rrset.data() { + if existing_data != &data { + rrset.push_data(existing_data.clone()); + } + } + + trace!("Removing single RR of {rtype} so updating RRSET"); + writable + .update_rrset(SharedRrset::new(rrset)) + .await + .map_err(|_| Error::EventHandlerError)?; + } + } + } + } + + XfrEvent::AddRecord(_serial, rec) => { + self.init_batch().await?; + + if !self.first_event_seen && rec.rtype() == Rtype::SOA { + // If the first event is the addition of a SOA record to + // the zone, this must be a complete replacement of the + // zone (as you can't have two SOA records), i.e. + // something like an AXFR transfer. We can't add records + // from a new version of the zone to an existing zone + // because if the old version contained a record which the + // new version does not, it would get left behind. So in + // this case we have to mark all of the existing records + // in the zone as "removed" and then add new records. This + // allows the old records to continue being served to + // current consumers while the zone is being updated. + self.write + .remove_all() + .await + .map_err(|_| Error::EventHandlerError)?; + } + + let (rtype, data, end_node, mut rrset) = + self.prep_add_del(rec).await?; + + let writable = self.write.writable.as_ref().unwrap(); + + trace!("Adding RR: {:?}", rrset); + rrset.push_data(data); + + match end_node { + Some(n) => { + trace!("Adding RR at end_node"); + + if let Some(existing_rrset) = n + .get_rrset(rtype) + .await + .map_err(|_| Error::EventHandlerError)? + { + for existing_data in existing_rrset.data() { + rrset.push_data(existing_data.clone()); + } + } + + n.update_rrset(SharedRrset::new(rrset)) + .await + .map_err(|_| Error::EventHandlerError)?; + } + None => { + trace!("Adding RR at root"); + writable + .update_rrset(SharedRrset::new(rrset)) + .await + .map_err(|_| Error::EventHandlerError)?; + } + } + } + + XfrEvent::BeginBatchDelete(_) => { + if self.batching { + // Commit the previous batch. + self.write.commit().await?; + } + + self.batching = true; + } + + XfrEvent::BeginBatchAdd(_) => { + self.batching = true; + } + + XfrEvent::EndOfTransfer => { + // Commit the previous batch. + self.write.commit().await?; + } + + XfrEvent::ProcessingFailed => { + // ??? + } + } + + self.first_event_seen = true; + + Ok(()) + } +} + +//------------ WriteState ----------------------------------------------------- + +struct WriteState { + write: Box, + writable: Option>, +} + +impl WriteState { + async fn new(zone: &Zone) -> std::io::Result { + let write = zone.write().await; + let writable = Some(write.open(true).await?); + Ok(Self { write, writable }) + } + + async fn remove_all(&mut self) -> std::io::Result<()> { + if let Some(writable) = &mut self.writable { + writable.remove_all().await?; + } + + Ok(()) + } + + async fn commit(&mut self) -> Result<(), Error> { + // Commit the deletes and adds that just occurred + if let Some(writable) = self.writable.take() { + // Ensure that there are no dangling references to the created + // diff (otherwise commit() will panic). + drop(writable); + self.write + .commit(false) + .await + .map_err(|_| Error::EventHandlerError)?; + } + + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use core::str::FromStr; + + use crate::base::iana::Class; + use crate::zonetree::ZoneBuilder; + + use super::*; + use crate::base::{ParsedName, Serial, Ttl}; + use crate::rdata::Soa; + + #[tokio::test] + async fn simple_test() { + init_logging(); + + let zone = mk_empty_zone("example.com"); + + let mut evt_handler = + ZoneUpdateEventHandler::new(zone.clone()).await.unwrap(); + + let s = Serial::now(); + let soa = mk_soa(s); + let soa = ZoneRecordData::Soa(soa); + let soa = Record::new( + ParsedName::from(Name::from_str("example.com").unwrap()), + Class::IN, + Ttl::from_secs(0), + soa, + ); + + evt_handler + .handle_event(XfrEvent::AddRecord(s, soa)) + .await + .unwrap(); + + evt_handler + .handle_event(XfrEvent::EndOfTransfer) + .await + .unwrap(); + } + + //------------ Helper functions ------------------------------------------- + + fn init_logging() { + // Initialize tracing based logging. Override with env var RUST_LOG, e.g. + // RUST_LOG=trace. DEBUG level will show the .rpl file name, Stelline step + // numbers and types as they are being executed. + tracing_subscriber::fmt() + .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) + .with_thread_ids(true) + .without_time() + .try_init() + .ok(); + } + + fn mk_empty_zone(apex_name: &str) -> Zone { + ZoneBuilder::new(Name::from_str(apex_name).unwrap(), Class::IN) + .build() + } + + fn mk_soa(serial: Serial) -> Soa> { + let mname = ParsedName::from(Name::from_str("mname").unwrap()); + let rname = ParsedName::from(Name::from_str("rname").unwrap()); + let ttl = Ttl::from_secs(0); + Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl) + } +} diff --git a/src/zonetree/zone.rs b/src/zonetree/zone.rs index 0af5d396d..dfa7a6013 100644 --- a/src/zonetree/zone.rs +++ b/src/zonetree/zone.rs @@ -13,6 +13,11 @@ use super::traits::WritableZone; use super::types::StoredName; use super::{parsed, ReadableZone, ZoneStore}; +//------------ ZoneKey ------------------------------------------------------- + +/// TODO +pub type ZoneKey = (StoredName, Class); + //------------ Zone ---------------------------------------------------------- /// A single DNS zone. @@ -21,6 +26,12 @@ pub struct Zone { store: Arc, } +impl AsRef for Zone { + fn as_ref(&self) -> &dyn ZoneStore { + self.store.as_ref() + } +} + impl Zone { /// Creates a new [`Zone`] instance with the given data. pub fn new(data: impl ZoneStore + 'static) -> Self { @@ -29,6 +40,13 @@ impl Zone { } } + /// TODO + pub fn into_inner(self) -> Arc { + self.store + } +} + +impl Zone { /// Gets the CLASS of this zone. pub fn class(&self) -> Class { self.store.class() @@ -47,9 +65,18 @@ impl Zone { /// Gets a write interface to this zone. pub fn write( &self, - ) -> Pin>>> { + ) -> Pin> + Send + Sync>> + { self.store.clone().write() } + + /// Gets a key that uniquely identifies this zone. + /// + /// Note: Assumes that there is only ever one instance of a zone with a + /// given apex name and class in a set of zones. + pub fn key(&self) -> ZoneKey { + (self.apex_name().clone(), self.class()) + } } //--- TryFrom From c7638d77b2e0439c990fbc67fd2f197d9cac611f Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 22 Aug 2024 13:22:34 +0200 Subject: [PATCH 032/333] A little bit of simplification. --- src/zonetree/xfr_event_handler.rs | 106 ++++++++++-------------------- 1 file changed, 36 insertions(+), 70 deletions(-) diff --git a/src/zonetree/xfr_event_handler.rs b/src/zonetree/xfr_event_handler.rs index 773916b39..cc4fa6735 100644 --- a/src/zonetree/xfr_event_handler.rs +++ b/src/zonetree/xfr_event_handler.rs @@ -1,16 +1,17 @@ //! Support for applying XFR changes to a [`Zone`]. -use crate::net::client::xfr::{Error, XfrEvent, XfrEventHandler, XfrRecord}; +use std::borrow::ToOwned; +use std::boxed::Box; + +use bytes::Bytes; +use tracing::{error, trace}; use super::error::OutOfZone; use super::{WritableZone, WritableZoneNode, Zone}; use crate::base::name::{FlattenInto, Label, ToLabelIter}; -use crate::base::{Name, Record, Rtype, ToName}; +use crate::base::{Name, Rtype, ToName}; +use crate::net::client::xfr::{Error, XfrEvent, XfrEventHandler, XfrRecord}; use crate::rdata::ZoneRecordData; use crate::zonetree::{Rrset, SharedRrset}; -use bytes::Bytes; -use std::borrow::ToOwned; -use std::boxed::Box; -use tracing::{error, trace}; struct ZoneUpdateEventHandler { zone: Zone, @@ -118,48 +119,24 @@ impl XfrEventHandler for ZoneUpdateEventHandler { trace!("Deleting RR for {rtype}"); - match end_node { - Some(n) => { - trace!("Deleting RR at end_node"); - - if let Some(existing_rrset) = n - .get_rrset(rtype) - .await - .map_err(|_| Error::EventHandlerError)? - { - for existing_data in existing_rrset.data() { - if existing_data != &data { - rrset.push_data(existing_data.clone()); - } - } - - trace!("Removing single RR of {rtype} so updating RRSET"); - n.update_rrset(SharedRrset::new(rrset)) - .await - .map_err(|_| Error::EventHandlerError)?; - } - } - None => { - trace!("Deleting RR at root"); - if let Some(existing_rrset) = writable - .get_rrset(rtype) - .await - .map_err(|_| Error::EventHandlerError)? - { - for existing_data in existing_rrset.data() { - if existing_data != &data { - rrset.push_data(existing_data.clone()); - } - } - - trace!("Removing single RR of {rtype} so updating RRSET"); - writable - .update_rrset(SharedRrset::new(rrset)) - .await - .map_err(|_| Error::EventHandlerError)?; + let node = end_node.as_ref().unwrap_or(writable); + + if let Some(existing_rrset) = node + .get_rrset(rtype) + .await + .map_err(|_| Error::EventHandlerError)? + { + for existing_data in existing_rrset.data() { + if existing_data != &data { + rrset.push_data(existing_data.clone()); } } } + + trace!("Removing single RR of {rtype} so updating RRSET"); + node.update_rrset(SharedRrset::new(rrset)) + .await + .map_err(|_| Error::EventHandlerError)?; } XfrEvent::AddRecord(_serial, rec) => { @@ -191,32 +168,21 @@ impl XfrEventHandler for ZoneUpdateEventHandler { trace!("Adding RR: {:?}", rrset); rrset.push_data(data); - match end_node { - Some(n) => { - trace!("Adding RR at end_node"); - - if let Some(existing_rrset) = n - .get_rrset(rtype) - .await - .map_err(|_| Error::EventHandlerError)? - { - for existing_data in existing_rrset.data() { - rrset.push_data(existing_data.clone()); - } - } + let node = end_node.as_ref().unwrap_or(writable); - n.update_rrset(SharedRrset::new(rrset)) - .await - .map_err(|_| Error::EventHandlerError)?; - } - None => { - trace!("Adding RR at root"); - writable - .update_rrset(SharedRrset::new(rrset)) - .await - .map_err(|_| Error::EventHandlerError)?; + if let Some(existing_rrset) = node + .get_rrset(rtype) + .await + .map_err(|_| Error::EventHandlerError)? + { + for existing_data in existing_rrset.data() { + rrset.push_data(existing_data.clone()); } } + + node.update_rrset(SharedRrset::new(rrset)) + .await + .map_err(|_| Error::EventHandlerError)?; } XfrEvent::BeginBatchDelete(_) => { @@ -291,11 +257,11 @@ mod tests { use core::str::FromStr; use crate::base::iana::Class; + use crate::base::{ParsedName, Record, Serial, Ttl}; + use crate::rdata::Soa; use crate::zonetree::ZoneBuilder; use super::*; - use crate::base::{ParsedName, Serial, Ttl}; - use crate::rdata::Soa; #[tokio::test] async fn simple_test() { From 8e9ff88b098639208374e11516f5de3548d513d1 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 22 Aug 2024 13:22:53 +0200 Subject: [PATCH 033/333] Cargo fmt. --- src/net/client/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/client/mod.rs b/src/net/client/mod.rs index cb3efad86..b49d4336b 100644 --- a/src/net/client/mod.rs +++ b/src/net/client/mod.rs @@ -190,7 +190,7 @@ pub mod request; pub mod stream; #[cfg(feature = "tsig")] pub mod tsig; -pub mod xfr; #[cfg(feature = "unstable-validator")] pub mod validator; pub mod validator_test; +pub mod xfr; From e78dbea0630b32f600013d376125a31aff3bbf56 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 22 Aug 2024 13:50:47 +0200 Subject: [PATCH 034/333] Add missing RustDoc comments to satisfy Clippy. --- src/net/client/xfr.rs | 129 ++++++++++++++++++++++++++++++++++-------- 1 file changed, 105 insertions(+), 24 deletions(-) diff --git a/src/net/client/xfr.rs b/src/net/client/xfr.rs index 0ced33de8..fc910b7f2 100644 --- a/src/net/client/xfr.rs +++ b/src/net/client/xfr.rs @@ -29,10 +29,19 @@ pub type XfrRecord = /// [`XfrResponseProcessor`] can be invoked on one ore more sequentially /// AXFR/IXFR received response messages to verify them and during processing /// emit events which an implementor of [`XfrEventHandler`] can handle. +/// +/// Each instance of [`XfrResponseProcessosr`] should process a single XFR +/// response sequence. Once an instance of [`XfrResponseProcessosr`] has +/// finished processing an XFR response sequence it must be discarded. +/// Attempting to use it once processing has finished will result in an error. +/// To process another XFR response sequence create another instance of +/// [`XfrResponseProcessor`]. pub struct XfrResponseProcessor { + /// The event handler that events will be sent to for handling. evt_handler: T, - mode: Mode, + /// The current processing state. + state: State, } impl XfrResponseProcessor { @@ -42,7 +51,7 @@ impl XfrResponseProcessor { pub fn new(evt_handler: T) -> Self { Self { evt_handler, - mode: Mode::default(), + state: State::default(), } } @@ -77,10 +86,10 @@ impl XfrResponseProcessor { // file or not? let mut records = answer.limit_to::>(); - match self.mode { + match self.state { // When given the first response in a sequence, do some initial // setup. - Mode::AwaitingFirstAnswer => { + State::AwaitingFirstAnswer => { let Some(Ok(record)) = records.next() else { return Err(Error::Malformed); }; @@ -93,7 +102,7 @@ impl XfrResponseProcessor { } // For subsequent messages make sure that the XFR - Mode::AwaitingNextAnswer { + State::AwaitingNextAnswer { initial_xfr_type, initial_query_id, .. @@ -109,13 +118,13 @@ impl XfrResponseProcessor { } } - Mode::TransferComplete => { + State::TransferComplete => { // We already finished processing an XFR response sequence. We // don't expect there to be any more messages to process!. return Err(Error::Malformed); } - Mode::TransferFailed => { + State::TransferFailed => { // We had ot terminate processing of the XFR response sequence // due to a problem with the received data, so we don't expect // to be invoked again with another response message! @@ -123,7 +132,7 @@ impl XfrResponseProcessor { } }; - let Mode::AwaitingNextAnswer { read, .. } = &mut self.mode else { + let State::AwaitingNextAnswer { read, .. } = &mut self.state else { unreachable!(); }; @@ -133,13 +142,13 @@ impl XfrResponseProcessor { if let Some(event) = read.record(record).await? { match event { XfrEvent::EndOfTransfer => { - self.mode = Mode::TransferComplete; + self.state = State::TransferComplete; self.evt_handler.handle_event(event).await?; return Ok(true); } XfrEvent::ProcessingFailed => { - self.mode = Mode::TransferFailed; + self.state = State::TransferFailed; let _ = self.evt_handler.handle_event(event).await; return Err(Error::Malformed); } @@ -156,14 +165,22 @@ impl XfrResponseProcessor { Ok(false) } + /// Check if an XFR response header is valid. + /// + /// Enforce the rules defined in 2. AXFR Messages of RFC 5936. See: + /// https://www.rfc-editor.org/rfc/rfc5936.html#section-2 + /// + /// Takes a request as well as a response as the response is checked to + /// see if it is in reply to the given request. + /// + /// Returns Ok on success, Err otherwise. On success the type of XFR that + /// was determined is returned as well as the answer section from the XFR + /// response. async fn check_is_xfr_answer<'a>( &mut self, req: &Message, resp: &'a Message, ) -> Result<(XfrType, RecordSection<'a, Bytes>), CheckError> { - // Enforce the rules defined in 2. AXFR Messages of RFC 5936. - // See: https://www.rfc-editor.org/rfc/rfc5936.html#section-2 - // Check the request. let req_header = req.header(); let req_counts = req.header_counts(); @@ -215,7 +232,7 @@ impl XfrResponseProcessor { // 2.2.1. Header Values // "QDCOUNT MUST be 1 in the first message; // MUST be 0 or 1 in all following messages;" - if matches!(self.mode, Mode::AwaitingFirstAnswer) + if matches!(self.state, State::AwaitingFirstAnswer) && (resp_counts.qdcount() != 1 || resp.sole_question() != req.sole_question()) { @@ -227,22 +244,26 @@ impl XfrResponseProcessor { Ok((xfr_type, answer)) } + /// Initialise the processosr. + /// + /// Records the initial SOA record and other details will will be used + /// while processing the rest of the response. async fn initialize( &mut self, initial_xfr_type: XfrType, initial_query_id: u16, - record: XfrRecord, + soa_record: XfrRecord, ) -> Result<(), CheckError> { // The initial record should be a SOA record. - let data = record.into_data(); + let data = soa_record.into_data(); let ZoneRecordData::Soa(soa) = data else { return Err(CheckError::NotValidXfrResponse); }; - let read = ReadState::new(initial_xfr_type, soa); + let read = ParsingState::new(initial_xfr_type, soa); - self.mode = Mode::AwaitingNextAnswer { + self.state = State::AwaitingNextAnswer { initial_xfr_type, initial_query_id, read, @@ -252,36 +273,68 @@ impl XfrResponseProcessor { } } -//------------ Mode ----------------------------------------------------------- +//------------ State ---------------------------------------------------------- +/// The current processing state. #[derive(Default)] -enum Mode { +enum State { + /// Waiting for the first XFR response message. #[default] AwaitingFirstAnswer, + /// Waiting for a subsequent XFR response message. AwaitingNextAnswer { + /// The type of XFR response sequence expected based on the initial + /// request and response. initial_xfr_type: XfrType, + + /// The header ID of the original XFR request. initial_query_id: u16, - read: ReadState, + + /// The current parsing state. + read: ParsingState, }, + /// The end of the XFR response sequence was detected. TransferComplete, + /// An unrecoverable problem occurred while processing the XFR response + /// sequence. TransferFailed, } -//------------ ReadState ------------------------------------------------------ +//------------ ParsingState --------------------------------------------------- +/// State related to parsing the XFR response sequence. #[derive(Debug)] -struct ReadState { +struct ParsingState { + /// The type of XFR response sequence being parsed. + /// + /// This can differ to the type of XFR response sequence that we expected + /// to parse because the server can fallback from IXFR to AXFR. actual_xfr_type: XfrType, + + /// The initial SOA record that signals the start and end of both AXFR and + /// IXFR response sequences. initial_soa: Soa>, + + /// The current SOA record. + /// + /// For AXFR response sequences this will be the same as `initial_soa`. + /// For IXFR response sequences this will be the last SOA record parsed as + /// each diff sequence contains two SOA records: one at the start of the + /// delete sequence and one at the start of the add sequence. current_soa: Soa>, + + /// The kind of records currently being processed, either adds or deletes. ixfr_update_mode: IxfrUpdateMode, + + /// The number of resource records parsed so far. rr_count: usize, } -impl ReadState { +impl ParsingState { + /// Create a new parsing state. fn new( initial_xfr_type: XfrType, initial_soa: Soa>, @@ -295,6 +348,10 @@ impl ReadState { } } + /// Parse a single resource record. + /// + /// Returns an [`XfrEvent`] that should be emitted for the parsed record, + /// if any. async fn record( &mut self, rec: XfrRecord, @@ -522,14 +579,22 @@ pub trait XfrEventHandler { //------------ IxfrUpdateMode ------------------------------------------------- +/// The kind of records currently being processed, either adds or deletes. #[derive(Copy, Clone, Debug, Default, PartialEq, Eq)] enum IxfrUpdateMode { + /// The records being parsed are deletions. + /// + /// Deletions come before additions. #[default] Deleting, + /// The records being parsed are additions. + Adding, } + impl IxfrUpdateMode { + /// Toggle between the possible [`IxfrUpdateMode`] variants. fn toggle(&mut self) { match self { IxfrUpdateMode::Deleting => *self = IxfrUpdateMode::Adding, @@ -574,12 +639,19 @@ impl Error { //------------ PrepareError --------------------------------------------------- +/// Errors that can occur during intiial checking of an XFR response sequence. #[derive(Debug)] enum CheckError { + /// A parsing error occurred while checking the original request and + /// response messages. ParseError(ParseError), + /// The XFR request is not valid according to the rules defined by RFC + /// 5936 (AXFR) or RFC 1995 (IXFR). NotValidXfrQuery, + /// The XFR response is not valid according to the rules defined by RFC + /// 5936 (AXFR) or RFC 1995 (IXFR). NotValidXfrResponse, } @@ -593,9 +665,18 @@ impl From for CheckError { //------------ XfrType -------------------------------------------------------- +/// The type of XFR response sequence. #[derive(Copy, Clone, Debug, PartialEq, Eq)] enum XfrType { + /// RFC 5936 AXFR. + /// + /// A complete snapshot of a zone at a particular version. Axfr, + + /// RFC 1995 IXFR. + /// + /// An incremental diff of the version of the zone that the server has + /// compared to the version of the zone that the client has. Ixfr, } From d2fb357bd96535cd8b96c4482225946cc7b14bca Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 22 Aug 2024 13:50:51 +0200 Subject: [PATCH 035/333] Cargo fmt. --- src/net/client/xfr.rs | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/src/net/client/xfr.rs b/src/net/client/xfr.rs index fc910b7f2..c98ca5562 100644 --- a/src/net/client/xfr.rs +++ b/src/net/client/xfr.rs @@ -245,8 +245,8 @@ impl XfrResponseProcessor { } /// Initialise the processosr. - /// - /// Records the initial SOA record and other details will will be used + /// + /// Records the initial SOA record and other details will will be used /// while processing the rest of the response. async fn initialize( &mut self, @@ -309,7 +309,7 @@ enum State { #[derive(Debug)] struct ParsingState { /// The type of XFR response sequence being parsed. - /// + /// /// This can differ to the type of XFR response sequence that we expected /// to parse because the server can fallback from IXFR to AXFR. actual_xfr_type: XfrType, @@ -319,7 +319,7 @@ struct ParsingState { initial_soa: Soa>, /// The current SOA record. - /// + /// /// For AXFR response sequences this will be the same as `initial_soa`. /// For IXFR response sequences this will be the last SOA record parsed as /// each diff sequence contains two SOA records: one at the start of the @@ -349,7 +349,7 @@ impl ParsingState { } /// Parse a single resource record. - /// + /// /// Returns an [`XfrEvent`] that should be emitted for the parsed record, /// if any. async fn record( @@ -583,13 +583,12 @@ pub trait XfrEventHandler { #[derive(Copy, Clone, Debug, Default, PartialEq, Eq)] enum IxfrUpdateMode { /// The records being parsed are deletions. - /// + /// /// Deletions come before additions. #[default] Deleting, /// The records being parsed are additions. - Adding, } @@ -669,12 +668,12 @@ impl From for CheckError { #[derive(Copy, Clone, Debug, PartialEq, Eq)] enum XfrType { /// RFC 5936 AXFR. - /// + /// /// A complete snapshot of a zone at a particular version. Axfr, /// RFC 1995 IXFR. - /// + /// /// An incremental diff of the version of the zone that the server has /// compared to the version of the zone that the client has. Ixfr, From 63280618b04af96f38b0aa4eff041ac57e7e49a9 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 22 Aug 2024 13:52:58 +0200 Subject: [PATCH 036/333] Review feedback. --- src/net/client/xfr.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/net/client/xfr.rs b/src/net/client/xfr.rs index c98ca5562..c9d876760 100644 --- a/src/net/client/xfr.rs +++ b/src/net/client/xfr.rs @@ -63,7 +63,7 @@ impl XfrResponseProcessor { /// Returns Ok(true) if the XFR response was the last in the seqence, /// Ok(false) if more XFR response messages are needed to complete the /// sequence, or Err on error. - pub async fn answer( + pub async fn process_answer( &mut self, req: &Message, resp: Message, @@ -112,8 +112,8 @@ impl XfrResponseProcessor { { // The XFR type is extracted from the request. If we were // given a different request with a different question and - // qtype on a subsequent invocation of answer() that would - // be unexpected. + // qtype on a subsequent invocation of process_answer() + // that would be unexpected. return Err(Error::NotValidXfrQuery); } } @@ -125,7 +125,7 @@ impl XfrResponseProcessor { } State::TransferFailed => { - // We had ot terminate processing of the XFR response sequence + // We had to terminate processing of the XFR response sequence // due to a problem with the received data, so we don't expect // to be invoked again with another response message! return Err(Error::Terminated); From 458da3f65e384aa2ac94adfbd7fd02ddd4ba7076 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 22 Aug 2024 13:53:20 +0200 Subject: [PATCH 037/333] Add another missing RustDoc comment to satisfy Clippy. --- src/net/client/xfr.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/net/client/xfr.rs b/src/net/client/xfr.rs index c9d876760..93ed6b4d4 100644 --- a/src/net/client/xfr.rs +++ b/src/net/client/xfr.rs @@ -624,6 +624,7 @@ pub enum Error { } impl Error { + /// Convert a [`CheckError`] to an [`Error`]. fn from_check_error( msg: Message, prepare_err: CheckError, @@ -1134,7 +1135,7 @@ mod tests { evt_handler: TestXfrEventHandler, processor: &mut XfrResponseProcessor, ) { - let res = processor.answer(req, resp).await; + let res = processor.process_answer(req, resp).await; // Verify that the XFR processor returns an error. assert!( From 000194cf9c32d730d604d70a32119f544f9a1457 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 22 Aug 2024 13:54:18 +0200 Subject: [PATCH 038/333] Review feedback. --- src/net/client/xfr.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/client/xfr.rs b/src/net/client/xfr.rs index 93ed6b4d4..dd19fce1f 100644 --- a/src/net/client/xfr.rs +++ b/src/net/client/xfr.rs @@ -500,7 +500,7 @@ impl ParsingState { //------------ RecordResult --------------------------------------------------- -/// An event emitted by [`XfrResponseProcessor`] during transfer processing.` +/// An event emitted by [`XfrResponseProcessor`] during transfer processing. #[derive(Copy, Clone, Debug, PartialEq, Eq)] pub enum XfrEvent { /// Delete record R in zone serial S. From 3690c864cabb4970069be92bb1e8d75b40f04b13 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 22 Aug 2024 13:56:09 +0200 Subject: [PATCH 039/333] Review feedback. --- src/net/client/xfr.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/client/xfr.rs b/src/net/client/xfr.rs index dd19fce1f..76a4ea556 100644 --- a/src/net/client/xfr.rs +++ b/src/net/client/xfr.rs @@ -696,7 +696,6 @@ impl TryFrom for XfrType { #[cfg(test)] mod tests { - use core::net::Ipv4Addr; use core::str::FromStr; use std::string::String; @@ -711,6 +710,7 @@ mod tests { use crate::base::message_builder::{ AnswerBuilder, AuthorityBuilder, QuestionBuilder, }; + use crate::base::net::Ipv4Addr; use crate::base::rdata::ComposeRecordData; use crate::base::{MessageBuilder, Ttl}; use crate::base::{Name, ToName}; From 17d4f9eb8c788d65825645cef54aa1c5013101b5 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 22 Aug 2024 14:21:38 +0200 Subject: [PATCH 040/333] Walk over all records, not just those expected to exist in zone files, as AXFR received data should be faithfully replicated by the receiving secondary server per RFC 5936 section 3 Zone Contents. --- src/net/client/xfr.rs | 28 +++++++++++++++++++--------- 1 file changed, 19 insertions(+), 9 deletions(-) diff --git a/src/net/client/xfr.rs b/src/net/client/xfr.rs index 76a4ea556..450ab4b3c 100644 --- a/src/net/client/xfr.rs +++ b/src/net/client/xfr.rs @@ -14,13 +14,13 @@ use crate::base::wire::ParseError; use crate::base::{ Message, ParsedName, Record, RecordSection, Rtype, Serial, }; -use crate::rdata::{Soa, ZoneRecordData}; +use crate::rdata::{AllRecordData, Soa}; //------------ XfrRecord ------------------------------------------------------ /// The type of record processed by [`XfrResponseProcessor`]. pub type XfrRecord = - Record, ZoneRecordData>>; + Record, AllRecordData>>; //------------ XfrResponseProcessor ------------------------------------------- @@ -80,11 +80,21 @@ impl XfrResponseProcessor { Err(err) => return Err(Error::from_check_error(resp, err)), }; - // Walk over all records that are of the type that can exist in a zone - // file. TODO: Should this actually walk over all possible record - // types irrespective of whether they are intended to exist in a zone - // file or not? - let mut records = answer.limit_to::>(); + // https://datatracker.ietf.org/doc/html/rfc5936#section-3 + // 3. Zone Contents "The objective of the AXFR session is to request + // and transfer the contents of a zone, in order to permit the AXFR + // client to faithfully reconstruct the zone as it exists at the + // primary server for the given zone serial number. The word + // "exists" here designates the externally visible behavior, i.e., + // the zone content that is being served (handed out to clients) -- + // not its persistent representation in a zone file or database + // used by the server -- and that for consistency should be served + // subsequently by the AXFR client in an identical manner." + // + // So, walk over all the records in the answer, not just those that + // might be expected to exist in a zone (i.e. not just ZoneRecordData + // record types). + let mut records = answer.into_records(); match self.state { // When given the first response in a sequence, do some initial @@ -257,7 +267,7 @@ impl XfrResponseProcessor { // The initial record should be a SOA record. let data = soa_record.into_data(); - let ZoneRecordData::Soa(soa) = data else { + let AllRecordData::Soa(soa) = data else { return Err(CheckError::NotValidXfrResponse); }; @@ -373,7 +383,7 @@ impl ParsingState { // having no effect as it is already present. let soa = match rec.data() { - ZoneRecordData::Soa(soa) => Some(soa), + AllRecordData::Soa(soa) => Some(soa), _ => None, }; From 44d3d8699e7e4c0993d1870058097f4a5ff6a9cf Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 22 Aug 2024 20:55:08 +0200 Subject: [PATCH 041/333] Review feedback: switch to an iterator based approach. --- src/net/client/xfr.rs | 692 +++++++++++++++++++----------------------- 1 file changed, 312 insertions(+), 380 deletions(-) diff --git a/src/net/client/xfr.rs b/src/net/client/xfr.rs index 450ab4b3c..feb698f9b 100644 --- a/src/net/client/xfr.rs +++ b/src/net/client/xfr.rs @@ -4,16 +4,17 @@ //! one or more AXFR/IXFR response messages in terms of the high level //! [`XfrEvent`]s that they represent without having to deal with the //! AXFR/IXFR protocol details. +use core::iter::Flatten; + use std::fmt::Debug; use bytes::Bytes; use tracing::trace; use crate::base::iana::Opcode; +use crate::base::message::AnyRecordIter; use crate::base::wire::ParseError; -use crate::base::{ - Message, ParsedName, Record, RecordSection, Rtype, Serial, -}; +use crate::base::{Message, ParsedName, Record, Rtype, Serial}; use crate::rdata::{AllRecordData, Soa}; //------------ XfrRecord ------------------------------------------------------ @@ -26,59 +27,40 @@ pub type XfrRecord = /// An AXFR/IXFR response processor. /// -/// [`XfrResponseProcessor`] can be invoked on one ore more sequentially +/// [`XfrResponseProcessor`] can be invoked on one or more sequentially /// AXFR/IXFR received response messages to verify them and during processing -/// emit events which an implementor of [`XfrEventHandler`] can handle. +/// emit events which can be consumed via the iterator returned by +/// [`process_answer()`]. /// -/// Each instance of [`XfrResponseProcessosr`] should process a single XFR -/// response sequence. Once an instance of [`XfrResponseProcessosr`] has -/// finished processing an XFR response sequence it must be discarded. -/// Attempting to use it once processing has finished will result in an error. -/// To process another XFR response sequence create another instance of -/// [`XfrResponseProcessor`]. -pub struct XfrResponseProcessor { - /// The event handler that events will be sent to for handling. - evt_handler: T, - - /// The current processing state. - state: State, -} - -impl XfrResponseProcessor { - /// Create a new XFR response processor. - /// - /// Events will be emitted to the given [`XfrEventHandler`] implementation. - pub fn new(evt_handler: T) -> Self { - Self { - evt_handler, - state: State::default(), - } - } +/// Each [`XfrEventIterator`] produces events for a single response message. +/// If the end of the XFR response sequence has been reached the iterator will +/// emit an [`XfrEvent::TransferComplete`] event. +/// +/// If the `TransferComplete` event has not been seen it means that the +/// sequence is incomplete and the next response message in the sequence +/// should be passed to [`process_next_answer()`] along with the exhausted +/// iterator. The result will be a new [`XfrEventIterator`] to consume. +/// +/// The process of producing and consuming iterators continues until the end +/// of the transfer is detected or a parsing error occurs. +pub struct XfrResponseProcessor; +impl XfrResponseProcessor { /// Process a single AXFR/IXFR response message. /// - /// During processing events will be emitted to the registered - /// [`XfrEventHandler`] for handling. + /// Return an [`XfrEventIterator`] over [`XfrEvent`]s emitted during + /// processing. /// - /// Returns Ok(true) if the XFR response was the last in the seqence, - /// Ok(false) if more XFR response messages are needed to complete the - /// sequence, or Err on error. - pub async fn process_answer( - &mut self, - req: &Message, - resp: Message, - ) -> Result { + /// If the returned iterator does not emit an + /// [`XfrEvent::TransferComplete`] event, call [`process_next_answer()`] + /// with the next response message to continue iterating over the transfer + /// responses. + pub async fn process_answer<'a>( + req: &'a Message, + resp: &'a Message, + ) -> Result, Error> { // Check that the given message is a DNS XFR response. - let res = self.check_is_xfr_answer(req, &resp).await; - - // Unpack the XFR type and answer object. We cannot do this in the - // line above using `map_err()` and `?` as the Rust compiler complains - // about attempting to return `resp` while a reference to it still - // exists. - let (xfr_type, answer) = match res { - Ok(values) => values, - Err(err) => return Err(Error::from_check_error(resp, err)), - }; + let xfr_type = Self::check_is_xfr_answer(req, resp).await?; // https://datatracker.ietf.org/doc/html/rfc5936#section-3 // 3. Zone Contents "The objective of the AXFR session is to request @@ -94,85 +76,61 @@ impl XfrResponseProcessor { // So, walk over all the records in the answer, not just those that // might be expected to exist in a zone (i.e. not just ZoneRecordData // record types). + let answer = resp.answer().map_err(Error::ParseError)?; let mut records = answer.into_records(); - match self.state { - // When given the first response in a sequence, do some initial - // setup. - State::AwaitingFirstAnswer => { - let Some(Ok(record)) = records.next() else { - return Err(Error::Malformed); - }; - - if let Err(err) = - self.initialize(xfr_type, req.header().id(), record).await - { - return Err(Error::from_check_error(resp, err)); - } - } + let Some(Ok(record)) = records.next() else { + return Err(Error::Malformed); + }; + + let state = Self::initialize(xfr_type, record).await?; - // For subsequent messages make sure that the XFR - State::AwaitingNextAnswer { - initial_xfr_type, - initial_query_id, - .. - } => { - if xfr_type != initial_xfr_type - || req.header().id() != initial_query_id - { - // The XFR type is extracted from the request. If we were - // given a different request with a different question and - // qtype on a subsequent invocation of process_answer() - // that would be unexpected. - return Err(Error::NotValidXfrQuery); + let records = records.flatten(); + + let iter = XfrEventIterator { + req, + state, + records, + }; + + Ok(iter) + } + + /// Process a subsequent XFR response message. + /// + /// Revives the given iterator with new data based on the given response. + pub async fn process_next_answer<'a>( + it: &mut XfrEventIterator<'a>, + resp: &'a Message, + ) -> Result<(), Error> { + // Verify that the given iterator is in the expected state + match it.state { + State::AwaitingAnswer { .. } => { + // Verify that the given iterator is exhausted + if it.records.next().is_some() { + return Err(Error::AnswerNotFullyProcessed); } + + let _ = Self::check_is_xfr_answer(it.req, resp).await?; + let answer = resp.answer().map_err(Error::ParseError)?; + it.records = answer.into_records().flatten(); + + Ok(()) } State::TransferComplete => { // We already finished processing an XFR response sequence. We // don't expect there to be any more messages to process!. - return Err(Error::Malformed); + Err(Error::Malformed) } State::TransferFailed => { // We had to terminate processing of the XFR response sequence // due to a problem with the received data, so we don't expect // to be invoked again with another response message! - return Err(Error::Terminated); - } - }; - - let State::AwaitingNextAnswer { read, .. } = &mut self.state else { - unreachable!(); - }; - - for record in records.flatten() { - trace!("XFR record {}: {record:?}", read.rr_count); - - if let Some(event) = read.record(record).await? { - match event { - XfrEvent::EndOfTransfer => { - self.state = State::TransferComplete; - self.evt_handler.handle_event(event).await?; - return Ok(true); - } - - XfrEvent::ProcessingFailed => { - self.state = State::TransferFailed; - let _ = self.evt_handler.handle_event(event).await; - return Err(Error::Malformed); - } - - _ => { - self.evt_handler.handle_event(event).await?; - } - } + Err(Error::Terminated) } } - - // Finished processing this message but did not yet reach the end of - // the transfer, more responses are expected. - Ok(false) } /// Check if an XFR response header is valid. @@ -186,11 +144,10 @@ impl XfrResponseProcessor { /// Returns Ok on success, Err otherwise. On success the type of XFR that /// was determined is returned as well as the answer section from the XFR /// response. - async fn check_is_xfr_answer<'a>( - &mut self, + async fn check_is_xfr_answer( req: &Message, - resp: &'a Message, - ) -> Result<(XfrType, RecordSection<'a, Bytes>), CheckError> { + resp: &Message, + ) -> Result { // Check the request. let req_header = req.header(); let req_counts = req.header_counts(); @@ -242,16 +199,13 @@ impl XfrResponseProcessor { // 2.2.1. Header Values // "QDCOUNT MUST be 1 in the first message; // MUST be 0 or 1 in all following messages;" - if matches!(self.state, State::AwaitingFirstAnswer) - && (resp_counts.qdcount() != 1 - || resp.sole_question() != req.sole_question()) + if resp_counts.qdcount() != 1 + || resp.sole_question() != req.sole_question() { return Err(CheckError::NotValidXfrResponse); } - let answer = resp.answer().map_err(CheckError::ParseError)?; - - Ok((xfr_type, answer)) + Ok(xfr_type) } /// Initialise the processosr. @@ -259,11 +213,9 @@ impl XfrResponseProcessor { /// Records the initial SOA record and other details will will be used /// while processing the rest of the response. async fn initialize( - &mut self, initial_xfr_type: XfrType, - initial_query_id: u16, soa_record: XfrRecord, - ) -> Result<(), CheckError> { + ) -> Result { // The initial record should be a SOA record. let data = soa_record.into_data(); @@ -271,39 +223,20 @@ impl XfrResponseProcessor { return Err(CheckError::NotValidXfrResponse); }; - let read = ParsingState::new(initial_xfr_type, soa); - - self.state = State::AwaitingNextAnswer { + Ok(State::AwaitingAnswer(ParsingState::new( initial_xfr_type, - initial_query_id, - read, - }; - - Ok(()) + soa, + ))) } } //------------ State ---------------------------------------------------------- /// The current processing state. -#[derive(Default)] +#[derive(Debug)] enum State { - /// Waiting for the first XFR response message. - #[default] - AwaitingFirstAnswer, - - /// Waiting for a subsequent XFR response message. - AwaitingNextAnswer { - /// The type of XFR response sequence expected based on the initial - /// request and response. - initial_xfr_type: XfrType, - - /// The header ID of the original XFR request. - initial_query_id: u16, - - /// The current parsing state. - read: ParsingState, - }, + /// Waiting for an XFR response message. + AwaitingAnswer(ParsingState), /// The end of the XFR response sequence was detected. TransferComplete, @@ -362,7 +295,7 @@ impl ParsingState { /// /// Returns an [`XfrEvent`] that should be emitted for the parsed record, /// if any. - async fn record( + fn parse_record( &mut self, rec: XfrRecord, ) -> Result>, Error> { @@ -571,20 +504,96 @@ impl std::fmt::Display for XfrEvent { } } -//------------ XfrEventHandler --------------------------------------------------- +//------------ XfrEventIterator ----------------------------------------------- -/// A trait for implementing handlers of [`XfrEvent`]s. -pub trait XfrEventHandler { - /// Handle the given [`XfrEvent`]. +/// An iterator over [`XfrResponseProcessor`] generated [`XfrEvent`]s. +#[derive(Debug)] +pub struct XfrEventIterator<'a> { + /// The original XFR request. /// - /// Returning an Err will cause transfer processsing to be aborted and the - /// error to be returned to the client of [`XfrResponseProcessor`], except in - /// the case of [`XfrEvent::ProcessingFailed`] for which the return value of - /// this handler will be ignored by [`XfrResponseProcessor`]. - fn handle_event( - &self, - evt: XfrEvent, - ) -> impl std::future::Future> + Send; + /// Used to check if responses relate to the original request. + req: &'a Message, + + /// The current parsing state. + state: State, + + /// An iterator over the records in the current response. + records: Flatten< + AnyRecordIter<'a, Bytes, AllRecordData>>, + >, +} + +impl<'a> Iterator for XfrEventIterator<'a> { + type Item = Result, XfrEventIteratorError>; + + fn next(&mut self) -> Option { + match &mut self.state { + State::AwaitingAnswer(parsing_state) => { + #[allow(clippy::blocks_in_conditions)] + let event = self.records.next().and_then(|record| { + trace!( + "XFR record {}: {record:?}", + parsing_state.rr_count + ); + parsing_state.parse_record(record).unwrap_or_default() + }); + + match event { + Some(XfrEvent::EndOfTransfer) => { + // Record that the transfer completed. + self.state = State::TransferComplete; + + // Return the end of transfer event so that the client + // can distinguish this condition from None (end of + // iteration) which can happen when the message body + // has been consumed but more records are needed from + // subsequent responses to complete the transfer. + Some(Ok(XfrEvent::EndOfTransfer)) + } + + Some(XfrEvent::ProcessingFailed) => { + // Record that the transfer failed. + self.state = State::TransferFailed; + + // Return an error. + Some(Err(XfrEventIteratorError::ProcessingFailed)) + } + + Some(e) => { + // Return the event. + Some(Ok(e)) + } + + None => { + // No more events available: end iteration for now. + // The client can revive this iterator by passing it + // to XfrResponseProcessor::process_next_answer(). + None + } + } + } + + State::TransferComplete => { + // The transfer was completed parsed. No more events available. + None + } + + State::TransferFailed => { + // We had to terminate processing of the XFR response sequence + // due to a problem with the received data. No more events available. + None + } + } + } +} + +//------------ XfrEventIteratorError ------------------------------------------ + +/// Errors that can occur during XfrEventIterator iteration. +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +pub enum XfrEventIteratorError { + /// Transfer processing failed. + ProcessingFailed, } //------------ IxfrUpdateMode ------------------------------------------------- @@ -618,31 +627,33 @@ impl IxfrUpdateMode { #[derive(Debug)] pub enum Error { /// The message could not be parsed. - ParseError(ParseError, Message), + ParseError(ParseError), /// The request message is not an XFR query/ NotValidXfrQuery, /// The response message is not an XFR response. - NotValidXfrResponse(Message), + NotValidXfrResponse, /// At least one record in the XFR response sequence is incorrect. Malformed, + /// At least one record in the XFR response sequence was not consumed + /// by the caller. + AnswerNotFullyProcessed, + /// Processing was already terminated for this XFR response sequence. Terminated, } -impl Error { - /// Convert a [`CheckError`] to an [`Error`]. - fn from_check_error( - msg: Message, - prepare_err: CheckError, - ) -> Self { - match prepare_err { - CheckError::ParseError(err) => Self::ParseError(err, msg), +//--- From + +impl From for Error { + fn from(err: CheckError) -> Self { + match err { + CheckError::ParseError(err) => Self::ParseError(err), CheckError::NotValidXfrQuery => Self::NotValidXfrQuery, - CheckError::NotValidXfrResponse => Self::NotValidXfrResponse(msg), + CheckError::NotValidXfrResponse => Self::NotValidXfrResponse, } } } @@ -708,13 +719,8 @@ impl TryFrom for XfrType { mod tests { use core::str::FromStr; - use std::string::String; - use std::sync::Arc; - use std::vec::Vec; - use bytes::BytesMut; use octseq::Octets; - use tokio::sync::Mutex; use crate::base::iana::Rcode; use crate::base::message_builder::{ @@ -726,6 +732,7 @@ mod tests { use crate::base::{Name, ToName}; use crate::rdata::A; + use super::XfrEvent as XE; use super::*; #[tokio::test] @@ -737,13 +744,8 @@ mod tests { // Process the request and assert that it is rejected as not being // a valid XFR response and that no XFR processor events were emitted. - assert_xfr_response( - &req.clone(), - req, - |res| matches!(res, Err(Error::NotValidXfrResponse(_))), - &[], - ) - .await; + let res = XfrResponseProcessor::process_answer(&req, &req).await; + assert!(matches!(res, Err(Error::NotValidXfrResponse))); } #[tokio::test] @@ -759,13 +761,9 @@ mod tests { // Process the response and assert that it is rejected as not being // a valid XFR response and that no XFR processor events were emitted. - assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Err(Error::NotValidXfrResponse(_))), - &[], - ) - .await; + let resp = answer.into_message(); + let res = XfrResponseProcessor::process_answer(&req, &resp).await; + assert!(matches!(res, Err(Error::NotValidXfrResponse))); } #[tokio::test] @@ -780,13 +778,9 @@ mod tests { // Process the response and assert that it is rejected as not being // a valid XFR response and that no XFR processor events were emitted. - assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Err(Error::NotValidXfrResponse(_))), - &[], - ) - .await; + let resp = answer.into_message(); + let res = XfrResponseProcessor::process_answer(&req, &resp).await; + assert!(matches!(res, Err(Error::NotValidXfrResponse))); } #[tokio::test] @@ -804,13 +798,9 @@ mod tests { // Process the response and assert that it is rejected as not being // a valid XFR response and that no XFR processor events were emitted. - assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Err(Error::NotValidXfrResponse(_))), - &[], - ) - .await; + let resp = answer.into_message(); + let res = XfrResponseProcessor::process_answer(&req, &resp).await; + assert!(matches!(res, Err(Error::NotValidXfrResponse))); } #[tokio::test] @@ -828,16 +818,14 @@ mod tests { let mut answer = mk_empty_answer(&req, Rcode::NOERROR); add_answer_record(&req, &mut answer, mk_soa(Serial::now())); - // Process the response and assert that Ok(false) is returned by the - // XFR processor indicating that the XFR response was incomplete. Also - // verify the events emitted by the XFR processor. - assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Ok(false)), - &[], - ) - .await; + // Process the response. + let resp = answer.into_message(); + let mut it = XfrResponseProcessor::process_answer(&req, &resp) + .await + .unwrap(); + + // Verify that no events are by the XFR processor. + assert!(it.next().is_none()); } #[tokio::test] @@ -857,16 +845,15 @@ mod tests { add_answer_record(&req, &mut answer, soa.clone()); add_answer_record(&req, &mut answer, soa); - // Process the response and assert that Ok(true) is returned by the - // XFR processor indicating that the XFR response was complete. Also - // verify the events emitted by the XFR processor. - assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Ok(true)), - &["EndOfTransfer"], - ) - .await; + // Process the response. + let resp = answer.into_message(); + let mut it = XfrResponseProcessor::process_answer(&req, &resp) + .await + .unwrap(); + + // Verify the events emitted by the XFR processor. + assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); + assert!(it.next().is_none()); } #[tokio::test] @@ -885,33 +872,28 @@ mod tests { let soa = mk_soa(Serial::now()); add_answer_record(&req, &mut answer, soa.clone()); - // Process the response and assert that Ok(true) is returned by the - // XFR processor indicating that the XFR response was complete. Also - // verify the events emitted by the XFR processor. - let (evt_handler, mut processor) = assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Ok(false)), - &[], - ) - .await; + // Process the response. + let resp = answer.into_message(); + let mut it = XfrResponseProcessor::process_answer(&req, &resp) + .await + .unwrap(); + + // Verify the events emitted by the XFR processor. + assert!(it.next().is_none()); // Create another AXFR response to complete the transfer. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); add_answer_record(&req, &mut answer, soa); - // Process the response and assert that Ok(true) is returned by the - // XFR processor indicating that the XFR response was complete. Also - // verify the events emitted by the XFR processor. - assert_xfr_response_with_processor( - &req, - answer.into_message(), - |res| matches!(res, Ok(true)), - &["EndOfTransfer"], - evt_handler, - &mut processor, - ) - .await; + // Process the response. + let resp = answer.into_message(); + XfrResponseProcessor::process_next_answer(&mut it, &resp) + .await + .unwrap(); + + // Verify the events emitted by the XFR processor. + assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); + assert!(it.next().is_none()); } #[tokio::test] @@ -923,22 +905,25 @@ mod tests { // Create an AXFR response. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - let soa = mk_soa(Serial::now()); + let serial = Serial::now(); + let soa = mk_soa(serial); add_answer_record(&req, &mut answer, soa.clone()); add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); add_answer_record(&req, &mut answer, soa); - // Process the response and assert that Ok(true) is returned by the - // XFR processor indicating that the XFR response was complete. Also - // verify the events emitted by the XFR processor. - assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Ok(true)), - &["AddRecord", "AddRecord", "EndOfTransfer"], - ) - .await; + // Process the response. + let resp = answer.into_message(); + let mut it = XfrResponseProcessor::process_answer(&req, &resp) + .await + .unwrap(); + + // Verify the events emitted by the XFR processor. + let s = serial; + assert!(matches!(it.next(), Some(Ok(XE::AddRecord(n, _))) if n == s)); + assert!(matches!(it.next(), Some(Ok(XE::AddRecord(n, _))) if n == s)); + assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); + assert!(it.next().is_none()); } #[tokio::test] @@ -979,24 +964,53 @@ mod tests { // Closing SOA with servers current SOA add_answer_record(&req, &mut answer, new_soa); - // Process the response and assert that Ok(true) is returned by the - // XFR processor indicating that the XFR response was complete. Also - // verify the events emitted by the XFR processor. - assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Ok(true)), - &[ - "BeginBatchDelete", - "DeleteRecord", - "DeleteRecord", - "BeginBatchAdd", - "AddRecord", - "AddRecord", - "EndOfTransfer", - ], - ) - .await; + // Process the response. + let resp = answer.into_message(); + let it = XfrResponseProcessor::process_answer(&req, &resp) + .await + .unwrap(); + + // Verify the events emitted by the XFR processor. + let owner = ParsedName::from(Name::from_str("example.com").unwrap()); + let expected_events = [ + Ok(XfrEvent::BeginBatchDelete(old_serial)), + Ok(XfrEvent::DeleteRecord( + old_serial, + Record::from(( + owner.clone(), + 0, + AllRecordData::A(A::new(Ipv4Addr::LOCALHOST)), + )), + )), + Ok(XfrEvent::DeleteRecord( + old_serial, + Record::from(( + owner.clone(), + 0, + AllRecordData::A(A::new(Ipv4Addr::BROADCAST)), + )), + )), + Ok(XfrEvent::BeginBatchAdd(new_serial)), + Ok(XfrEvent::AddRecord( + new_serial, + Record::from(( + owner.clone(), + 0, + AllRecordData::A(A::new(Ipv4Addr::BROADCAST)), + )), + )), + Ok(XfrEvent::AddRecord( + new_serial, + Record::from(( + owner, + 0, + AllRecordData::A(A::new(Ipv4Addr::LOCALHOST)), + )), + )), + Ok(XfrEvent::EndOfTransfer), + ]; + + assert!(it.eq(expected_events)); } #[tokio::test] @@ -1028,16 +1042,15 @@ mod tests { add_answer_record(&req, &mut answer, old_soa); add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - // Process the response and assert that Ok(true) is returned by the - // XFR processor indicating that the XFR response was complete. Also - // verify the events emitted by the XFR processor. - let (evt_handler, mut processor) = assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Ok(false)), - &["BeginBatchDelete", "DeleteRecord"], - ) - .await; + // Process the response. + let resp = answer.into_message(); + let mut it = XfrResponseProcessor::process_answer(&req, &resp) + .await + .unwrap(); + + // Verify the events emitted by the XFR processor. + assert!(matches!(it.next(), Some(Ok(XE::BeginBatchDelete(_))))); + assert!(matches!(it.next(), Some(Ok(XE::DeleteRecord(..))))); // Craete a second IXFR response that completes the transfer let mut answer = mk_empty_answer(&req, Rcode::NOERROR); @@ -1051,51 +1064,19 @@ mod tests { // Closing SOA with servers current SOA add_answer_record(&req, &mut answer, new_soa); - assert_xfr_response_with_processor( - &req, - answer.into_message(), - |res| matches!(res, Ok(true)), - &[ - "BeginBatchDelete", // Seen during processing of the 1st answer - "DeleteRecord", // Seen during processing of the 1st answer - "DeleteRecord", // Seen during processing of the 2nd answer - "BeginBatchAdd", // Seen during processing of the 2nd answer - "AddRecord", // Seen during processing of the 2nd answer - "AddRecord", // Seen during processing of the 2nd answer - "EndOfTransfer", // Seen during processing of the 2nd answer - ], - evt_handler, - &mut processor, - ) - .await; - } - - //------------ TestXfrEventHandler ---------------------------------------- - - #[derive(Clone, Default)] - struct TestXfrEventHandler { - events: Arc>>, - } - - impl TestXfrEventHandler { - pub fn new() -> Self { - Self::default() - } - - pub async fn events(self) -> Vec { - self.events.lock().await.clone() - } - } + // Process the response. + let resp = answer.into_message(); + XfrResponseProcessor::process_next_answer(&mut it, &resp) + .await + .unwrap(); - impl XfrEventHandler for TestXfrEventHandler { - async fn handle_event( - &self, - evt: XfrEvent, - ) -> Result<(), Error> { - trace!("Received event: {evt}"); - self.events.lock().await.push(format!("{evt}")); - Ok(()) - } + // Verify the events emitted by the XFR processor. + assert!(matches!(it.next(), Some(Ok(XE::DeleteRecord(..))))); + assert!(matches!(it.next(), Some(Ok(XE::BeginBatchAdd(_))))); + assert!(matches!(it.next(), Some(Ok(XE::AddRecord(..))))); + assert!(matches!(it.next(), Some(Ok(XE::AddRecord(..))))); + assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); + assert!(it.next().is_none()); } //------------ Helper functions ------------------------------------------- @@ -1112,55 +1093,6 @@ mod tests { .ok(); } - async fn assert_xfr_response( - req: &Message, - resp: Message, - res_check_cb: fn(&Result) -> bool, - expected_events: &[&str], - ) -> ( - TestXfrEventHandler, - XfrResponseProcessor, - ) { - let evt_handler = TestXfrEventHandler::new(); - let mut processor = XfrResponseProcessor::new(evt_handler.clone()); - - assert_xfr_response_with_processor( - req, - resp, - res_check_cb, - expected_events, - evt_handler.clone(), - &mut processor, - ) - .await; - - (evt_handler, processor) - } - - async fn assert_xfr_response_with_processor( - req: &Message, - resp: Message, - res_check_cb: fn(&Result) -> bool, - expected_events: &[&str], - evt_handler: TestXfrEventHandler, - processor: &mut XfrResponseProcessor, - ) { - let res = processor.process_answer(req, resp).await; - - // Verify that the XFR processor returns an error. - assert!( - res_check_cb(&res), - "Unexpected result {res:?} from the XFR processor", - ); - - // Verify that no XFR processing events were emitted. - assert_eq!( - &evt_handler.clone().events().await, - expected_events, - "Unexpected events were emitted by the XFR processor" - ); - } - fn mk_request(qname: &str, qtype: Rtype) -> QuestionBuilder { let req = MessageBuilder::new_bytes(); let mut req = req.question(); From 127f0500f37a6a5333dfcd6841e23f86f6085420 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 22 Aug 2024 21:25:10 +0200 Subject: [PATCH 042/333] Correct outdated comment. --- src/net/client/xfr.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/net/client/xfr.rs b/src/net/client/xfr.rs index feb698f9b..b00ea2225 100644 --- a/src/net/client/xfr.rs +++ b/src/net/client/xfr.rs @@ -39,7 +39,8 @@ pub type XfrRecord = /// If the `TransferComplete` event has not been seen it means that the /// sequence is incomplete and the next response message in the sequence /// should be passed to [`process_next_answer()`] along with the exhausted -/// iterator. The result will be a new [`XfrEventIterator`] to consume. +/// iterator. This will populate thr [`XfrEventIterator`] with more records +/// to parse thereby causing iteration to resume. /// /// The process of producing and consuming iterators continues until the end /// of the transfer is detected or a parsing error occurs. From 95e1b9ec92320039c44a8566da29619562c93601 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 26 Aug 2024 13:19:47 +0200 Subject: [PATCH 043/333] Review feedback: Move out of of net::client. --- src/net/client/mod.rs | 1 - src/net/mod.rs | 1 + src/net/xfr/mod.rs | 1 + src/net/{client/xfr.rs => xfr/processor.rs} | 0 4 files changed, 2 insertions(+), 1 deletion(-) create mode 100644 src/net/xfr/mod.rs rename src/net/{client/xfr.rs => xfr/processor.rs} (100%) diff --git a/src/net/client/mod.rs b/src/net/client/mod.rs index b49d4336b..0275f3bef 100644 --- a/src/net/client/mod.rs +++ b/src/net/client/mod.rs @@ -193,4 +193,3 @@ pub mod tsig; #[cfg(feature = "unstable-validator")] pub mod validator; pub mod validator_test; -pub mod xfr; diff --git a/src/net/mod.rs b/src/net/mod.rs index 98657acec..cbd7def0b 100644 --- a/src/net/mod.rs +++ b/src/net/mod.rs @@ -18,3 +18,4 @@ pub mod client; pub mod server; +pub mod xfr; diff --git a/src/net/xfr/mod.rs b/src/net/xfr/mod.rs new file mode 100644 index 000000000..3cde65f09 --- /dev/null +++ b/src/net/xfr/mod.rs @@ -0,0 +1 @@ +pub mod processor; diff --git a/src/net/client/xfr.rs b/src/net/xfr/processor.rs similarity index 100% rename from src/net/client/xfr.rs rename to src/net/xfr/processor.rs From 8359a8dff35aa7f8f59a3348354fc2e74952bb16 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 26 Aug 2024 13:20:03 +0200 Subject: [PATCH 044/333] Add missing RFC URL to comment block. --- src/net/xfr/processor.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/xfr/processor.rs b/src/net/xfr/processor.rs index 450ab4b3c..d7d554c31 100644 --- a/src/net/xfr/processor.rs +++ b/src/net/xfr/processor.rs @@ -238,7 +238,7 @@ impl XfrResponseProcessor { return Err(CheckError::NotValidXfrResponse); } - // + // https://datatracker.ietf.org/doc/html/rfc1995#section-2.2.1 // 2.2.1. Header Values // "QDCOUNT MUST be 1 in the first message; // MUST be 0 or 1 in all following messages;" From 2f0bbdafd859dc23443f5105d691694c1cacc595 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 26 Aug 2024 13:39:25 +0200 Subject: [PATCH 045/333] Review feedback: Fix broken minimal versions check error: "`impl Trait` only allowed in function and inherent method return types, not in trait method return" --- src/net/xfr/processor.rs | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/src/net/xfr/processor.rs b/src/net/xfr/processor.rs index d7d554c31..c6f712160 100644 --- a/src/net/xfr/processor.rs +++ b/src/net/xfr/processor.rs @@ -575,16 +575,15 @@ impl std::fmt::Display for XfrEvent { /// A trait for implementing handlers of [`XfrEvent`]s. pub trait XfrEventHandler { + type Fut: std::future::Future>; + /// Handle the given [`XfrEvent`]. /// /// Returning an Err will cause transfer processsing to be aborted and the /// error to be returned to the client of [`XfrResponseProcessor`], except in /// the case of [`XfrEvent::ProcessingFailed`] for which the return value of /// this handler will be ignored by [`XfrResponseProcessor`]. - fn handle_event( - &self, - evt: XfrEvent, - ) -> impl std::future::Future> + Send; + fn handle_event(&self, evt: XfrEvent) -> Self::Fut; } //------------ IxfrUpdateMode ------------------------------------------------- @@ -706,15 +705,17 @@ impl TryFrom for XfrType { #[cfg(test)] mod tests { + use core::future::ready; + use core::future::Ready; use core::str::FromStr; use std::string::String; use std::sync::Arc; + use std::sync::Mutex; use std::vec::Vec; use bytes::BytesMut; use octseq::Octets; - use tokio::sync::Mutex; use crate::base::iana::Rcode; use crate::base::message_builder::{ @@ -1083,18 +1084,17 @@ mod tests { } pub async fn events(self) -> Vec { - self.events.lock().await.clone() + self.events.lock().unwrap().clone() } } impl XfrEventHandler for TestXfrEventHandler { - async fn handle_event( - &self, - evt: XfrEvent, - ) -> Result<(), Error> { + type Fut = Ready>; + + fn handle_event(&self, evt: XfrEvent) -> Self::Fut { trace!("Received event: {evt}"); - self.events.lock().await.push(format!("{evt}")); - Ok(()) + self.events.lock().unwrap().push(format!("{evt}")); + ready(Ok(())) } } From a40d3cbaff584e62ecc43a17c569ef88f5bdd1eb Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 26 Aug 2024 13:52:34 +0200 Subject: [PATCH 046/333] Fix: Return the correct error code and rename it as XxxReuest for consistency with XxxResponse. --- src/net/xfr/processor.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/net/xfr/processor.rs b/src/net/xfr/processor.rs index c6f712160..c93514c3d 100644 --- a/src/net/xfr/processor.rs +++ b/src/net/xfr/processor.rs @@ -201,17 +201,17 @@ impl XfrResponseProcessor { || req_counts.ancount() != 0 || req_header.opcode() != Opcode::QUERY { - return Err(CheckError::NotValidXfrQuery); + return Err(CheckError::NotValidXfrRequest); } let Some(qtype) = req.qtype() else { - return Err(CheckError::NotValidXfrResponse); + return Err(CheckError::NotValidXfrRequest); }; let xfr_type = match qtype { Rtype::AXFR => XfrType::Axfr, Rtype::IXFR => XfrType::Ixfr, - _ => return Err(CheckError::NotValidXfrResponse), + _ => return Err(CheckError::NotValidXfrRequest), }; // https://datatracker.ietf.org/doc/html/rfc1995#section-3 @@ -640,7 +640,7 @@ impl Error { ) -> Self { match prepare_err { CheckError::ParseError(err) => Self::ParseError(err, msg), - CheckError::NotValidXfrQuery => Self::NotValidXfrQuery, + CheckError::NotValidXfrRequest => Self::NotValidXfrQuery, CheckError::NotValidXfrResponse => Self::NotValidXfrResponse(msg), } } @@ -657,7 +657,7 @@ enum CheckError { /// The XFR request is not valid according to the rules defined by RFC /// 5936 (AXFR) or RFC 1995 (IXFR). - NotValidXfrQuery, + NotValidXfrRequest, /// The XFR response is not valid according to the rules defined by RFC /// 5936 (AXFR) or RFC 1995 (IXFR). From a922b9397f50a8af2feab290c5235432b3e8f56d Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 26 Aug 2024 13:59:10 +0200 Subject: [PATCH 047/333] Minor fixes. --- src/net/xfr/processor.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/net/xfr/processor.rs b/src/net/xfr/processor.rs index c93514c3d..e7d80ed3e 100644 --- a/src/net/xfr/processor.rs +++ b/src/net/xfr/processor.rs @@ -124,7 +124,7 @@ impl XfrResponseProcessor { // given a different request with a different question and // qtype on a subsequent invocation of process_answer() // that would be unexpected. - return Err(Error::NotValidXfrQuery); + return Err(Error::NotValidXfrRequest); } } @@ -620,7 +620,7 @@ pub enum Error { ParseError(ParseError, Message), /// The request message is not an XFR query/ - NotValidXfrQuery, + NotValidXfrRequest, /// The response message is not an XFR response. NotValidXfrResponse(Message), @@ -640,7 +640,7 @@ impl Error { ) -> Self { match prepare_err { CheckError::ParseError(err) => Self::ParseError(err, msg), - CheckError::NotValidXfrRequest => Self::NotValidXfrQuery, + CheckError::NotValidXfrRequest => Self::NotValidXfrRequest, CheckError::NotValidXfrResponse => Self::NotValidXfrResponse(msg), } } @@ -741,7 +741,7 @@ mod tests { assert_xfr_response( &req.clone(), req, - |res| matches!(res, Err(Error::NotValidXfrResponse(_))), + |res| matches!(res, Err(Error::NotValidXfrRequest)), &[], ) .await; @@ -752,7 +752,7 @@ mod tests { init_logging(); // Create a non-XFR request to reply to. - let req = mk_request("example.com", Rtype::A).into_message(); + let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create a non-XFR response. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); From 5a32215ec5680fceee1eef651ccb9b92ed4c4fde Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 26 Aug 2024 14:02:07 +0200 Subject: [PATCH 048/333] Comment fix. --- src/net/xfr/processor.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/xfr/processor.rs b/src/net/xfr/processor.rs index e7d80ed3e..28460f10f 100644 --- a/src/net/xfr/processor.rs +++ b/src/net/xfr/processor.rs @@ -508,7 +508,7 @@ impl ParsingState { } } -//------------ RecordResult --------------------------------------------------- +//------------ XfrEvent ------------------------------------------------------- /// An event emitted by [`XfrResponseProcessor`] during transfer processing. #[derive(Copy, Clone, Debug, PartialEq, Eq)] From aa294fbe9cb84552301d0f4f3c43c6f306184149 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 27 Aug 2024 11:58:02 +0200 Subject: [PATCH 049/333] Review feedback: SERVFAIL if question cannot be built when preparing an error message. --- src/base/message_builder.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/base/message_builder.rs b/src/base/message_builder.rs index f5dccd323..d1cc2be40 100644 --- a/src/base/message_builder.rs +++ b/src/base/message_builder.rs @@ -274,14 +274,18 @@ impl MessageBuilder { header.set_qr(true); header.set_opcode(msg.header().opcode()); header.set_rd(msg.header().rd()); + header.set_rcode(rcode); } + let mut builder = self.question(); for item in msg.question().flatten() { if builder.push(item).is_err() { + builder.rewind(); + builder.header_mut().set_rcode(Rcode::SERVFAIL); break; } } - builder.header_mut().set_rcode(rcode); + builder.answer() } From 79f7ebac5d0f9702f9fc15fdef7e6bd5889b28e4 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 27 Aug 2024 20:51:25 +0200 Subject: [PATCH 050/333] Initial working version of iterator based XfrEventProcessor. --- src/net/client/xfr.rs | 659 ++++++++++++++++++++++-------------------- 1 file changed, 339 insertions(+), 320 deletions(-) diff --git a/src/net/client/xfr.rs b/src/net/client/xfr.rs index b00ea2225..2089cde5f 100644 --- a/src/net/client/xfr.rs +++ b/src/net/client/xfr.rs @@ -4,9 +4,8 @@ //! one or more AXFR/IXFR response messages in terms of the high level //! [`XfrEvent`]s that they represent without having to deal with the //! AXFR/IXFR protocol details. -use core::iter::Flatten; - use std::fmt::Debug; +use std::sync::{Arc, Mutex}; use bytes::Bytes; use tracing::trace; @@ -44,9 +43,65 @@ pub type XfrRecord = /// /// The process of producing and consuming iterators continues until the end /// of the transfer is detected or a parsing error occurs. -pub struct XfrResponseProcessor; +pub struct XfrResponseProcessor { + req: Message, + + /// The response message currently being processed. + resp: Option>, + + state: Option>>, +} + +impl XfrResponseProcessor { + /// Creates a new instance of [`XfrMessageProcessor`]. + /// + /// Processes a single XFR response stream. + pub fn new(req: Message) -> Result { + Self::check_request(&req)?; + Ok(Self { + req, + resp: None, + state: None, + }) + } +} impl XfrResponseProcessor { + /// Check if an XFR request is valid. + pub fn check_request(req: &Message) -> Result<(), Error> { + let req_header = req.header(); + let req_counts = req.header_counts(); + + if req.is_error() + || req_header.qr() + || req_counts.qdcount() != 1 + || req_counts.ancount() != 0 + || req_header.opcode() != Opcode::QUERY + { + return Err(Error::NotValidXfrRequest); + } + + let Some(qtype) = req.qtype() else { + return Err(Error::NotValidXfrRequest); + }; + + if !matches!(qtype, Rtype::AXFR | Rtype::IXFR) { + return Err(Error::NotValidXfrRequest); + } + + // https://datatracker.ietf.org/doc/html/rfc1995#section-3 + // 3. Query Format + // "The IXFR query packet format is the same as that of a normal DNS + // query, but with the query type being IXFR and the authority + // section containing the SOA record of client's version of the + // zone." + if matches!(qtype, Rtype::IXFR) && req_counts.nscount() != 1 { + return Err(Error::NotValidXfrRequest); + } + + Ok(()) + } + /// Process a single AXFR/IXFR response message. /// /// Return an [`XfrEventIterator`] over [`XfrEvent`]s emitted during @@ -56,12 +111,12 @@ impl XfrResponseProcessor { /// [`XfrEvent::TransferComplete`] event, call [`process_next_answer()`] /// with the next response message to continue iterating over the transfer /// responses. - pub async fn process_answer<'a>( - req: &'a Message, - resp: &'a Message, - ) -> Result, Error> { + pub fn process_answer( + &mut self, + resp: Message, + ) -> Result { // Check that the given message is a DNS XFR response. - let xfr_type = Self::check_is_xfr_answer(req, resp).await?; + self.check_response(&resp)?; // https://datatracker.ietf.org/doc/html/rfc5936#section-3 // 3. Zone Contents "The objective of the AXFR session is to request @@ -77,62 +132,56 @@ impl XfrResponseProcessor { // So, walk over all the records in the answer, not just those that // might be expected to exist in a zone (i.e. not just ZoneRecordData // record types). - let answer = resp.answer().map_err(Error::ParseError)?; - let mut records = answer.into_records(); - - let Some(Ok(record)) = records.next() else { - return Err(Error::Malformed); - }; - - let state = Self::initialize(xfr_type, record).await?; - - let records = records.flatten(); - let iter = XfrEventIterator { - req, - state, - records, + let state; + let iter = if self.state.is_none() { + let (state_clone, iter) = self.initialize(resp)?; + state = state_clone; + iter + } else { + state = self.state.as_ref().unwrap().clone(); + self.update(resp)? }; - Ok(iter) + Ok(XfrEventIterator::new(state, iter)) } - /// Process a subsequent XFR response message. - /// - /// Revives the given iterator with new data based on the given response. - pub async fn process_next_answer<'a>( - it: &mut XfrEventIterator<'a>, - resp: &'a Message, - ) -> Result<(), Error> { - // Verify that the given iterator is in the expected state - match it.state { - State::AwaitingAnswer { .. } => { - // Verify that the given iterator is exhausted - if it.records.next().is_some() { - return Err(Error::AnswerNotFullyProcessed); - } - - let _ = Self::check_is_xfr_answer(it.req, resp).await?; - let answer = resp.answer().map_err(Error::ParseError)?; - it.records = answer.into_records().flatten(); - - Ok(()) - } - - State::TransferComplete => { - // We already finished processing an XFR response sequence. We - // don't expect there to be any more messages to process!. - Err(Error::Malformed) - } - - State::TransferFailed => { - // We had to terminate processing of the XFR response sequence - // due to a problem with the received data, so we don't expect - // to be invoked again with another response message! - Err(Error::Terminated) - } - } - } + // /// Process a subsequent XFR response message. + // /// + // /// Revives the given iterator with new data based on the given response. + // pub async fn process_next_answer<'a>( + // it: &mut XfrEventIterator<'a>, + // resp: &'a Message, + // ) -> Result<(), Error> { + // // Verify that the given iterator is in the expected state + // match it.state { + // State::AwaitingAnswer { .. } => { + // // Verify that the given iterator is exhausted + // if it.iter.next().is_some() { + // return Err(Error::AnswerNotFullyProcessed); + // } + + // let _ = Self::check_respons(it.req, resp).await?; + // let answer = resp.answer().map_err(Error::ParseError)?; + // it.iter = answer.into_records().flatten(); + + // Ok(()) + // } + + // State::TransferComplete => { + // // We already finished processing an XFR response sequence. We + // // don't expect there to be any more messages to process!. + // Err(Error::Malformed) + // } + + // State::TransferFailed => { + // // We had to terminate processing of the XFR response sequence + // // due to a problem with the received data, so we don't expect + // // to be invoked again with another response message! + // Err(Error::Terminated) + // } + // } + // } /// Check if an XFR response header is valid. /// @@ -145,113 +194,107 @@ impl XfrResponseProcessor { /// Returns Ok on success, Err otherwise. On success the type of XFR that /// was determined is returned as well as the answer section from the XFR /// response. - async fn check_is_xfr_answer( - req: &Message, - resp: &Message, - ) -> Result { - // Check the request. - let req_header = req.header(); - let req_counts = req.header_counts(); - - if req.is_error() - || req_header.qr() - || req_counts.qdcount() != 1 - || req_counts.ancount() != 0 - || req_header.opcode() != Opcode::QUERY - { - return Err(CheckError::NotValidXfrQuery); - } - - let Some(qtype) = req.qtype() else { - return Err(CheckError::NotValidXfrResponse); - }; - - let xfr_type = match qtype { - Rtype::AXFR => XfrType::Axfr, - Rtype::IXFR => XfrType::Ixfr, - _ => return Err(CheckError::NotValidXfrResponse), - }; - - // https://datatracker.ietf.org/doc/html/rfc1995#section-3 - // 3. Query Format - // "The IXFR query packet format is the same as that of a normal DNS - // query, but with the query type being IXFR and the authority - // section containing the SOA record of client's version of the - // zone." - if matches!(xfr_type, XfrType::Ixfr) && req_counts.nscount() != 1 { - return Err(CheckError::NotValidXfrResponse); - } - - // Check the response. + fn check_response(&self, resp: &Message) -> Result<(), Error> { let resp_header = resp.header(); let resp_counts = resp.header_counts(); if resp.is_error() - || !resp.is_answer_header(req) + || !resp.is_answer_header(&self.req) || resp_header.opcode() != Opcode::QUERY || resp_header.tc() || resp_counts.ancount() == 0 || resp_counts.nscount() != 0 { - return Err(CheckError::NotValidXfrResponse); + return Err(Error::NotValidXfrResponse); } - // + //https://www.rfc-editor.org/rfc/rfc5936.html#section-2.2.1 // 2.2.1. Header Values // "QDCOUNT MUST be 1 in the first message; // MUST be 0 or 1 in all following messages;" - if resp_counts.qdcount() != 1 - || resp.sole_question() != req.sole_question() + let qdcount = resp_counts.qdcount(); + if (self.state.is_none() && qdcount != 1) + || (self.state.is_some() && qdcount > 1) { - return Err(CheckError::NotValidXfrResponse); + return Err(Error::NotValidXfrResponse); } - Ok(xfr_type) + Ok(()) } /// Initialise the processosr. /// /// Records the initial SOA record and other details will will be used /// while processing the rest of the response. - async fn initialize( - initial_xfr_type: XfrType, - soa_record: XfrRecord, - ) -> Result { - // The initial record should be a SOA record. - let data = soa_record.into_data(); + fn initialize( + &mut self, + resp: Message, + ) -> Result< + ( + Arc>, + AnyRecordIter<'_, Bytes, AllRecordData>>, + ), + Error, + > { + let xfr_type = match self.req.qtype() { + Some(Rtype::AXFR) => XfrType::Axfr, + Some(Rtype::IXFR) => XfrType::Ixfr, + _ => unreachable!(), // Checked already in check_request(). + }; - let AllRecordData::Soa(soa) = data else { - return Err(CheckError::NotValidXfrResponse); + self.resp = Some(resp); + + let Some(resp) = &self.resp else { + unreachable!(); }; - Ok(State::AwaitingAnswer(ParsingState::new( - initial_xfr_type, - soa, - ))) - } -} + let answer = resp.answer().map_err(Error::ParseError)?; -//------------ State ---------------------------------------------------------- + let mut records = answer.into_records(); -/// The current processing state. -#[derive(Debug)] -enum State { - /// Waiting for an XFR response message. - AwaitingAnswer(ParsingState), + let Some(Ok(record)) = records.next() else { + return Err(Error::Malformed); + }; + + // The initial record should be a SOA record. + let AllRecordData::Soa(soa) = record.into_data() else { + return Err(Error::NotValidXfrResponse); + }; + + let state = Arc::new(Mutex::new(State::new(xfr_type, soa))); + self.state.replace(state.clone()); + + Ok((state, records)) + } + + /// Initialise the processosr. + /// + /// Records the initial SOA record and other details will will be used + /// while processing the rest of the response. + fn update( + &mut self, + resp: Message, + ) -> Result< + AnyRecordIter<'_, Bytes, AllRecordData>>, + Error, + > { + self.resp = Some(resp); + + let Some(resp) = &self.resp else { + unreachable!(); + }; - /// The end of the XFR response sequence was detected. - TransferComplete, + let answer = resp.answer().map_err(Error::ParseError)?; - /// An unrecoverable problem occurred while processing the XFR response - /// sequence. - TransferFailed, + Ok(answer.into_records()) + } } -//------------ ParsingState --------------------------------------------------- +//------------ State ---------------------------------------------------------- /// State related to parsing the XFR response sequence. #[derive(Debug)] -struct ParsingState { +struct State { /// The type of XFR response sequence being parsed. /// /// This can differ to the type of XFR response sequence that we expected @@ -277,7 +320,7 @@ struct ParsingState { rr_count: usize, } -impl ParsingState { +impl State { /// Create a new parsing state. fn new( initial_xfr_type: XfrType, @@ -296,10 +339,7 @@ impl ParsingState { /// /// Returns an [`XfrEvent`] that should be emitted for the parsed record, /// if any. - fn parse_record( - &mut self, - rec: XfrRecord, - ) -> Result>, Error> { + fn parse_record(&mut self, rec: XfrRecord) -> XfrEvent { self.rr_count += 1; // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2 @@ -333,11 +373,11 @@ impl ParsingState { // MUST conclude with the same SOA resource record. // Intermediate messages MUST NOT contain the SOA resource // record." - Ok(Some(XfrEvent::EndOfTransfer)) + XfrEvent::EndOfTransfer } XfrType::Axfr => { - Ok(Some(XfrEvent::AddRecord(self.current_soa.serial(), rec))) + XfrEvent::AddRecord(self.current_soa.serial(), rec) } XfrType::Ixfr if self.rr_count < 2 => unreachable!(), @@ -345,7 +385,7 @@ impl ParsingState { XfrType::Ixfr if self.rr_count == 2 => { if record_matches_initial_soa { // IXFR not available, AXFR of empty zone detected. - Ok(Some(XfrEvent::EndOfTransfer)) + XfrEvent::EndOfTransfer } else if let Some(soa) = soa { // This SOA record is the start of an IXFR diff sequence. self.current_soa = soa.clone(); @@ -357,7 +397,7 @@ impl ParsingState { IxfrUpdateMode::Deleting ); - Ok(Some(XfrEvent::BeginBatchDelete(soa.serial()))) + XfrEvent::BeginBatchDelete(soa.serial()) } else { // https://datatracker.ietf.org/doc/html/rfc1995#section-4 // 4. Response Format @@ -387,10 +427,7 @@ impl ParsingState { // assume that "incremental zone transfer is not available" // and so "the behaviour is the same as an AXFR response", self.actual_xfr_type = XfrType::Axfr; - Ok(Some(XfrEvent::AddRecord( - self.current_soa.serial(), - rec, - ))) + XfrEvent::AddRecord(self.current_soa.serial(), rec) } } @@ -405,36 +442,30 @@ impl ParsingState { // Is this the end of the transfer, or the start // of a new diff sequence? if record_matches_initial_soa { - Ok(Some(XfrEvent::EndOfTransfer)) + XfrEvent::EndOfTransfer } else { - Ok(Some(XfrEvent::BeginBatchDelete( + XfrEvent::BeginBatchDelete( self.current_soa.serial(), - ))) + ) } } IxfrUpdateMode::Adding => { // We just switched from the Delete phase of a // diff sequence to the add phase of the diff // sequence. - Ok(Some(XfrEvent::BeginBatchAdd( - self.current_soa.serial(), - ))) + XfrEvent::BeginBatchAdd(self.current_soa.serial()) } } } else { match self.ixfr_update_mode { - IxfrUpdateMode::Deleting => { - Ok(Some(XfrEvent::DeleteRecord( - self.current_soa.serial(), - rec, - ))) - } - IxfrUpdateMode::Adding => { - Ok(Some(XfrEvent::AddRecord( - self.current_soa.serial(), - rec, - ))) - } + IxfrUpdateMode::Deleting => XfrEvent::DeleteRecord( + self.current_soa.serial(), + rec, + ), + IxfrUpdateMode::Adding => XfrEvent::AddRecord( + self.current_soa.serial(), + rec, + ), } } } @@ -508,80 +539,50 @@ impl std::fmt::Display for XfrEvent { //------------ XfrEventIterator ----------------------------------------------- /// An iterator over [`XfrResponseProcessor`] generated [`XfrEvent`]s. -#[derive(Debug)] pub struct XfrEventIterator<'a> { - /// The original XFR request. - /// - /// Used to check if responses relate to the original request. - req: &'a Message, - - /// The current parsing state. - state: State, + /// The parent processor. + processor_state: Arc>, /// An iterator over the records in the current response. - records: Flatten< - AnyRecordIter<'a, Bytes, AllRecordData>>, - >, + iter: AnyRecordIter<'a, Bytes, AllRecordData>>, +} + +impl<'a> XfrEventIterator<'a> { + fn new( + processor_state: Arc>, + iter: AnyRecordIter< + 'a, + Bytes, + AllRecordData>, + >, + ) -> Self { + Self { + processor_state, + iter, + } + } } impl<'a> Iterator for XfrEventIterator<'a> { type Item = Result, XfrEventIteratorError>; fn next(&mut self) -> Option { - match &mut self.state { - State::AwaitingAnswer(parsing_state) => { - #[allow(clippy::blocks_in_conditions)] - let event = self.records.next().and_then(|record| { - trace!( - "XFR record {}: {record:?}", - parsing_state.rr_count - ); - parsing_state.parse_record(record).unwrap_or_default() - }); - - match event { - Some(XfrEvent::EndOfTransfer) => { - // Record that the transfer completed. - self.state = State::TransferComplete; - - // Return the end of transfer event so that the client - // can distinguish this condition from None (end of - // iteration) which can happen when the message body - // has been consumed but more records are needed from - // subsequent responses to complete the transfer. - Some(Ok(XfrEvent::EndOfTransfer)) - } + match self.iter.next() { + Some(Ok(record)) => { + let mut state = self.processor_state.lock().unwrap(); - Some(XfrEvent::ProcessingFailed) => { - // Record that the transfer failed. - self.state = State::TransferFailed; + trace!("XFR record {}: {record:?}", state.rr_count); + let event = state.parse_record(record); - // Return an error. - Some(Err(XfrEventIteratorError::ProcessingFailed)) - } - - Some(e) => { - // Return the event. - Some(Ok(e)) - } - - None => { - // No more events available: end iteration for now. - // The client can revive this iterator by passing it - // to XfrResponseProcessor::process_next_answer(). - None - } - } + Some(Ok(event)) } - State::TransferComplete => { - // The transfer was completed parsed. No more events available. - None + Some(Err(err)) => { + Some(Err(XfrEventIteratorError::ParseError(err))) } - State::TransferFailed => { - // We had to terminate processing of the XFR response sequence - // due to a problem with the received data. No more events available. + None => { + // No more events available: end iteration. None } } @@ -594,7 +595,7 @@ impl<'a> Iterator for XfrEventIterator<'a> { #[derive(Copy, Clone, Debug, PartialEq, Eq)] pub enum XfrEventIteratorError { /// Transfer processing failed. - ProcessingFailed, + ParseError(ParseError), } //------------ IxfrUpdateMode ------------------------------------------------- @@ -631,7 +632,7 @@ pub enum Error { ParseError(ParseError), /// The request message is not an XFR query/ - NotValidXfrQuery, + NotValidXfrRequest, /// The response message is not an XFR response. NotValidXfrResponse, @@ -647,43 +648,43 @@ pub enum Error { Terminated, } -//--- From +// //--- From -impl From for Error { - fn from(err: CheckError) -> Self { - match err { - CheckError::ParseError(err) => Self::ParseError(err), - CheckError::NotValidXfrQuery => Self::NotValidXfrQuery, - CheckError::NotValidXfrResponse => Self::NotValidXfrResponse, - } - } -} +// impl From for Error { +// fn from(err: CheckError) -> Self { +// match err { +// CheckError::ParseError(err) => Self::ParseError(err), +// CheckError::NotValidXfrQuery => Self::NotValidXfrQuery, +// CheckError::NotValidXfrResponse => Self::NotValidXfrResponse, +// } +// } +// } -//------------ PrepareError --------------------------------------------------- +// //------------ PrepareError --------------------------------------------------- -/// Errors that can occur during intiial checking of an XFR response sequence. -#[derive(Debug)] -enum CheckError { - /// A parsing error occurred while checking the original request and - /// response messages. - ParseError(ParseError), +// /// Errors that can occur during intiial checking of an XFR response sequence. +// #[derive(Debug)] +// enum CheckError { +// /// A parsing error occurred while checking the original request and +// /// response messages. +// ParseError(ParseError), - /// The XFR request is not valid according to the rules defined by RFC - /// 5936 (AXFR) or RFC 1995 (IXFR). - NotValidXfrQuery, +// /// The XFR request is not valid according to the rules defined by RFC +// /// 5936 (AXFR) or RFC 1995 (IXFR). +// NotValidXfrQuery, - /// The XFR response is not valid according to the rules defined by RFC - /// 5936 (AXFR) or RFC 1995 (IXFR). - NotValidXfrResponse, -} +// /// The XFR response is not valid according to the rules defined by RFC +// /// 5936 (AXFR) or RFC 1995 (IXFR). +// NotValidXfrResponse, +// } -//--- From +// //--- From -impl From for CheckError { - fn from(err: ParseError) -> Self { - Self::ParseError(err) - } -} +// impl From for CheckError { +// fn from(err: ParseError) -> Self { +// Self::ParseError(err) +// } +// } //------------ XfrType -------------------------------------------------------- @@ -736,81 +737,100 @@ mod tests { use super::XfrEvent as XE; use super::*; - #[tokio::test] - async fn request_message_is_rejected() { + #[test] + fn request_message_is_rejected() { init_logging(); - // Create a non-XFR request to reply to. + // Create a no/n-XFR request to reply to. let req = mk_request("example.com", Rtype::A).into_message(); + // Create an XFR response processor. + let res = XfrResponseProcessor::new(req.clone()); + // Process the request and assert that it is rejected as not being // a valid XFR response and that no XFR processor events were emitted. - let res = XfrResponseProcessor::process_answer(&req, &req).await; - assert!(matches!(res, Err(Error::NotValidXfrResponse))); + assert!(matches!(res, Err(Error::NotValidXfrRequest))); } - #[tokio::test] - async fn non_xfr_response_is_rejected() { + #[test] + fn non_xfr_response_is_rejected() { init_logging(); - // Create a non-XFR request to reply to. - let req = mk_request("example.com", Rtype::A).into_message(); + // Create an AXFR-like request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); // Create a non-XFR response. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + let resp = answer.into_message(); // Process the response and assert that it is rejected as not being // a valid XFR response and that no XFR processor events were emitted. - let resp = answer.into_message(); - let res = XfrResponseProcessor::process_answer(&req, &resp).await; - assert!(matches!(res, Err(Error::NotValidXfrResponse))); + assert!(matches!( + processor.process_answer(resp), + Err(Error::NotValidXfrResponse) + )); } - #[tokio::test] - async fn axfr_response_with_no_answers_is_rejected() { + #[test] + fn axfr_response_with_no_answers_is_rejected() { init_logging(); // Create an AXFR request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + // Create a response that lacks answers. - let answer = mk_empty_answer(&req, Rcode::NOERROR); + let resp = mk_empty_answer(&req, Rcode::NOERROR).into_message(); // Process the response and assert that it is rejected as not being // a valid XFR response and that no XFR processor events were emitted. - let resp = answer.into_message(); - let res = XfrResponseProcessor::process_answer(&req, &resp).await; - assert!(matches!(res, Err(Error::NotValidXfrResponse))); + assert!(matches!( + processor.process_answer(resp), + Err(Error::NotValidXfrResponse) + )); } - #[tokio::test] - async fn error_axfr_response_is_rejected() { + #[test] + fn error_axfr_response_is_rejected() { init_logging(); // Create an AXFR request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + // Create a minimal valid AXFR response, just something that should // not be rejected by the XFR processor due to its content. It should // however be rejected due to the non-NOERROR rcode. let mut answer = mk_empty_answer(&req, Rcode::SERVFAIL); add_answer_record(&req, &mut answer, mk_soa(Serial::now())); + let resp = answer.into_message(); // Process the response and assert that it is rejected as not being // a valid XFR response and that no XFR processor events were emitted. - let resp = answer.into_message(); - let res = XfrResponseProcessor::process_answer(&req, &resp).await; - assert!(matches!(res, Err(Error::NotValidXfrResponse))); + assert!(matches!( + processor.process_answer(resp), + Err(Error::NotValidXfrResponse) + )); } - #[tokio::test] - async fn incomplete_axfr_response_is_accepted() { + #[test] + fn incomplete_axfr_response_is_accepted() { init_logging(); // Create an AXFR request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + // Create an incomplete AXFR response. A proper AXFR response has at // least two identical SOA records, one at the start and one at the // end, but this response contains only a single SOA record. This is @@ -818,24 +838,25 @@ mod tests { // still provide the missing SOA record. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); add_answer_record(&req, &mut answer, mk_soa(Serial::now())); + let resp = answer.into_message(); // Process the response. - let resp = answer.into_message(); - let mut it = XfrResponseProcessor::process_answer(&req, &resp) - .await - .unwrap(); + let mut it = processor.process_answer(resp).unwrap(); // Verify that no events are by the XFR processor. assert!(it.next().is_none()); } - #[tokio::test] - async fn axfr_response_with_only_soas_is_accepted() { + #[test] + fn axfr_response_with_only_soas_is_accepted() { init_logging(); // Create an AXFR request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + // Create a complete but minimal AXFR response. A proper AXFR response // has at least two identical SOA records, one at the start and one at // the end, with actual zone records in between. This response has only @@ -845,25 +866,26 @@ mod tests { let soa = mk_soa(Serial::now()); add_answer_record(&req, &mut answer, soa.clone()); add_answer_record(&req, &mut answer, soa); + let resp = answer.into_message(); // Process the response. - let resp = answer.into_message(); - let mut it = XfrResponseProcessor::process_answer(&req, &resp) - .await - .unwrap(); + let mut it = processor.process_answer(resp).unwrap(); // Verify the events emitted by the XFR processor. assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); assert!(it.next().is_none()); } - #[tokio::test] - async fn axfr_multi_response_with_only_soas_is_accepted() { + #[test] + fn axfr_multi_response_with_only_soas_is_accepted() { init_logging(); // Create an AXFR request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + // Create a complete but minimal AXFR response. A proper AXFR response // has at least two identical SOA records, one at the start and one at // the end, with actual zone records in between. This response has only @@ -872,12 +894,10 @@ mod tests { let mut answer = mk_empty_answer(&req, Rcode::NOERROR); let soa = mk_soa(Serial::now()); add_answer_record(&req, &mut answer, soa.clone()); + let resp = answer.into_message(); // Process the response. - let resp = answer.into_message(); - let mut it = XfrResponseProcessor::process_answer(&req, &resp) - .await - .unwrap(); + let mut it = processor.process_answer(resp).unwrap(); // Verify the events emitted by the XFR processor. assert!(it.next().is_none()); @@ -885,25 +905,26 @@ mod tests { // Create another AXFR response to complete the transfer. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); add_answer_record(&req, &mut answer, soa); + let resp = answer.into_message(); // Process the response. - let resp = answer.into_message(); - XfrResponseProcessor::process_next_answer(&mut it, &resp) - .await - .unwrap(); + let mut it = processor.process_answer(resp).unwrap(); // Verify the events emitted by the XFR processor. assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); assert!(it.next().is_none()); } - #[tokio::test] - async fn axfr_response_generates_expected_events() { + #[test] + fn axfr_response_generates_expected_events() { init_logging(); // Create an AXFR request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + // Create an AXFR response. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); let serial = Serial::now(); @@ -912,12 +933,10 @@ mod tests { add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); add_answer_record(&req, &mut answer, soa); + let resp = answer.into_message(); // Process the response. - let resp = answer.into_message(); - let mut it = XfrResponseProcessor::process_answer(&req, &resp) - .await - .unwrap(); + let mut it = processor.process_answer(resp).unwrap(); // Verify the events emitted by the XFR processor. let s = serial; @@ -927,8 +946,8 @@ mod tests { assert!(it.next().is_none()); } - #[tokio::test] - async fn ixfr_response_generates_expected_events() { + #[test] + fn ixfr_response_generates_expected_events() { init_logging(); // Create an IXFR request to reply to. @@ -939,6 +958,9 @@ mod tests { add_authority_record(&mut authority, soa); let req = authority.into_message(); + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + // Prepare some serial numbers and SOA records to use in the IXFR response. let old_serial = client_serial; let new_serial = client_serial.add(1); @@ -964,12 +986,10 @@ mod tests { add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); // Closing SOA with servers current SOA add_answer_record(&req, &mut answer, new_soa); + let resp = answer.into_message(); // Process the response. - let resp = answer.into_message(); - let it = XfrResponseProcessor::process_answer(&req, &resp) - .await - .unwrap(); + let it = processor.process_answer(resp).unwrap(); // Verify the events emitted by the XFR processor. let owner = ParsedName::from(Name::from_str("example.com").unwrap()); @@ -1014,8 +1034,8 @@ mod tests { assert!(it.eq(expected_events)); } - #[tokio::test] - async fn multi_ixfr_response_generates_expected_events() { + #[test] + fn multi_ixfr_response_generates_expected_events() { init_logging(); // Create an IXFR request to reply to. @@ -1026,6 +1046,9 @@ mod tests { add_authority_record(&mut authority, soa); let req = authority.into_message(); + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + // Prepare some serial numbers and SOA records to use in the IXFR response. let old_serial = client_serial; let new_serial = client_serial.add(1); @@ -1042,12 +1065,10 @@ mod tests { // new version of the zone. add_answer_record(&req, &mut answer, old_soa); add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + let resp = answer.into_message(); // Process the response. - let resp = answer.into_message(); - let mut it = XfrResponseProcessor::process_answer(&req, &resp) - .await - .unwrap(); + let mut it = processor.process_answer(resp).unwrap(); // Verify the events emitted by the XFR processor. assert!(matches!(it.next(), Some(Ok(XE::BeginBatchDelete(_))))); @@ -1064,12 +1085,10 @@ mod tests { add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); // Closing SOA with servers current SOA add_answer_record(&req, &mut answer, new_soa); + let resp = answer.into_message(); // Process the response. - let resp = answer.into_message(); - XfrResponseProcessor::process_next_answer(&mut it, &resp) - .await - .unwrap(); + let mut it = processor.process_answer(resp).unwrap(); // Verify the events emitted by the XFR processor. assert!(matches!(it.next(), Some(Ok(XE::DeleteRecord(..))))); From 3f78708ce9f7b4c7fe341ee1b89a5d6ee62c8702 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 27 Aug 2024 21:50:38 +0200 Subject: [PATCH 051/333] Get rid of Arc and remove commented out code. --- src/net/client/xfr.rs | 338 +++++++++++++++++------------------------- 1 file changed, 137 insertions(+), 201 deletions(-) diff --git a/src/net/client/xfr.rs b/src/net/client/xfr.rs index 2089cde5f..aaa19723f 100644 --- a/src/net/client/xfr.rs +++ b/src/net/client/xfr.rs @@ -5,7 +5,6 @@ //! [`XfrEvent`]s that they represent without having to deal with the //! AXFR/IXFR protocol details. use std::fmt::Debug; -use std::sync::{Arc, Mutex}; use bytes::Bytes; use tracing::trace; @@ -44,12 +43,20 @@ pub type XfrRecord = /// The process of producing and consuming iterators continues until the end /// of the transfer is detected or a parsing error occurs. pub struct XfrResponseProcessor { + /// The XFR request for which responses should be processed. req: Message, + /// Internal state. + inner: Inner, +} + +#[derive(Default)] +struct Inner { /// The response message currently being processed. resp: Option>, - state: Option>>, + /// The processing state. + iteration_state: Option, } impl XfrResponseProcessor { @@ -60,15 +67,39 @@ impl XfrResponseProcessor { Self::check_request(&req)?; Ok(Self { req, - resp: None, - state: None, + inner: Default::default(), }) } } impl XfrResponseProcessor { + /// Process a single AXFR/IXFR response message. + /// + /// Return an [`XfrEventIterator`] over [`XfrEvent`]s emitted during + /// processing. + /// + /// If the returned iterator does not emit an + /// [`XfrEvent::TransferComplete`] event, call [`process_next_answer()`] + /// with the next response message to continue iterating over the transfer + /// responses. + pub fn process_answer( + &mut self, + resp: Message, + ) -> Result { + // Check that the given message is a DNS XFR response. + self.check_response(&resp)?; + + let is_first = self.inner.initialize(&self.req, resp)?; + + XfrEventIterator::new( + is_first, + &mut self.inner.iteration_state, + self.inner.resp.as_ref().unwrap(), + ) + } + /// Check if an XFR request is valid. - pub fn check_request(req: &Message) -> Result<(), Error> { + fn check_request(req: &Message) -> Result<(), Error> { let req_header = req.header(); let req_counts = req.header_counts(); @@ -102,87 +133,6 @@ impl XfrResponseProcessor { Ok(()) } - /// Process a single AXFR/IXFR response message. - /// - /// Return an [`XfrEventIterator`] over [`XfrEvent`]s emitted during - /// processing. - /// - /// If the returned iterator does not emit an - /// [`XfrEvent::TransferComplete`] event, call [`process_next_answer()`] - /// with the next response message to continue iterating over the transfer - /// responses. - pub fn process_answer( - &mut self, - resp: Message, - ) -> Result { - // Check that the given message is a DNS XFR response. - self.check_response(&resp)?; - - // https://datatracker.ietf.org/doc/html/rfc5936#section-3 - // 3. Zone Contents "The objective of the AXFR session is to request - // and transfer the contents of a zone, in order to permit the AXFR - // client to faithfully reconstruct the zone as it exists at the - // primary server for the given zone serial number. The word - // "exists" here designates the externally visible behavior, i.e., - // the zone content that is being served (handed out to clients) -- - // not its persistent representation in a zone file or database - // used by the server -- and that for consistency should be served - // subsequently by the AXFR client in an identical manner." - // - // So, walk over all the records in the answer, not just those that - // might be expected to exist in a zone (i.e. not just ZoneRecordData - // record types). - - let state; - let iter = if self.state.is_none() { - let (state_clone, iter) = self.initialize(resp)?; - state = state_clone; - iter - } else { - state = self.state.as_ref().unwrap().clone(); - self.update(resp)? - }; - - Ok(XfrEventIterator::new(state, iter)) - } - - // /// Process a subsequent XFR response message. - // /// - // /// Revives the given iterator with new data based on the given response. - // pub async fn process_next_answer<'a>( - // it: &mut XfrEventIterator<'a>, - // resp: &'a Message, - // ) -> Result<(), Error> { - // // Verify that the given iterator is in the expected state - // match it.state { - // State::AwaitingAnswer { .. } => { - // // Verify that the given iterator is exhausted - // if it.iter.next().is_some() { - // return Err(Error::AnswerNotFullyProcessed); - // } - - // let _ = Self::check_respons(it.req, resp).await?; - // let answer = resp.answer().map_err(Error::ParseError)?; - // it.iter = answer.into_records().flatten(); - - // Ok(()) - // } - - // State::TransferComplete => { - // // We already finished processing an XFR response sequence. We - // // don't expect there to be any more messages to process!. - // Err(Error::Malformed) - // } - - // State::TransferFailed => { - // // We had to terminate processing of the XFR response sequence - // // due to a problem with the received data, so we don't expect - // // to be invoked again with another response message! - // Err(Error::Terminated) - // } - // } - // } - /// Check if an XFR response header is valid. /// /// Enforce the rules defined in 2. AXFR Messages of RFC 5936. See: @@ -213,80 +163,75 @@ impl XfrResponseProcessor { // "QDCOUNT MUST be 1 in the first message; // MUST be 0 or 1 in all following messages;" let qdcount = resp_counts.qdcount(); - if (self.state.is_none() && qdcount != 1) - || (self.state.is_some() && qdcount > 1) + if (self.inner.iteration_state.is_none() && qdcount != 1) + || (self.inner.iteration_state.is_some() && qdcount > 1) { return Err(Error::NotValidXfrResponse); } Ok(()) } +} +impl Inner { /// Initialise the processosr. /// /// Records the initial SOA record and other details will will be used /// while processing the rest of the response. fn initialize( &mut self, + req: &Message, resp: Message, - ) -> Result< - ( - Arc>, - AnyRecordIter<'_, Bytes, AllRecordData>>, - ), - Error, - > { - let xfr_type = match self.req.qtype() { - Some(Rtype::AXFR) => XfrType::Axfr, - Some(Rtype::IXFR) => XfrType::Ixfr, - _ => unreachable!(), // Checked already in check_request(). - }; - - self.resp = Some(resp); - - let Some(resp) = &self.resp else { - unreachable!(); - }; - - let answer = resp.answer().map_err(Error::ParseError)?; - - let mut records = answer.into_records(); - - let Some(Ok(record)) = records.next() else { - return Err(Error::Malformed); - }; - - // The initial record should be a SOA record. - let AllRecordData::Soa(soa) = record.into_data() else { - return Err(Error::NotValidXfrResponse); - }; - - let state = Arc::new(Mutex::new(State::new(xfr_type, soa))); - self.state.replace(state.clone()); - - Ok((state, records)) - } - - /// Initialise the processosr. - /// - /// Records the initial SOA record and other details will will be used - /// while processing the rest of the response. - fn update( - &mut self, - resp: Message, - ) -> Result< - AnyRecordIter<'_, Bytes, AllRecordData>>, - Error, - > { + ) -> Result { self.resp = Some(resp); - let Some(resp) = &self.resp else { - unreachable!(); - }; - - let answer = resp.answer().map_err(Error::ParseError)?; - - Ok(answer.into_records()) + if self.iteration_state.is_none() { + let Some(resp) = &self.resp else { + unreachable!(); + }; + + let answer = resp.answer().map_err(Error::ParseError)?; + + // https://datatracker.ietf.org/doc/html/rfc5936#section-3 + // 3. Zone Contents + // "The objective of the AXFR session is to request and transfer + // the contents of a zone, in order to permit the AXFR client + // to faithfully reconstruct the zone as it exists at the + // primary server for the given zone serial number. The word + // "exists" here designates the externally visible behavior, + // i.e., the zone content that is being served (handed out to + // clients) -- not its persistent representation in a zone file + // or database used by the server -- and that for consistency + // should be served subsequently by the AXFR client in an + // identical manner." + // + // So, walk over all the records in the answer, not just those + // that might be expected to exist in a zone (i.e. not just + // ZoneRecordData record types). + + let mut records = answer.into_records(); + + let xfr_type = match req.qtype() { + Some(Rtype::AXFR) => XfrType::Axfr, + Some(Rtype::IXFR) => XfrType::Ixfr, + _ => unreachable!(), // Checked already in check_request(). + }; + + let Some(Ok(record)) = records.next() else { + return Err(Error::Malformed); + }; + + // The initial record should be a SOA record. + let AllRecordData::Soa(soa) = record.into_data() else { + return Err(Error::NotValidXfrResponse); + }; + + self.iteration_state.replace(State::new(xfr_type, soa)); + + Ok(true) + } else { + Ok(false) + } } } @@ -539,40 +484,69 @@ impl std::fmt::Display for XfrEvent { //------------ XfrEventIterator ----------------------------------------------- /// An iterator over [`XfrResponseProcessor`] generated [`XfrEvent`]s. -pub struct XfrEventIterator<'a> { +pub struct XfrEventIterator<'a, 'b> { /// The parent processor. - processor_state: Arc>, + iteration_state: &'a mut Option, /// An iterator over the records in the current response. - iter: AnyRecordIter<'a, Bytes, AllRecordData>>, + iter: AnyRecordIter<'b, Bytes, AllRecordData>>, } -impl<'a> XfrEventIterator<'a> { +impl<'a, 'b> XfrEventIterator<'a, 'b> { fn new( - processor_state: Arc>, - iter: AnyRecordIter< - 'a, - Bytes, - AllRecordData>, - >, - ) -> Self { - Self { - processor_state, - iter, + is_first: bool, + iteration_state: &'a mut Option, + resp: &'b Message, + ) -> Result { + let answer = resp.answer().map_err(Error::ParseError)?; + + // https://datatracker.ietf.org/doc/html/rfc5936#section-3 + // 3. Zone Contents + // "The objective of the AXFR session is to request and transfer + // the contents of a zone, in order to permit the AXFR client + // to faithfully reconstruct the zone as it exists at the + // primary server for the given zone serial number. The word + // "exists" here designates the externally visible behavior, + // i.e., the zone content that is being served (handed out to + // clients) -- not its persistent representation in a zone file + // or database used by the server -- and that for consistency + // should be served subsequently by the AXFR client in an + // identical manner." + // + // So, walk over all the records in the answer, not just those + // that might be expected to exist in a zone (i.e. not just + // ZoneRecordData record types). + + let mut iter = answer.into_records(); + + if is_first { + let Some(Ok(_)) = iter.next() else { + return Err(Error::Malformed); + }; } + + Ok(Self { + iteration_state, + iter, + }) } } -impl<'a> Iterator for XfrEventIterator<'a> { +impl<'a, 'b> Iterator for XfrEventIterator<'a, 'b> { type Item = Result, XfrEventIteratorError>; fn next(&mut self) -> Option { + let Some(iteration_state) = self.iteration_state else { + unreachable!(); + }; + match self.iter.next() { Some(Ok(record)) => { - let mut state = self.processor_state.lock().unwrap(); - - trace!("XFR record {}: {record:?}", state.rr_count); - let event = state.parse_record(record); + trace!( + "XFR record {}: {record:?}", + iteration_state.rr_count + ); + let event = iteration_state.parse_record(record); Some(Ok(event)) } @@ -648,44 +622,6 @@ pub enum Error { Terminated, } -// //--- From - -// impl From for Error { -// fn from(err: CheckError) -> Self { -// match err { -// CheckError::ParseError(err) => Self::ParseError(err), -// CheckError::NotValidXfrQuery => Self::NotValidXfrQuery, -// CheckError::NotValidXfrResponse => Self::NotValidXfrResponse, -// } -// } -// } - -// //------------ PrepareError --------------------------------------------------- - -// /// Errors that can occur during intiial checking of an XFR response sequence. -// #[derive(Debug)] -// enum CheckError { -// /// A parsing error occurred while checking the original request and -// /// response messages. -// ParseError(ParseError), - -// /// The XFR request is not valid according to the rules defined by RFC -// /// 5936 (AXFR) or RFC 1995 (IXFR). -// NotValidXfrQuery, - -// /// The XFR response is not valid according to the rules defined by RFC -// /// 5936 (AXFR) or RFC 1995 (IXFR). -// NotValidXfrResponse, -// } - -// //--- From - -// impl From for CheckError { -// fn from(err: ParseError) -> Self { -// Self::ParseError(err) -// } -// } - //------------ XfrType -------------------------------------------------------- /// The type of XFR response sequence. From 6e25d252e126361afed1115c32b211cdde7a5428 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 27 Aug 2024 21:56:14 +0200 Subject: [PATCH 052/333] Don't pass Some. --- src/net/client/xfr.rs | 35 ++++++++++++----------------------- 1 file changed, 12 insertions(+), 23 deletions(-) diff --git a/src/net/client/xfr.rs b/src/net/client/xfr.rs index aaa19723f..6af36be0c 100644 --- a/src/net/client/xfr.rs +++ b/src/net/client/xfr.rs @@ -55,8 +55,8 @@ struct Inner { /// The response message currently being processed. resp: Option>, - /// The processing state. - iteration_state: Option, + /// The iteration state. + state: Option, } impl XfrResponseProcessor { @@ -93,7 +93,7 @@ impl XfrResponseProcessor { XfrEventIterator::new( is_first, - &mut self.inner.iteration_state, + self.inner.state.as_mut().unwrap(), self.inner.resp.as_ref().unwrap(), ) } @@ -163,8 +163,8 @@ impl XfrResponseProcessor { // "QDCOUNT MUST be 1 in the first message; // MUST be 0 or 1 in all following messages;" let qdcount = resp_counts.qdcount(); - if (self.inner.iteration_state.is_none() && qdcount != 1) - || (self.inner.iteration_state.is_some() && qdcount > 1) + if (self.inner.state.is_none() && qdcount != 1) + || (self.inner.state.is_some() && qdcount > 1) { return Err(Error::NotValidXfrResponse); } @@ -185,7 +185,7 @@ impl Inner { ) -> Result { self.resp = Some(resp); - if self.iteration_state.is_none() { + if self.state.is_none() { let Some(resp) = &self.resp else { unreachable!(); }; @@ -226,7 +226,7 @@ impl Inner { return Err(Error::NotValidXfrResponse); }; - self.iteration_state.replace(State::new(xfr_type, soa)); + self.state.replace(State::new(xfr_type, soa)); Ok(true) } else { @@ -486,7 +486,7 @@ impl std::fmt::Display for XfrEvent { /// An iterator over [`XfrResponseProcessor`] generated [`XfrEvent`]s. pub struct XfrEventIterator<'a, 'b> { /// The parent processor. - iteration_state: &'a mut Option, + state: &'a mut State, /// An iterator over the records in the current response. iter: AnyRecordIter<'b, Bytes, AllRecordData>>, @@ -495,7 +495,7 @@ pub struct XfrEventIterator<'a, 'b> { impl<'a, 'b> XfrEventIterator<'a, 'b> { fn new( is_first: bool, - iteration_state: &'a mut Option, + state: &'a mut State, resp: &'b Message, ) -> Result { let answer = resp.answer().map_err(Error::ParseError)?; @@ -525,10 +525,7 @@ impl<'a, 'b> XfrEventIterator<'a, 'b> { }; } - Ok(Self { - iteration_state, - iter, - }) + Ok(Self { state, iter }) } } @@ -536,18 +533,10 @@ impl<'a, 'b> Iterator for XfrEventIterator<'a, 'b> { type Item = Result, XfrEventIteratorError>; fn next(&mut self) -> Option { - let Some(iteration_state) = self.iteration_state else { - unreachable!(); - }; - match self.iter.next() { Some(Ok(record)) => { - trace!( - "XFR record {}: {record:?}", - iteration_state.rr_count - ); - let event = iteration_state.parse_record(record); - + trace!("XFR record {}: {record:?}", self.state.rr_count); + let event = self.state.parse_record(record); Some(Ok(event)) } From 59f36c5a784d7b557e70de62bfdaf20cc00824ed Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 27 Aug 2024 23:32:20 +0200 Subject: [PATCH 053/333] Cleanup. --- src/net/client/mod.rs | 1 - src/net/client/xfr.rs | 1091 ------------------------------- src/net/mod.rs | 1 + src/net/xfr/client/iterator.rs | 81 +++ src/net/xfr/client/mod.rs | 12 + src/net/xfr/client/processor.rs | 410 ++++++++++++ src/net/xfr/client/tests.rs | 451 +++++++++++++ src/net/xfr/client/types.rs | 167 +++++ src/net/xfr/mod.rs | 2 + 9 files changed, 1124 insertions(+), 1092 deletions(-) delete mode 100644 src/net/client/xfr.rs create mode 100644 src/net/xfr/client/iterator.rs create mode 100644 src/net/xfr/client/mod.rs create mode 100644 src/net/xfr/client/processor.rs create mode 100644 src/net/xfr/client/tests.rs create mode 100644 src/net/xfr/client/types.rs create mode 100644 src/net/xfr/mod.rs diff --git a/src/net/client/mod.rs b/src/net/client/mod.rs index b49d4336b..0275f3bef 100644 --- a/src/net/client/mod.rs +++ b/src/net/client/mod.rs @@ -193,4 +193,3 @@ pub mod tsig; #[cfg(feature = "unstable-validator")] pub mod validator; pub mod validator_test; -pub mod xfr; diff --git a/src/net/client/xfr.rs b/src/net/client/xfr.rs deleted file mode 100644 index 6af36be0c..000000000 --- a/src/net/client/xfr.rs +++ /dev/null @@ -1,1091 +0,0 @@ -//! Parsing of AXFR/IXFR response messages for higher level processing. -//! -//! This module provides [`XfrResponseProcessor`] which enables you to process -//! one or more AXFR/IXFR response messages in terms of the high level -//! [`XfrEvent`]s that they represent without having to deal with the -//! AXFR/IXFR protocol details. -use std::fmt::Debug; - -use bytes::Bytes; -use tracing::trace; - -use crate::base::iana::Opcode; -use crate::base::message::AnyRecordIter; -use crate::base::wire::ParseError; -use crate::base::{Message, ParsedName, Record, Rtype, Serial}; -use crate::rdata::{AllRecordData, Soa}; - -//------------ XfrRecord ------------------------------------------------------ - -/// The type of record processed by [`XfrResponseProcessor`]. -pub type XfrRecord = - Record, AllRecordData>>; - -//------------ XfrResponseProcessor ------------------------------------------- - -/// An AXFR/IXFR response processor. -/// -/// [`XfrResponseProcessor`] can be invoked on one or more sequentially -/// AXFR/IXFR received response messages to verify them and during processing -/// emit events which can be consumed via the iterator returned by -/// [`process_answer()`]. -/// -/// Each [`XfrEventIterator`] produces events for a single response message. -/// If the end of the XFR response sequence has been reached the iterator will -/// emit an [`XfrEvent::TransferComplete`] event. -/// -/// If the `TransferComplete` event has not been seen it means that the -/// sequence is incomplete and the next response message in the sequence -/// should be passed to [`process_next_answer()`] along with the exhausted -/// iterator. This will populate thr [`XfrEventIterator`] with more records -/// to parse thereby causing iteration to resume. -/// -/// The process of producing and consuming iterators continues until the end -/// of the transfer is detected or a parsing error occurs. -pub struct XfrResponseProcessor { - /// The XFR request for which responses should be processed. - req: Message, - - /// Internal state. - inner: Inner, -} - -#[derive(Default)] -struct Inner { - /// The response message currently being processed. - resp: Option>, - - /// The iteration state. - state: Option, -} - -impl XfrResponseProcessor { - /// Creates a new instance of [`XfrMessageProcessor`]. - /// - /// Processes a single XFR response stream. - pub fn new(req: Message) -> Result { - Self::check_request(&req)?; - Ok(Self { - req, - inner: Default::default(), - }) - } -} - -impl XfrResponseProcessor { - /// Process a single AXFR/IXFR response message. - /// - /// Return an [`XfrEventIterator`] over [`XfrEvent`]s emitted during - /// processing. - /// - /// If the returned iterator does not emit an - /// [`XfrEvent::TransferComplete`] event, call [`process_next_answer()`] - /// with the next response message to continue iterating over the transfer - /// responses. - pub fn process_answer( - &mut self, - resp: Message, - ) -> Result { - // Check that the given message is a DNS XFR response. - self.check_response(&resp)?; - - let is_first = self.inner.initialize(&self.req, resp)?; - - XfrEventIterator::new( - is_first, - self.inner.state.as_mut().unwrap(), - self.inner.resp.as_ref().unwrap(), - ) - } - - /// Check if an XFR request is valid. - fn check_request(req: &Message) -> Result<(), Error> { - let req_header = req.header(); - let req_counts = req.header_counts(); - - if req.is_error() - || req_header.qr() - || req_counts.qdcount() != 1 - || req_counts.ancount() != 0 - || req_header.opcode() != Opcode::QUERY - { - return Err(Error::NotValidXfrRequest); - } - - let Some(qtype) = req.qtype() else { - return Err(Error::NotValidXfrRequest); - }; - - if !matches!(qtype, Rtype::AXFR | Rtype::IXFR) { - return Err(Error::NotValidXfrRequest); - } - - // https://datatracker.ietf.org/doc/html/rfc1995#section-3 - // 3. Query Format - // "The IXFR query packet format is the same as that of a normal DNS - // query, but with the query type being IXFR and the authority - // section containing the SOA record of client's version of the - // zone." - if matches!(qtype, Rtype::IXFR) && req_counts.nscount() != 1 { - return Err(Error::NotValidXfrRequest); - } - - Ok(()) - } - - /// Check if an XFR response header is valid. - /// - /// Enforce the rules defined in 2. AXFR Messages of RFC 5936. See: - /// https://www.rfc-editor.org/rfc/rfc5936.html#section-2 - /// - /// Takes a request as well as a response as the response is checked to - /// see if it is in reply to the given request. - /// - /// Returns Ok on success, Err otherwise. On success the type of XFR that - /// was determined is returned as well as the answer section from the XFR - /// response. - fn check_response(&self, resp: &Message) -> Result<(), Error> { - let resp_header = resp.header(); - let resp_counts = resp.header_counts(); - - if resp.is_error() - || !resp.is_answer_header(&self.req) - || resp_header.opcode() != Opcode::QUERY - || resp_header.tc() - || resp_counts.ancount() == 0 - || resp_counts.nscount() != 0 - { - return Err(Error::NotValidXfrResponse); - } - - //https://www.rfc-editor.org/rfc/rfc5936.html#section-2.2.1 - // 2.2.1. Header Values - // "QDCOUNT MUST be 1 in the first message; - // MUST be 0 or 1 in all following messages;" - let qdcount = resp_counts.qdcount(); - if (self.inner.state.is_none() && qdcount != 1) - || (self.inner.state.is_some() && qdcount > 1) - { - return Err(Error::NotValidXfrResponse); - } - - Ok(()) - } -} - -impl Inner { - /// Initialise the processosr. - /// - /// Records the initial SOA record and other details will will be used - /// while processing the rest of the response. - fn initialize( - &mut self, - req: &Message, - resp: Message, - ) -> Result { - self.resp = Some(resp); - - if self.state.is_none() { - let Some(resp) = &self.resp else { - unreachable!(); - }; - - let answer = resp.answer().map_err(Error::ParseError)?; - - // https://datatracker.ietf.org/doc/html/rfc5936#section-3 - // 3. Zone Contents - // "The objective of the AXFR session is to request and transfer - // the contents of a zone, in order to permit the AXFR client - // to faithfully reconstruct the zone as it exists at the - // primary server for the given zone serial number. The word - // "exists" here designates the externally visible behavior, - // i.e., the zone content that is being served (handed out to - // clients) -- not its persistent representation in a zone file - // or database used by the server -- and that for consistency - // should be served subsequently by the AXFR client in an - // identical manner." - // - // So, walk over all the records in the answer, not just those - // that might be expected to exist in a zone (i.e. not just - // ZoneRecordData record types). - - let mut records = answer.into_records(); - - let xfr_type = match req.qtype() { - Some(Rtype::AXFR) => XfrType::Axfr, - Some(Rtype::IXFR) => XfrType::Ixfr, - _ => unreachable!(), // Checked already in check_request(). - }; - - let Some(Ok(record)) = records.next() else { - return Err(Error::Malformed); - }; - - // The initial record should be a SOA record. - let AllRecordData::Soa(soa) = record.into_data() else { - return Err(Error::NotValidXfrResponse); - }; - - self.state.replace(State::new(xfr_type, soa)); - - Ok(true) - } else { - Ok(false) - } - } -} - -//------------ State ---------------------------------------------------------- - -/// State related to parsing the XFR response sequence. -#[derive(Debug)] -struct State { - /// The type of XFR response sequence being parsed. - /// - /// This can differ to the type of XFR response sequence that we expected - /// to parse because the server can fallback from IXFR to AXFR. - actual_xfr_type: XfrType, - - /// The initial SOA record that signals the start and end of both AXFR and - /// IXFR response sequences. - initial_soa: Soa>, - - /// The current SOA record. - /// - /// For AXFR response sequences this will be the same as `initial_soa`. - /// For IXFR response sequences this will be the last SOA record parsed as - /// each diff sequence contains two SOA records: one at the start of the - /// delete sequence and one at the start of the add sequence. - current_soa: Soa>, - - /// The kind of records currently being processed, either adds or deletes. - ixfr_update_mode: IxfrUpdateMode, - - /// The number of resource records parsed so far. - rr_count: usize, -} - -impl State { - /// Create a new parsing state. - fn new( - initial_xfr_type: XfrType, - initial_soa: Soa>, - ) -> Self { - Self { - actual_xfr_type: initial_xfr_type, - initial_soa: initial_soa.clone(), - current_soa: initial_soa, - rr_count: 1, - ixfr_update_mode: Default::default(), - } - } - - /// Parse a single resource record. - /// - /// Returns an [`XfrEvent`] that should be emitted for the parsed record, - /// if any. - fn parse_record(&mut self, rec: XfrRecord) -> XfrEvent { - self.rr_count += 1; - - // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2 - // 2.2. AXFR Response - // "..clients MUST accept any ordering and grouping of the non-SOA - // RRs. Each RR SHOULD be transmitted only once, and AXFR clients - // MUST ignore any duplicate RRs received." - // - // Note: We do NOT implement this MUST here because it would be very - // inefficient to actually check that any received non-SOA RR has not - // been seen before during the in-progress transfer. Clients of - // XfrResponseProcessor are better placed to enforce this rule if - // needed, e.g. at the moment of insertion into a zone tree checking - // that the record is not already present or insertion of a duplicate - // having no effect as it is already present. - - let soa = match rec.data() { - AllRecordData::Soa(soa) => Some(soa), - _ => None, - }; - - let record_matches_initial_soa = soa == Some(&self.initial_soa); - - match self.actual_xfr_type { - XfrType::Axfr if record_matches_initial_soa => { - // https://www.rfc-editor.org/rfc/rfc5936.html#section-2.2 - // 2.2. AXFR Response - // ... - // "In such a series, the first message MUST begin with the - // SOA resource record of the zone, and the last message - // MUST conclude with the same SOA resource record. - // Intermediate messages MUST NOT contain the SOA resource - // record." - XfrEvent::EndOfTransfer - } - - XfrType::Axfr => { - XfrEvent::AddRecord(self.current_soa.serial(), rec) - } - - XfrType::Ixfr if self.rr_count < 2 => unreachable!(), - - XfrType::Ixfr if self.rr_count == 2 => { - if record_matches_initial_soa { - // IXFR not available, AXFR of empty zone detected. - XfrEvent::EndOfTransfer - } else if let Some(soa) = soa { - // This SOA record is the start of an IXFR diff sequence. - self.current_soa = soa.clone(); - - // We don't need to set the IXFR update more here as it - // should already be set to Deleting. - debug_assert_eq!( - self.ixfr_update_mode, - IxfrUpdateMode::Deleting - ); - - XfrEvent::BeginBatchDelete(soa.serial()) - } else { - // https://datatracker.ietf.org/doc/html/rfc1995#section-4 - // 4. Response Format - // "If incremental zone transfer is not available, the - // entire zone is returned. The first and the last RR - // of the response is the SOA record of the zone. I.e. - // the behavior is the same as an AXFR response except - // the query type is IXFR. - // - // If incremental zone transfer is available, one or - // more difference sequences is returned. The list of - // difference sequences is preceded and followed by a - // copy of the server's current version of the SOA. - // - // Each difference sequence represents one update to - // the zone (one SOA serial change) consisting of - // deleted RRs and added RRs. The first RR of the - // deleted RRs is the older SOA RR and the first RR of - // the added RRs is the newer SOA RR. - // - // Modification of an RR is performed first by removing - // the original RR and then adding the modified one." - - // As this is IXFR and this is the second record, it should - // be the "first RR of the deleted RRs" which should be - // "the older SOA RR". However, it isn't a SOA RR. As such - // assume that "incremental zone transfer is not available" - // and so "the behaviour is the same as an AXFR response", - self.actual_xfr_type = XfrType::Axfr; - XfrEvent::AddRecord(self.current_soa.serial(), rec) - } - } - - XfrType::Ixfr => { - if let Some(soa) = soa { - self.ixfr_update_mode.toggle(); - self.current_soa = soa.clone(); - - match self.ixfr_update_mode { - IxfrUpdateMode::Deleting => { - // We just finished a (Delete, Add) diff sequence. - // Is this the end of the transfer, or the start - // of a new diff sequence? - if record_matches_initial_soa { - XfrEvent::EndOfTransfer - } else { - XfrEvent::BeginBatchDelete( - self.current_soa.serial(), - ) - } - } - IxfrUpdateMode::Adding => { - // We just switched from the Delete phase of a - // diff sequence to the add phase of the diff - // sequence. - XfrEvent::BeginBatchAdd(self.current_soa.serial()) - } - } - } else { - match self.ixfr_update_mode { - IxfrUpdateMode::Deleting => XfrEvent::DeleteRecord( - self.current_soa.serial(), - rec, - ), - IxfrUpdateMode::Adding => XfrEvent::AddRecord( - self.current_soa.serial(), - rec, - ), - } - } - } - } - } -} - -//------------ RecordResult --------------------------------------------------- - -/// An event emitted by [`XfrResponseProcessor`] during transfer processing. -#[derive(Copy, Clone, Debug, PartialEq, Eq)] -pub enum XfrEvent { - /// Delete record R in zone serial S. - /// - /// The transfer signalled that the given record should be deleted from - /// the zone version with the given serial number. - /// - /// Note: If the transfer contains N deletions of fhe same record then - /// this event will occur N times. - DeleteRecord(Serial, R), - - /// Add record R in zone serial S. - /// - /// The transfer signalled that the given record should be added to the - /// zone version with the given serial number. - /// - /// Note: If the transfer contains N additions of fhe same record then - /// this event will occur N times. - AddRecord(Serial, R), - - /// Prepare to delete records in zone serial S. - /// - /// The transfer signalled that zero or more record deletions will follow, - /// all for the zone version with the given serial number. - BeginBatchDelete(Serial), - - /// Prepare to add records in zone serial S. - /// - /// The transfer signalled that zero or more record additions will follow, - /// all for the zone version with the given serial number. - BeginBatchAdd(Serial), - - /// Transfer completed successfully. - /// - /// Note: This event is not emitted until the final record of the final - /// response in a set of one or more transfer responss has been seen. - EndOfTransfer, - - /// Transfer processing failed. - /// - /// This event indicates that there is a problem with the transfer data - /// and that transfer processing cannot continue. - ProcessingFailed, -} - -//--- Display - -impl std::fmt::Display for XfrEvent { - fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { - match self { - XfrEvent::DeleteRecord(_, _) => f.write_str("DeleteRecord"), - XfrEvent::AddRecord(_, _) => f.write_str("AddRecord"), - XfrEvent::BeginBatchDelete(_) => f.write_str("BeginBatchDelete"), - XfrEvent::BeginBatchAdd(_) => f.write_str("BeginBatchAdd"), - XfrEvent::EndOfTransfer => f.write_str("EndOfTransfer"), - XfrEvent::ProcessingFailed => f.write_str("ProcessingFailed"), - } - } -} - -//------------ XfrEventIterator ----------------------------------------------- - -/// An iterator over [`XfrResponseProcessor`] generated [`XfrEvent`]s. -pub struct XfrEventIterator<'a, 'b> { - /// The parent processor. - state: &'a mut State, - - /// An iterator over the records in the current response. - iter: AnyRecordIter<'b, Bytes, AllRecordData>>, -} - -impl<'a, 'b> XfrEventIterator<'a, 'b> { - fn new( - is_first: bool, - state: &'a mut State, - resp: &'b Message, - ) -> Result { - let answer = resp.answer().map_err(Error::ParseError)?; - - // https://datatracker.ietf.org/doc/html/rfc5936#section-3 - // 3. Zone Contents - // "The objective of the AXFR session is to request and transfer - // the contents of a zone, in order to permit the AXFR client - // to faithfully reconstruct the zone as it exists at the - // primary server for the given zone serial number. The word - // "exists" here designates the externally visible behavior, - // i.e., the zone content that is being served (handed out to - // clients) -- not its persistent representation in a zone file - // or database used by the server -- and that for consistency - // should be served subsequently by the AXFR client in an - // identical manner." - // - // So, walk over all the records in the answer, not just those - // that might be expected to exist in a zone (i.e. not just - // ZoneRecordData record types). - - let mut iter = answer.into_records(); - - if is_first { - let Some(Ok(_)) = iter.next() else { - return Err(Error::Malformed); - }; - } - - Ok(Self { state, iter }) - } -} - -impl<'a, 'b> Iterator for XfrEventIterator<'a, 'b> { - type Item = Result, XfrEventIteratorError>; - - fn next(&mut self) -> Option { - match self.iter.next() { - Some(Ok(record)) => { - trace!("XFR record {}: {record:?}", self.state.rr_count); - let event = self.state.parse_record(record); - Some(Ok(event)) - } - - Some(Err(err)) => { - Some(Err(XfrEventIteratorError::ParseError(err))) - } - - None => { - // No more events available: end iteration. - None - } - } - } -} - -//------------ XfrEventIteratorError ------------------------------------------ - -/// Errors that can occur during XfrEventIterator iteration. -#[derive(Copy, Clone, Debug, PartialEq, Eq)] -pub enum XfrEventIteratorError { - /// Transfer processing failed. - ParseError(ParseError), -} - -//------------ IxfrUpdateMode ------------------------------------------------- - -/// The kind of records currently being processed, either adds or deletes. -#[derive(Copy, Clone, Debug, Default, PartialEq, Eq)] -enum IxfrUpdateMode { - /// The records being parsed are deletions. - /// - /// Deletions come before additions. - #[default] - Deleting, - - /// The records being parsed are additions. - Adding, -} - -impl IxfrUpdateMode { - /// Toggle between the possible [`IxfrUpdateMode`] variants. - fn toggle(&mut self) { - match self { - IxfrUpdateMode::Deleting => *self = IxfrUpdateMode::Adding, - IxfrUpdateMode::Adding => *self = IxfrUpdateMode::Deleting, - } - } -} - -//------------ Error ---------------------------------------------------------- - -/// An error reported by [`XfrResponseProcessor`]. -#[derive(Debug)] -pub enum Error { - /// The message could not be parsed. - ParseError(ParseError), - - /// The request message is not an XFR query/ - NotValidXfrRequest, - - /// The response message is not an XFR response. - NotValidXfrResponse, - - /// At least one record in the XFR response sequence is incorrect. - Malformed, - - /// At least one record in the XFR response sequence was not consumed - /// by the caller. - AnswerNotFullyProcessed, - - /// Processing was already terminated for this XFR response sequence. - Terminated, -} - -//------------ XfrType -------------------------------------------------------- - -/// The type of XFR response sequence. -#[derive(Copy, Clone, Debug, PartialEq, Eq)] -enum XfrType { - /// RFC 5936 AXFR. - /// - /// A complete snapshot of a zone at a particular version. - Axfr, - - /// RFC 1995 IXFR. - /// - /// An incremental diff of the version of the zone that the server has - /// compared to the version of the zone that the client has. - Ixfr, -} - -//--- From - -impl TryFrom for XfrType { - type Error = (); - - fn try_from(rtype: Rtype) -> Result { - match rtype { - Rtype::AXFR => Ok(XfrType::Axfr), - Rtype::IXFR => Ok(XfrType::Ixfr), - _ => Err(()), - } - } -} - -#[cfg(test)] -mod tests { - use core::str::FromStr; - - use bytes::BytesMut; - use octseq::Octets; - - use crate::base::iana::Rcode; - use crate::base::message_builder::{ - AnswerBuilder, AuthorityBuilder, QuestionBuilder, - }; - use crate::base::net::Ipv4Addr; - use crate::base::rdata::ComposeRecordData; - use crate::base::{MessageBuilder, Ttl}; - use crate::base::{Name, ToName}; - use crate::rdata::A; - - use super::XfrEvent as XE; - use super::*; - - #[test] - fn request_message_is_rejected() { - init_logging(); - - // Create a no/n-XFR request to reply to. - let req = mk_request("example.com", Rtype::A).into_message(); - - // Create an XFR response processor. - let res = XfrResponseProcessor::new(req.clone()); - - // Process the request and assert that it is rejected as not being - // a valid XFR response and that no XFR processor events were emitted. - assert!(matches!(res, Err(Error::NotValidXfrRequest))); - } - - #[test] - fn non_xfr_response_is_rejected() { - init_logging(); - - // Create an AXFR-like request to reply to. - let req = mk_request("example.com", Rtype::AXFR).into_message(); - - // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); - - // Create a non-XFR response. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - let resp = answer.into_message(); - - // Process the response and assert that it is rejected as not being - // a valid XFR response and that no XFR processor events were emitted. - assert!(matches!( - processor.process_answer(resp), - Err(Error::NotValidXfrResponse) - )); - } - - #[test] - fn axfr_response_with_no_answers_is_rejected() { - init_logging(); - - // Create an AXFR request to reply to. - let req = mk_request("example.com", Rtype::AXFR).into_message(); - - // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); - - // Create a response that lacks answers. - let resp = mk_empty_answer(&req, Rcode::NOERROR).into_message(); - - // Process the response and assert that it is rejected as not being - // a valid XFR response and that no XFR processor events were emitted. - assert!(matches!( - processor.process_answer(resp), - Err(Error::NotValidXfrResponse) - )); - } - - #[test] - fn error_axfr_response_is_rejected() { - init_logging(); - - // Create an AXFR request to reply to. - let req = mk_request("example.com", Rtype::AXFR).into_message(); - - // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); - - // Create a minimal valid AXFR response, just something that should - // not be rejected by the XFR processor due to its content. It should - // however be rejected due to the non-NOERROR rcode. - let mut answer = mk_empty_answer(&req, Rcode::SERVFAIL); - add_answer_record(&req, &mut answer, mk_soa(Serial::now())); - let resp = answer.into_message(); - - // Process the response and assert that it is rejected as not being - // a valid XFR response and that no XFR processor events were emitted. - assert!(matches!( - processor.process_answer(resp), - Err(Error::NotValidXfrResponse) - )); - } - - #[test] - fn incomplete_axfr_response_is_accepted() { - init_logging(); - - // Create an AXFR request to reply to. - let req = mk_request("example.com", Rtype::AXFR).into_message(); - - // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); - - // Create an incomplete AXFR response. A proper AXFR response has at - // least two identical SOA records, one at the start and one at the - // end, but this response contains only a single SOA record. This is - // not considered invalid however because a subsequent response could - // still provide the missing SOA record. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - add_answer_record(&req, &mut answer, mk_soa(Serial::now())); - let resp = answer.into_message(); - - // Process the response. - let mut it = processor.process_answer(resp).unwrap(); - - // Verify that no events are by the XFR processor. - assert!(it.next().is_none()); - } - - #[test] - fn axfr_response_with_only_soas_is_accepted() { - init_logging(); - - // Create an AXFR request to reply to. - let req = mk_request("example.com", Rtype::AXFR).into_message(); - - // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); - - // Create a complete but minimal AXFR response. A proper AXFR response - // has at least two identical SOA records, one at the start and one at - // the end, with actual zone records in between. This response has only - // the start and end SOA and no content in between. RFC 5936 doesn't - // seem to disallow this. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - let soa = mk_soa(Serial::now()); - add_answer_record(&req, &mut answer, soa.clone()); - add_answer_record(&req, &mut answer, soa); - let resp = answer.into_message(); - - // Process the response. - let mut it = processor.process_answer(resp).unwrap(); - - // Verify the events emitted by the XFR processor. - assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); - assert!(it.next().is_none()); - } - - #[test] - fn axfr_multi_response_with_only_soas_is_accepted() { - init_logging(); - - // Create an AXFR request to reply to. - let req = mk_request("example.com", Rtype::AXFR).into_message(); - - // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); - - // Create a complete but minimal AXFR response. A proper AXFR response - // has at least two identical SOA records, one at the start and one at - // the end, with actual zone records in between. This response has only - // the start and end SOA and no content in between. RFC 5936 doesn't - // seem to disallow this. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - let soa = mk_soa(Serial::now()); - add_answer_record(&req, &mut answer, soa.clone()); - let resp = answer.into_message(); - - // Process the response. - let mut it = processor.process_answer(resp).unwrap(); - - // Verify the events emitted by the XFR processor. - assert!(it.next().is_none()); - - // Create another AXFR response to complete the transfer. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - add_answer_record(&req, &mut answer, soa); - let resp = answer.into_message(); - - // Process the response. - let mut it = processor.process_answer(resp).unwrap(); - - // Verify the events emitted by the XFR processor. - assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); - assert!(it.next().is_none()); - } - - #[test] - fn axfr_response_generates_expected_events() { - init_logging(); - - // Create an AXFR request to reply to. - let req = mk_request("example.com", Rtype::AXFR).into_message(); - - // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); - - // Create an AXFR response. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - let serial = Serial::now(); - let soa = mk_soa(serial); - add_answer_record(&req, &mut answer, soa.clone()); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); - add_answer_record(&req, &mut answer, soa); - let resp = answer.into_message(); - - // Process the response. - let mut it = processor.process_answer(resp).unwrap(); - - // Verify the events emitted by the XFR processor. - let s = serial; - assert!(matches!(it.next(), Some(Ok(XE::AddRecord(n, _))) if n == s)); - assert!(matches!(it.next(), Some(Ok(XE::AddRecord(n, _))) if n == s)); - assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); - assert!(it.next().is_none()); - } - - #[test] - fn ixfr_response_generates_expected_events() { - init_logging(); - - // Create an IXFR request to reply to. - let req = mk_request("example.com", Rtype::IXFR); - let mut authority = req.authority(); - let client_serial = Serial::now(); - let soa = mk_soa(client_serial); - add_authority_record(&mut authority, soa); - let req = authority.into_message(); - - // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); - - // Prepare some serial numbers and SOA records to use in the IXFR response. - let old_serial = client_serial; - let new_serial = client_serial.add(1); - let old_soa = mk_soa(old_serial); - let new_soa = mk_soa(new_serial); - - // Create an IXFR response. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - // Outer SOA with servers current SOA - add_answer_record(&req, &mut answer, new_soa.clone()); - // Start of diff sequence: SOA of the servers' previous zone version - // (which matches that of the client) followed by records to be - // deleted as they were in that version of the zone but are not in the - // new version of the zone. - add_answer_record(&req, &mut answer, old_soa); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); - // SOA of the servers` new zone version (which is ahead of that of the - // client) followed by records to be added as they were added in this - // new version of the zone.` - add_answer_record(&req, &mut answer, new_soa.clone()); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - // Closing SOA with servers current SOA - add_answer_record(&req, &mut answer, new_soa); - let resp = answer.into_message(); - - // Process the response. - let it = processor.process_answer(resp).unwrap(); - - // Verify the events emitted by the XFR processor. - let owner = ParsedName::from(Name::from_str("example.com").unwrap()); - let expected_events = [ - Ok(XfrEvent::BeginBatchDelete(old_serial)), - Ok(XfrEvent::DeleteRecord( - old_serial, - Record::from(( - owner.clone(), - 0, - AllRecordData::A(A::new(Ipv4Addr::LOCALHOST)), - )), - )), - Ok(XfrEvent::DeleteRecord( - old_serial, - Record::from(( - owner.clone(), - 0, - AllRecordData::A(A::new(Ipv4Addr::BROADCAST)), - )), - )), - Ok(XfrEvent::BeginBatchAdd(new_serial)), - Ok(XfrEvent::AddRecord( - new_serial, - Record::from(( - owner.clone(), - 0, - AllRecordData::A(A::new(Ipv4Addr::BROADCAST)), - )), - )), - Ok(XfrEvent::AddRecord( - new_serial, - Record::from(( - owner, - 0, - AllRecordData::A(A::new(Ipv4Addr::LOCALHOST)), - )), - )), - Ok(XfrEvent::EndOfTransfer), - ]; - - assert!(it.eq(expected_events)); - } - - #[test] - fn multi_ixfr_response_generates_expected_events() { - init_logging(); - - // Create an IXFR request to reply to. - let req = mk_request("example.com", Rtype::IXFR); - let mut authority = req.authority(); - let client_serial = Serial::now(); - let soa = mk_soa(client_serial); - add_authority_record(&mut authority, soa); - let req = authority.into_message(); - - // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); - - // Prepare some serial numbers and SOA records to use in the IXFR response. - let old_serial = client_serial; - let new_serial = client_serial.add(1); - let old_soa = mk_soa(old_serial); - let new_soa = mk_soa(new_serial); - - // Create a partial IXFR response. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - // Outer SOA with servers current SOA - add_answer_record(&req, &mut answer, new_soa.clone()); - // Start of diff sequence: SOA of the servers' previous zone version - // (which matches that of the client) followed by records to be - // deleted as they were in that version of the zone but are not in the - // new version of the zone. - add_answer_record(&req, &mut answer, old_soa); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - let resp = answer.into_message(); - - // Process the response. - let mut it = processor.process_answer(resp).unwrap(); - - // Verify the events emitted by the XFR processor. - assert!(matches!(it.next(), Some(Ok(XE::BeginBatchDelete(_))))); - assert!(matches!(it.next(), Some(Ok(XE::DeleteRecord(..))))); - - // Craete a second IXFR response that completes the transfer - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); - // SOA of the servers` new zone version (which is ahead of that of the - // client) followed by records to be added as they were added in this - // new version of the zone.` - add_answer_record(&req, &mut answer, new_soa.clone()); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - // Closing SOA with servers current SOA - add_answer_record(&req, &mut answer, new_soa); - let resp = answer.into_message(); - - // Process the response. - let mut it = processor.process_answer(resp).unwrap(); - - // Verify the events emitted by the XFR processor. - assert!(matches!(it.next(), Some(Ok(XE::DeleteRecord(..))))); - assert!(matches!(it.next(), Some(Ok(XE::BeginBatchAdd(_))))); - assert!(matches!(it.next(), Some(Ok(XE::AddRecord(..))))); - assert!(matches!(it.next(), Some(Ok(XE::AddRecord(..))))); - assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); - assert!(it.next().is_none()); - } - - //------------ Helper functions ------------------------------------------- - - fn init_logging() { - // Initialize tracing based logging. Override with env var RUST_LOG, e.g. - // RUST_LOG=trace. DEBUG level will show the .rpl file name, Stelline step - // numbers and types as they are being executed. - tracing_subscriber::fmt() - .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) - .with_thread_ids(true) - .without_time() - .try_init() - .ok(); - } - - fn mk_request(qname: &str, qtype: Rtype) -> QuestionBuilder { - let req = MessageBuilder::new_bytes(); - let mut req = req.question(); - req.push((Name::vec_from_str(qname).unwrap(), qtype)) - .unwrap(); - req - } - - fn mk_empty_answer( - req: &Message, - rcode: Rcode, - ) -> AnswerBuilder { - let builder = MessageBuilder::new_bytes(); - builder.start_answer(req, rcode).unwrap() - } - - fn add_answer_record( - req: &Message, - answer: &mut AnswerBuilder, - item: T, - ) { - let question = req.sole_question().unwrap(); - let qname = question.qname(); - let qclass = question.qclass(); - answer - .push((qname, qclass, Ttl::from_secs(0), item)) - .unwrap(); - } - - fn add_authority_record( - authority: &mut AuthorityBuilder, - item: T, - ) { - let (qname, qclass) = { - let question = authority.as_message().sole_question().unwrap(); - let qname = question.qname().to_bytes(); - let qclass = question.qclass(); - (qname, qclass) - }; - authority - .push((qname, qclass, Ttl::from_secs(0), item)) - .unwrap(); - } - - fn mk_soa(serial: Serial) -> Soa> { - let mname = Name::from_str("mname").unwrap(); - let rname = Name::from_str("rname").unwrap(); - let ttl = Ttl::from_secs(0); - Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl) - } -} diff --git a/src/net/mod.rs b/src/net/mod.rs index 98657acec..f07663fb3 100644 --- a/src/net/mod.rs +++ b/src/net/mod.rs @@ -18,3 +18,4 @@ pub mod client; pub mod server; +pub mod xfr; \ No newline at end of file diff --git a/src/net/xfr/client/iterator.rs b/src/net/xfr/client/iterator.rs new file mode 100644 index 000000000..54a235cc8 --- /dev/null +++ b/src/net/xfr/client/iterator.rs @@ -0,0 +1,81 @@ +//! Emit events while iterating over an XFR response message. + +use bytes::Bytes; +use tracing::trace; + +use crate::base::{message::AnyRecordIter, Message, ParsedName}; +use crate::rdata::AllRecordData; + +use super::processor::RecordProcessor; +use super::types::{Error, XfrEvent, XfrEventIteratorError, XfrRecord}; + +///------------ XfrEventIterator ----------------------------------------------- + +/// An iterator over [`XfrResponseProcessor`] generated [`XfrEvent`]s. +pub struct XfrEventIterator<'a, 'b> { + /// The parent processor. + state: &'a mut RecordProcessor, + + /// An iterator over the records in the current response. + iter: AnyRecordIter<'b, Bytes, AllRecordData>>, +} + +impl<'a, 'b> XfrEventIterator<'a, 'b> { + pub(super) fn new( + state: &'a mut RecordProcessor, + resp: &'b Message, + ) -> Result { + let answer = resp.answer().map_err(Error::ParseError)?; + + // https://datatracker.ietf.org/doc/html/rfc5936#section-3 + // 3. Zone Contents + // "The objective of the AXFR session is to request and transfer + // the contents of a zone, in order to permit the AXFR client + // to faithfully reconstruct the zone as it exists at the + // primary server for the given zone serial number. The word + // "exists" here designates the externally visible behavior, + // i.e., the zone content that is being served (handed out to + // clients) -- not its persistent representation in a zone file + // or database used by the server -- and that for consistency + // should be served subsequently by the AXFR client in an + // identical manner." + // + // So, walk over all the records in the answer, not just those + // that might be expected to exist in a zone (i.e. not just + // ZoneRecordData record types). + + let mut iter = answer.into_records(); + + if state.rr_count == 0 { + let Some(Ok(_)) = iter.next() else { + return Err(Error::Malformed); + }; + state.rr_count += 1; + } + + Ok(Self { state, iter }) + } +} + +impl<'a, 'b> Iterator for XfrEventIterator<'a, 'b> { + type Item = Result, XfrEventIteratorError>; + + fn next(&mut self) -> Option { + match self.iter.next() { + Some(Ok(record)) => { + trace!("XFR record {}: {record:?}", self.state.rr_count); + let event = self.state.process_record(record); + Some(Ok(event)) + } + + Some(Err(err)) => { + Some(Err(XfrEventIteratorError::ParseError(err))) + } + + None => { + // No more events available: end iteration. + None + } + } + } +} diff --git a/src/net/xfr/client/mod.rs b/src/net/xfr/client/mod.rs new file mode 100644 index 000000000..2a8fe5706 --- /dev/null +++ b/src/net/xfr/client/mod.rs @@ -0,0 +1,12 @@ +//! Parsing of AXFR/IXFR response messages for higher level processing. +//! +//! This module provides [`XfrResponseProcessor`] which enables you to process +//! one or more AXFR/IXFR response messages in terms of the high level +//! [`XfrEvent`]s that they represent without having to deal with the +//! AXFR/IXFR protocol details. +pub mod iterator; +pub mod processor; +pub mod types; + +#[cfg(test)] +mod tests; \ No newline at end of file diff --git a/src/net/xfr/client/processor.rs b/src/net/xfr/client/processor.rs new file mode 100644 index 000000000..5b4b216b7 --- /dev/null +++ b/src/net/xfr/client/processor.rs @@ -0,0 +1,410 @@ +use std::fmt::Debug; + +use bytes::Bytes; + +use crate::base::iana::Opcode; +use crate::base::{Message, ParsedName, Rtype}; +use crate::rdata::{AllRecordData, Soa}; + +use super::iterator::XfrEventIterator; +use super::types::{Error, IxfrUpdateMode, XfrEvent, XfrRecord, XfrType}; + +//------------ XfrResponseProcessor ------------------------------------------- + +/// An AXFR/IXFR response processor. +/// +/// [`XfrResponseProcessor`] can be invoked on one or more sequentially +/// AXFR/IXFR received response messages to verify them and during processing +/// emit events which can be consumed via the iterator returned by +/// [`process_answer()`]. +/// +/// Each [`XfrEventIterator`] produces events for a single response message. +/// If the end of the XFR response sequence has been reached the iterator will +/// emit an [`XfrEvent::TransferComplete`] event. +/// +/// If the `TransferComplete` event has not been seen it means that the +/// sequence is incomplete and the next response message in the sequence +/// should be passed to [`process_next_answer()`] along with the exhausted +/// iterator. This will populate thr [`XfrEventIterator`] with more records +/// to parse thereby causing iteration to resume. +/// +/// The process of producing and consuming iterators continues until the end +/// of the transfer is detected or a parsing error occurs. +pub struct XfrResponseProcessor { + /// The XFR request for which responses should be processed. + req: Message, + + /// Internal state. + /// + /// None until the first call to [`process_answer()`]. + inner: Option, +} + +impl XfrResponseProcessor { + /// Creates a new instance of [`XfrMessageProcessor`]. + /// + /// Processes a single XFR response stream. + pub fn new(req: Message) -> Result { + Self::check_request(&req)?; + Ok(Self { req, inner: None }) + } +} + +impl XfrResponseProcessor { + /// Process a single AXFR/IXFR response message. + /// + /// Return an [`XfrEventIterator`] over [`XfrEvent`]s emitted during + /// processing. + /// + /// If the returned iterator does not emit an + /// [`XfrEvent::TransferComplete`] event, call [`process_next_answer()`] + /// with the next response message to continue iterating over the transfer + /// responses. + pub fn process_answer( + &mut self, + resp: Message, + ) -> Result { + // Check that the given message is a DNS XFR response. + self.check_response(&resp)?; + + if let Some(inner) = &mut self.inner { + inner.resp = resp; + } else { + self.initialize(resp)?; + } + + let inner = self.inner.as_mut().unwrap(); + + XfrEventIterator::new(&mut inner.state, &inner.resp) + } +} + +impl XfrResponseProcessor { + /// Initialize inner state. + fn initialize(&mut self, resp: Message) -> Result<(), Error> { + self.inner = Some(Inner::new(&self.req, resp)?); + Ok(()) + } + + /// Check if an XFR request is valid. + fn check_request(req: &Message) -> Result<(), Error> { + let req_header = req.header(); + let req_counts = req.header_counts(); + + if req.is_error() + || req_header.qr() + || req_counts.qdcount() != 1 + || req_counts.ancount() != 0 + || req_header.opcode() != Opcode::QUERY + { + return Err(Error::NotValidXfrRequest); + } + + let Some(qtype) = req.qtype() else { + return Err(Error::NotValidXfrRequest); + }; + + if !matches!(qtype, Rtype::AXFR | Rtype::IXFR) { + return Err(Error::NotValidXfrRequest); + } + + // https://datatracker.ietf.org/doc/html/rfc1995#section-3 + // 3. Query Format + // "The IXFR query packet format is the same as that of a normal DNS + // query, but with the query type being IXFR and the authority + // section containing the SOA record of client's version of the + // zone." + if matches!(qtype, Rtype::IXFR) && req_counts.nscount() != 1 { + return Err(Error::NotValidXfrRequest); + } + + Ok(()) + } + + /// Check if an XFR response header is valid. + /// + /// Enforce the rules defined in 2. AXFR Messages of RFC 5936. See: + /// https://www.rfc-editor.org/rfc/rfc5936.html#section-2 + /// + /// Takes a request as well as a response as the response is checked to + /// see if it is in reply to the given request. + /// + /// Returns Ok on success, Err otherwise. On success the type of XFR that + /// was determined is returned as well as the answer section from the XFR + /// response. + fn check_response(&self, resp: &Message) -> Result<(), Error> { + let resp_header = resp.header(); + let resp_counts = resp.header_counts(); + + if resp.is_error() + || !resp.is_answer_header(&self.req) + || resp_header.opcode() != Opcode::QUERY + || resp_header.tc() + || resp_counts.ancount() == 0 + || resp_counts.nscount() != 0 + { + return Err(Error::NotValidXfrResponse); + } + + //https://www.rfc-editor.org/rfc/rfc5936.html#section-2.2.1 + // 2.2.1. Header Values + // "QDCOUNT MUST be 1 in the first message; + // MUST be 0 or 1 in all following messages;" + let qdcount = resp_counts.qdcount(); + let first_message = self.inner.is_none(); + if (first_message && qdcount != 1) || (!first_message && qdcount > 1) + { + return Err(Error::NotValidXfrResponse); + } + + Ok(()) + } +} + +//------------ Inner ---------------------------------------------------------- + +/// Internal dynamic state of [`XfrResponseProcessor`]. +/// +/// Separated out from [`XfrResponseProcessor`] to avoid needing multiple +/// mutable self references in [`process_answer()`]. +struct Inner { + /// The response message currently being processed. + resp: Message, + + /// State that influences and is influenced by resposne processing. + state: RecordProcessor, +} + +impl Inner { + /// Initialise the processosr. + /// + /// Records the initial SOA record and other details will will be used + /// while processing the rest of the response. + fn new( + req: &Message, + resp: Message, + ) -> Result { + let answer = resp.answer().map_err(Error::ParseError)?; + + // https://datatracker.ietf.org/doc/html/rfc5936#section-3 + // 3. Zone Contents + // "The objective of the AXFR session is to request and transfer + // the contents of a zone, in order to permit the AXFR client + // to faithfully reconstruct the zone as it exists at the + // primary server for the given zone serial number. The word + // "exists" here designates the externally visible behavior, + // i.e., the zone content that is being served (handed out to + // clients) -- not its persistent representation in a zone file + // or database used by the server -- and that for consistency + // should be served subsequently by the AXFR client in an + // identical manner." + // + // So, walk over all the records in the answer, not just those + // that might be expected to exist in a zone (i.e. not just + // ZoneRecordData record types). + + let mut records = answer.into_records(); + + let xfr_type = match req.qtype() { + Some(Rtype::AXFR) => XfrType::Axfr, + Some(Rtype::IXFR) => XfrType::Ixfr, + _ => unreachable!(), // Checked already in check_request(). + }; + + let Some(Ok(record)) = records.next() else { + return Err(Error::Malformed); + }; + + // The initial record should be a SOA record. + let AllRecordData::Soa(soa) = record.into_data() else { + return Err(Error::NotValidXfrResponse); + }; + + let state = RecordProcessor::new(xfr_type, soa); + + Ok(Inner { resp, state }) + } +} + +//------------ State ---------------------------------------------------------- + +/// State related to processing the XFR response sequence. +#[derive(Debug)] +pub(super) struct RecordProcessor { + /// The type of XFR response sequence being parsed. + /// + /// This can differ to the type of XFR response sequence that we expected + /// to parse because the server can fallback from IXFR to AXFR. + pub(super) actual_xfr_type: XfrType, + + /// The initial SOA record that signals the start and end of both AXFR and + /// IXFR response sequences. + pub(super) initial_soa: Soa>, + + /// The current SOA record. + /// + /// For AXFR response sequences this will be the same as `initial_soa`. + /// For IXFR response sequences this will be the last SOA record parsed as + /// each diff sequence contains two SOA records: one at the start of the + /// delete sequence and one at the start of the add sequence. + pub(super) current_soa: Soa>, + + /// The kind of records currently being processed, either adds or deletes. + pub(super) ixfr_update_mode: IxfrUpdateMode, + + /// The number of resource records parsed so far. + pub(super) rr_count: usize, +} + +impl RecordProcessor { + /// Create a new [`RecordProcessor`]. + fn new( + initial_xfr_type: XfrType, + initial_soa: Soa>, + ) -> Self { + Self { + actual_xfr_type: initial_xfr_type, + initial_soa: initial_soa.clone(), + current_soa: initial_soa, + rr_count: 0, + ixfr_update_mode: Default::default(), + } + } + + /// Process a single resource record. + /// + /// Returns an [`XfrEvent`] that should be emitted for the processed + /// record, if any. + pub(super) fn process_record(&mut self, rec: XfrRecord) -> XfrEvent { + self.rr_count += 1; + + // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2 + // 2.2. AXFR Response + // "..clients MUST accept any ordering and grouping of the non-SOA + // RRs. Each RR SHOULD be transmitted only once, and AXFR clients + // MUST ignore any duplicate RRs received." + // + // Note: We do NOT implement this MUST here because it would be very + // inefficient to actually check that any received non-SOA RR has not + // been seen before during the in-progress transfer. Clients of + // XfrResponseProcessor are better placed to enforce this rule if + // needed, e.g. at the moment of insertion into a zone tree checking + // that the record is not already present or insertion of a duplicate + // having no effect as it is already present. + + let soa = match rec.data() { + AllRecordData::Soa(soa) => Some(soa), + _ => None, + }; + + let record_matches_initial_soa = soa == Some(&self.initial_soa); + + match self.actual_xfr_type { + XfrType::Axfr if record_matches_initial_soa => { + // https://www.rfc-editor.org/rfc/rfc5936.html#section-2.2 + // 2.2. AXFR Response + // ... + // "In such a series, the first message MUST begin with the + // SOA resource record of the zone, and the last message + // MUST conclude with the same SOA resource record. + // Intermediate messages MUST NOT contain the SOA resource + // record." + XfrEvent::EndOfTransfer + } + + XfrType::Axfr => { + XfrEvent::AddRecord(self.current_soa.serial(), rec) + } + + XfrType::Ixfr if self.rr_count < 2 => unreachable!(), + + XfrType::Ixfr if self.rr_count == 2 => { + if record_matches_initial_soa { + // IXFR not available, AXFR of empty zone detected. + XfrEvent::EndOfTransfer + } else if let Some(soa) = soa { + // This SOA record is the start of an IXFR diff sequence. + self.current_soa = soa.clone(); + + // We don't need to set the IXFR update more here as it + // should already be set to Deleting. + debug_assert_eq!( + self.ixfr_update_mode, + IxfrUpdateMode::Deleting + ); + + XfrEvent::BeginBatchDelete(soa.serial()) + } else { + // https://datatracker.ietf.org/doc/html/rfc1995#section-4 + // 4. Response Format + // "If incremental zone transfer is not available, the + // entire zone is returned. The first and the last RR + // of the response is the SOA record of the zone. I.e. + // the behavior is the same as an AXFR response except + // the query type is IXFR. + // + // If incremental zone transfer is available, one or + // more difference sequences is returned. The list of + // difference sequences is preceded and followed by a + // copy of the server's current version of the SOA. + // + // Each difference sequence represents one update to + // the zone (one SOA serial change) consisting of + // deleted RRs and added RRs. The first RR of the + // deleted RRs is the older SOA RR and the first RR of + // the added RRs is the newer SOA RR. + // + // Modification of an RR is performed first by removing + // the original RR and then adding the modified one." + + // As this is IXFR and this is the second record, it should + // be the "first RR of the deleted RRs" which should be + // "the older SOA RR". However, it isn't a SOA RR. As such + // assume that "incremental zone transfer is not available" + // and so "the behaviour is the same as an AXFR response", + self.actual_xfr_type = XfrType::Axfr; + XfrEvent::AddRecord(self.current_soa.serial(), rec) + } + } + + XfrType::Ixfr => { + if let Some(soa) = soa { + self.ixfr_update_mode.toggle(); + self.current_soa = soa.clone(); + + match self.ixfr_update_mode { + IxfrUpdateMode::Deleting => { + // We just finished a (Delete, Add) diff sequence. + // Is this the end of the transfer, or the start + // of a new diff sequence? + if record_matches_initial_soa { + XfrEvent::EndOfTransfer + } else { + XfrEvent::BeginBatchDelete( + self.current_soa.serial(), + ) + } + } + IxfrUpdateMode::Adding => { + // We just switched from the Delete phase of a + // diff sequence to the add phase of the diff + // sequence. + XfrEvent::BeginBatchAdd(self.current_soa.serial()) + } + } + } else { + match self.ixfr_update_mode { + IxfrUpdateMode::Deleting => XfrEvent::DeleteRecord( + self.current_soa.serial(), + rec, + ), + IxfrUpdateMode::Adding => XfrEvent::AddRecord( + self.current_soa.serial(), + rec, + ), + } + } + } + } + } +} diff --git a/src/net/xfr/client/tests.rs b/src/net/xfr/client/tests.rs new file mode 100644 index 000000000..37b5721e4 --- /dev/null +++ b/src/net/xfr/client/tests.rs @@ -0,0 +1,451 @@ +use core::str::FromStr; + +use bytes::{Bytes, BytesMut}; +use octseq::Octets; + +use crate::base::iana::Rcode; +use crate::base::message_builder::{ + AnswerBuilder, AuthorityBuilder, QuestionBuilder, +}; +use crate::base::net::Ipv4Addr; +use crate::base::rdata::ComposeRecordData; +use crate::base::{ + Message, MessageBuilder, ParsedName, Record, Rtype, Serial, Ttl, +}; +use crate::base::{Name, ToName}; +use crate::rdata::{AllRecordData, Soa, A}; + +use super::processor::XfrResponseProcessor; +use super::types::{ + Error, XfrEvent, XfrEvent as XE, XfrEventIteratorError, XfrRecord, +}; + +#[test] +fn request_message_is_rejected() { + init_logging(); + + // Create a no/n-XFR request to reply to. + let req = mk_request("example.com", Rtype::A).into_message(); + + // Create an XFR response processor. + let res = XfrResponseProcessor::new(req.clone()); + + // Process the request and assert that it is rejected as not being + // a valid XFR response and that no XFR processor events were emitted. + assert!(matches!(res, Err(Error::NotValidXfrRequest))); +} + +#[test] +fn non_xfr_response_is_rejected() { + init_logging(); + + // Create an AXFR-like request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + + // Create a non-XFR response. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + let resp = answer.into_message(); + + // Process the response and assert that it is rejected as not being + // a valid XFR response and that no XFR processor events were emitted. + assert!(matches!( + processor.process_answer(resp), + Err(Error::NotValidXfrResponse) + )); +} + +#[test] +fn axfr_response_with_no_answers_is_rejected() { + init_logging(); + + // Create an AXFR request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + + // Create a response that lacks answers. + let resp = mk_empty_answer(&req, Rcode::NOERROR).into_message(); + + // Process the response and assert that it is rejected as not being + // a valid XFR response and that no XFR processor events were emitted. + assert!(matches!( + processor.process_answer(resp), + Err(Error::NotValidXfrResponse) + )); +} + +#[test] +fn error_axfr_response_is_rejected() { + init_logging(); + + // Create an AXFR request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + + // Create a minimal valid AXFR response, just something that should + // not be rejected by the XFR processor due to its content. It should + // however be rejected due to the non-NOERROR rcode. + let mut answer = mk_empty_answer(&req, Rcode::SERVFAIL); + add_answer_record(&req, &mut answer, mk_soa(Serial::now())); + let resp = answer.into_message(); + + // Process the response and assert that it is rejected as not being + // a valid XFR response and that no XFR processor events were emitted. + assert!(matches!( + processor.process_answer(resp), + Err(Error::NotValidXfrResponse) + )); +} + +#[test] +fn incomplete_axfr_response_is_accepted() { + init_logging(); + + // Create an AXFR request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + + // Create an incomplete AXFR response. A proper AXFR response has at + // least two identical SOA records, one at the start and one at the + // end, but this response contains only a single SOA record. This is + // not considered invalid however because a subsequent response could + // still provide the missing SOA record. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + add_answer_record(&req, &mut answer, mk_soa(Serial::now())); + let resp = answer.into_message(); + + // Process the response. + let mut it = processor.process_answer(resp).unwrap(); + + // Verify that no events are by the XFR processor. + assert!(it.next().is_none()); +} + +#[test] +fn axfr_response_with_only_soas_is_accepted() { + init_logging(); + + // Create an AXFR request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + + // Create a complete but minimal AXFR response. A proper AXFR response + // has at least two identical SOA records, one at the start and one at + // the end, with actual zone records in between. This response has only + // the start and end SOA and no content in between. RFC 5936 doesn't + // seem to disallow this. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + let soa = mk_soa(Serial::now()); + add_answer_record(&req, &mut answer, soa.clone()); + add_answer_record(&req, &mut answer, soa); + let resp = answer.into_message(); + + // Process the response. + let mut it = processor.process_answer(resp).unwrap(); + + // Verify the events emitted by the XFR processor. + assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); + assert!(it.next().is_none()); +} + +#[test] +fn axfr_multi_response_with_only_soas_is_accepted() { + init_logging(); + + // Create an AXFR request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + + // Create a complete but minimal AXFR response. A proper AXFR response + // has at least two identical SOA records, one at the start and one at + // the end, with actual zone records in between. This response has only + // the start and end SOA and no content in between. RFC 5936 doesn't + // seem to disallow this. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + let soa = mk_soa(Serial::now()); + add_answer_record(&req, &mut answer, soa.clone()); + let resp = answer.into_message(); + + // Process the response. + let mut it = processor.process_answer(resp).unwrap(); + + // Verify the events emitted by the XFR processor. + assert!(it.next().is_none()); + + // Create another AXFR response to complete the transfer. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + add_answer_record(&req, &mut answer, soa); + let resp = answer.into_message(); + + // Process the response. + let mut it = processor.process_answer(resp).unwrap(); + + // Verify the events emitted by the XFR processor. + assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); + assert!(it.next().is_none()); +} + +#[test] +fn axfr_response_generates_expected_events() { + init_logging(); + + // Create an AXFR request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + + // Create an AXFR response. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + let serial = Serial::now(); + let soa = mk_soa(serial); + add_answer_record(&req, &mut answer, soa.clone()); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); + add_answer_record(&req, &mut answer, soa); + let resp = answer.into_message(); + + // Process the response. + let mut it = processor.process_answer(resp).unwrap(); + + // Verify the events emitted by the XFR processor. + let s = serial; + assert!(matches!(it.next(), Some(Ok(XE::AddRecord(n, _))) if n == s)); + assert!(matches!(it.next(), Some(Ok(XE::AddRecord(n, _))) if n == s)); + assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); + assert!(it.next().is_none()); +} + +#[test] +fn ixfr_response_generates_expected_events() { + init_logging(); + + // Create an IXFR request to reply to. + let req = mk_request("example.com", Rtype::IXFR); + let mut authority = req.authority(); + let client_serial = Serial::now(); + let soa = mk_soa(client_serial); + add_authority_record(&mut authority, soa); + let req = authority.into_message(); + + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + + // Prepare some serial numbers and SOA records to use in the IXFR response. + let old_serial = client_serial; + let new_serial = client_serial.add(1); + let old_soa = mk_soa(old_serial); + let new_soa = mk_soa(new_serial); + + // Create an IXFR response. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + // Outer SOA with servers current SOA + add_answer_record(&req, &mut answer, new_soa.clone()); + // Start of diff sequence: SOA of the servers' previous zone version + // (which matches that of the client) followed by records to be + // deleted as they were in that version of the zone but are not in the + // new version of the zone. + add_answer_record(&req, &mut answer, old_soa); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); + // SOA of the servers` new zone version (which is ahead of that of the + // client) followed by records to be added as they were added in this + // new version of the zone.` + add_answer_record(&req, &mut answer, new_soa.clone()); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + // Closing SOA with servers current SOA + add_answer_record(&req, &mut answer, new_soa); + let resp = answer.into_message(); + + // Process the response. + let it = processor.process_answer(resp).unwrap(); + + // Verify the events emitted by the XFR processor. + let owner = + ParsedName::::from(Name::from_str("example.com").unwrap()); + let expected_events: [Result, XfrEventIteratorError>; + 7] = [ + Ok(XfrEvent::BeginBatchDelete(old_serial)), + Ok(XfrEvent::DeleteRecord( + old_serial, + Record::from(( + owner.clone(), + 0, + AllRecordData::A(A::new(Ipv4Addr::LOCALHOST)), + )), + )), + Ok(XfrEvent::DeleteRecord( + old_serial, + Record::from(( + owner.clone(), + 0, + AllRecordData::A(A::new(Ipv4Addr::BROADCAST)), + )), + )), + Ok(XfrEvent::BeginBatchAdd(new_serial)), + Ok(XfrEvent::AddRecord( + new_serial, + Record::from(( + owner.clone(), + 0, + AllRecordData::A(A::new(Ipv4Addr::BROADCAST)), + )), + )), + Ok(XfrEvent::AddRecord( + new_serial, + Record::from(( + owner, + 0, + AllRecordData::A(A::new(Ipv4Addr::LOCALHOST)), + )), + )), + Ok(XfrEvent::EndOfTransfer), + ]; + + assert!(it.eq(expected_events)); +} + +#[test] +fn multi_ixfr_response_generates_expected_events() { + init_logging(); + + // Create an IXFR request to reply to. + let req = mk_request("example.com", Rtype::IXFR); + let mut authority = req.authority(); + let client_serial = Serial::now(); + let soa = mk_soa(client_serial); + add_authority_record(&mut authority, soa); + let req = authority.into_message(); + + // Create an XFR response processor. + let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + + // Prepare some serial numbers and SOA records to use in the IXFR response. + let old_serial = client_serial; + let new_serial = client_serial.add(1); + let old_soa = mk_soa(old_serial); + let new_soa = mk_soa(new_serial); + + // Create a partial IXFR response. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + // Outer SOA with servers current SOA + add_answer_record(&req, &mut answer, new_soa.clone()); + // Start of diff sequence: SOA of the servers' previous zone version + // (which matches that of the client) followed by records to be + // deleted as they were in that version of the zone but are not in the + // new version of the zone. + add_answer_record(&req, &mut answer, old_soa); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + let resp = answer.into_message(); + + // Process the response. + let mut it = processor.process_answer(resp).unwrap(); + + // Verify the events emitted by the XFR processor. + assert!(matches!(it.next(), Some(Ok(XE::BeginBatchDelete(_))))); + assert!(matches!(it.next(), Some(Ok(XE::DeleteRecord(..))))); + assert!(it.next().is_none()); + + // Craete a second IXFR response that completes the transfer + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); + // SOA of the servers` new zone version (which is ahead of that of the + // client) followed by records to be added as they were added in this + // new version of the zone.` + add_answer_record(&req, &mut answer, new_soa.clone()); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); + add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + // Closing SOA with servers current SOA + add_answer_record(&req, &mut answer, new_soa); + let resp = answer.into_message(); + + // Process the response. + let mut it = processor.process_answer(resp).unwrap(); + + // Verify the events emitted by the XFR processor. + assert!(matches!(it.next(), Some(Ok(XE::DeleteRecord(..))))); + assert!(matches!(it.next(), Some(Ok(XE::BeginBatchAdd(_))))); + assert!(matches!(it.next(), Some(Ok(XE::AddRecord(..))))); + assert!(matches!(it.next(), Some(Ok(XE::AddRecord(..))))); + assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); + assert!(it.next().is_none()); +} + +//------------ Helper functions ------------------------------------------- + +fn init_logging() { + // Initialize tracing based logging. Override with env var RUST_LOG, e.g. + // RUST_LOG=trace. DEBUG level will show the .rpl file name, Stelline step + // numbers and types as they are being executed. + tracing_subscriber::fmt() + .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) + .with_thread_ids(true) + .without_time() + .try_init() + .ok(); +} + +fn mk_request(qname: &str, qtype: Rtype) -> QuestionBuilder { + let req = MessageBuilder::new_bytes(); + let mut req = req.question(); + req.push((Name::vec_from_str(qname).unwrap(), qtype)) + .unwrap(); + req +} + +fn mk_empty_answer( + req: &Message, + rcode: Rcode, +) -> AnswerBuilder { + let builder = MessageBuilder::new_bytes(); + builder.start_answer(req, rcode).unwrap() +} + +fn add_answer_record( + req: &Message, + answer: &mut AnswerBuilder, + item: T, +) { + let question = req.sole_question().unwrap(); + let qname = question.qname(); + let qclass = question.qclass(); + answer + .push((qname, qclass, Ttl::from_secs(0), item)) + .unwrap(); +} + +fn add_authority_record( + authority: &mut AuthorityBuilder, + item: T, +) { + let (qname, qclass) = { + let question = authority.as_message().sole_question().unwrap(); + let qname = question.qname().to_bytes(); + let qclass = question.qclass(); + (qname, qclass) + }; + authority + .push((qname, qclass, Ttl::from_secs(0), item)) + .unwrap(); +} + +fn mk_soa(serial: Serial) -> Soa> { + let mname = Name::from_str("mname").unwrap(); + let rname = Name::from_str("rname").unwrap(); + let ttl = Ttl::from_secs(0); + Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl) +} diff --git a/src/net/xfr/client/types.rs b/src/net/xfr/client/types.rs new file mode 100644 index 000000000..fb6f48564 --- /dev/null +++ b/src/net/xfr/client/types.rs @@ -0,0 +1,167 @@ +//! XFR related types. + +//------------ XfrRecord ------------------------------------------------------ + +use bytes::Bytes; + +use crate::{ + base::{wire::ParseError, ParsedName, Record, Rtype, Serial}, + rdata::AllRecordData, +}; + +/// The type of record processed by [`XfrResponseProcessor`]. +pub type XfrRecord = + Record, AllRecordData>>; + +//------------ XfrType -------------------------------------------------------- + +/// The type of XFR response sequence. +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +pub(super) enum XfrType { + /// RFC 5936 AXFR. + /// + /// A complete snapshot of a zone at a particular version. + Axfr, + + /// RFC 1995 IXFR. + /// + /// An incremental diff of the version of the zone that the server has + /// compared to the version of the zone that the client has. + Ixfr, +} + +//--- From + +impl TryFrom for XfrType { + type Error = (); + + fn try_from(rtype: Rtype) -> Result { + match rtype { + Rtype::AXFR => Ok(XfrType::Axfr), + Rtype::IXFR => Ok(XfrType::Ixfr), + _ => Err(()), + } + } +} + +//------------ XfrEvent ------------------------------------------------------- + +/// An event emitted by [`XfrResponseProcessor`] during transfer processing. +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +pub enum XfrEvent { + /// Delete record R in zone serial S. + /// + /// The transfer signalled that the given record should be deleted from + /// the zone version with the given serial number. + /// + /// Note: If the transfer contains N deletions of fhe same record then + /// this event will occur N times. + DeleteRecord(Serial, R), + + /// Add record R in zone serial S. + /// + /// The transfer signalled that the given record should be added to the + /// zone version with the given serial number. + /// + /// Note: If the transfer contains N additions of fhe same record then + /// this event will occur N times. + AddRecord(Serial, R), + + /// Prepare to delete records in zone serial S. + /// + /// The transfer signalled that zero or more record deletions will follow, + /// all for the zone version with the given serial number. + BeginBatchDelete(Serial), + + /// Prepare to add records in zone serial S. + /// + /// The transfer signalled that zero or more record additions will follow, + /// all for the zone version with the given serial number. + BeginBatchAdd(Serial), + + /// Transfer completed successfully. + /// + /// Note: This event is not emitted until the final record of the final + /// response in a set of one or more transfer responss has been seen. + EndOfTransfer, + + /// Transfer processing failed. + /// + /// This event indicates that there is a problem with the transfer data + /// and that transfer processing cannot continue. + ProcessingFailed, +} + +//--- Display + +impl std::fmt::Display for XfrEvent { + fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { + match self { + XfrEvent::DeleteRecord(_, _) => f.write_str("DeleteRecord"), + XfrEvent::AddRecord(_, _) => f.write_str("AddRecord"), + XfrEvent::BeginBatchDelete(_) => f.write_str("BeginBatchDelete"), + XfrEvent::BeginBatchAdd(_) => f.write_str("BeginBatchAdd"), + XfrEvent::EndOfTransfer => f.write_str("EndOfTransfer"), + XfrEvent::ProcessingFailed => f.write_str("ProcessingFailed"), + } + } +} + +//------------ IxfrUpdateMode ------------------------------------------------- + +/// The kind of records currently being processed, either adds or deletes. +#[derive(Copy, Clone, Debug, Default, PartialEq, Eq)] +pub(super) enum IxfrUpdateMode { + /// The records being parsed are deletions. + /// + /// Deletions come before additions. + #[default] + Deleting, + + /// The records being parsed are additions. + Adding, +} + +impl IxfrUpdateMode { + /// Toggle between the possible [`IxfrUpdateMode`] variants. + pub fn toggle(&mut self) { + match self { + IxfrUpdateMode::Deleting => *self = IxfrUpdateMode::Adding, + IxfrUpdateMode::Adding => *self = IxfrUpdateMode::Deleting, + } + } +} + +//------------ Error ---------------------------------------------------------- + +/// An error reported by [`XfrResponseProcessor`]. +#[derive(Debug)] +pub enum Error { + /// The message could not be parsed. + ParseError(ParseError), + + /// The request message is not an XFR query/ + NotValidXfrRequest, + + /// The response message is not an XFR response. + NotValidXfrResponse, + + /// At least one record in the XFR response sequence is incorrect. + Malformed, + + /// At least one record in the XFR response sequence was not consumed + /// by the caller. + AnswerNotFullyProcessed, + + /// Processing was already terminated for this XFR response sequence. + Terminated, +} + +//------------ XfrEventIteratorError ------------------------------------------ + +/// Errors that can occur during XfrEventIterator iteration. +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +pub enum XfrEventIteratorError { + /// Transfer processing failed. + ParseError(ParseError), +} diff --git a/src/net/xfr/mod.rs b/src/net/xfr/mod.rs new file mode 100644 index 000000000..87f8b3438 --- /dev/null +++ b/src/net/xfr/mod.rs @@ -0,0 +1,2 @@ +//! XFR client related functionality. +pub mod client; \ No newline at end of file From beef368c72b8064b81c14f52f0d5d1577f2bf998 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 27 Aug 2024 23:41:07 +0200 Subject: [PATCH 054/333] Review feedback: remove specialized fn extracted from is_answer() only for AXFR use. --- src/base/message.rs | 17 ++--------------- 1 file changed, 2 insertions(+), 15 deletions(-) diff --git a/src/base/message.rs b/src/base/message.rs index 398e11b0a..0dfac4d9c 100644 --- a/src/base/message.rs +++ b/src/base/message.rs @@ -428,7 +428,8 @@ impl Message { &self, query: &Message, ) -> bool { - if !self.is_answer_header(query) + if !self.header().qr() + || self.header().id() != query.header().id() || self.header_counts().qdcount() != query.header_counts().qdcount() { @@ -438,20 +439,6 @@ impl Message { } } - /// Returns whether this is the answer to some other message. - /// - /// The method checks whether the ID fields of the headers are the same, - /// whether the QR flag is set in this message. It does NOT check whether - /// the questions are the same which is useful in case of responses such - /// as subsequent RFC 5936 AXFR responses which are not required to copy - /// the question from the request to the response. - pub fn is_answer_header( - &self, - query: &Message, - ) -> bool { - self.header().qr() && self.header().id() == query.header().id() - } - /// Could this message result in a stream of responses? /// /// Most DNS queries result in a single response, but some (only AXFR and From 706411910cb565ee5a90c808b58cb214c51b2b7d Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 28 Aug 2024 00:15:08 +0200 Subject: [PATCH 055/333] Cleanup, RustDocs, refactoring, added a feature gate. --- Cargo.toml | 1 + src/lib.rs | 1 + src/net/mod.rs | 9 +- src/net/xfr/mod.rs | 10 ++- .../xfr/{client => processing}/iterator.rs | 16 ++-- src/net/xfr/{client => processing}/mod.rs | 12 ++- .../xfr/{client => processing}/processor.rs | 90 ++++++++++--------- src/net/xfr/{client => processing}/tests.rs | 12 +-- src/net/xfr/{client => processing}/types.rs | 25 +++--- 9 files changed, 102 insertions(+), 74 deletions(-) rename src/net/xfr/{client => processing}/iterator.rs (82%) rename src/net/xfr/{client => processing}/mod.rs (63%) rename src/net/xfr/{client => processing}/processor.rs (83%) rename src/net/xfr/{client => processing}/tests.rs (97%) rename src/net/xfr/{client => processing}/types.rs (89%) diff --git a/Cargo.toml b/Cargo.toml index 554aae9ff..7c06d3ef2 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -64,6 +64,7 @@ unstable-client-transport = ["moka", "net", "tracing"] unstable-server-transport = ["arc-swap", "chrono/clock", "libc", "net", "siphasher", "tracing"] unstable-stelline = ["tokio/test-util", "tracing", "tracing-subscriber", "unstable-server-transport", "zonefile"] unstable-validator = ["validate", "zonefile", "unstable-client-transport"] +unstable-xfr = [] unstable-zonetree = ["futures", "parking_lot", "serde", "tokio", "tracing"] [dev-dependencies] diff --git a/src/lib.rs b/src/lib.rs index cf1f04610..8e06f2d4c 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -137,6 +137,7 @@ //! a server perspective; primarily the `net::server` module. //! * `unstable-validator`: a DNSSEC validator, primarily the `validator` //! and the `net::client::validator` modules. +//! * `unstable-xfr`: zone transfer related functionality.. //! * `unstable-zonetree`: building & querying zone trees; primarily the //! `zonetree` module. //! diff --git a/src/net/mod.rs b/src/net/mod.rs index f07663fb3..5b2f37ebe 100644 --- a/src/net/mod.rs +++ b/src/net/mod.rs @@ -6,12 +6,15 @@ //! Currently, the module only provides the unstable #![cfg_attr(feature = "unstable-client-transport", doc = " [`client`]")] #![cfg_attr(not(feature = "unstable-client-transport"), doc = " `client`")] -//! sub-module intended for sending requests and receiving responses to them, -//! and the unstable +//! sub-module intended for sending requests and receiving responses to them, +//! the unstable #![cfg_attr(feature = "unstable-server-transport", doc = " [`server`]")] #![cfg_attr(not(feature = "unstable-server-transport"), doc = " `server`")] //! sub-module intended for receiving requests and sending responses to them. -//! sub-module for sending requests and receiving responses to them. +//! and the unstable +#![cfg_attr(feature = "unstable-xfr", doc = " [`xfr`]")] +#![cfg_attr(not(feature = "unstable-xfr"), doc = " `xfr`")] +//! sub-module for zone transfer related functionality. //! #![cfg(feature = "net")] #![cfg_attr(docsrs, doc(cfg(feature = "net")))] diff --git a/src/net/xfr/mod.rs b/src/net/xfr/mod.rs index 87f8b3438..566a1f0f5 100644 --- a/src/net/xfr/mod.rs +++ b/src/net/xfr/mod.rs @@ -1,2 +1,8 @@ -//! XFR client related functionality. -pub mod client; \ No newline at end of file +#![cfg_attr( + not(feature = "unstable-xfr"), + doc = " The `unstable-xfr` feature is necessary to enable this module." +)] +// #![warn(missing_docs)] +// #![warn(clippy::missing_docs_in_private_items)] +//! Zone transfer related functionality. +pub mod processing; \ No newline at end of file diff --git a/src/net/xfr/client/iterator.rs b/src/net/xfr/processing/iterator.rs similarity index 82% rename from src/net/xfr/client/iterator.rs rename to src/net/xfr/processing/iterator.rs index 54a235cc8..0f0899efb 100644 --- a/src/net/xfr/client/iterator.rs +++ b/src/net/xfr/processing/iterator.rs @@ -7,11 +7,13 @@ use crate::base::{message::AnyRecordIter, Message, ParsedName}; use crate::rdata::AllRecordData; use super::processor::RecordProcessor; -use super::types::{Error, XfrEvent, XfrEventIteratorError, XfrRecord}; +use super::types::{ProcessingError, XfrEvent, IterationError, XfrRecord}; -///------------ XfrEventIterator ----------------------------------------------- +//------------ XfrEventIterator ----------------------------------------------- /// An iterator over [`XfrResponseProcessor`] generated [`XfrEvent`]s. +/// +/// [`XfrResponseProcessor`]: super::processor::XfrResponseProcessor pub struct XfrEventIterator<'a, 'b> { /// The parent processor. state: &'a mut RecordProcessor, @@ -24,8 +26,8 @@ impl<'a, 'b> XfrEventIterator<'a, 'b> { pub(super) fn new( state: &'a mut RecordProcessor, resp: &'b Message, - ) -> Result { - let answer = resp.answer().map_err(Error::ParseError)?; + ) -> Result { + let answer = resp.answer().map_err(ProcessingError::ParseError)?; // https://datatracker.ietf.org/doc/html/rfc5936#section-3 // 3. Zone Contents @@ -48,7 +50,7 @@ impl<'a, 'b> XfrEventIterator<'a, 'b> { if state.rr_count == 0 { let Some(Ok(_)) = iter.next() else { - return Err(Error::Malformed); + return Err(ProcessingError::Malformed); }; state.rr_count += 1; } @@ -58,7 +60,7 @@ impl<'a, 'b> XfrEventIterator<'a, 'b> { } impl<'a, 'b> Iterator for XfrEventIterator<'a, 'b> { - type Item = Result, XfrEventIteratorError>; + type Item = Result, IterationError>; fn next(&mut self) -> Option { match self.iter.next() { @@ -69,7 +71,7 @@ impl<'a, 'b> Iterator for XfrEventIterator<'a, 'b> { } Some(Err(err)) => { - Some(Err(XfrEventIteratorError::ParseError(err))) + Some(Err(IterationError::ParseError(err))) } None => { diff --git a/src/net/xfr/client/mod.rs b/src/net/xfr/processing/mod.rs similarity index 63% rename from src/net/xfr/client/mod.rs rename to src/net/xfr/processing/mod.rs index 2a8fe5706..ea9c17377 100644 --- a/src/net/xfr/client/mod.rs +++ b/src/net/xfr/processing/mod.rs @@ -4,9 +4,13 @@ //! one or more AXFR/IXFR response messages in terms of the high level //! [`XfrEvent`]s that they represent without having to deal with the //! AXFR/IXFR protocol details. -pub mod iterator; -pub mod processor; -pub mod types; +mod iterator; +mod processor; +mod types; #[cfg(test)] -mod tests; \ No newline at end of file +mod tests; + +pub use iterator::XfrEventIterator; +pub use processor::XfrResponseProcessor; +pub use types::{ProcessingError, XfrEvent, IterationError, XfrRecord}; diff --git a/src/net/xfr/client/processor.rs b/src/net/xfr/processing/processor.rs similarity index 83% rename from src/net/xfr/client/processor.rs rename to src/net/xfr/processing/processor.rs index 5b4b216b7..2c8dc36af 100644 --- a/src/net/xfr/client/processor.rs +++ b/src/net/xfr/processing/processor.rs @@ -7,29 +7,32 @@ use crate::base::{Message, ParsedName, Rtype}; use crate::rdata::{AllRecordData, Soa}; use super::iterator::XfrEventIterator; -use super::types::{Error, IxfrUpdateMode, XfrEvent, XfrRecord, XfrType}; +use super::types::{ProcessingError, IxfrUpdateMode, XfrEvent, XfrRecord, XfrType}; //------------ XfrResponseProcessor ------------------------------------------- /// An AXFR/IXFR response processor. /// -/// [`XfrResponseProcessor`] can be invoked on one or more sequentially -/// AXFR/IXFR received response messages to verify them and during processing -/// emit events which can be consumed via the iterator returned by -/// [`process_answer()`]. +/// Use [`XfrResponseProcessor`] to process a sequence of AXFR or IXFR +/// response messages into a corresponding sequence of high level +/// [`XfrEvent`]s. /// -/// Each [`XfrEventIterator`] produces events for a single response message. -/// If the end of the XFR response sequence has been reached the iterator will -/// emit an [`XfrEvent::TransferComplete`] event. +/// # Usage +/// +/// For each response stream to be processed, construct an +/// [`XfrResponseProcessor`] for the corresponding XFR request message, then +/// pass each XFR response message to [`process_answer()`]. +/// +/// Each call to [`process_answer()`] will return an [`XfrEventIterator`] +/// which when iterated over will produce a sequence of [`XfrEvent`]s for a +/// single response message. The iterator emits an [`XfrEvent::EndOfTransfer`] +/// event when the last record in the transfer is reached. /// -/// If the `TransferComplete` event has not been seen it means that the -/// sequence is incomplete and the next response message in the sequence -/// should be passed to [`process_next_answer()`] along with the exhausted -/// iterator. This will populate thr [`XfrEventIterator`] with more records -/// to parse thereby causing iteration to resume. -/// -/// The process of producing and consuming iterators continues until the end -/// of the transfer is detected or a parsing error occurs. +/// If [`XfrEvent::EndOfTransfer`] event has not yet been emitted it means +/// that the sequence is incomplete and the next response message in the +/// sequence should be passed to [`process_answer()`]. +/// +/// [`process_answer()`]: XfrResponseProcessor::process_answer() pub struct XfrResponseProcessor { /// The XFR request for which responses should be processed. req: Message, @@ -41,10 +44,11 @@ pub struct XfrResponseProcessor { } impl XfrResponseProcessor { - /// Creates a new instance of [`XfrMessageProcessor`]. + /// Creates a new XFR message processor. /// - /// Processes a single XFR response stream. - pub fn new(req: Message) -> Result { + /// The processor can be used to process response messages that relate to + /// the given XFR request message. + pub fn new(req: Message) -> Result { Self::check_request(&req)?; Ok(Self { req, inner: None }) } @@ -53,17 +57,16 @@ impl XfrResponseProcessor { impl XfrResponseProcessor { /// Process a single AXFR/IXFR response message. /// - /// Return an [`XfrEventIterator`] over [`XfrEvent`]s emitted during + /// Returns an [`XfrEventIterator`] over [`XfrEvent`]s emitted during /// processing. /// - /// If the returned iterator does not emit an - /// [`XfrEvent::TransferComplete`] event, call [`process_next_answer()`] - /// with the next response message to continue iterating over the transfer - /// responses. + /// If the returned iterator does not emit an [`XfrEvent::EndOfTransfer`] + /// event, call this function with the next outstanding response message + /// to continue iterating over the incomplete transfer. pub fn process_answer( &mut self, resp: Message, - ) -> Result { + ) -> Result { // Check that the given message is a DNS XFR response. self.check_response(&resp)?; @@ -81,13 +84,13 @@ impl XfrResponseProcessor { impl XfrResponseProcessor { /// Initialize inner state. - fn initialize(&mut self, resp: Message) -> Result<(), Error> { + fn initialize(&mut self, resp: Message) -> Result<(), ProcessingError> { self.inner = Some(Inner::new(&self.req, resp)?); Ok(()) } /// Check if an XFR request is valid. - fn check_request(req: &Message) -> Result<(), Error> { + fn check_request(req: &Message) -> Result<(), ProcessingError> { let req_header = req.header(); let req_counts = req.header_counts(); @@ -97,15 +100,15 @@ impl XfrResponseProcessor { || req_counts.ancount() != 0 || req_header.opcode() != Opcode::QUERY { - return Err(Error::NotValidXfrRequest); + return Err(ProcessingError::NotValidXfrRequest); } let Some(qtype) = req.qtype() else { - return Err(Error::NotValidXfrRequest); + return Err(ProcessingError::NotValidXfrRequest); }; if !matches!(qtype, Rtype::AXFR | Rtype::IXFR) { - return Err(Error::NotValidXfrRequest); + return Err(ProcessingError::NotValidXfrRequest); } // https://datatracker.ietf.org/doc/html/rfc1995#section-3 @@ -115,7 +118,7 @@ impl XfrResponseProcessor { // section containing the SOA record of client's version of the // zone." if matches!(qtype, Rtype::IXFR) && req_counts.nscount() != 1 { - return Err(Error::NotValidXfrRequest); + return Err(ProcessingError::NotValidXfrRequest); } Ok(()) @@ -132,18 +135,22 @@ impl XfrResponseProcessor { /// Returns Ok on success, Err otherwise. On success the type of XFR that /// was determined is returned as well as the answer section from the XFR /// response. - fn check_response(&self, resp: &Message) -> Result<(), Error> { + fn check_response(&self, resp: &Message) -> Result<(), ProcessingError> { let resp_header = resp.header(); let resp_counts = resp.header_counts(); + // Note: We don't call Message::is_answer() here because that requires + // the message to have a question but subsequent AXFR responses are + // not required to have a question. if resp.is_error() - || !resp.is_answer_header(&self.req) + || !resp_header.qr() + || resp_header.id() != self.req.header().id() || resp_header.opcode() != Opcode::QUERY || resp_header.tc() || resp_counts.ancount() == 0 || resp_counts.nscount() != 0 { - return Err(Error::NotValidXfrResponse); + return Err(ProcessingError::NotValidXfrResponse); } //https://www.rfc-editor.org/rfc/rfc5936.html#section-2.2.1 @@ -154,7 +161,7 @@ impl XfrResponseProcessor { let first_message = self.inner.is_none(); if (first_message && qdcount != 1) || (!first_message && qdcount > 1) { - return Err(Error::NotValidXfrResponse); + return Err(ProcessingError::NotValidXfrResponse); } Ok(()) @@ -183,8 +190,8 @@ impl Inner { fn new( req: &Message, resp: Message, - ) -> Result { - let answer = resp.answer().map_err(Error::ParseError)?; + ) -> Result { + let answer = resp.answer().map_err(ProcessingError::ParseError)?; // https://datatracker.ietf.org/doc/html/rfc5936#section-3 // 3. Zone Contents @@ -212,12 +219,12 @@ impl Inner { }; let Some(Ok(record)) = records.next() else { - return Err(Error::Malformed); + return Err(ProcessingError::Malformed); }; // The initial record should be a SOA record. let AllRecordData::Soa(soa) = record.into_data() else { - return Err(Error::NotValidXfrResponse); + return Err(ProcessingError::NotValidXfrResponse); }; let state = RecordProcessor::new(xfr_type, soa); @@ -275,7 +282,10 @@ impl RecordProcessor { /// /// Returns an [`XfrEvent`] that should be emitted for the processed /// record, if any. - pub(super) fn process_record(&mut self, rec: XfrRecord) -> XfrEvent { + pub(super) fn process_record( + &mut self, + rec: XfrRecord, + ) -> XfrEvent { self.rr_count += 1; // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2 diff --git a/src/net/xfr/client/tests.rs b/src/net/xfr/processing/tests.rs similarity index 97% rename from src/net/xfr/client/tests.rs rename to src/net/xfr/processing/tests.rs index 37b5721e4..ccd7c1af2 100644 --- a/src/net/xfr/client/tests.rs +++ b/src/net/xfr/processing/tests.rs @@ -17,7 +17,7 @@ use crate::rdata::{AllRecordData, Soa, A}; use super::processor::XfrResponseProcessor; use super::types::{ - Error, XfrEvent, XfrEvent as XE, XfrEventIteratorError, XfrRecord, + ProcessingError, XfrEvent, XfrEvent as XE, IterationError, XfrRecord, }; #[test] @@ -32,7 +32,7 @@ fn request_message_is_rejected() { // Process the request and assert that it is rejected as not being // a valid XFR response and that no XFR processor events were emitted. - assert!(matches!(res, Err(Error::NotValidXfrRequest))); + assert!(matches!(res, Err(ProcessingError::NotValidXfrRequest))); } #[test] @@ -54,7 +54,7 @@ fn non_xfr_response_is_rejected() { // a valid XFR response and that no XFR processor events were emitted. assert!(matches!( processor.process_answer(resp), - Err(Error::NotValidXfrResponse) + Err(ProcessingError::NotValidXfrResponse) )); } @@ -75,7 +75,7 @@ fn axfr_response_with_no_answers_is_rejected() { // a valid XFR response and that no XFR processor events were emitted. assert!(matches!( processor.process_answer(resp), - Err(Error::NotValidXfrResponse) + Err(ProcessingError::NotValidXfrResponse) )); } @@ -100,7 +100,7 @@ fn error_axfr_response_is_rejected() { // a valid XFR response and that no XFR processor events were emitted. assert!(matches!( processor.process_answer(resp), - Err(Error::NotValidXfrResponse) + Err(ProcessingError::NotValidXfrResponse) )); } @@ -277,7 +277,7 @@ fn ixfr_response_generates_expected_events() { // Verify the events emitted by the XFR processor. let owner = ParsedName::::from(Name::from_str("example.com").unwrap()); - let expected_events: [Result, XfrEventIteratorError>; + let expected_events: [Result, IterationError>; 7] = [ Ok(XfrEvent::BeginBatchDelete(old_serial)), Ok(XfrEvent::DeleteRecord( diff --git a/src/net/xfr/client/types.rs b/src/net/xfr/processing/types.rs similarity index 89% rename from src/net/xfr/client/types.rs rename to src/net/xfr/processing/types.rs index fb6f48564..6effcba75 100644 --- a/src/net/xfr/client/types.rs +++ b/src/net/xfr/processing/types.rs @@ -10,6 +10,8 @@ use crate::{ }; /// The type of record processed by [`XfrResponseProcessor`]. +/// +/// [`XfrResponseProcessor`]: super::processor::XfrResponseProcessor pub type XfrRecord = Record, AllRecordData>>; @@ -47,6 +49,8 @@ impl TryFrom for XfrType { //------------ XfrEvent ------------------------------------------------------- /// An event emitted by [`XfrResponseProcessor`] during transfer processing. +/// +/// [`XfrResponseProcessor`]: super::processor::XfrResponseProcessor #[derive(Copy, Clone, Debug, PartialEq, Eq)] pub enum XfrEvent { /// Delete record R in zone serial S. @@ -132,11 +136,13 @@ impl IxfrUpdateMode { } } -//------------ Error ---------------------------------------------------------- +//------------ ProcessingError ------------------------------------------------ /// An error reported by [`XfrResponseProcessor`]. +/// +/// [`XfrResponseProcessor`]: super::processor::XfrResponseProcessor #[derive(Debug)] -pub enum Error { +pub enum ProcessingError { /// The message could not be parsed. ParseError(ParseError), @@ -148,20 +154,15 @@ pub enum Error { /// At least one record in the XFR response sequence is incorrect. Malformed, - - /// At least one record in the XFR response sequence was not consumed - /// by the caller. - AnswerNotFullyProcessed, - - /// Processing was already terminated for this XFR response sequence. - Terminated, } -//------------ XfrEventIteratorError ------------------------------------------ +//------------ IterationError ------------------------------------------------- -/// Errors that can occur during XfrEventIterator iteration. +/// Errors that can occur during [`XfrEventIterator`]` iteration. +/// +/// [`XfrEventIterator`]: super::iterator::XfrEventIterator #[derive(Copy, Clone, Debug, PartialEq, Eq)] -pub enum XfrEventIteratorError { +pub enum IterationError { /// Transfer processing failed. ParseError(ParseError), } From 94c163a37d9345023720e6ced9f0d431c6fb7c5e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 28 Aug 2024 00:15:26 +0200 Subject: [PATCH 056/333] Cargo fnt. --- src/net/mod.rs | 4 ++-- src/net/xfr/mod.rs | 2 +- src/net/xfr/processing/iterator.rs | 8 +++----- src/net/xfr/processing/mod.rs | 2 +- src/net/xfr/processing/processor.rs | 20 ++++++++++++++------ src/net/xfr/processing/tests.rs | 5 ++--- src/net/xfr/processing/types.rs | 8 ++++---- 7 files changed, 27 insertions(+), 22 deletions(-) diff --git a/src/net/mod.rs b/src/net/mod.rs index 5b2f37ebe..d9528f48c 100644 --- a/src/net/mod.rs +++ b/src/net/mod.rs @@ -6,7 +6,7 @@ //! Currently, the module only provides the unstable #![cfg_attr(feature = "unstable-client-transport", doc = " [`client`]")] #![cfg_attr(not(feature = "unstable-client-transport"), doc = " `client`")] -//! sub-module intended for sending requests and receiving responses to them, +//! sub-module intended for sending requests and receiving responses to them, //! the unstable #![cfg_attr(feature = "unstable-server-transport", doc = " [`server`]")] #![cfg_attr(not(feature = "unstable-server-transport"), doc = " `server`")] @@ -21,4 +21,4 @@ pub mod client; pub mod server; -pub mod xfr; \ No newline at end of file +pub mod xfr; diff --git a/src/net/xfr/mod.rs b/src/net/xfr/mod.rs index 566a1f0f5..5f6ec8604 100644 --- a/src/net/xfr/mod.rs +++ b/src/net/xfr/mod.rs @@ -5,4 +5,4 @@ // #![warn(missing_docs)] // #![warn(clippy::missing_docs_in_private_items)] //! Zone transfer related functionality. -pub mod processing; \ No newline at end of file +pub mod processing; diff --git a/src/net/xfr/processing/iterator.rs b/src/net/xfr/processing/iterator.rs index 0f0899efb..d18f036c6 100644 --- a/src/net/xfr/processing/iterator.rs +++ b/src/net/xfr/processing/iterator.rs @@ -7,12 +7,12 @@ use crate::base::{message::AnyRecordIter, Message, ParsedName}; use crate::rdata::AllRecordData; use super::processor::RecordProcessor; -use super::types::{ProcessingError, XfrEvent, IterationError, XfrRecord}; +use super::types::{IterationError, ProcessingError, XfrEvent, XfrRecord}; //------------ XfrEventIterator ----------------------------------------------- /// An iterator over [`XfrResponseProcessor`] generated [`XfrEvent`]s. -/// +/// /// [`XfrResponseProcessor`]: super::processor::XfrResponseProcessor pub struct XfrEventIterator<'a, 'b> { /// The parent processor. @@ -70,9 +70,7 @@ impl<'a, 'b> Iterator for XfrEventIterator<'a, 'b> { Some(Ok(event)) } - Some(Err(err)) => { - Some(Err(IterationError::ParseError(err))) - } + Some(Err(err)) => Some(Err(IterationError::ParseError(err))), None => { // No more events available: end iteration. diff --git a/src/net/xfr/processing/mod.rs b/src/net/xfr/processing/mod.rs index ea9c17377..91227d878 100644 --- a/src/net/xfr/processing/mod.rs +++ b/src/net/xfr/processing/mod.rs @@ -13,4 +13,4 @@ mod tests; pub use iterator::XfrEventIterator; pub use processor::XfrResponseProcessor; -pub use types::{ProcessingError, XfrEvent, IterationError, XfrRecord}; +pub use types::{IterationError, ProcessingError, XfrEvent, XfrRecord}; diff --git a/src/net/xfr/processing/processor.rs b/src/net/xfr/processing/processor.rs index 2c8dc36af..f342cfba1 100644 --- a/src/net/xfr/processing/processor.rs +++ b/src/net/xfr/processing/processor.rs @@ -7,7 +7,9 @@ use crate::base::{Message, ParsedName, Rtype}; use crate::rdata::{AllRecordData, Soa}; use super::iterator::XfrEventIterator; -use super::types::{ProcessingError, IxfrUpdateMode, XfrEvent, XfrRecord, XfrType}; +use super::types::{ + IxfrUpdateMode, ProcessingError, XfrEvent, XfrRecord, XfrType, +}; //------------ XfrResponseProcessor ------------------------------------------- @@ -18,11 +20,11 @@ use super::types::{ProcessingError, IxfrUpdateMode, XfrEvent, XfrRecord, XfrType /// [`XfrEvent`]s. /// /// # Usage -/// +/// /// For each response stream to be processed, construct an /// [`XfrResponseProcessor`] for the corresponding XFR request message, then /// pass each XFR response message to [`process_answer()`]. -/// +/// /// Each call to [`process_answer()`] will return an [`XfrEventIterator`] /// which when iterated over will produce a sequence of [`XfrEvent`]s for a /// single response message. The iterator emits an [`XfrEvent::EndOfTransfer`] @@ -31,7 +33,7 @@ use super::types::{ProcessingError, IxfrUpdateMode, XfrEvent, XfrRecord, XfrType /// If [`XfrEvent::EndOfTransfer`] event has not yet been emitted it means /// that the sequence is incomplete and the next response message in the /// sequence should be passed to [`process_answer()`]. -/// +/// /// [`process_answer()`]: XfrResponseProcessor::process_answer() pub struct XfrResponseProcessor { /// The XFR request for which responses should be processed. @@ -84,7 +86,10 @@ impl XfrResponseProcessor { impl XfrResponseProcessor { /// Initialize inner state. - fn initialize(&mut self, resp: Message) -> Result<(), ProcessingError> { + fn initialize( + &mut self, + resp: Message, + ) -> Result<(), ProcessingError> { self.inner = Some(Inner::new(&self.req, resp)?); Ok(()) } @@ -135,7 +140,10 @@ impl XfrResponseProcessor { /// Returns Ok on success, Err otherwise. On success the type of XFR that /// was determined is returned as well as the answer section from the XFR /// response. - fn check_response(&self, resp: &Message) -> Result<(), ProcessingError> { + fn check_response( + &self, + resp: &Message, + ) -> Result<(), ProcessingError> { let resp_header = resp.header(); let resp_counts = resp.header_counts(); diff --git a/src/net/xfr/processing/tests.rs b/src/net/xfr/processing/tests.rs index ccd7c1af2..9c62ca8e2 100644 --- a/src/net/xfr/processing/tests.rs +++ b/src/net/xfr/processing/tests.rs @@ -17,7 +17,7 @@ use crate::rdata::{AllRecordData, Soa, A}; use super::processor::XfrResponseProcessor; use super::types::{ - ProcessingError, XfrEvent, XfrEvent as XE, IterationError, XfrRecord, + IterationError, ProcessingError, XfrEvent, XfrEvent as XE, XfrRecord, }; #[test] @@ -277,8 +277,7 @@ fn ixfr_response_generates_expected_events() { // Verify the events emitted by the XFR processor. let owner = ParsedName::::from(Name::from_str("example.com").unwrap()); - let expected_events: [Result, IterationError>; - 7] = [ + let expected_events: [Result, IterationError>; 7] = [ Ok(XfrEvent::BeginBatchDelete(old_serial)), Ok(XfrEvent::DeleteRecord( old_serial, diff --git a/src/net/xfr/processing/types.rs b/src/net/xfr/processing/types.rs index 6effcba75..fb896e39a 100644 --- a/src/net/xfr/processing/types.rs +++ b/src/net/xfr/processing/types.rs @@ -10,7 +10,7 @@ use crate::{ }; /// The type of record processed by [`XfrResponseProcessor`]. -/// +/// /// [`XfrResponseProcessor`]: super::processor::XfrResponseProcessor pub type XfrRecord = Record, AllRecordData>>; @@ -49,7 +49,7 @@ impl TryFrom for XfrType { //------------ XfrEvent ------------------------------------------------------- /// An event emitted by [`XfrResponseProcessor`] during transfer processing. -/// +/// /// [`XfrResponseProcessor`]: super::processor::XfrResponseProcessor #[derive(Copy, Clone, Debug, PartialEq, Eq)] pub enum XfrEvent { @@ -139,7 +139,7 @@ impl IxfrUpdateMode { //------------ ProcessingError ------------------------------------------------ /// An error reported by [`XfrResponseProcessor`]. -/// +/// /// [`XfrResponseProcessor`]: super::processor::XfrResponseProcessor #[derive(Debug)] pub enum ProcessingError { @@ -159,7 +159,7 @@ pub enum ProcessingError { //------------ IterationError ------------------------------------------------- /// Errors that can occur during [`XfrEventIterator`]` iteration. -/// +/// /// [`XfrEventIterator`]: super::iterator::XfrEventIterator #[derive(Copy, Clone, Debug, PartialEq, Eq)] pub enum IterationError { From e26b7d6743746756e186d8370c51ff11010c354d Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 28 Aug 2024 10:18:06 +0200 Subject: [PATCH 057/333] Remove file accidentally left behind from merging. --- src/net/xfr/processor.rs | 1214 -------------------------------------- 1 file changed, 1214 deletions(-) delete mode 100644 src/net/xfr/processor.rs diff --git a/src/net/xfr/processor.rs b/src/net/xfr/processor.rs deleted file mode 100644 index 28460f10f..000000000 --- a/src/net/xfr/processor.rs +++ /dev/null @@ -1,1214 +0,0 @@ -//! Parsing of AXFR/IXFR response messages for higher level processing. -//! -//! This module provides [`XfrResponseProcessor`] which enables you to process -//! one or more AXFR/IXFR response messages in terms of the high level -//! [`XfrEvent`]s that they represent without having to deal with the -//! AXFR/IXFR protocol details. -use std::fmt::Debug; - -use bytes::Bytes; -use tracing::trace; - -use crate::base::iana::Opcode; -use crate::base::wire::ParseError; -use crate::base::{ - Message, ParsedName, Record, RecordSection, Rtype, Serial, -}; -use crate::rdata::{AllRecordData, Soa}; - -//------------ XfrRecord ------------------------------------------------------ - -/// The type of record processed by [`XfrResponseProcessor`]. -pub type XfrRecord = - Record, AllRecordData>>; - -//------------ XfrResponseProcessor ------------------------------------------- - -/// An AXFR/IXFR response processor. -/// -/// [`XfrResponseProcessor`] can be invoked on one ore more sequentially -/// AXFR/IXFR received response messages to verify them and during processing -/// emit events which an implementor of [`XfrEventHandler`] can handle. -/// -/// Each instance of [`XfrResponseProcessosr`] should process a single XFR -/// response sequence. Once an instance of [`XfrResponseProcessosr`] has -/// finished processing an XFR response sequence it must be discarded. -/// Attempting to use it once processing has finished will result in an error. -/// To process another XFR response sequence create another instance of -/// [`XfrResponseProcessor`]. -pub struct XfrResponseProcessor { - /// The event handler that events will be sent to for handling. - evt_handler: T, - - /// The current processing state. - state: State, -} - -impl XfrResponseProcessor { - /// Create a new XFR response processor. - /// - /// Events will be emitted to the given [`XfrEventHandler`] implementation. - pub fn new(evt_handler: T) -> Self { - Self { - evt_handler, - state: State::default(), - } - } - - /// Process a single AXFR/IXFR response message. - /// - /// During processing events will be emitted to the registered - /// [`XfrEventHandler`] for handling. - /// - /// Returns Ok(true) if the XFR response was the last in the seqence, - /// Ok(false) if more XFR response messages are needed to complete the - /// sequence, or Err on error. - pub async fn process_answer( - &mut self, - req: &Message, - resp: Message, - ) -> Result { - // Check that the given message is a DNS XFR response. - let res = self.check_is_xfr_answer(req, &resp).await; - - // Unpack the XFR type and answer object. We cannot do this in the - // line above using `map_err()` and `?` as the Rust compiler complains - // about attempting to return `resp` while a reference to it still - // exists. - let (xfr_type, answer) = match res { - Ok(values) => values, - Err(err) => return Err(Error::from_check_error(resp, err)), - }; - - // https://datatracker.ietf.org/doc/html/rfc5936#section-3 - // 3. Zone Contents "The objective of the AXFR session is to request - // and transfer the contents of a zone, in order to permit the AXFR - // client to faithfully reconstruct the zone as it exists at the - // primary server for the given zone serial number. The word - // "exists" here designates the externally visible behavior, i.e., - // the zone content that is being served (handed out to clients) -- - // not its persistent representation in a zone file or database - // used by the server -- and that for consistency should be served - // subsequently by the AXFR client in an identical manner." - // - // So, walk over all the records in the answer, not just those that - // might be expected to exist in a zone (i.e. not just ZoneRecordData - // record types). - let mut records = answer.into_records(); - - match self.state { - // When given the first response in a sequence, do some initial - // setup. - State::AwaitingFirstAnswer => { - let Some(Ok(record)) = records.next() else { - return Err(Error::Malformed); - }; - - if let Err(err) = - self.initialize(xfr_type, req.header().id(), record).await - { - return Err(Error::from_check_error(resp, err)); - } - } - - // For subsequent messages make sure that the XFR - State::AwaitingNextAnswer { - initial_xfr_type, - initial_query_id, - .. - } => { - if xfr_type != initial_xfr_type - || req.header().id() != initial_query_id - { - // The XFR type is extracted from the request. If we were - // given a different request with a different question and - // qtype on a subsequent invocation of process_answer() - // that would be unexpected. - return Err(Error::NotValidXfrRequest); - } - } - - State::TransferComplete => { - // We already finished processing an XFR response sequence. We - // don't expect there to be any more messages to process!. - return Err(Error::Malformed); - } - - State::TransferFailed => { - // We had to terminate processing of the XFR response sequence - // due to a problem with the received data, so we don't expect - // to be invoked again with another response message! - return Err(Error::Terminated); - } - }; - - let State::AwaitingNextAnswer { read, .. } = &mut self.state else { - unreachable!(); - }; - - for record in records.flatten() { - trace!("XFR record {}: {record:?}", read.rr_count); - - if let Some(event) = read.record(record).await? { - match event { - XfrEvent::EndOfTransfer => { - self.state = State::TransferComplete; - self.evt_handler.handle_event(event).await?; - return Ok(true); - } - - XfrEvent::ProcessingFailed => { - self.state = State::TransferFailed; - let _ = self.evt_handler.handle_event(event).await; - return Err(Error::Malformed); - } - - _ => { - self.evt_handler.handle_event(event).await?; - } - } - } - } - - // Finished processing this message but did not yet reach the end of - // the transfer, more responses are expected. - Ok(false) - } - - /// Check if an XFR response header is valid. - /// - /// Enforce the rules defined in 2. AXFR Messages of RFC 5936. See: - /// https://www.rfc-editor.org/rfc/rfc5936.html#section-2 - /// - /// Takes a request as well as a response as the response is checked to - /// see if it is in reply to the given request. - /// - /// Returns Ok on success, Err otherwise. On success the type of XFR that - /// was determined is returned as well as the answer section from the XFR - /// response. - async fn check_is_xfr_answer<'a>( - &mut self, - req: &Message, - resp: &'a Message, - ) -> Result<(XfrType, RecordSection<'a, Bytes>), CheckError> { - // Check the request. - let req_header = req.header(); - let req_counts = req.header_counts(); - - if req.is_error() - || req_header.qr() - || req_counts.qdcount() != 1 - || req_counts.ancount() != 0 - || req_header.opcode() != Opcode::QUERY - { - return Err(CheckError::NotValidXfrRequest); - } - - let Some(qtype) = req.qtype() else { - return Err(CheckError::NotValidXfrRequest); - }; - - let xfr_type = match qtype { - Rtype::AXFR => XfrType::Axfr, - Rtype::IXFR => XfrType::Ixfr, - _ => return Err(CheckError::NotValidXfrRequest), - }; - - // https://datatracker.ietf.org/doc/html/rfc1995#section-3 - // 3. Query Format - // "The IXFR query packet format is the same as that of a normal DNS - // query, but with the query type being IXFR and the authority - // section containing the SOA record of client's version of the - // zone." - if matches!(xfr_type, XfrType::Ixfr) && req_counts.nscount() != 1 { - return Err(CheckError::NotValidXfrResponse); - } - - // Check the response. - let resp_header = resp.header(); - let resp_counts = resp.header_counts(); - - if resp.is_error() - || !resp.is_answer_header(req) - || resp_header.opcode() != Opcode::QUERY - || resp_header.tc() - || resp_counts.ancount() == 0 - || resp_counts.nscount() != 0 - { - return Err(CheckError::NotValidXfrResponse); - } - - // https://datatracker.ietf.org/doc/html/rfc1995#section-2.2.1 - // 2.2.1. Header Values - // "QDCOUNT MUST be 1 in the first message; - // MUST be 0 or 1 in all following messages;" - if matches!(self.state, State::AwaitingFirstAnswer) - && (resp_counts.qdcount() != 1 - || resp.sole_question() != req.sole_question()) - { - return Err(CheckError::NotValidXfrResponse); - } - - let answer = resp.answer().map_err(CheckError::ParseError)?; - - Ok((xfr_type, answer)) - } - - /// Initialise the processosr. - /// - /// Records the initial SOA record and other details will will be used - /// while processing the rest of the response. - async fn initialize( - &mut self, - initial_xfr_type: XfrType, - initial_query_id: u16, - soa_record: XfrRecord, - ) -> Result<(), CheckError> { - // The initial record should be a SOA record. - let data = soa_record.into_data(); - - let AllRecordData::Soa(soa) = data else { - return Err(CheckError::NotValidXfrResponse); - }; - - let read = ParsingState::new(initial_xfr_type, soa); - - self.state = State::AwaitingNextAnswer { - initial_xfr_type, - initial_query_id, - read, - }; - - Ok(()) - } -} - -//------------ State ---------------------------------------------------------- - -/// The current processing state. -#[derive(Default)] -enum State { - /// Waiting for the first XFR response message. - #[default] - AwaitingFirstAnswer, - - /// Waiting for a subsequent XFR response message. - AwaitingNextAnswer { - /// The type of XFR response sequence expected based on the initial - /// request and response. - initial_xfr_type: XfrType, - - /// The header ID of the original XFR request. - initial_query_id: u16, - - /// The current parsing state. - read: ParsingState, - }, - - /// The end of the XFR response sequence was detected. - TransferComplete, - - /// An unrecoverable problem occurred while processing the XFR response - /// sequence. - TransferFailed, -} - -//------------ ParsingState --------------------------------------------------- - -/// State related to parsing the XFR response sequence. -#[derive(Debug)] -struct ParsingState { - /// The type of XFR response sequence being parsed. - /// - /// This can differ to the type of XFR response sequence that we expected - /// to parse because the server can fallback from IXFR to AXFR. - actual_xfr_type: XfrType, - - /// The initial SOA record that signals the start and end of both AXFR and - /// IXFR response sequences. - initial_soa: Soa>, - - /// The current SOA record. - /// - /// For AXFR response sequences this will be the same as `initial_soa`. - /// For IXFR response sequences this will be the last SOA record parsed as - /// each diff sequence contains two SOA records: one at the start of the - /// delete sequence and one at the start of the add sequence. - current_soa: Soa>, - - /// The kind of records currently being processed, either adds or deletes. - ixfr_update_mode: IxfrUpdateMode, - - /// The number of resource records parsed so far. - rr_count: usize, -} - -impl ParsingState { - /// Create a new parsing state. - fn new( - initial_xfr_type: XfrType, - initial_soa: Soa>, - ) -> Self { - Self { - actual_xfr_type: initial_xfr_type, - initial_soa: initial_soa.clone(), - current_soa: initial_soa, - rr_count: 1, - ixfr_update_mode: Default::default(), - } - } - - /// Parse a single resource record. - /// - /// Returns an [`XfrEvent`] that should be emitted for the parsed record, - /// if any. - async fn record( - &mut self, - rec: XfrRecord, - ) -> Result>, Error> { - self.rr_count += 1; - - // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2 - // 2.2. AXFR Response - // "..clients MUST accept any ordering and grouping of the non-SOA - // RRs. Each RR SHOULD be transmitted only once, and AXFR clients - // MUST ignore any duplicate RRs received." - // - // Note: We do NOT implement this MUST here because it would be very - // inefficient to actually check that any received non-SOA RR has not - // been seen before during the in-progress transfer. Clients of - // XfrResponseProcessor are better placed to enforce this rule if - // needed, e.g. at the moment of insertion into a zone tree checking - // that the record is not already present or insertion of a duplicate - // having no effect as it is already present. - - let soa = match rec.data() { - AllRecordData::Soa(soa) => Some(soa), - _ => None, - }; - - let record_matches_initial_soa = soa == Some(&self.initial_soa); - - match self.actual_xfr_type { - XfrType::Axfr if record_matches_initial_soa => { - // https://www.rfc-editor.org/rfc/rfc5936.html#section-2.2 - // 2.2. AXFR Response - // ... - // "In such a series, the first message MUST begin with the - // SOA resource record of the zone, and the last message - // MUST conclude with the same SOA resource record. - // Intermediate messages MUST NOT contain the SOA resource - // record." - Ok(Some(XfrEvent::EndOfTransfer)) - } - - XfrType::Axfr => { - Ok(Some(XfrEvent::AddRecord(self.current_soa.serial(), rec))) - } - - XfrType::Ixfr if self.rr_count < 2 => unreachable!(), - - XfrType::Ixfr if self.rr_count == 2 => { - if record_matches_initial_soa { - // IXFR not available, AXFR of empty zone detected. - Ok(Some(XfrEvent::EndOfTransfer)) - } else if let Some(soa) = soa { - // This SOA record is the start of an IXFR diff sequence. - self.current_soa = soa.clone(); - - // We don't need to set the IXFR update more here as it - // should already be set to Deleting. - debug_assert_eq!( - self.ixfr_update_mode, - IxfrUpdateMode::Deleting - ); - - Ok(Some(XfrEvent::BeginBatchDelete(soa.serial()))) - } else { - // https://datatracker.ietf.org/doc/html/rfc1995#section-4 - // 4. Response Format - // "If incremental zone transfer is not available, the - // entire zone is returned. The first and the last RR - // of the response is the SOA record of the zone. I.e. - // the behavior is the same as an AXFR response except - // the query type is IXFR. - // - // If incremental zone transfer is available, one or - // more difference sequences is returned. The list of - // difference sequences is preceded and followed by a - // copy of the server's current version of the SOA. - // - // Each difference sequence represents one update to - // the zone (one SOA serial change) consisting of - // deleted RRs and added RRs. The first RR of the - // deleted RRs is the older SOA RR and the first RR of - // the added RRs is the newer SOA RR. - // - // Modification of an RR is performed first by removing - // the original RR and then adding the modified one." - - // As this is IXFR and this is the second record, it should - // be the "first RR of the deleted RRs" which should be - // "the older SOA RR". However, it isn't a SOA RR. As such - // assume that "incremental zone transfer is not available" - // and so "the behaviour is the same as an AXFR response", - self.actual_xfr_type = XfrType::Axfr; - Ok(Some(XfrEvent::AddRecord( - self.current_soa.serial(), - rec, - ))) - } - } - - XfrType::Ixfr => { - if let Some(soa) = soa { - self.ixfr_update_mode.toggle(); - self.current_soa = soa.clone(); - - match self.ixfr_update_mode { - IxfrUpdateMode::Deleting => { - // We just finished a (Delete, Add) diff sequence. - // Is this the end of the transfer, or the start - // of a new diff sequence? - if record_matches_initial_soa { - Ok(Some(XfrEvent::EndOfTransfer)) - } else { - Ok(Some(XfrEvent::BeginBatchDelete( - self.current_soa.serial(), - ))) - } - } - IxfrUpdateMode::Adding => { - // We just switched from the Delete phase of a - // diff sequence to the add phase of the diff - // sequence. - Ok(Some(XfrEvent::BeginBatchAdd( - self.current_soa.serial(), - ))) - } - } - } else { - match self.ixfr_update_mode { - IxfrUpdateMode::Deleting => { - Ok(Some(XfrEvent::DeleteRecord( - self.current_soa.serial(), - rec, - ))) - } - IxfrUpdateMode::Adding => { - Ok(Some(XfrEvent::AddRecord( - self.current_soa.serial(), - rec, - ))) - } - } - } - } - } - } -} - -//------------ XfrEvent ------------------------------------------------------- - -/// An event emitted by [`XfrResponseProcessor`] during transfer processing. -#[derive(Copy, Clone, Debug, PartialEq, Eq)] -pub enum XfrEvent { - /// Delete record R in zone serial S. - /// - /// The transfer signalled that the given record should be deleted from - /// the zone version with the given serial number. - /// - /// Note: If the transfer contains N deletions of fhe same record then - /// this event will occur N times. - DeleteRecord(Serial, R), - - /// Add record R in zone serial S. - /// - /// The transfer signalled that the given record should be added to the - /// zone version with the given serial number. - /// - /// Note: If the transfer contains N additions of fhe same record then - /// this event will occur N times. - AddRecord(Serial, R), - - /// Prepare to delete records in zone serial S. - /// - /// The transfer signalled that zero or more record deletions will follow, - /// all for the zone version with the given serial number. - BeginBatchDelete(Serial), - - /// Prepare to add records in zone serial S. - /// - /// The transfer signalled that zero or more record additions will follow, - /// all for the zone version with the given serial number. - BeginBatchAdd(Serial), - - /// Transfer completed successfully. - /// - /// Note: This event is not emitted until the final record of the final - /// response in a set of one or more transfer responss has been seen. - EndOfTransfer, - - /// Transfer processing failed. - /// - /// This event indicates that there is a problem with the transfer data - /// and that transfer processing cannot continue. - ProcessingFailed, -} - -//--- Display - -impl std::fmt::Display for XfrEvent { - fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { - match self { - XfrEvent::DeleteRecord(_, _) => f.write_str("DeleteRecord"), - XfrEvent::AddRecord(_, _) => f.write_str("AddRecord"), - XfrEvent::BeginBatchDelete(_) => f.write_str("BeginBatchDelete"), - XfrEvent::BeginBatchAdd(_) => f.write_str("BeginBatchAdd"), - XfrEvent::EndOfTransfer => f.write_str("EndOfTransfer"), - XfrEvent::ProcessingFailed => f.write_str("ProcessingFailed"), - } - } -} - -//------------ XfrEventHandler --------------------------------------------------- - -/// A trait for implementing handlers of [`XfrEvent`]s. -pub trait XfrEventHandler { - type Fut: std::future::Future>; - - /// Handle the given [`XfrEvent`]. - /// - /// Returning an Err will cause transfer processsing to be aborted and the - /// error to be returned to the client of [`XfrResponseProcessor`], except in - /// the case of [`XfrEvent::ProcessingFailed`] for which the return value of - /// this handler will be ignored by [`XfrResponseProcessor`]. - fn handle_event(&self, evt: XfrEvent) -> Self::Fut; -} - -//------------ IxfrUpdateMode ------------------------------------------------- - -/// The kind of records currently being processed, either adds or deletes. -#[derive(Copy, Clone, Debug, Default, PartialEq, Eq)] -enum IxfrUpdateMode { - /// The records being parsed are deletions. - /// - /// Deletions come before additions. - #[default] - Deleting, - - /// The records being parsed are additions. - Adding, -} - -impl IxfrUpdateMode { - /// Toggle between the possible [`IxfrUpdateMode`] variants. - fn toggle(&mut self) { - match self { - IxfrUpdateMode::Deleting => *self = IxfrUpdateMode::Adding, - IxfrUpdateMode::Adding => *self = IxfrUpdateMode::Deleting, - } - } -} - -//------------ Error ---------------------------------------------------------- - -/// An error reported by [`XfrResponseProcessor`]. -#[derive(Debug)] -pub enum Error { - /// The message could not be parsed. - ParseError(ParseError, Message), - - /// The request message is not an XFR query/ - NotValidXfrRequest, - - /// The response message is not an XFR response. - NotValidXfrResponse(Message), - - /// At least one record in the XFR response sequence is incorrect. - Malformed, - - /// Processing was already terminated for this XFR response sequence. - Terminated, -} - -impl Error { - /// Convert a [`CheckError`] to an [`Error`]. - fn from_check_error( - msg: Message, - prepare_err: CheckError, - ) -> Self { - match prepare_err { - CheckError::ParseError(err) => Self::ParseError(err, msg), - CheckError::NotValidXfrRequest => Self::NotValidXfrRequest, - CheckError::NotValidXfrResponse => Self::NotValidXfrResponse(msg), - } - } -} - -//------------ PrepareError --------------------------------------------------- - -/// Errors that can occur during intiial checking of an XFR response sequence. -#[derive(Debug)] -enum CheckError { - /// A parsing error occurred while checking the original request and - /// response messages. - ParseError(ParseError), - - /// The XFR request is not valid according to the rules defined by RFC - /// 5936 (AXFR) or RFC 1995 (IXFR). - NotValidXfrRequest, - - /// The XFR response is not valid according to the rules defined by RFC - /// 5936 (AXFR) or RFC 1995 (IXFR). - NotValidXfrResponse, -} - -//--- From - -impl From for CheckError { - fn from(err: ParseError) -> Self { - Self::ParseError(err) - } -} - -//------------ XfrType -------------------------------------------------------- - -/// The type of XFR response sequence. -#[derive(Copy, Clone, Debug, PartialEq, Eq)] -enum XfrType { - /// RFC 5936 AXFR. - /// - /// A complete snapshot of a zone at a particular version. - Axfr, - - /// RFC 1995 IXFR. - /// - /// An incremental diff of the version of the zone that the server has - /// compared to the version of the zone that the client has. - Ixfr, -} - -//--- From - -impl TryFrom for XfrType { - type Error = (); - - fn try_from(rtype: Rtype) -> Result { - match rtype { - Rtype::AXFR => Ok(XfrType::Axfr), - Rtype::IXFR => Ok(XfrType::Ixfr), - _ => Err(()), - } - } -} - -#[cfg(test)] -mod tests { - use core::future::ready; - use core::future::Ready; - use core::str::FromStr; - - use std::string::String; - use std::sync::Arc; - use std::sync::Mutex; - use std::vec::Vec; - - use bytes::BytesMut; - use octseq::Octets; - - use crate::base::iana::Rcode; - use crate::base::message_builder::{ - AnswerBuilder, AuthorityBuilder, QuestionBuilder, - }; - use crate::base::net::Ipv4Addr; - use crate::base::rdata::ComposeRecordData; - use crate::base::{MessageBuilder, Ttl}; - use crate::base::{Name, ToName}; - use crate::rdata::A; - - use super::*; - - #[tokio::test] - async fn request_message_is_rejected() { - init_logging(); - - // Create a non-XFR request to reply to. - let req = mk_request("example.com", Rtype::A).into_message(); - - // Process the request and assert that it is rejected as not being - // a valid XFR response and that no XFR processor events were emitted. - assert_xfr_response( - &req.clone(), - req, - |res| matches!(res, Err(Error::NotValidXfrRequest)), - &[], - ) - .await; - } - - #[tokio::test] - async fn non_xfr_response_is_rejected() { - init_logging(); - - // Create a non-XFR request to reply to. - let req = mk_request("example.com", Rtype::AXFR).into_message(); - - // Create a non-XFR response. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - - // Process the response and assert that it is rejected as not being - // a valid XFR response and that no XFR processor events were emitted. - assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Err(Error::NotValidXfrResponse(_))), - &[], - ) - .await; - } - - #[tokio::test] - async fn axfr_response_with_no_answers_is_rejected() { - init_logging(); - - // Create an AXFR request to reply to. - let req = mk_request("example.com", Rtype::AXFR).into_message(); - - // Create a response that lacks answers. - let answer = mk_empty_answer(&req, Rcode::NOERROR); - - // Process the response and assert that it is rejected as not being - // a valid XFR response and that no XFR processor events were emitted. - assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Err(Error::NotValidXfrResponse(_))), - &[], - ) - .await; - } - - #[tokio::test] - async fn error_axfr_response_is_rejected() { - init_logging(); - - // Create an AXFR request to reply to. - let req = mk_request("example.com", Rtype::AXFR).into_message(); - - // Create a minimal valid AXFR response, just something that should - // not be rejected by the XFR processor due to its content. It should - // however be rejected due to the non-NOERROR rcode. - let mut answer = mk_empty_answer(&req, Rcode::SERVFAIL); - add_answer_record(&req, &mut answer, mk_soa(Serial::now())); - - // Process the response and assert that it is rejected as not being - // a valid XFR response and that no XFR processor events were emitted. - assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Err(Error::NotValidXfrResponse(_))), - &[], - ) - .await; - } - - #[tokio::test] - async fn incomplete_axfr_response_is_accepted() { - init_logging(); - - // Create an AXFR request to reply to. - let req = mk_request("example.com", Rtype::AXFR).into_message(); - - // Create an incomplete AXFR response. A proper AXFR response has at - // least two identical SOA records, one at the start and one at the - // end, but this response contains only a single SOA record. This is - // not considered invalid however because a subsequent response could - // still provide the missing SOA record. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - add_answer_record(&req, &mut answer, mk_soa(Serial::now())); - - // Process the response and assert that Ok(false) is returned by the - // XFR processor indicating that the XFR response was incomplete. Also - // verify the events emitted by the XFR processor. - assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Ok(false)), - &[], - ) - .await; - } - - #[tokio::test] - async fn axfr_response_with_only_soas_is_accepted() { - init_logging(); - - // Create an AXFR request to reply to. - let req = mk_request("example.com", Rtype::AXFR).into_message(); - - // Create a complete but minimal AXFR response. A proper AXFR response - // has at least two identical SOA records, one at the start and one at - // the end, with actual zone records in between. This response has only - // the start and end SOA and no content in between. RFC 5936 doesn't - // seem to disallow this. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - let soa = mk_soa(Serial::now()); - add_answer_record(&req, &mut answer, soa.clone()); - add_answer_record(&req, &mut answer, soa); - - // Process the response and assert that Ok(true) is returned by the - // XFR processor indicating that the XFR response was complete. Also - // verify the events emitted by the XFR processor. - assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Ok(true)), - &["EndOfTransfer"], - ) - .await; - } - - #[tokio::test] - async fn axfr_multi_response_with_only_soas_is_accepted() { - init_logging(); - - // Create an AXFR request to reply to. - let req = mk_request("example.com", Rtype::AXFR).into_message(); - - // Create a complete but minimal AXFR response. A proper AXFR response - // has at least two identical SOA records, one at the start and one at - // the end, with actual zone records in between. This response has only - // the start and end SOA and no content in between. RFC 5936 doesn't - // seem to disallow this. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - let soa = mk_soa(Serial::now()); - add_answer_record(&req, &mut answer, soa.clone()); - - // Process the response and assert that Ok(true) is returned by the - // XFR processor indicating that the XFR response was complete. Also - // verify the events emitted by the XFR processor. - let (evt_handler, mut processor) = assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Ok(false)), - &[], - ) - .await; - - // Create another AXFR response to complete the transfer. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - add_answer_record(&req, &mut answer, soa); - - // Process the response and assert that Ok(true) is returned by the - // XFR processor indicating that the XFR response was complete. Also - // verify the events emitted by the XFR processor. - assert_xfr_response_with_processor( - &req, - answer.into_message(), - |res| matches!(res, Ok(true)), - &["EndOfTransfer"], - evt_handler, - &mut processor, - ) - .await; - } - - #[tokio::test] - async fn axfr_response_generates_expected_events() { - init_logging(); - - // Create an AXFR request to reply to. - let req = mk_request("example.com", Rtype::AXFR).into_message(); - - // Create an AXFR response. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - let soa = mk_soa(Serial::now()); - add_answer_record(&req, &mut answer, soa.clone()); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); - add_answer_record(&req, &mut answer, soa); - - // Process the response and assert that Ok(true) is returned by the - // XFR processor indicating that the XFR response was complete. Also - // verify the events emitted by the XFR processor. - assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Ok(true)), - &["AddRecord", "AddRecord", "EndOfTransfer"], - ) - .await; - } - - #[tokio::test] - async fn ixfr_response_generates_expected_events() { - init_logging(); - - // Create an IXFR request to reply to. - let req = mk_request("example.com", Rtype::IXFR); - let mut authority = req.authority(); - let client_serial = Serial::now(); - let soa = mk_soa(client_serial); - add_authority_record(&mut authority, soa); - let req = authority.into_message(); - - // Prepare some serial numbers and SOA records to use in the IXFR response. - let old_serial = client_serial; - let new_serial = client_serial.add(1); - let old_soa = mk_soa(old_serial); - let new_soa = mk_soa(new_serial); - - // Create an IXFR response. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - // Outer SOA with servers current SOA - add_answer_record(&req, &mut answer, new_soa.clone()); - // Start of diff sequence: SOA of the servers' previous zone version - // (which matches that of the client) followed by records to be - // deleted as they were in that version of the zone but are not in the - // new version of the zone. - add_answer_record(&req, &mut answer, old_soa); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); - // SOA of the servers` new zone version (which is ahead of that of the - // client) followed by records to be added as they were added in this - // new version of the zone.` - add_answer_record(&req, &mut answer, new_soa.clone()); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - // Closing SOA with servers current SOA - add_answer_record(&req, &mut answer, new_soa); - - // Process the response and assert that Ok(true) is returned by the - // XFR processor indicating that the XFR response was complete. Also - // verify the events emitted by the XFR processor. - assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Ok(true)), - &[ - "BeginBatchDelete", - "DeleteRecord", - "DeleteRecord", - "BeginBatchAdd", - "AddRecord", - "AddRecord", - "EndOfTransfer", - ], - ) - .await; - } - - #[tokio::test] - async fn multi_ixfr_response_generates_expected_events() { - init_logging(); - - // Create an IXFR request to reply to. - let req = mk_request("example.com", Rtype::IXFR); - let mut authority = req.authority(); - let client_serial = Serial::now(); - let soa = mk_soa(client_serial); - add_authority_record(&mut authority, soa); - let req = authority.into_message(); - - // Prepare some serial numbers and SOA records to use in the IXFR response. - let old_serial = client_serial; - let new_serial = client_serial.add(1); - let old_soa = mk_soa(old_serial); - let new_soa = mk_soa(new_serial); - - // Create a partial IXFR response. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - // Outer SOA with servers current SOA - add_answer_record(&req, &mut answer, new_soa.clone()); - // Start of diff sequence: SOA of the servers' previous zone version - // (which matches that of the client) followed by records to be - // deleted as they were in that version of the zone but are not in the - // new version of the zone. - add_answer_record(&req, &mut answer, old_soa); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - - // Process the response and assert that Ok(true) is returned by the - // XFR processor indicating that the XFR response was complete. Also - // verify the events emitted by the XFR processor. - let (evt_handler, mut processor) = assert_xfr_response( - &req, - answer.into_message(), - |res| matches!(res, Ok(false)), - &["BeginBatchDelete", "DeleteRecord"], - ) - .await; - - // Craete a second IXFR response that completes the transfer - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); - // SOA of the servers` new zone version (which is ahead of that of the - // client) followed by records to be added as they were added in this - // new version of the zone.` - add_answer_record(&req, &mut answer, new_soa.clone()); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - // Closing SOA with servers current SOA - add_answer_record(&req, &mut answer, new_soa); - - assert_xfr_response_with_processor( - &req, - answer.into_message(), - |res| matches!(res, Ok(true)), - &[ - "BeginBatchDelete", // Seen during processing of the 1st answer - "DeleteRecord", // Seen during processing of the 1st answer - "DeleteRecord", // Seen during processing of the 2nd answer - "BeginBatchAdd", // Seen during processing of the 2nd answer - "AddRecord", // Seen during processing of the 2nd answer - "AddRecord", // Seen during processing of the 2nd answer - "EndOfTransfer", // Seen during processing of the 2nd answer - ], - evt_handler, - &mut processor, - ) - .await; - } - - //------------ TestXfrEventHandler ---------------------------------------- - - #[derive(Clone, Default)] - struct TestXfrEventHandler { - events: Arc>>, - } - - impl TestXfrEventHandler { - pub fn new() -> Self { - Self::default() - } - - pub async fn events(self) -> Vec { - self.events.lock().unwrap().clone() - } - } - - impl XfrEventHandler for TestXfrEventHandler { - type Fut = Ready>; - - fn handle_event(&self, evt: XfrEvent) -> Self::Fut { - trace!("Received event: {evt}"); - self.events.lock().unwrap().push(format!("{evt}")); - ready(Ok(())) - } - } - - //------------ Helper functions ------------------------------------------- - - fn init_logging() { - // Initialize tracing based logging. Override with env var RUST_LOG, e.g. - // RUST_LOG=trace. DEBUG level will show the .rpl file name, Stelline step - // numbers and types as they are being executed. - tracing_subscriber::fmt() - .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) - .with_thread_ids(true) - .without_time() - .try_init() - .ok(); - } - - async fn assert_xfr_response( - req: &Message, - resp: Message, - res_check_cb: fn(&Result) -> bool, - expected_events: &[&str], - ) -> ( - TestXfrEventHandler, - XfrResponseProcessor, - ) { - let evt_handler = TestXfrEventHandler::new(); - let mut processor = XfrResponseProcessor::new(evt_handler.clone()); - - assert_xfr_response_with_processor( - req, - resp, - res_check_cb, - expected_events, - evt_handler.clone(), - &mut processor, - ) - .await; - - (evt_handler, processor) - } - - async fn assert_xfr_response_with_processor( - req: &Message, - resp: Message, - res_check_cb: fn(&Result) -> bool, - expected_events: &[&str], - evt_handler: TestXfrEventHandler, - processor: &mut XfrResponseProcessor, - ) { - let res = processor.process_answer(req, resp).await; - - // Verify that the XFR processor returns an error. - assert!( - res_check_cb(&res), - "Unexpected result {res:?} from the XFR processor", - ); - - // Verify that no XFR processing events were emitted. - assert_eq!( - &evt_handler.clone().events().await, - expected_events, - "Unexpected events were emitted by the XFR processor" - ); - } - - fn mk_request(qname: &str, qtype: Rtype) -> QuestionBuilder { - let req = MessageBuilder::new_bytes(); - let mut req = req.question(); - req.push((Name::vec_from_str(qname).unwrap(), qtype)) - .unwrap(); - req - } - - fn mk_empty_answer( - req: &Message, - rcode: Rcode, - ) -> AnswerBuilder { - let builder = MessageBuilder::new_bytes(); - builder.start_answer(req, rcode).unwrap() - } - - fn add_answer_record( - req: &Message, - answer: &mut AnswerBuilder, - item: T, - ) { - let question = req.sole_question().unwrap(); - let qname = question.qname(); - let qclass = question.qclass(); - answer - .push((qname, qclass, Ttl::from_secs(0), item)) - .unwrap(); - } - - fn add_authority_record( - authority: &mut AuthorityBuilder, - item: T, - ) { - let (qname, qclass) = { - let question = authority.as_message().sole_question().unwrap(); - let qname = question.qname().to_bytes(); - let qclass = question.qclass(); - (qname, qclass) - }; - authority - .push((qname, qclass, Ttl::from_secs(0), item)) - .unwrap(); - } - - fn mk_soa(serial: Serial) -> Soa> { - let mname = Name::from_str("mname").unwrap(); - let rname = Name::from_str("rname").unwrap(); - let ttl = Ttl::from_secs(0); - Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl) - } -} From b0766b658c2b91aab387ff8c5d0be1794e4b2d76 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Sat, 31 Aug 2024 00:24:30 +0200 Subject: [PATCH 058/333] Remove request argument to XfrMessageProcessor() as the finalized request is not available to sender of the request (as it supplies a request template) or the recipient of the response. --- examples/client-transports.rs | 14 ++++++- src/net/xfr/processing/processor.rs | 57 +++-------------------------- src/net/xfr/processing/tests.rs | 33 +++++------------ src/net/xfr/processing/types.rs | 3 -- 4 files changed, 27 insertions(+), 80 deletions(-) diff --git a/examples/client-transports.rs b/examples/client-transports.rs index 98a5739a5..084cfd500 100644 --- a/examples/client-transports.rs +++ b/examples/client-transports.rs @@ -354,6 +354,8 @@ where + Sync + 'static, { + use domain::net::xfr::processing::XfrResponseProcessor; + // Create a signing key. let key_name = KeyName::from_str("demo-key").unwrap(); let secret = domain::utils::base64::decode::>( @@ -376,12 +378,20 @@ where let mut request = tsig_conn.send_request(req); // Get the reply + let mut processor = XfrResponseProcessor::new(); loop { println!("Waiting for signed reply"); let reply = request.get_response().await.unwrap(); println!("Signed reply: {:?}", reply); - if reply.is_none() { - break; + match reply { + Some(reply) => { + let it = processor.process_answer(reply).unwrap(); + for event in it { + let event = event.unwrap(); + println!("XFR event: {event}"); + } + } + None => break, } } } diff --git a/src/net/xfr/processing/processor.rs b/src/net/xfr/processing/processor.rs index f342cfba1..90f01539d 100644 --- a/src/net/xfr/processing/processor.rs +++ b/src/net/xfr/processing/processor.rs @@ -35,10 +35,8 @@ use super::types::{ /// sequence should be passed to [`process_answer()`]. /// /// [`process_answer()`]: XfrResponseProcessor::process_answer() +#[derive(Default)] pub struct XfrResponseProcessor { - /// The XFR request for which responses should be processed. - req: Message, - /// Internal state. /// /// None until the first call to [`process_answer()`]. @@ -47,12 +45,8 @@ pub struct XfrResponseProcessor { impl XfrResponseProcessor { /// Creates a new XFR message processor. - /// - /// The processor can be used to process response messages that relate to - /// the given XFR request message. - pub fn new(req: Message) -> Result { - Self::check_request(&req)?; - Ok(Self { req, inner: None }) + pub fn new() -> Self { + Self::default() } } @@ -90,42 +84,7 @@ impl XfrResponseProcessor { &mut self, resp: Message, ) -> Result<(), ProcessingError> { - self.inner = Some(Inner::new(&self.req, resp)?); - Ok(()) - } - - /// Check if an XFR request is valid. - fn check_request(req: &Message) -> Result<(), ProcessingError> { - let req_header = req.header(); - let req_counts = req.header_counts(); - - if req.is_error() - || req_header.qr() - || req_counts.qdcount() != 1 - || req_counts.ancount() != 0 - || req_header.opcode() != Opcode::QUERY - { - return Err(ProcessingError::NotValidXfrRequest); - } - - let Some(qtype) = req.qtype() else { - return Err(ProcessingError::NotValidXfrRequest); - }; - - if !matches!(qtype, Rtype::AXFR | Rtype::IXFR) { - return Err(ProcessingError::NotValidXfrRequest); - } - - // https://datatracker.ietf.org/doc/html/rfc1995#section-3 - // 3. Query Format - // "The IXFR query packet format is the same as that of a normal DNS - // query, but with the query type being IXFR and the authority - // section containing the SOA record of client's version of the - // zone." - if matches!(qtype, Rtype::IXFR) && req_counts.nscount() != 1 { - return Err(ProcessingError::NotValidXfrRequest); - } - + self.inner = Some(Inner::new(resp)?); Ok(()) } @@ -152,7 +111,6 @@ impl XfrResponseProcessor { // not required to have a question. if resp.is_error() || !resp_header.qr() - || resp_header.id() != self.req.header().id() || resp_header.opcode() != Opcode::QUERY || resp_header.tc() || resp_counts.ancount() == 0 @@ -195,10 +153,7 @@ impl Inner { /// /// Records the initial SOA record and other details will will be used /// while processing the rest of the response. - fn new( - req: &Message, - resp: Message, - ) -> Result { + fn new(resp: Message) -> Result { let answer = resp.answer().map_err(ProcessingError::ParseError)?; // https://datatracker.ietf.org/doc/html/rfc5936#section-3 @@ -220,7 +175,7 @@ impl Inner { let mut records = answer.into_records(); - let xfr_type = match req.qtype() { + let xfr_type = match resp.qtype() { Some(Rtype::AXFR) => XfrType::Axfr, Some(Rtype::IXFR) => XfrType::Ixfr, _ => unreachable!(), // Checked already in check_request(). diff --git a/src/net/xfr/processing/tests.rs b/src/net/xfr/processing/tests.rs index 9c62ca8e2..0067e8af0 100644 --- a/src/net/xfr/processing/tests.rs +++ b/src/net/xfr/processing/tests.rs @@ -20,21 +20,6 @@ use super::types::{ IterationError, ProcessingError, XfrEvent, XfrEvent as XE, XfrRecord, }; -#[test] -fn request_message_is_rejected() { - init_logging(); - - // Create a no/n-XFR request to reply to. - let req = mk_request("example.com", Rtype::A).into_message(); - - // Create an XFR response processor. - let res = XfrResponseProcessor::new(req.clone()); - - // Process the request and assert that it is rejected as not being - // a valid XFR response and that no XFR processor events were emitted. - assert!(matches!(res, Err(ProcessingError::NotValidXfrRequest))); -} - #[test] fn non_xfr_response_is_rejected() { init_logging(); @@ -43,7 +28,7 @@ fn non_xfr_response_is_rejected() { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + let mut processor = XfrResponseProcessor::new(); // Create a non-XFR response. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); @@ -66,7 +51,7 @@ fn axfr_response_with_no_answers_is_rejected() { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + let mut processor = XfrResponseProcessor::new(); // Create a response that lacks answers. let resp = mk_empty_answer(&req, Rcode::NOERROR).into_message(); @@ -87,7 +72,7 @@ fn error_axfr_response_is_rejected() { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + let mut processor = XfrResponseProcessor::new(); // Create a minimal valid AXFR response, just something that should // not be rejected by the XFR processor due to its content. It should @@ -112,7 +97,7 @@ fn incomplete_axfr_response_is_accepted() { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + let mut processor = XfrResponseProcessor::new(); // Create an incomplete AXFR response. A proper AXFR response has at // least two identical SOA records, one at the start and one at the @@ -138,7 +123,7 @@ fn axfr_response_with_only_soas_is_accepted() { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + let mut processor = XfrResponseProcessor::new(); // Create a complete but minimal AXFR response. A proper AXFR response // has at least two identical SOA records, one at the start and one at @@ -167,7 +152,7 @@ fn axfr_multi_response_with_only_soas_is_accepted() { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + let mut processor = XfrResponseProcessor::new(); // Create a complete but minimal AXFR response. A proper AXFR response // has at least two identical SOA records, one at the start and one at @@ -206,7 +191,7 @@ fn axfr_response_generates_expected_events() { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + let mut processor = XfrResponseProcessor::new(); // Create an AXFR response. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); @@ -242,7 +227,7 @@ fn ixfr_response_generates_expected_events() { let req = authority.into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + let mut processor = XfrResponseProcessor::new(); // Prepare some serial numbers and SOA records to use in the IXFR response. let old_serial = client_serial; @@ -331,7 +316,7 @@ fn multi_ixfr_response_generates_expected_events() { let req = authority.into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(req.clone()).unwrap(); + let mut processor = XfrResponseProcessor::new(); // Prepare some serial numbers and SOA records to use in the IXFR response. let old_serial = client_serial; diff --git a/src/net/xfr/processing/types.rs b/src/net/xfr/processing/types.rs index fb896e39a..4f4e46886 100644 --- a/src/net/xfr/processing/types.rs +++ b/src/net/xfr/processing/types.rs @@ -146,9 +146,6 @@ pub enum ProcessingError { /// The message could not be parsed. ParseError(ParseError), - /// The request message is not an XFR query/ - NotValidXfrRequest, - /// The response message is not an XFR response. NotValidXfrResponse, From dc7b0394620dfcc3792d26809ba7fcce889d1dc5 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 15:38:40 +0200 Subject: [PATCH 059/333] Revert local change made for testing. --- examples/client-transports.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/examples/client-transports.rs b/examples/client-transports.rs index 98a5739a5..d4a0dd5aa 100644 --- a/examples/client-transports.rs +++ b/examples/client-transports.rs @@ -51,8 +51,7 @@ async fn main() { let req = RequestMessage::new(msg).unwrap(); // Destination for UDP and TCP - let server_addr = - SocketAddr::new(IpAddr::from_str("127.0.0.1").unwrap(), 8055); + let server_addr = SocketAddr::new(IpAddr::from_str("::1").unwrap(), 53); let mut stream_config = stream::Config::new(); stream_config.set_response_timeout(Duration::from_millis(100)); From 1d7c952298026a513c3aa4aac19398c37f00cade Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 15:39:13 +0200 Subject: [PATCH 060/333] Import used type. --- examples/client-transports.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/examples/client-transports.rs b/examples/client-transports.rs index d4a0dd5aa..b9046169c 100644 --- a/examples/client-transports.rs +++ b/examples/client-transports.rs @@ -257,10 +257,7 @@ async fn main() { }); // Send a request message. - let mut request = domain::net::client::request::SendRequest::send_request( - &tcp, - req.clone(), - ); + let mut request = SendRequest::send_request(&tcp, req.clone()); // Get the reply let reply = request.get_response().await; From bf79abad69eae856c00d23781f29883c58f00fa8 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 15:42:17 +0200 Subject: [PATCH 061/333] Remove outdated comment. --- src/net/client/multi_stream.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/net/client/multi_stream.rs b/src/net/client/multi_stream.rs index 641a1a12b..d0c65c753 100644 --- a/src/net/client/multi_stream.rs +++ b/src/net/client/multi_stream.rs @@ -356,9 +356,6 @@ impl GetResponse for Request { > { Box::pin(Self::get_response(self)) } - - // TODO: Override stream_complete() and is_stream_complete() like - // net::client::stream does? } //------------ Transport ------------------------------------------------ From 326207d856946283f1366a66d0eead0d582b3f27 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 15:53:12 +0200 Subject: [PATCH 062/333] Remove commented out code. --- src/net/client/tsig.rs | 23 ----------------------- 1 file changed, 23 deletions(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index a7d5a13cc..d781d3498 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -96,29 +96,6 @@ where //------------ SendRequestMulti ---------------------------------------------------- -/* -impl SendRequestMulti for Connection -where - CR: ComposeRequestMulti + 'static, - Upstream: SendRequestMulti> - + Send - + Sync - + 'static, - K: Clone + AsRef + Send + Sync + 'static, -{ - fn send_request( - &self, - request_msg: CR, - ) -> Box { - Box::new(RequestMulti::::new( - request_msg, - self.key.clone(), - self.upstream.clone(), - )) - } -} -*/ - impl SendRequestMulti for Connection where CR: ComposeRequestMulti + 'static, From 620bcef246f1b23e4235223bcfe3496b98f6b6c3 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 15:58:40 +0200 Subject: [PATCH 063/333] Minor fix in client-transport demo logging. --- examples/client-transports.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/client-transports.rs b/examples/client-transports.rs index 869a29aaa..34ae0a005 100644 --- a/examples/client-transports.rs +++ b/examples/client-transports.rs @@ -378,9 +378,9 @@ where loop { println!("Waiting for signed reply"); let reply = request.get_response().await.unwrap(); - println!("Signed reply: {:?}", reply); match reply { Some(reply) => { + println!("Signed reply: {:?}", reply); let it = processor.process_answer(reply).unwrap(); for event in it { let event = event.unwrap(); From 42b02cfbda65e22c80eb3b6f31fb2f3c83eb1043 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 16:13:47 +0200 Subject: [PATCH 064/333] Silence Clippy for now. --- src/zonetree/in_memory/nodes.rs | 1 + src/zonetree/tree.rs | 1 + 2 files changed, 2 insertions(+) diff --git a/src/zonetree/in_memory/nodes.rs b/src/zonetree/in_memory/nodes.rs index 1c8d7ef1b..73c5ef98e 100644 --- a/src/zonetree/in_memory/nodes.rs +++ b/src/zonetree/in_memory/nodes.rs @@ -341,6 +341,7 @@ impl NodeRrset { self.rrsets.rollback(version); } + #[allow(dead_code)] pub fn clean(&mut self, version: Version) { self.rrsets.rollback(version); } diff --git a/src/zonetree/tree.rs b/src/zonetree/tree.rs index 863d2c7af..d02423870 100644 --- a/src/zonetree/tree.rs +++ b/src/zonetree/tree.rs @@ -185,6 +185,7 @@ impl ZoneSetNode { //------------ ZoneSetIter --------------------------------------------------- +/// TODO pub struct ZoneSetIter<'a> { roots: hash_map::Values<'a, Class, ZoneSetNode>, nodes: NodesIter<'a>, From 86282799d28a927e67aded5bed567b6fbd1fbc08 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 16:16:36 +0200 Subject: [PATCH 065/333] Clippy. --- src/zonetree/in_memory/versioned.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/zonetree/in_memory/versioned.rs b/src/zonetree/in_memory/versioned.rs index 85b1d99e7..034729444 100644 --- a/src/zonetree/in_memory/versioned.rs +++ b/src/zonetree/in_memory/versioned.rs @@ -87,7 +87,7 @@ impl Versioned { // clients to continue seeing the old version, but clients of the zone // after it is committed will see the new version, i.e. the empty // value which will cause get() to return None. - if self.data.last().map(|item| item.0) != None { + if self.data.last().map(|item| item.0).is_some() { self.data.push((version, None)); } } From 649836a7d444de9c1fbe8a056585a4ae79da2577 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 16:19:55 +0200 Subject: [PATCH 066/333] Review feedback: Respond with SERVFAIL with whatever questions we were able to push. --- src/base/message_builder.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/base/message_builder.rs b/src/base/message_builder.rs index d1cc2be40..369466ebb 100644 --- a/src/base/message_builder.rs +++ b/src/base/message_builder.rs @@ -280,7 +280,6 @@ impl MessageBuilder { let mut builder = self.question(); for item in msg.question().flatten() { if builder.push(item).is_err() { - builder.rewind(); builder.header_mut().set_rcode(Rcode::SERVFAIL); break; } From 80752e8babcfe333d3109b4452d7a116d24138cf Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 22:35:42 +0200 Subject: [PATCH 067/333] Simplify comment. --- examples/client-transports.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/examples/client-transports.rs b/examples/client-transports.rs index b9046169c..fc4c6bb50 100644 --- a/examples/client-transports.rs +++ b/examples/client-transports.rs @@ -363,9 +363,6 @@ where // Create a signing transport. This assumes that the server being // connected to is configured with a key with the same name, algorithm and // secret and to allow that key to be used for the request we are making. - // I'm not aware of any public server with a publically announced TSIG key - // that can be used for testing so this will fail, but has been tested to - // work locally with an appropriately configured NSD server. let tsig_conn = tsig::Connection::new(Some(key), conn); // Send a query message. From f8c58a1b6a80882f4c6a82552d6498047da11488 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 22:42:00 +0200 Subject: [PATCH 068/333] Minor RustDoc improvement. --- src/net/server/mod.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/net/server/mod.rs b/src/net/server/mod.rs index 4a9d3fcc3..f29314852 100644 --- a/src/net/server/mod.rs +++ b/src/net/server/mod.rs @@ -27,7 +27,7 @@ //! <-- (optional) middleware services - post-processes responses //! <-- server - serializes responses //! <-- network source - writes bytes to the client -//! ```` +//! ``` //! //! # Getting started //! @@ -38,15 +38,15 @@ //! application [`Service`] impl at the peak. //! //! Whether using [`DgramServer`] or [`StreamServer`] the required steps are -//! the same. +//! the same: //! -//! - Create an appropriate network source (more on this below). -//! - Construct a server transport with `new()` passing in the network +//! 1. Create an appropriate network source (more on this below). +//! 2. Construct a server transport with `new()` passing in the network //! source and service instance as arguments. //! - (optional) Tune the server behaviour via builder functions such as //! `with_config()`. -//! - `run()` the server. -//! - `shutdown()` the server, explicitly or on [`drop`]. +//! 3. `run()` the server. +//! 4. `shutdown()` the server, explicitly or on [`drop`]. //! //! See [`DgramServer`] and [`StreamServer`] for example code to help you get //! started. From 9aa5795bba359c1e88a25e573d89970424eb70e7 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 22:43:08 +0200 Subject: [PATCH 069/333] Minor RustDoc improvement. --- src/net/client/tsig.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index d781d3498..c2c935f05 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -1,4 +1,4 @@ -//! TODO +//! A TSIG signing & verifying passthrough transport. #![cfg(all(feature = "tsig", feature = "unstable-client-transport"))] #![warn(missing_docs)] #![warn(clippy::missing_docs_in_private_items)] From efe4972241edd152c53dad0de67dc56298c176b1 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 22:45:00 +0200 Subject: [PATCH 070/333] Remove Unpin constraint on Service RequestOctets generic type. --- src/net/server/message.rs | 2 +- src/net/server/service.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/net/server/message.rs b/src/net/server/message.rs index 46f9f0e60..84889a45b 100644 --- a/src/net/server/message.rs +++ b/src/net/server/message.rs @@ -157,7 +157,7 @@ impl From for TransportSpecificContext { #[derive(Debug)] pub struct Request where - Octs: AsRef<[u8]> + Send + Sync + Unpin, + Octs: AsRef<[u8]> + Send + Sync, { /// The network address of the connected client. client_addr: std::net::SocketAddr, diff --git a/src/net/server/service.rs b/src/net/server/service.rs index cbca6a4b2..f4b0fa1fc 100644 --- a/src/net/server/service.rs +++ b/src/net/server/service.rs @@ -173,7 +173,7 @@ pub type ServiceResult = Result, ServiceError>; /// [`call`]: Self::call() /// [`service_fn`]: crate::net::server::util::service_fn() pub trait Service< - RequestOctets: AsRef<[u8]> + Send + Sync + Unpin = Vec, + RequestOctets: AsRef<[u8]> + Send + Sync = Vec, RequestMeta: Clone + Default = (), > { From 9f447508ccab9b860f6f26d82057af82920c9fb7 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 22:50:08 +0200 Subject: [PATCH 071/333] Clippy. --- src/net/server/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/net/server/mod.rs b/src/net/server/mod.rs index f29314852..ab7a3b7dc 100644 --- a/src/net/server/mod.rs +++ b/src/net/server/mod.rs @@ -42,9 +42,9 @@ //! //! 1. Create an appropriate network source (more on this below). //! 2. Construct a server transport with `new()` passing in the network -//! source and service instance as arguments. -//! - (optional) Tune the server behaviour via builder functions such as -//! `with_config()`. +//! source and service instance as arguments. +//! - (optional) Tune the server behaviour via builder functions such as +//! `with_config()`. //! 3. `run()` the server. //! 4. `shutdown()` the server, explicitly or on [`drop`]. //! From 7c26d7635878cef098c893f3307983a11f089dfc Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:10:49 +0200 Subject: [PATCH 072/333] Add RustDoc for the multiple response support. --- src/net/client/mod.rs | 41 ++++++++++++++++++++++++++++++++++++++++- 1 file changed, 40 insertions(+), 1 deletion(-) diff --git a/src/net/client/mod.rs b/src/net/client/mod.rs index fa6144b67..2b7ba4a79 100644 --- a/src/net/client/mod.rs +++ b/src/net/client/mod.rs @@ -32,7 +32,7 @@ //! 1) Creating a request message, //! 2) Creating a DNS transport, //! 3) Sending the request, and -//! 4) Receiving the reply. +//! 4) Receiving the reply or replies. //! //! The first and second step are independent and can happen in any order. //! The third step uses the resuts of the first and second step. @@ -144,6 +144,45 @@ //! let reply = request.get_response().await; //! # } //! ``` +//! +//!
+//! +//! **Support for multiple responses:** +//! +//! [RequestMessage][request::RequestMessage] is designed for the most common +//! use case: single request, single response. +//! +//! However, zone transfers (e.g. using the `AXFR` or `IXFR` query types) can +//! result in multiple responses. Attempting to create a +//! [RequestMessage][request::RequestMessage] for such a query will result in +//! [Error::FormError][request::Error::FormError]. +//! +//! For zone transfers you should use +//! [RequestMessageMulti][request::RequestMessageMulti] instead which can be +//! used like so: +//! +//! ```no_run +//! # use crate::domain::net::client::request::{RequestMessageMulti, SendRequest}; +//! # use std::net::{IpAddr, SocketAddr}; +//! # use std::str::FromStr; +//! # async fn _test() { +//! # let (conn, _) = domain::net::client::stream::Connection::<_, RequestMessageMulti>>::new( +//! # domain::net::client::protocol::TcpConnect::new( +//! # SocketAddr::new(IpAddr::from_str("::1").unwrap(), 53) +//! # ) +//! # ); +//! # let req = domain::net::client::request::RequestMessageMulti::new( +//! # domain::base::MessageBuilder::new_vec() +//! # ).unwrap(); +//! # let mut request = tls_conn.send_request(req); +//! while let Some(reply) = request.get_response().await { +//! // ... +//! } +//! # } +//! ``` +//! +//!
+//! //! # Limitations //! From 88d7087ef2489171e7db36d9a0d6d9529a779143 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:12:35 +0200 Subject: [PATCH 073/333] Remove outdated comment. --- src/net/client/multi_stream.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/net/client/multi_stream.rs b/src/net/client/multi_stream.rs index 641a1a12b..d0c65c753 100644 --- a/src/net/client/multi_stream.rs +++ b/src/net/client/multi_stream.rs @@ -356,9 +356,6 @@ impl GetResponse for Request { > { Box::pin(Self::get_response(self)) } - - // TODO: Override stream_complete() and is_stream_complete() like - // net::client::stream does? } //------------ Transport ------------------------------------------------ From 18da3ee0259c3c7391ae7243bdede645a9588cad Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:16:53 +0200 Subject: [PATCH 074/333] Replace incorrect conversion (as it will return Error::FormError for XFR query types) by explicit RequestMessageMulti support. --- src/net/client/request.rs | 14 ------ src/stelline/client.rs | 92 +++++++++++++++++++++++++++------------ 2 files changed, 63 insertions(+), 43 deletions(-) diff --git a/src/net/client/request.rs b/src/net/client/request.rs index 527ca09ee..1f9cd19d6 100644 --- a/src/net/client/request.rs +++ b/src/net/client/request.rs @@ -620,20 +620,6 @@ impl + Debug + Octets + Send + Sync> ComposeRequestMulti } } -//--- From - -impl> From> - for RequestMessageMulti -{ - fn from(req: RequestMessage) -> Self { - RequestMessageMulti { - msg: req.msg, - header: req.header, - opt: req.opt, - } - } -} - //------------ Error --------------------------------------------------------- /// Error type for client transports. diff --git a/src/stelline/client.rs b/src/stelline/client.rs index f5c91a116..6fbd5b9bd 100644 --- a/src/stelline/client.rs +++ b/src/stelline/client.rs @@ -19,14 +19,15 @@ use crate::base::iana::{Opcode, OptionCode}; use crate::base::opt::{ComposeOptData, OptData}; use crate::base::{Message, MessageBuilder}; use crate::net::client::request::{ - ComposeRequest, Error, GetResponse, GetResponseMulti, RequestMessage, - RequestMessageMulti, SendRequest, SendRequestMulti, + ComposeRequest, ComposeRequestMulti, Error, GetResponse, + GetResponseMulti, RequestMessage, RequestMessageMulti, SendRequest, + SendRequestMulti, }; use crate::stelline::matches::match_multi_msg; use crate::zonefile::inplace::Entry::Record; use super::matches::match_msg; -use super::parse_stelline::{Entry, Reply, Stelline, StepType}; +use super::parse_stelline::{Entry, Reply, Sections, Stelline, StepType}; use super::channel::DEF_CLIENT_ADDR; use core::ops::Deref; @@ -236,15 +237,18 @@ impl Dispatcher { entry: &Entry, ) -> Result { if let Some(client) = &self.0 { - let reqmsg = entry2reqmsg(entry); - trace!(?reqmsg); let res = match client.deref() { Client::Single(client) => { + let reqmsg = entry2reqmsg(entry); + trace!(?reqmsg); Response::Single(client.send_request(reqmsg)) } - Client::Multi(client) => Response::Multi( - client.send_request(RequestMessageMulti::from(reqmsg)), - ), + + Client::Multi(client) => { + let reqmsg = entry2reqmsg_multi(entry); + trace!(?reqmsg); + Response::Multi(client.send_request(reqmsg)) + } }; return Ok(res); } @@ -679,6 +683,56 @@ fn init_logging() { } fn entry2reqmsg(entry: &Entry) -> RequestMessage> { + let (sections, reply, msg) = entry2msg(entry); + + let mut reqmsg = RequestMessage::new(msg).unwrap(); + if !entry + .matches + .as_ref() + .map(|v| v.mock_client) + .unwrap_or_default() + { + reqmsg.set_dnssec_ok(reply.fl_do); + } + if reply.notify { + reqmsg.header_mut().set_opcode(Opcode::NOTIFY); + } + + let edns_bytes = §ions.additional.edns_bytes; + if !edns_bytes.is_empty() { + let raw_opt = RawOptData { bytes: edns_bytes }; + reqmsg.add_opt(&raw_opt).unwrap(); + } + + reqmsg +} + +fn entry2reqmsg_multi(entry: &Entry) -> RequestMessageMulti> { + let (sections, reply, msg) = entry2msg(entry); + + let mut reqmsg = RequestMessageMulti::new(msg).unwrap(); + if !entry + .matches + .as_ref() + .map(|v| v.mock_client) + .unwrap_or_default() + { + reqmsg.set_dnssec_ok(reply.fl_do); + } + if reply.notify { + reqmsg.header_mut().set_opcode(Opcode::NOTIFY); + } + + let edns_bytes = §ions.additional.edns_bytes; + if !edns_bytes.is_empty() { + let raw_opt = RawOptData { bytes: edns_bytes }; + reqmsg.add_opt(&raw_opt).unwrap(); + } + + reqmsg +} + +fn entry2msg(entry: &Entry) -> (&Sections, Reply, Message>) { let sections = entry.sections.as_ref().unwrap(); let mut msg = MessageBuilder::new_vec().question(); if let Some(opcode) = entry.opcode { @@ -712,27 +766,7 @@ fn entry2reqmsg(entry: &Entry) -> RequestMessage> { header.set_ad(reply.ad); header.set_cd(reply.cd); let msg = msg.into_message(); - - let mut reqmsg = RequestMessage::new(msg).unwrap(); - if !entry - .matches - .as_ref() - .map(|v| v.mock_client) - .unwrap_or_default() - { - reqmsg.set_dnssec_ok(reply.fl_do); - } - if reply.notify { - reqmsg.header_mut().set_opcode(Opcode::NOTIFY); - } - - let edns_bytes = §ions.additional.edns_bytes; - if !edns_bytes.is_empty() { - let raw_opt = RawOptData { bytes: edns_bytes }; - reqmsg.add_opt(&raw_opt).unwrap(); - } - - reqmsg + (sections, reply, msg) } #[derive(Debug)] From b99e6f7f8e110d852b9380d72e130acb85f0ef35 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:18:01 +0200 Subject: [PATCH 075/333] Remove errant whitespace in RustDoc comments. --- src/net/client/stream.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index dcff40bc3..b213f736f 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -122,7 +122,7 @@ impl Config { /// Sets the response timeout. /// /// For requests where ComposeRequest::is_streaming() returns true see - /// set_streaming_response_timeout() instead. + /// set_streaming_response_timeout() instead. /// /// Excessive values are quietly trimmed. // @@ -158,7 +158,7 @@ impl Config { /// /// By default the stream is immediately closed if there are no pending /// requests or responses. - /// + /// /// Set this to allow requests to be sent in sequence with delays between /// such as a SOA query followed by AXFR for more efficient use of the /// stream per RFC 9103. From 024b1d1755e5e28861d111d61cfc6f7f8683b255 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:22:39 +0200 Subject: [PATCH 076/333] Undo reformatting of imports in otherwise untouched files. --- src/net/client/request.rs | 22 ++++++++++------------ src/net/client/stream.rs | 34 ++++++++++++++-------------------- 2 files changed, 24 insertions(+), 32 deletions(-) diff --git a/src/net/client/request.rs b/src/net/client/request.rs index 1f9cd19d6..2f79b2dbd 100644 --- a/src/net/client/request.rs +++ b/src/net/client/request.rs @@ -1,16 +1,4 @@ //! Constructing and sending requests. -use std::boxed::Box; -use std::fmt::Debug; -use std::future::Future; -use std::pin::Pin; -use std::sync::Arc; -use std::vec::Vec; -use std::{error, fmt}; - -use bytes::Bytes; -use octseq::Octets; -use tracing::trace; - use crate::base::iana::{Opcode, Rcode}; use crate::base::message::{CopyRecordsError, ShortMessage}; use crate::base::message_builder::{ @@ -20,6 +8,16 @@ use crate::base::opt::{ComposeOptData, LongOptData, OptRecord}; use crate::base::wire::{Composer, ParseError}; use crate::base::{Header, Message, ParsedName, Rtype, StaticCompressor}; use crate::rdata::AllRecordData; +use bytes::Bytes; +use octseq::Octets; +use std::boxed::Box; +use std::fmt::Debug; +use std::future::Future; +use std::pin::Pin; +use std::sync::Arc; +use std::vec::Vec; +use std::{error, fmt}; +use tracing::trace; #[cfg(feature = "tsig")] use crate::tsig; diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index b213f736f..f6d8a8c1b 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -13,40 +13,34 @@ // - timeouts // - request timeout // - create new connection after end/failure of previous one - +use super::request::{ + ComposeRequest, ComposeRequestMulti, Error, GetResponse, + GetResponseMulti, SendRequest, SendRequestMulti, +}; +use crate::base::iana::{Rcode, Rtype}; +use crate::base::message::Message; +use crate::base::message_builder::StreamTarget; +use crate::base::opt::{AllOptData, OptRecord, TcpKeepalive}; +use crate::base::{ParsedName, Serial}; +use crate::rdata::AllRecordData; +use crate::utils::config::DefMinMax; +use bytes::{Bytes, BytesMut}; use core::cmp; -use core::future::ready; +use octseq::Octets; use std::boxed::Box; use std::fmt::Debug; +use std::future::ready; use std::future::Future; use std::pin::Pin; use std::sync::Arc; use std::time::{Duration, Instant}; use std::vec::Vec; - -use bytes::{Bytes, BytesMut}; -use octseq::Octets; use tokio::io::{AsyncRead, AsyncReadExt, AsyncWrite, AsyncWriteExt}; use tokio::sync::{mpsc, oneshot}; use tokio::time::sleep; use tracing::trace; -use crate::base::iana::Rcode; -use crate::base::message::Message; -use crate::base::message_builder::StreamTarget; -use crate::base::opt::{AllOptData, OptRecord, TcpKeepalive}; -use crate::base::{ParsedName, Rtype, Serial}; -use crate::net::client::request::{ - ComposeRequest, Error, GetResponse, SendRequest, -}; -use crate::rdata::AllRecordData; -use crate::utils::config::DefMinMax; - -use super::request::{ - ComposeRequestMulti, GetResponseMulti, SendRequestMulti, -}; - //------------ Configuration Constants ---------------------------------------- /// Default response timeout. From 0d2a488897b5a812631f2fce6da50acd229e3c46 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:23:09 +0200 Subject: [PATCH 077/333] Undo reformatting in otherwise untouched file. --- src/net/client/stream.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index f6d8a8c1b..b216e7fc6 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -27,7 +27,6 @@ use crate::utils::config::DefMinMax; use bytes::{Bytes, BytesMut}; use core::cmp; use octseq::Octets; - use std::boxed::Box; use std::fmt::Debug; use std::future::ready; From d6d5794891305ee3cbd3d4b262fa958efacd3ba9 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:23:29 +0200 Subject: [PATCH 078/333] Undo reformatting in otherwise untouched file. --- src/net/client/stream.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index b216e7fc6..19714babd 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -13,6 +13,7 @@ // - timeouts // - request timeout // - create new connection after end/failure of previous one + use super::request::{ ComposeRequest, ComposeRequestMulti, Error, GetResponse, GetResponseMulti, SendRequest, SendRequestMulti, From 801866386ae30280dc1c0e92463d4b959173e8a8 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:43:03 +0200 Subject: [PATCH 079/333] Remove outdated comment. --- src/net/client/tsig.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index c2c935f05..84438f54d 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -310,8 +310,6 @@ where RequestStateMulti::Init => { let tsig_client = Arc::new(std::sync::Mutex::new(None)); - // TODO: TSIG sign the request, and send the signed version - // upstream. let msg = AuthenticatedRequestMessageMulti { request: self.request_msg.take().unwrap(), key: self.key.clone(), From 88b5c750ca2012459c6c205d9c76ac8370d8f786 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:54:11 +0200 Subject: [PATCH 080/333] More TSIG client RustDoc. --- src/net/client/tsig.rs | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index 84438f54d..d5cdb51f3 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -1,4 +1,26 @@ //! A TSIG signing & verifying passthrough transport. +//! +//! This module provides a transport that wraps the [high-level support for +//! signing message exchanges with TSIG][crate::tsig]. +//! +//! # Usage +//! +//! 1. Create a signing [Key]. +//! 2. Create a [Connection] that wraps an upstream connection and uses the +//! key. +//! 3. [Send a request][Connection::send_request] using the connection. +//! 4. [Receive the response][Request::get_response] or responses. +//! +//! # How it works +//! +//! Supplying the key is optional. The transport only affects the request and +//! response if a key is supplied. This allows for optional signing without +//! having to construct a different client stack. +//! +//! When a key is supplied, requests are automatically signed and response +//! signatures are automatically verified. On verification failure +//! [Error::ValidationError][crate::net::client::request::Error] will be +//! returned. #![cfg(all(feature = "tsig", feature = "unstable-client-transport"))] #![warn(missing_docs)] #![warn(clippy::missing_docs_in_private_items)] From ca3cccf4618225f5bde2d57189c89903e086d89c Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:54:50 +0200 Subject: [PATCH 081/333] Minor code simplification. --- src/net/client/tsig.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index d5cdb51f3..39949728f 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -359,7 +359,7 @@ where None => { match &mut self.state { RequestStateMulti::Init => { - debug!("Ignoring attempt to complete TSIG stream that hasn't been read from yet."); + unreachable!() } RequestStateMulti::GetResponse( From 119ab450aa80426c140eaadcc66092c7566026b5 Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Tue, 3 Sep 2024 15:59:05 +0200 Subject: [PATCH 082/333] Fix example. --- src/net/client/mod.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/net/client/mod.rs b/src/net/client/mod.rs index 2b7ba4a79..36567610b 100644 --- a/src/net/client/mod.rs +++ b/src/net/client/mod.rs @@ -162,11 +162,11 @@ //! used like so: //! //! ```no_run -//! # use crate::domain::net::client::request::{RequestMessageMulti, SendRequest}; +//! # use crate::domain::net::client::request::{RequestMessage, SendRequestMulti}; //! # use std::net::{IpAddr, SocketAddr}; //! # use std::str::FromStr; //! # async fn _test() { -//! # let (conn, _) = domain::net::client::stream::Connection::<_, RequestMessageMulti>>::new( +//! # let (conn, _) = domain::net::client::stream::Connection::>, _>::new( //! # domain::net::client::protocol::TcpConnect::new( //! # SocketAddr::new(IpAddr::from_str("::1").unwrap(), 53) //! # ) @@ -174,8 +174,8 @@ //! # let req = domain::net::client::request::RequestMessageMulti::new( //! # domain::base::MessageBuilder::new_vec() //! # ).unwrap(); -//! # let mut request = tls_conn.send_request(req); -//! while let Some(reply) = request.get_response().await { +//! # let mut request = conn.send_request(req); +//! while let Ok(reply) = request.get_response().await { //! // ... //! } //! # } From 6adac7a2794aaaaecc43e33d80cf7833f1dedd6b Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Tue, 3 Sep 2024 16:09:13 +0200 Subject: [PATCH 083/333] Rename Message::is_stream to Message::is_xfr. Remove is_stream from ComposeRequest and ComposeRequestMulti. --- src/base/message.rs | 4 +++- src/net/client/request.rs | 16 +--------------- src/net/client/stream.rs | 30 ++++++++---------------------- 3 files changed, 12 insertions(+), 38 deletions(-) diff --git a/src/base/message.rs b/src/base/message.rs index 8b13c4ed4..7f52ed42c 100644 --- a/src/base/message.rs +++ b/src/base/message.rs @@ -439,7 +439,9 @@ impl Message { } } - pub fn is_streaming(&self) -> bool { + /// Returns whether the message has a question that is either AXFR or + /// IXFR. + pub fn is_xfr(&self) -> bool { self.first_question() .map(|q| matches!(q.qtype(), Rtype::AXFR | Rtype::IXFR)) .unwrap_or_default() diff --git a/src/net/client/request.rs b/src/net/client/request.rs index 2f79b2dbd..793f3f132 100644 --- a/src/net/client/request.rs +++ b/src/net/client/request.rs @@ -60,9 +60,6 @@ pub trait ComposeRequest: Debug + Send + Sync { /// Returns whether a message is an answer to the request. fn is_answer(&self, answer: &Message<[u8]>) -> bool; - /// Returns whether a message results in a response stream or not. - fn is_streaming(&self) -> bool; - /// Return the status of the DNSSEC OK flag. fn dnssec_ok(&self) -> bool; } @@ -104,9 +101,6 @@ pub trait ComposeRequestMulti: Debug + Send + Sync { /// Returns whether a message is an answer to the request. fn is_answer(&self, answer: &Message<[u8]>) -> bool; - /// Returns whether a message results in a response stream or not. - fn is_streaming(&self) -> bool; - /// Return the status of the DNSSEC OK flag. fn dnssec_ok(&self) -> bool; } @@ -386,10 +380,6 @@ impl + Debug + Octets + Send + Sync> ComposeRequest } } - fn is_streaming(&self) -> bool { - self.msg.is_streaming() - } - fn dnssec_ok(&self) -> bool { match &self.opt { None => false, @@ -422,7 +412,7 @@ impl + Debug + Octets> RequestMessageMulti { let msg = msg.into(); // Only accept the streaming types (IXFR and AXFR). - if !msg.is_streaming() { + if !msg.is_xfr() { return Err(Error::FormError); } let header = msg.header(); @@ -606,10 +596,6 @@ impl + Debug + Octets + Send + Sync> ComposeRequestMulti } } - fn is_streaming(&self) -> bool { - self.msg.is_streaming() - } - fn dnssec_ok(&self) -> bool { match &self.opt { None => false, diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index 19714babd..9441a22fa 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -30,7 +30,6 @@ use core::cmp; use octseq::Octets; use std::boxed::Box; use std::fmt::Debug; -use std::future::ready; use std::future::Future; use std::pin::Pin; use std::sync::Arc; @@ -261,14 +260,8 @@ where /// Returns a request handler for this connection. pub fn get_request(&self, request_msg: Req) -> Request { - if request_msg.is_streaming() { - Request { - fut: Box::pin(ready(Err(Error::FormError))), - } - } else { - Request { - fut: Box::pin(self.clone().handle_request_impl(request_msg)), - } + Request { + fut: Box::pin(self.clone().handle_request_impl(request_msg)), } } @@ -278,19 +271,12 @@ where request_msg: ReqMulti, ) -> RequestMulti { let (sender, receiver) = mpsc::channel(DEF_CHAN_CAP); - if !request_msg.is_streaming() { - RequestMulti { - stream: receiver, - fut: Some(Box::pin(ready(Err(Error::FormError)))), - } - } else { - RequestMulti { - stream: receiver, - fut: Some(Box::pin( - self.clone() - .handle_streaming_request_impl(request_msg, sender), - )), - } + RequestMulti { + stream: receiver, + fut: Some(Box::pin( + self.clone() + .handle_streaming_request_impl(request_msg, sender), + )), } } } From 5d424e79497c57adc8c06c1b8e821da99eb556f9 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 3 Sep 2024 22:47:53 +0200 Subject: [PATCH 084/333] Remove is_streaming() fn as it has been removed in upstream branches. --- src/net/client/tsig.rs | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index 39949728f..9bc2c681b 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -571,10 +571,6 @@ where self.request.set_udp_payload_size(value) } - fn is_streaming(&self) -> bool { - self.request.is_streaming() - } - fn set_dnssec_ok(&mut self, value: bool) { self.request.set_dnssec_ok(value) } @@ -703,10 +699,6 @@ where self.request.is_answer(answer) } - fn is_streaming(&self) -> bool { - self.request.is_streaming() - } - fn dnssec_ok(&self) -> bool { self.request.dnssec_ok() } From b9571615feb225cc019d662506b459fbd1fb0e0e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 3 Sep 2024 23:21:53 +0200 Subject: [PATCH 085/333] Cherry pick the Stelline server testing README from the `xfr` branch. --- test-data/server/README.md | 74 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 74 insertions(+) create mode 100644 test-data/server/README.md diff --git a/test-data/server/README.md b/test-data/server/README.md new file mode 100644 index 000000000..41ec4e477 --- /dev/null +++ b/test-data/server/README.md @@ -0,0 +1,74 @@ +# Stelline server tests + +The test recipes defined in this directory follow a common pattern, each +involving two or three parties that communicate via mock network sockets: + + 1. A real in-memory server to be tested. + The behaviour of the server is controlled by the `config:` block that + preceeds the `CONFIG_END` directive. + + This server is powered by actual running domain server code, not + test/mock code. + + This server has no network address at which it listens as such, instead + client requests are delivered via a mock network socket to which the + server is directly connected. + + 2. Real clients querying the server under test. + The behaviour of these clients is controlled by pairs of `STEP` + directives: `QUERY` then `CHECK_ANSWER`. + + These clients are powered by actual running domain client code, not + test/mock code. + + From the perspective of the server the clients appear to send requests + from 127.0.0.1 unless otherwise specified via + `STEP QUERY ADDRESS `. + + 3. **[Optional]** Mock servers for use with the above clients & server. + These servers do not actually exist, they are just a set of mock replies + which the Stelline test framework will provide as responses to matching + queries (from either the clients or the server under test). + + Queries are matched by both step number (only steps in the defined range + can match) and by properties of the DNS query. + +# Known limitations + +- The `QUERY` step only sets expectations, the actual sending of the query, the + receiving and checking of the answer all happen in the `CHECK_ANSWER` step. + +- The first use of a particular client IP address (or the default if none is + specified via `ADDRESS`) will determine the configuration of that client. If + two different steps define a different configuration for the same client, + e.g. a latter step specifying that requests should be signed using a + particular TSIG key, the former step will determine that the client sends + unsigned requests and the latter TSIG key specification will be ignored. + +- By default server responses are expected to consist of only a single reply. + The `EXTRA_PACKET` directive can be used to denote that a subsequent reply + should also be expected, e.g. for XFR responses. The `EXTRA_PACKET` + directive can be used more than once. The number of replies must exactly + match the number of expected replies and each reply must have the correct + subset of the total RRs across all "packets" (response messages). If the + number and/or composition of extra responses is not known use `MATCH + EXTRA_PACKETS` (note the plural) to tell the Stelline test framework to + gather all replies into one then test the RR present (order insensitive). + +- Specifying a TSIG key via `STEP QUERY KEY ` will cause the real + TSIG client code to TSIG sign the request, and to strip out the TSIG signing + in the reply before the comparison is made with the expected reply contents + as definedby the `SECTION` blocks of a `CHECK_ANSWER` step. As such the + actual effects of TSIG signing are not visible nor explicitly tested for in + test steps that use `KEY `. + +- Using a real client limits the control the tests have over the sent messages. + For example, irrespective of the query defined by a test, the client code + will inject an EDNS TCP keep-alive OPT record before sending it. This can be + worked around by using `MATCH MOCK_CLIENT` which will instruct the Stelline + test framework to use a different UDP client which does not modify the + request before sending. + +- Tests are run in a single thread. Actions that need to take place in the + background may therefore need an explicit chance to run, via use of + `STEP TIME_PASSES ELAPSE 0`. From b7db22cf26a716b15bb0de2d6107a3a7d87c8700 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 27 Aug 2024 11:58:02 +0200 Subject: [PATCH 086/333] Review feedback: SERVFAIL if question cannot be built when preparing an error message. --- src/base/message_builder.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/base/message_builder.rs b/src/base/message_builder.rs index f5dccd323..d1cc2be40 100644 --- a/src/base/message_builder.rs +++ b/src/base/message_builder.rs @@ -274,14 +274,18 @@ impl MessageBuilder { header.set_qr(true); header.set_opcode(msg.header().opcode()); header.set_rd(msg.header().rd()); + header.set_rcode(rcode); } + let mut builder = self.question(); for item in msg.question().flatten() { if builder.push(item).is_err() { + builder.rewind(); + builder.header_mut().set_rcode(Rcode::SERVFAIL); break; } } - builder.header_mut().set_rcode(rcode); + builder.answer() } From 45400d068acb32b1404b94c208e9ee78d543d72c Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 16:19:55 +0200 Subject: [PATCH 087/333] Review feedback: Respond with SERVFAIL with whatever questions we were able to push. --- src/base/message_builder.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/base/message_builder.rs b/src/base/message_builder.rs index d1cc2be40..369466ebb 100644 --- a/src/base/message_builder.rs +++ b/src/base/message_builder.rs @@ -280,7 +280,6 @@ impl MessageBuilder { let mut builder = self.question(); for item in msg.question().flatten() { if builder.push(item).is_err() { - builder.rewind(); builder.header_mut().set_rcode(Rcode::SERVFAIL); break; } From fe588d10366aff8ae8f01f7d15ee557b6a169016 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 22:42:00 +0200 Subject: [PATCH 088/333] Minor RustDoc improvement. --- src/net/server/mod.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/net/server/mod.rs b/src/net/server/mod.rs index 4a9d3fcc3..f29314852 100644 --- a/src/net/server/mod.rs +++ b/src/net/server/mod.rs @@ -27,7 +27,7 @@ //! <-- (optional) middleware services - post-processes responses //! <-- server - serializes responses //! <-- network source - writes bytes to the client -//! ```` +//! ``` //! //! # Getting started //! @@ -38,15 +38,15 @@ //! application [`Service`] impl at the peak. //! //! Whether using [`DgramServer`] or [`StreamServer`] the required steps are -//! the same. +//! the same: //! -//! - Create an appropriate network source (more on this below). -//! - Construct a server transport with `new()` passing in the network +//! 1. Create an appropriate network source (more on this below). +//! 2. Construct a server transport with `new()` passing in the network //! source and service instance as arguments. //! - (optional) Tune the server behaviour via builder functions such as //! `with_config()`. -//! - `run()` the server. -//! - `shutdown()` the server, explicitly or on [`drop`]. +//! 3. `run()` the server. +//! 4. `shutdown()` the server, explicitly or on [`drop`]. //! //! See [`DgramServer`] and [`StreamServer`] for example code to help you get //! started. From 331b247c4536c1caf25ed399c3a223b3fb779003 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 22:45:00 +0200 Subject: [PATCH 089/333] Remove Unpin constraint on Service RequestOctets generic type. --- src/net/server/message.rs | 2 +- src/net/server/service.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/net/server/message.rs b/src/net/server/message.rs index 46f9f0e60..84889a45b 100644 --- a/src/net/server/message.rs +++ b/src/net/server/message.rs @@ -157,7 +157,7 @@ impl From for TransportSpecificContext { #[derive(Debug)] pub struct Request where - Octs: AsRef<[u8]> + Send + Sync + Unpin, + Octs: AsRef<[u8]> + Send + Sync, { /// The network address of the connected client. client_addr: std::net::SocketAddr, diff --git a/src/net/server/service.rs b/src/net/server/service.rs index cbca6a4b2..f4b0fa1fc 100644 --- a/src/net/server/service.rs +++ b/src/net/server/service.rs @@ -173,7 +173,7 @@ pub type ServiceResult = Result, ServiceError>; /// [`call`]: Self::call() /// [`service_fn`]: crate::net::server::util::service_fn() pub trait Service< - RequestOctets: AsRef<[u8]> + Send + Sync + Unpin = Vec, + RequestOctets: AsRef<[u8]> + Send + Sync = Vec, RequestMeta: Clone + Default = (), > { From a10b8f41e486a63e39a74b6580f171ac0a6adf7d Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 22:50:08 +0200 Subject: [PATCH 090/333] Clippy. --- src/net/server/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/net/server/mod.rs b/src/net/server/mod.rs index f29314852..ab7a3b7dc 100644 --- a/src/net/server/mod.rs +++ b/src/net/server/mod.rs @@ -42,9 +42,9 @@ //! //! 1. Create an appropriate network source (more on this below). //! 2. Construct a server transport with `new()` passing in the network -//! source and service instance as arguments. -//! - (optional) Tune the server behaviour via builder functions such as -//! `with_config()`. +//! source and service instance as arguments. +//! - (optional) Tune the server behaviour via builder functions such as +//! `with_config()`. //! 3. `run()` the server. //! 4. `shutdown()` the server, explicitly or on [`drop`]. //! From 84761e0d87cf9bbd6436d7932b7c0bc391b5c944 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:43:03 +0200 Subject: [PATCH 091/333] Remove outdated comment. --- src/net/client/tsig.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index c2c935f05..84438f54d 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -310,8 +310,6 @@ where RequestStateMulti::Init => { let tsig_client = Arc::new(std::sync::Mutex::new(None)); - // TODO: TSIG sign the request, and send the signed version - // upstream. let msg = AuthenticatedRequestMessageMulti { request: self.request_msg.take().unwrap(), key: self.key.clone(), From ad432ac4957994abbb183eafe0696bce8adb99d3 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:54:11 +0200 Subject: [PATCH 092/333] More TSIG client RustDoc. --- src/net/client/tsig.rs | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index 84438f54d..d5cdb51f3 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -1,4 +1,26 @@ //! A TSIG signing & verifying passthrough transport. +//! +//! This module provides a transport that wraps the [high-level support for +//! signing message exchanges with TSIG][crate::tsig]. +//! +//! # Usage +//! +//! 1. Create a signing [Key]. +//! 2. Create a [Connection] that wraps an upstream connection and uses the +//! key. +//! 3. [Send a request][Connection::send_request] using the connection. +//! 4. [Receive the response][Request::get_response] or responses. +//! +//! # How it works +//! +//! Supplying the key is optional. The transport only affects the request and +//! response if a key is supplied. This allows for optional signing without +//! having to construct a different client stack. +//! +//! When a key is supplied, requests are automatically signed and response +//! signatures are automatically verified. On verification failure +//! [Error::ValidationError][crate::net::client::request::Error] will be +//! returned. #![cfg(all(feature = "tsig", feature = "unstable-client-transport"))] #![warn(missing_docs)] #![warn(clippy::missing_docs_in_private_items)] From af467d6fc2723eedabefa1743d78ebcb4ae4efe5 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:54:50 +0200 Subject: [PATCH 093/333] Minor code simplification. --- src/net/client/tsig.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index d5cdb51f3..39949728f 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -359,7 +359,7 @@ where None => { match &mut self.state { RequestStateMulti::Init => { - debug!("Ignoring attempt to complete TSIG stream that hasn't been read from yet."); + unreachable!() } RequestStateMulti::GetResponse( From 4bfef528df54b1a844f1e8e65e43692a2d41f13e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 3 Sep 2024 22:38:44 +0200 Subject: [PATCH 094/333] - Add TSIG response signing middleware. - Add TSIG Stelline test recipe. - Update Stelline server integration tests to use the new TSIG middleware. - Remove unnecessary Unpin bounds in net::server::message::Request. - Tsig module changes: - Derive Clone for Key. - Added Key::compose_len() for determining how many response bytes to reserve. - Added From> for ServerSequence. - Altered KeyStore blanket impls to avoid conflict with additional impl on Arc. - Various minor RustDoc improvements. - Made tsig::ServerError::unsigned() public. --- src/base/message_builder.rs | 3 +- src/net/server/message.rs | 4 +- src/net/server/middleware/cookies.rs | 2 +- src/net/server/middleware/edns.rs | 2 +- src/net/server/middleware/mod.rs | 2 + src/net/server/middleware/tsig.rs | 460 +++++++++++++++++++++++++++ src/net/server/tests/integration.rs | 109 +++++-- src/tsig/mod.rs | 162 ++++++++-- test-data/server/tsig.rpl | 138 ++++++++ 9 files changed, 831 insertions(+), 51 deletions(-) create mode 100644 src/net/server/middleware/tsig.rs create mode 100644 test-data/server/tsig.rpl diff --git a/src/base/message_builder.rs b/src/base/message_builder.rs index 369466ebb..e4c943f77 100644 --- a/src/base/message_builder.rs +++ b/src/base/message_builder.rs @@ -262,7 +262,8 @@ impl MessageBuilder { /// Starts creating an error for the given message. /// - /// Like [`start_answer()`] but infallible. Questions will be pushed if possible. + /// Like [`start_answer()`] but infallible. Questions will be pushed if + /// possible. pub fn start_error( mut self, msg: &Message, diff --git a/src/net/server/message.rs b/src/net/server/message.rs index 84889a45b..1ac11e3b1 100644 --- a/src/net/server/message.rs +++ b/src/net/server/message.rs @@ -192,7 +192,7 @@ where impl Request where - Octs: AsRef<[u8]> + Send + Sync + Unpin, + Octs: AsRef<[u8]> + Send + Sync, { /// Creates a new request wrapper around a message along with its context. pub fn new( @@ -270,7 +270,7 @@ where impl Clone for Request where - Octs: AsRef<[u8]> + Send + Sync + Unpin, + Octs: AsRef<[u8]> + Send + Sync, Metadata: Clone, { fn clone(&self) -> Self { diff --git a/src/net/server/middleware/cookies.rs b/src/net/server/middleware/cookies.rs index 58708e504..a864ec942 100644 --- a/src/net/server/middleware/cookies.rs +++ b/src/net/server/middleware/cookies.rs @@ -471,7 +471,7 @@ where Once::Item>>, ::Item, >; - type Future = core::future::Ready; + type Future = Ready; fn call( &self, diff --git a/src/net/server/middleware/edns.rs b/src/net/server/middleware/edns.rs index affe96362..6eae69350 100644 --- a/src/net/server/middleware/edns.rs +++ b/src/net/server/middleware/edns.rs @@ -361,7 +361,7 @@ where Once::Item>>, ::Item, >; - type Future = core::future::Ready; + type Future = Ready; fn call( &self, diff --git a/src/net/server/middleware/mod.rs b/src/net/server/middleware/mod.rs index b364a83ba..977814d03 100644 --- a/src/net/server/middleware/mod.rs +++ b/src/net/server/middleware/mod.rs @@ -30,3 +30,5 @@ pub mod cookies; pub mod edns; pub mod mandatory; pub mod stream; +#[cfg(feature = "tsig")] +pub mod tsig; diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs new file mode 100644 index 000000000..5cedac484 --- /dev/null +++ b/src/net/server/middleware/tsig.rs @@ -0,0 +1,460 @@ +//! TSIG message authentication middleware. + +use core::convert::Infallible; +use core::future::{ready, Ready}; +use core::marker::PhantomData; +use core::ops::{ControlFlow, DerefMut}; + +use std::sync::Arc; +use std::vec::Vec; + +use futures::stream::{once, Once, Stream}; +use octseq::{Octets, OctetsFrom}; +use tracing::{error, trace, warn}; + +use crate::base::iana::{Opcode, OptRcode, Rcode, TsigRcode}; +use crate::base::message_builder::AdditionalBuilder; +use crate::base::wire::Composer; +use crate::base::{Message, ParsedName, Question, Rtype, StreamTarget}; +use crate::net::server::message::Request; +use crate::net::server::service::{ + CallResult, Service, ServiceFeedback, ServiceResult, +}; +use crate::net::server::util::{mk_builder_for_target, mk_error_response}; +use crate::rdata::tsig::Time48; +use crate::tsig::{ + self, KeyName, KeyStore, ServerSequence, ServerTransaction, +}; + +use super::stream::{MiddlewareStream, PostprocessingStream}; + +//------------ TsigMiddlewareSvc ---------------------------------------------- + +/// TSIG message authentication middlware. +/// +/// This middleware service validates TSIG signatures on incoming requests, if +/// any, and adds TSIG signatures to responses to signed requests. +/// +/// | RFC | Status | +/// |--------|---------| +/// | [8945] | TBD | +/// +/// [8945]: https://datatracker.ietf.org/doc/rfc8945/ +#[derive(Clone, Debug)] +pub struct TsigMiddlewareSvc +where + KS: Clone + KeyStore, +{ + next_svc: NextSvc, + + key_store: KS, + + _phantom: PhantomData, +} + +impl TsigMiddlewareSvc +where + KS: Clone + KeyStore, +{ + /// Creates a new processor instance. + #[must_use] + pub fn new(next_svc: NextSvc, key_store: KS) -> Self { + Self { + next_svc, + key_store, + _phantom: PhantomData, + } + } +} + +impl TsigMiddlewareSvc +where + RequestOctets: Octets + OctetsFrom> + Send + Sync + Unpin, + NextSvc: Service, + NextSvc::Target: Composer + Default, + KS: Clone + KeyStore, + Infallible: From<>>::Error>, +{ + #[allow(clippy::type_complexity)] + fn preprocess( + req: &Request, + key_store: &KS, + ) -> ControlFlow< + AdditionalBuilder>, + Option<(Request, TsigSigner)>, + > { + if let Some(q) = Self::get_relevant_question(req.message()) { + let octets = req.message().as_slice().to_vec(); + let mut mut_msg = Message::from_octets(octets).unwrap(); + + match tsig::ServerTransaction::request( + key_store, + &mut mut_msg, + Time48::now(), + ) { + Ok(None) => { + // Message is not TSIG signed. + } + + Ok(Some(tsig)) => { + // Message is TSIG signed by a known key. + trace!( + "Request is signed with TSIG key '{}'", + tsig.key().name() + ); + + // Convert to RequestOctets so that the non-TSIG signed + // message case can just pass through the RequestOctets. + let source = mut_msg.into_octets(); + let octets = RequestOctets::octets_from(source); + let new_msg = Message::from_octets(octets).unwrap(); + + let mut new_req = Request::new( + req.client_addr(), + req.received_at(), + new_msg, + req.transport_ctx().clone(), + Authentication(Some(tsig.key().name().clone())), + ); + + let num_bytes_to_reserve = tsig.key().compose_len(); + new_req.reserve_bytes(num_bytes_to_reserve); + + return ControlFlow::Continue(Some(( + new_req, + TsigSigner::Transaction(tsig), + ))); + } + + Err(err) => { + // Message is incorrectly signed or signed with an unknown key. + warn!( + "{} for {} from {} refused: {err}", + q.qtype(), + q.qname(), + req.client_addr(), + ); + let builder = mk_builder_for_target(); + let additional = tsig::ServerError::::unsigned( + TsigRcode::BADKEY, + ) + .build_message(req.message(), builder) + .unwrap(); + return ControlFlow::Break(additional); + } + } + } + + ControlFlow::Continue(None) + } + + /// Sign the given response, or if necessary construct and return an + /// alternate response. + fn postprocess( + request: &Request, + response: &mut AdditionalBuilder>, + pp_config: PostprocessingConfig, + ) -> Option>> { + // Sign the response. + let mut tsig_signer = pp_config.tsig.lock().unwrap(); + + // Remove the limit we should have imposed during pre-processing so + // that we can use the space we reserved for the OPT RR. + response.clear_push_limit(); + + let signing_result = match tsig_signer.as_mut() { + Some(TsigSigner::Transaction(_)) => { + // Extract the single response signer and consume it in the + // signing process. + let Some(TsigSigner::Transaction(tsig)) = tsig_signer.take() + else { + unreachable!() + }; + trace!( + "Signing single response with TSIG key '{}'", + tsig.key().name() + ); + tsig.answer(response, Time48::now()) + } + + Some(TsigSigner::Sequence(tsig)) => { + // Use the multi-response signer to sign the response. + trace!( + "Signing response stream with TSIG key '{}'", + tsig.key().name() + ); + tsig.answer(response, Time48::now()) + } + + None => { + // Nothing to do as unsigned requests don't require response + // signing. + return None; + } + }; + + // Handle signing failure. This shouldn't happen because we reserve + // space in preprocess() for the TSIG RR that we add when signing. + if signing_result.is_err() { + // 5.3. Generation of TSIG on Answers + // "If addition of the TSIG record will cause the message to be + // truncated, the server MUST alter the response so that a TSIG + // can be included. This response contains only the question and + // a TSIG record, has the TC bit set, and has an RCODE of 0 + // (NOERROR). At this point, the client SHOULD retry the request + // using TCP (as per Section 4.2.2 of [RFC1035])." + + // We can't use the TSIG signer state we just had as that was consumed + // in the failed attempt to sign the answer, so we have to create a new + // TSIG state in order to sign the truncated response. + if request.transport_ctx().is_udp() { + return Self::mk_signed_truncated_response( + request, &pp_config, + ); + } else { + // In the TCP case there's not much we can do. The upstream + // service pushes response messages into the stream and we try + // and sign them. If there isn't enough space to add the TSIG + // signature RR to the response we can't signal the upstream + // to try again to produce a smaller response message as it + // may already have finished pushing into the stream or be + // several messages further on in its processsing. We also + // can't edit the response message content ourselves as we + // know nothing about the content. The only option left to us + // is to try and truncate the TSIG MAC and see if that helps, + // but we don't support that (yet? NSD doesn't support it + // either). + return Some(mk_error_response( + request.message(), + OptRcode::SERVFAIL, + )); + } + } + + None + } + + fn mk_signed_truncated_response( + request: &Request, + pp_config: &PostprocessingConfig, + ) -> Option>> { + let octets = request.message().as_slice().to_vec(); + let mut mut_msg = Message::from_octets(octets).unwrap(); + let res = ServerTransaction::request( + &pp_config.key_store, + &mut mut_msg, + Time48::now(), + ); + + match res { + Ok(None) => { + warn!("Ignoring attempt to create a signed truncated response for an unsigned request."); + None + } + + Ok(Some(tsig)) => { + let builder = mk_builder_for_target(); + let mut new_response = builder + .start_answer(request.message(), Rcode::NOERROR) + .unwrap(); + new_response.header_mut().set_tc(true); + let mut new_response = new_response.additional(); + + if let Err(err) = + tsig.answer(&mut new_response, Time48::now()) + { + error!("Unable to sign truncated TSIG response: {err}"); + Some(mk_error_response( + request.message(), + OptRcode::SERVFAIL, + )) + } else { + Some(new_response) + } + } + + Err(err) => { + error!("Unable to sign truncated TSIG response: {err}"); + Some(mk_error_response(request.message(), OptRcode::SERVFAIL)) + } + } + } + + fn get_relevant_question( + msg: &Message, + ) -> Option>>> { + if Opcode::QUERY == msg.header().opcode() && !msg.header().qr() { + if let Some(q) = msg.first_question() { + if matches!(q.qtype(), Rtype::SOA | Rtype::AXFR | Rtype::IXFR) + { + return Some(q); + } + } + } else if Opcode::NOTIFY == msg.header().opcode() + && !msg.header().qr() + { + if let Some(q) = msg.first_question() { + if matches!(q.qtype(), Rtype::SOA) { + return Some(q); + } + } + } + + None + } + + fn map_stream_item( + request: Request, + stream_item: ServiceResult, + pp_config: PostprocessingConfig, + ) -> ServiceResult { + if let Ok(mut call_res) = stream_item { + if matches!( + call_res.feedback(), + Some(ServiceFeedback::BeginTransaction) + ) { + // Does it need converting from the variant that supports + // single messages only (ServerTransaction) to the variant + // that supports signing multiple messages (ServerSequence)? + // Note: confusingly BeginTransaction and ServerTransaction + // use the term "transaction" to mean completely the opppsite + // of each other. With BeginTransaction we mean that the + // caller should instead a sequence of response messages + // instead of the usual single response message. With + // ServerTransaction the TSIG code means handling of single + // messages only and NOT sequences for which there is a + // separate ServerSequence type. Sigh. + let mut locked_tsig = pp_config.tsig.lock().unwrap(); + let mutable_tsig = locked_tsig.deref_mut(); + if let Some(TsigSigner::Transaction(tsig_txn)) = + mutable_tsig.take() + { + // Do the conversion and store the result for future + // invocations of this function for subsequent items + // in the response stream. + *mutable_tsig = Some(TsigSigner::Sequence( + ServerSequence::from(tsig_txn), + )); + } + } + + if let Some(response) = call_res.response_mut() { + if let Some(new_response) = + Self::postprocess(&request, response, pp_config) + { + *response = new_response; + } + } + + Ok(call_res) + } else { + stream_item + } + } +} + +//--- Service + +// Note: As the TSIG middleware must be the closest middleware to the server, +// it does not receive any special RequestMeta from the server, only (). +impl Service + for TsigMiddlewareSvc +where + RequestOctets: + Octets + OctetsFrom> + Send + Sync + 'static + Unpin, + NextSvc: Service, + NextSvc::Future: Unpin, + NextSvc::Target: Composer + Default, + KS: Clone + KeyStore + Unpin, + KS::Key: Clone + Unpin, + Infallible: From<>>::Error>, +{ + type Target = NextSvc::Target; + type Stream = MiddlewareStream< + NextSvc::Future, + NextSvc::Stream, + PostprocessingStream< + RequestOctets, + NextSvc::Future, + NextSvc::Stream, + (), + PostprocessingConfig, + >, + Once::Item>>, + ::Item, + >; + type Future = Ready; + + fn call(&self, request: Request) -> Self::Future { + match Self::preprocess(&request, &self.key_store) { + ControlFlow::Continue(Some((modified_req, tsig_opt))) => { + let tsig = Arc::new(std::sync::Mutex::new(Some(tsig_opt))); + + let svc_call_fut = self.next_svc.call(modified_req); + + let pp_config = PostprocessingConfig { + tsig, + key_store: self.key_store.clone(), + }; + + let map = PostprocessingStream::new( + svc_call_fut, + request, + pp_config, + Self::map_stream_item, + ); + + ready(MiddlewareStream::Map(map)) + } + + ControlFlow::Continue(None) => { + let request = request.with_new_metadata(Authentication(None)); + let svc_call_fut = self.next_svc.call(request); + ready(MiddlewareStream::IdentityFuture(svc_call_fut)) + } + + ControlFlow::Break(additional) => { + ready(MiddlewareStream::Result(once(ready(Ok( + CallResult::new(additional), + ))))) + } + } + } +} + +#[derive(Clone)] +pub struct PostprocessingConfig +where + KS: KeyStore + Clone, +{ + tsig: Arc::Key>>>>, + key_store: KS, +} + +#[derive(Clone, Debug)] +enum TsigSigner { + /// TODO + Transaction(ServerTransaction), + + /// TODO + Sequence(ServerSequence), +} + +//------------ MaybeAuthenticated --------------------------------------------- + +pub trait MaybeAuthenticated: + Clone + Default + Sync + Send + 'static +{ + fn key_name(&self) -> Option<&KeyName>; +} + +//------------ Authentication -------------------------------------------------- + +#[derive(Clone, Default)] +pub struct Authentication(pub Option); + +impl MaybeAuthenticated for Authentication { + fn key_name(&self) -> Option<&KeyName> { + self.0.as_ref() + } +} diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index 45106ab39..d455f1a85 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -1,5 +1,7 @@ +use core::str::FromStr; + use std::boxed::Box; -use std::collections::VecDeque; +use std::collections::{HashMap, VecDeque}; use std::fs::File; use std::net::SocketAddr; use std::path::PathBuf; @@ -8,6 +10,7 @@ use std::sync::Arc; use std::time::Duration; use std::vec::Vec; +use ring::test::rand::FixedByteRandom; use rstest::rstest; use tracing::instrument; use tracing::{trace, warn}; @@ -18,7 +21,10 @@ use crate::base::net::IpAddr; use crate::base::wire::Composer; use crate::base::Rtype; use crate::net::client::request::{RequestMessage, RequestMessageMulti}; -use crate::net::client::{dgram, stream}; +use crate::net::client::tsig::{ + AuthenticatedRequestMessage, AuthenticatedRequestMessageMulti, +}; +use crate::net::client::{dgram, stream, tsig}; use crate::net::server; use crate::net::server::buf::VecBufSource; use crate::net::server::dgram::DgramServer; @@ -36,6 +42,7 @@ use crate::stelline::client::{ }; use crate::stelline::parse_stelline::{self, parse_file, Config, Matches}; use crate::stelline::simple_dgram_client; +use crate::tsig::{Algorithm, Key, KeyName, KeyStore}; use crate::utils::base16; use crate::zonefile::inplace::{Entry, ScannedRecord, Zonefile}; @@ -59,6 +66,8 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { // Initialize tracing based logging. Override with env var RUST_LOG, e.g. // RUST_LOG=trace. DEBUG level will show the .rpl file name, Stelline step // numbers and types as they are being executed. + + use crate::net::server::middleware::tsig::TsigMiddlewareSvc; tracing_subscriber::fmt() .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) .with_thread_ids(true) @@ -66,11 +75,27 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { .try_init() .ok(); + // Load the test .rpl file that determines which queries will be sent + // and which responses will be expected, and how the server that + // answers them should be configured. let file = File::open(&rpl_file).unwrap(); let stelline = parse_file(&file, rpl_file.to_str().unwrap()); let server_config = parse_server_config(&stelline.config); - // Create a service to answer queries received by the DNS servers. + // Create a TSIG key store containing a 'TESTKEY' + let mut key_store = TestKeyStore::new(); + let key_name = KeyName::from_str("TESTKEY").unwrap(); + let rng = FixedByteRandom { byte: 0u8 }; + let (key, _) = + Key::generate(Algorithm::Sha256, &rng, key_name.clone(), None, None) + .unwrap(); + key_store.insert((key_name, Algorithm::Sha256), key); + let key_store = Arc::new(key_store); + + // Create a connection factory. + let dgram_server_conn = ClientServerChannel::new_dgram(); + let stream_server_conn = ClientServerChannel::new_stream(); + let zonefile = server_config.zonefile.clone(); let with_cookies = server_config.cookies.enabled @@ -105,12 +130,25 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { // 4. Mandatory DNS behaviour (e.g. RFC 1034/35 rules). let svc = MandatoryMiddlewareSvc::new(svc); + // 5. TSIG message authentication. + let svc = TsigMiddlewareSvc::new(svc, key_store.clone()); + + // NOTE: TSIG middleware *MUST* be the first middleware in the chain per + // RFC 8945 as it has to see incoming messages prior to any modification + // in order to verify the signature, and has to sign outgoing messages in + // their final state without any modification occuring thereafter. + // Create dgram and stream servers for answering requests - let (dgram_srv, dgram_conn, stream_srv, stream_conn) = - mk_servers(svc, &server_config); + let (dgram_srv, stream_srv) = mk_servers( + svc, + &server_config, + dgram_server_conn.clone(), + stream_server_conn.clone(), + ); // Create a client factory for sending requests - let client_factory = mk_client_factory(dgram_conn, stream_conn); + let client_factory = + mk_client_factory(dgram_server_conn, stream_server_conn, key_store); // Run the Stelline test! let step_value = Arc::new(CurrStepValue::new()); @@ -132,11 +170,11 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { fn mk_servers( service: Svc, server_config: &ServerConfig, + dgram_server_conn: ClientServerChannel, + stream_server_conn: ClientServerChannel, ) -> ( Arc>, - ClientServerChannel, Arc>, - ClientServerChannel, ) where Svc: Clone + Service + Send + Sync, @@ -149,8 +187,7 @@ where let (dgram_config, stream_config) = mk_server_configs(server_config); // Create a dgram server for handling UDP requests. - let dgram_server_conn = ClientServerChannel::new_dgram(); - let dgram_server = DgramServer::with_config( + let dgram_server = DgramServer::<_, _, Svc>::with_config( dgram_server_conn.clone(), VecBufSource, service.clone(), @@ -162,7 +199,6 @@ where // Create a stream server for handling TCP requests, i.e. Stelline queries // with "MATCH TCP". - let stream_server_conn = ClientServerChannel::new_stream(); let stream_server = StreamServer::with_config( stream_server_conn.clone(), VecBufSource, @@ -173,17 +209,13 @@ where let cloned_stream_server = stream_server.clone(); tokio::spawn(async move { cloned_stream_server.run().await }); - ( - dgram_server, - dgram_server_conn, - stream_server, - stream_server_conn, - ) + (dgram_server, stream_server) } fn mk_client_factory( dgram_server_conn: ClientServerChannel, stream_server_conn: ClientServerChannel, + key_store: Arc, ) -> impl ClientFactory { // Create a TCP client factory that only creates a client if (a) no // existing TCP client exists for the source address of the Stelline @@ -193,23 +225,31 @@ fn mk_client_factory( matches!(entry.matches, Some(Matches { tcp: true, .. })) }; + let tcp_key_store = key_store.clone(); let tcp_client_factory = PerClientAddressClientFactory::new( move |source_addr, entry| { + let key = entry.key_name.as_ref().and_then(|key_name| { + tcp_key_store.get_key(&key_name, Algorithm::Sha256) + }); let stream = stream_server_conn .connect(Some(SocketAddr::new(*source_addr, 0))); let (conn, transport) = stream::Connection::< - RequestMessage>, - RequestMessageMulti>, + AuthenticatedRequestMessage>, Key>, + AuthenticatedRequestMessageMulti< + RequestMessageMulti>, + Key, + >, >::new(stream); tokio::spawn(transport.run()); + let conn = Box::new(tsig::Connection::new(key, conn)); if let Some(sections) = &entry.sections { if let Some(q) = sections.question.first() { if matches!(q.qtype(), Rtype::AXFR | Rtype::IXFR) { - return Client::Multi(Box::new(conn)); + return Client::Multi(conn); } } } - Client::Single(Box::new(conn)) + Client::Single(conn) }, only_for_tcp_queries, ); @@ -221,16 +261,24 @@ fn mk_client_factory( let udp_client_factory = PerClientAddressClientFactory::new( move |source_addr, entry| { + let key = entry.key_name.as_ref().and_then(|key_name| { + key_store.get_key(&key_name, Algorithm::Sha256) + }); let connect = dgram_server_conn .new_client(Some(SocketAddr::new(*source_addr, 0))); match entry.matches.as_ref().map(|v| v.mock_client) { - Some(true) => Client::Single(Box::new( - simple_dgram_client::Connection::new(connect), - )), - _ => { - Client::Single(Box::new(dgram::Connection::new(connect))) + Some(true) => { + Client::Single(Box::new(tsig::Connection::new( + key, + simple_dgram_client::Connection::new(connect), + ))) } + + _ => Client::Single(Box::new(tsig::Connection::new( + key, + dgram::Connection::new(connect), + ))), } }, for_all_other_queries, @@ -274,8 +322,8 @@ fn mk_server_configs( // - Controlling the content of the `Zonefile` passed to instances of this // `Service` impl. #[allow(clippy::type_complexity)] -fn test_service( - request: Request>, +fn test_service( + request: Request, RequestMeta>, zonefile: Zonefile, ) -> ServiceResult> { fn as_record_and_dname( @@ -440,3 +488,8 @@ fn parse_server_config(config: &Config) -> ServerConfig { parsed_config } + +//------------ TestKeyStore --------------------------------------------------- + +// KeyStore is impl'd elsewhere for HashMap<(KeyName, Algorithm), K, S>. +type TestKeyStore = HashMap<(KeyName, Algorithm), Key>; diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index 89e7dd8c4..e0f48b44b 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -54,8 +54,6 @@ #![cfg(feature = "tsig")] #![cfg_attr(docsrs, doc(cfg(feature = "tsig")))] -mod interop; - use crate::base::header::HeaderSection; use crate::base::iana::{Class, Rcode, TsigRcode}; use crate::base::message::Message; @@ -67,6 +65,7 @@ use crate::base::record::Record; use crate::base::wire::{Composer, ParseError}; use crate::rdata::tsig::{Time48, Tsig}; use bytes::{Bytes, BytesMut}; +use core::fmt::Display; use core::{cmp, fmt, mem, str}; use octseq::octets::Octets; use ring::{constant_time, hkdf::KeyType, hmac, rand}; @@ -103,7 +102,7 @@ pub type KeyName = Name>; /// [`new`]: #method.new /// [`min_mac_len`]: #method.min_mac_len /// [`signing_len`]: #method.signing_len -#[derive(Debug)] +#[derive(Clone, Debug)] pub struct Key { /// The key’s bits and algorithm. key: hmac::Key, @@ -261,6 +260,70 @@ impl Key { self.signing_len } + /// Returns the space needed for a TSIG RR for this key. + pub fn compose_len(&self) -> u16 { + // The length of the generated TSIG RR is governed by variable and + // fixed parts. The variable parts are the key name, the algorithm + // name and the MAC. Given the key we can work this out. + + // https://datatracker.ietf.org/doc/html/rfc8945#section-4.2 + // 4.2. TSIG Record Format + // "The fields of the TSIG RR are described below. All multi-octet + // integers in the record are sent in network byte order (see + // Section 2.3.2 of [RFC1035]). + // + // Field Description RFC 1035 Size + // ----------------------------------------------------------------- + // NAME: The name of the key used, in domain name (variable) + // syntax. [...] + // TYPE: This MUST be TSIG (250: Transaction two octets + // SIGnature). + // CLASS: This MUST be ANY. two octets + // TTL: This MUST be 0. 32-bit + // RDLENGTH: (variable) 16-bit + // RDATA: The RDATA for a TSIG RR consists of a (variable) + // number of fields, described below: + // + // 1 1 1 1 1 1 1 1 1 1 2 2 2 2 2 2 2 2 2 2 3 3 + // 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 + // +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + // / Algorithm Name / + // +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + // | | + // | Time Signed +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + // | | Fudge | + // +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + // | MAC Size | / + // +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ MAC / + // / / + // +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + // | Original ID | Error | + // +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + // | Other Len | / + // +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ Other Data / + // / / + // +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + // " + let rdata_len = self.algorithm().to_name().compose_len() // Algorithm Name + + 6 // Time Signed + + 2 // Fudge + + 2 // MAC Size + + self.signing_len() as u16 + + 2 // Original ID + + 2 // Error + + 2; // Other Len + //+ 0; // Other Data (assume a successful response) + + let rr_len = self.name().compose_len() + + 2 // TYPE + + 2 // CLASS + + 4 // TTL + + 2 // RDLENGTH + + rdata_len; + + rr_len + } + /// Checks whether the key in the record is this key. fn check_tsig( &self, @@ -305,7 +368,7 @@ impl Key { /// if that is required. /// /// The method fails if the TSIG record doesn’t fit into the message - /// anymore, in which case the builder is returned unharmed. + /// anymore, in which case the builder is left unmodified. fn complete_message( &self, message: &mut AdditionalBuilder, @@ -325,6 +388,15 @@ impl AsRef for Key { } } +//--- Display + +#[cfg(feature = "std")] +impl Display for Key { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.write_fmt(format_args!("{}", self.name)) + } +} + //------------ KeyStore ------------------------------------------------------ /// A type that stores TSIG secret keys. @@ -359,7 +431,7 @@ pub trait KeyStore { ) -> Option; } -impl + Clone> KeyStore for K { +impl KeyStore for Key { type Key = Self; fn get_key( @@ -377,6 +449,24 @@ impl + Clone> KeyStore for K { } } +impl KeyStore for &'_ Key { + type Key = Self; + + fn get_key( + &self, + name: &N, + algorithm: Algorithm, + ) -> Option { + if self.as_ref().name() == name + && self.as_ref().algorithm() == algorithm + { + Some(self) + } else { + None + } + } +} + #[cfg(feature = "std")] impl KeyStore for HashMap<(KeyName, Algorithm), K, S> where @@ -396,9 +486,26 @@ where } } +#[cfg(feature = "std")] +impl KeyStore for std::sync::Arc +where + K: AsRef + Clone, + U: KeyStore, +{ + type Key = K; + + fn get_key( + &self, + name: &N, + algorithm: Algorithm, + ) -> Option { + (**self).get_key(name, algorithm) + } +} + //------------ ClientTransaction --------------------------------------------- -/// TSIG Client Transaction State. +/// TSIG Client transaction state. /// /// This types allows signing a DNS request with a given key and validate an /// answer received for it. @@ -425,14 +532,15 @@ impl> ClientTransaction { /// builder and a key. It signs the message with the key and adds the /// signature as a TSIG record to the message’s additional section. It /// also creates a transaction value that can later be used to validate - /// the response. It returns both the message and the transaction. + /// the response. It modifies the given message and returns the created + /// transaction. /// - /// The function can fail if the TSIG record doesn’t actually fit into - /// the message anymore. In this case, the function returns an error and - /// the untouched message. + /// The function can fail if the TSIG record doesn’t fit into the message. + /// In this case, the function returns an error and leave the given + /// message unmodified. /// - /// Unlike [`request_with_fudge`], this function uses the - /// recommended default value for _fudge:_ 300 seconds. + /// Unlike [`request_with_fudge`], this function uses the recommended + /// default value for _fudge:_ 300 seconds. /// /// [`request_with_fudge`]: #method.request_with_fudge pub fn request( @@ -586,8 +694,7 @@ impl> ServerTransaction { /// TSIG record must be the last record and returns it. /// /// If appending the TSIG record fails, which can only happen if there - /// isn’t enough space left, it returns the builder unchanged as the - /// error case. + /// isn’t enough space left, it returns an error. pub fn answer( self, message: &mut AdditionalBuilder, @@ -598,7 +705,7 @@ impl> ServerTransaction { /// Produces a signed answer with a given fudge. /// - /// This method is similar to [`answer`] but lets you explicitely state + /// This method is similar to [`answer`] but lets you explicitly state /// the `fudge`, i.e., the number of seconds the recipient’s clock is /// allowed to differ from your current time when checking the signature. /// The default, suggested by the RFC, is 300. @@ -954,6 +1061,25 @@ impl> ServerSequence { } } +//--- From + +/// Convert an unused [`ServerTransaction`] to a [`ServerSequence`] +/// +/// If [`ServerTransaction::request()`] was used to verify a TSIG signed +/// request but then you need to sign multiple responses, +/// [`ServerTransaction`] will not suffice as it can only sign a single +/// response. To resolve this you can use this function to convert the +/// [`ServerTransaction`] to a [`ServerSequence`] which can be used to sign +/// multiple responses. +impl From> for ServerSequence { + fn from(txn: ServerTransaction) -> Self { + Self { + context: txn.context, + first: true, + } + } +} + //------------ SigningContext ------------------------------------------------ /// A TSIG signing context. @@ -1404,8 +1530,8 @@ impl Variables { Class::ANY, 0, // The only reason creating TSIG record data can fail here is - // that the hmac is unreasonable large. Since we control its - // creation, panicing in this case is fine. + // that the hmac is unreasonably large. Since we control its + // creation, panicking in this case is fine. Tsig::new( key.algorithm().to_name(), self.time_signed, @@ -1619,7 +1745,7 @@ enum ServerErrorInner { } impl ServerError { - fn unsigned(error: TsigRcode) -> Self { + pub fn unsigned(error: TsigRcode) -> Self { ServerError(ServerErrorInner::Unsigned { error }) } diff --git a/test-data/server/tsig.rpl b/test-data/server/tsig.rpl new file mode 100644 index 000000000..7adaf1284 --- /dev/null +++ b/test-data/server/tsig.rpl @@ -0,0 +1,138 @@ +;------------ Server configuration -------------------------------------------- + +server: + ; Permit the server to respond to clients that send XFR requests from + ; 127.0.0.1 (the default). No TSIG signing is expected. + provide-xfr: 127.0.0.1 NOKEY + + ; Permit the server to respond to clients that send XFR requests from + ; 127.0.0.2. The requests must be TSIG signed using the specified key. + provide-xfr: 127.0.0.2 TESTKEY + + ; Permit the server to respond to clients that send XFR requests from + ; 127.0.0.3. The requests must be TSIG signed using the specified key. + provide-xfr: 127.0.0.3 TESTKEY + + ; Permit the server to respond to clients that send XFR requests from + ; 127.0.0.5. The requests must be TSIG signed using the specified key. + ; Responses will be sent in "backward compatible" mode as defined by + ; RFC 5936. + ; See: https://datatracker.ietf.org/doc/html/rfc5936#section-7.1. + provide-xfr: 127.0.0.4 TESTKEY COMPATIBLE + + ; Permit the server to respond to clients that send XFR requests from + ; 127.0.0.4. The requests must be TSIG signed using the specified key. + provide-xfr: 127.0.0.5 TESTKEY + + ; Define an in-memory zone to be served by the server. + local-data: "example.com. 3600 IN SOA ns.example.com. hostmaster.example.com. 1 3600 900 86400 3600" + local-data: "example.com. 3600 IN NS ns.example.net." + local-data: "www.example.com. 3600 IN A 1.2.3.4" +CONFIG_END + +;------------ Test definition ------------------------------------------------ + +SCENARIO_BEGIN Test TSIG signed queries and TSIG signed AXFR replies. + +;--- Mock replies + +; None + +;--- Test steps + +; Verify that when a client that is required to TSIG sign requests (127.0.0.2) +; submits an unsigned AXFR request over TCP that it is rejected (because while +; AXFR over TCP is allowed, the request is required to be correctly signed). +STEP 20 QUERY ADDRESS 127.0.0.2 +ENTRY_BEGIN +MATCH TCP +SECTION QUESTION + example.com. IN AXFR +ENTRY_END + +STEP 21 CHECK_ANSWER +ENTRY_BEGIN +MATCH all +REPLY QR AA REFUSED +ENTRY_END + +; Retrieve the zone via AXFR over TCP using the expected TSIG key. +STEP 30 QUERY ADDRESS 127.0.0.3 KEY TESTKEY +ENTRY_BEGIN +MATCH TCP +SECTION QUESTION + example.com. IN AXFR +ENTRY_END + +STEP 31 CHECK_ANSWER +ENTRY_BEGIN +MATCH all +REPLY QR AA NOERROR +SECTION QUESTION + example.com. IN AXFR +SECTION ANSWER + example.com. 3600 IN SOA ns.example.com. hostmaster.example.com. 1 3600 900 86400 3600 + example.com. 3600 IN NS ns.example.net. + www.example.com. 3600 IN A 1.2.3.4 + example.com. 3600 IN SOA ns.example.com. hostmaster.example.com. 1 3600 900 86400 3600 +ENTRY_END + +; Retrieve the zone using a client for which the server is configured to serve +; AXFR in backward compatible mode so that we can test multi-response TSIG. +STEP 40 QUERY ADDRESS 127.0.0.4 KEY TESTKEY +ENTRY_BEGIN +MATCH TCP +SECTION QUESTION + example.com. IN AXFR +ENTRY_END + +STEP 41 CHECK_ANSWER +ENTRY_BEGIN +MATCH all EXTRA_PACKETS +REPLY QR AA NOERROR +SECTION QUESTION + example.com. IN AXFR +SECTION ANSWER + example.com. 3600 IN SOA ns.example.com. hostmaster.example.com. 1 3600 900 86400 3600 + www.example.com. 3600 IN A 1.2.3.4 + example.com. 3600 IN NS ns.example.net. + example.com. 3600 IN SOA ns.example.com. hostmaster.example.com. 1 3600 900 86400 3600 +ENTRY_END + +; NOTE: See test-data/server/README.md regarding the effect of MOCK_CLIENT +; that is used here. +STEP 50 QUERY ADDRESS 127.0.0.5 +ENTRY_BEGIN +MATCH UDP +MATCH MOCK_CLIENT +SECTION QUESTION + example.com. IN SOA +SECTION ADDITIONAL +; Stelline doesn't support parsing zone entries that use the ( multiline ) +; format, otherwise we could use here. + TESTKEY 0 CLASS255 TYPE250 \# 61 0b686d61632d73686132353600 000000000000 012c 0020 a1c86ced1815d60903129a525a14494516895d99ea94bf0b5b04338126a4d625 0000 0000 0000 +; ^ Other Len +; ^ Error +; ^ Original ID +; ^ MAC +; ^ MAC Size +; ^ Fudge +; ^ Algorithm Name ^ Time Signed +; ^ RDATA byte length. +; ^ RFC 3597 CLASSNN TYPENN \# encoding of unknown DNS RR types. +; We use this so that we can define the RDATA using HEX bytes. +ENTRY_END + +STEP 51 CHECK_ANSWER +ENTRY_BEGIN +MATCH all +REPLY QR NOERROR +SECTION QUESTION + example.com. IN SOA +SECTION ANSWER + example.com. 3600 IN SOA ns.example.com. hostmaster.example.com. 1 3600 900 86400 3600 +SECTION ADDITIONAL + TESTKEY 0 CLASS255 TYPE250 \# 61 0b686d61632d73686132353600 000000000000 012c 0020 6d7f9c5a14c2b48d4a0549000af29808e5eb25f7a80c22a2b1c0cf2ef3929bcd 0000 0000 0000 +ENTRY_END + +SCENARIO_END \ No newline at end of file From 7c3bbbf4c96208be4ac856974446eaec9a0b8019 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 3 Sep 2024 23:36:49 +0200 Subject: [PATCH 095/333] Remove unnecessary Clone bound. --- src/net/client/dgram.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/client/dgram.rs b/src/net/client/dgram.rs index 6695a5492..e771fc372 100644 --- a/src/net/client/dgram.rs +++ b/src/net/client/dgram.rs @@ -322,7 +322,7 @@ where S: AsyncConnect + Clone + Send + Sync + 'static, S::Connection: AsyncDgramRecv + AsyncDgramSend + Send + Sync + Unpin + 'static, - Req: ComposeRequest + Clone + Send + Sync + 'static, + Req: ComposeRequest + Send + Sync + 'static, { fn send_request( &self, From 55ce15519bd40e5b296dc19b290ec15b78773777 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:16:53 +0200 Subject: [PATCH 096/333] Replace incorrect conversion (as it will return Error::FormError for XFR query types) by explicit RequestMessageMulti support. --- src/net/client/request.rs | 14 ------ src/stelline/client.rs | 92 +++++++++++++++++++++++++++------------ 2 files changed, 63 insertions(+), 43 deletions(-) diff --git a/src/net/client/request.rs b/src/net/client/request.rs index abff22486..e4c2731b3 100644 --- a/src/net/client/request.rs +++ b/src/net/client/request.rs @@ -620,20 +620,6 @@ impl + Debug + Octets + Send + Sync> ComposeRequestMulti } } -//--- From - -impl> From> - for RequestMessageMulti -{ - fn from(req: RequestMessage) -> Self { - RequestMessageMulti { - msg: req.msg, - header: req.header, - opt: req.opt, - } - } -} - //------------ Error --------------------------------------------------------- /// Error type for client transports. diff --git a/src/stelline/client.rs b/src/stelline/client.rs index f5c91a116..6fbd5b9bd 100644 --- a/src/stelline/client.rs +++ b/src/stelline/client.rs @@ -19,14 +19,15 @@ use crate::base::iana::{Opcode, OptionCode}; use crate::base::opt::{ComposeOptData, OptData}; use crate::base::{Message, MessageBuilder}; use crate::net::client::request::{ - ComposeRequest, Error, GetResponse, GetResponseMulti, RequestMessage, - RequestMessageMulti, SendRequest, SendRequestMulti, + ComposeRequest, ComposeRequestMulti, Error, GetResponse, + GetResponseMulti, RequestMessage, RequestMessageMulti, SendRequest, + SendRequestMulti, }; use crate::stelline::matches::match_multi_msg; use crate::zonefile::inplace::Entry::Record; use super::matches::match_msg; -use super::parse_stelline::{Entry, Reply, Stelline, StepType}; +use super::parse_stelline::{Entry, Reply, Sections, Stelline, StepType}; use super::channel::DEF_CLIENT_ADDR; use core::ops::Deref; @@ -236,15 +237,18 @@ impl Dispatcher { entry: &Entry, ) -> Result { if let Some(client) = &self.0 { - let reqmsg = entry2reqmsg(entry); - trace!(?reqmsg); let res = match client.deref() { Client::Single(client) => { + let reqmsg = entry2reqmsg(entry); + trace!(?reqmsg); Response::Single(client.send_request(reqmsg)) } - Client::Multi(client) => Response::Multi( - client.send_request(RequestMessageMulti::from(reqmsg)), - ), + + Client::Multi(client) => { + let reqmsg = entry2reqmsg_multi(entry); + trace!(?reqmsg); + Response::Multi(client.send_request(reqmsg)) + } }; return Ok(res); } @@ -679,6 +683,56 @@ fn init_logging() { } fn entry2reqmsg(entry: &Entry) -> RequestMessage> { + let (sections, reply, msg) = entry2msg(entry); + + let mut reqmsg = RequestMessage::new(msg).unwrap(); + if !entry + .matches + .as_ref() + .map(|v| v.mock_client) + .unwrap_or_default() + { + reqmsg.set_dnssec_ok(reply.fl_do); + } + if reply.notify { + reqmsg.header_mut().set_opcode(Opcode::NOTIFY); + } + + let edns_bytes = §ions.additional.edns_bytes; + if !edns_bytes.is_empty() { + let raw_opt = RawOptData { bytes: edns_bytes }; + reqmsg.add_opt(&raw_opt).unwrap(); + } + + reqmsg +} + +fn entry2reqmsg_multi(entry: &Entry) -> RequestMessageMulti> { + let (sections, reply, msg) = entry2msg(entry); + + let mut reqmsg = RequestMessageMulti::new(msg).unwrap(); + if !entry + .matches + .as_ref() + .map(|v| v.mock_client) + .unwrap_or_default() + { + reqmsg.set_dnssec_ok(reply.fl_do); + } + if reply.notify { + reqmsg.header_mut().set_opcode(Opcode::NOTIFY); + } + + let edns_bytes = §ions.additional.edns_bytes; + if !edns_bytes.is_empty() { + let raw_opt = RawOptData { bytes: edns_bytes }; + reqmsg.add_opt(&raw_opt).unwrap(); + } + + reqmsg +} + +fn entry2msg(entry: &Entry) -> (&Sections, Reply, Message>) { let sections = entry.sections.as_ref().unwrap(); let mut msg = MessageBuilder::new_vec().question(); if let Some(opcode) = entry.opcode { @@ -712,27 +766,7 @@ fn entry2reqmsg(entry: &Entry) -> RequestMessage> { header.set_ad(reply.ad); header.set_cd(reply.cd); let msg = msg.into_message(); - - let mut reqmsg = RequestMessage::new(msg).unwrap(); - if !entry - .matches - .as_ref() - .map(|v| v.mock_client) - .unwrap_or_default() - { - reqmsg.set_dnssec_ok(reply.fl_do); - } - if reply.notify { - reqmsg.header_mut().set_opcode(Opcode::NOTIFY); - } - - let edns_bytes = §ions.additional.edns_bytes; - if !edns_bytes.is_empty() { - let raw_opt = RawOptData { bytes: edns_bytes }; - reqmsg.add_opt(&raw_opt).unwrap(); - } - - reqmsg + (sections, reply, msg) } #[derive(Debug)] From cba3daf89fd7594823396b79609c5199cb22fc53 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:18:01 +0200 Subject: [PATCH 097/333] Remove errant whitespace in RustDoc comments. --- src/net/client/stream.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index dcff40bc3..b213f736f 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -122,7 +122,7 @@ impl Config { /// Sets the response timeout. /// /// For requests where ComposeRequest::is_streaming() returns true see - /// set_streaming_response_timeout() instead. + /// set_streaming_response_timeout() instead. /// /// Excessive values are quietly trimmed. // @@ -158,7 +158,7 @@ impl Config { /// /// By default the stream is immediately closed if there are no pending /// requests or responses. - /// + /// /// Set this to allow requests to be sent in sequence with delays between /// such as a SOA query followed by AXFR for more efficient use of the /// stream per RFC 9103. From 113fabd639f337efc86b282285057985213990bc Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:22:39 +0200 Subject: [PATCH 098/333] Undo reformatting of imports in otherwise untouched files. --- src/net/client/request.rs | 22 ++++++++++------------ src/net/client/stream.rs | 34 ++++++++++++++-------------------- 2 files changed, 24 insertions(+), 32 deletions(-) diff --git a/src/net/client/request.rs b/src/net/client/request.rs index e4c2731b3..7ab82020c 100644 --- a/src/net/client/request.rs +++ b/src/net/client/request.rs @@ -1,16 +1,4 @@ //! Constructing and sending requests. -use std::boxed::Box; -use std::fmt::Debug; -use std::future::Future; -use std::pin::Pin; -use std::sync::Arc; -use std::vec::Vec; -use std::{error, fmt}; - -use bytes::Bytes; -use octseq::Octets; -use tracing::trace; - use crate::base::iana::{Opcode, Rcode}; use crate::base::message::{CopyRecordsError, ShortMessage}; use crate::base::message_builder::{ @@ -20,6 +8,16 @@ use crate::base::opt::{ComposeOptData, LongOptData, OptRecord}; use crate::base::wire::{Composer, ParseError}; use crate::base::{Header, Message, ParsedName, Rtype, StaticCompressor}; use crate::rdata::AllRecordData; +use bytes::Bytes; +use octseq::Octets; +use std::boxed::Box; +use std::fmt::Debug; +use std::future::Future; +use std::pin::Pin; +use std::sync::Arc; +use std::vec::Vec; +use std::{error, fmt}; +use tracing::trace; #[cfg(feature = "tsig")] use crate::tsig; diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index b213f736f..f6d8a8c1b 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -13,40 +13,34 @@ // - timeouts // - request timeout // - create new connection after end/failure of previous one - +use super::request::{ + ComposeRequest, ComposeRequestMulti, Error, GetResponse, + GetResponseMulti, SendRequest, SendRequestMulti, +}; +use crate::base::iana::{Rcode, Rtype}; +use crate::base::message::Message; +use crate::base::message_builder::StreamTarget; +use crate::base::opt::{AllOptData, OptRecord, TcpKeepalive}; +use crate::base::{ParsedName, Serial}; +use crate::rdata::AllRecordData; +use crate::utils::config::DefMinMax; +use bytes::{Bytes, BytesMut}; use core::cmp; -use core::future::ready; +use octseq::Octets; use std::boxed::Box; use std::fmt::Debug; +use std::future::ready; use std::future::Future; use std::pin::Pin; use std::sync::Arc; use std::time::{Duration, Instant}; use std::vec::Vec; - -use bytes::{Bytes, BytesMut}; -use octseq::Octets; use tokio::io::{AsyncRead, AsyncReadExt, AsyncWrite, AsyncWriteExt}; use tokio::sync::{mpsc, oneshot}; use tokio::time::sleep; use tracing::trace; -use crate::base::iana::Rcode; -use crate::base::message::Message; -use crate::base::message_builder::StreamTarget; -use crate::base::opt::{AllOptData, OptRecord, TcpKeepalive}; -use crate::base::{ParsedName, Rtype, Serial}; -use crate::net::client::request::{ - ComposeRequest, Error, GetResponse, SendRequest, -}; -use crate::rdata::AllRecordData; -use crate::utils::config::DefMinMax; - -use super::request::{ - ComposeRequestMulti, GetResponseMulti, SendRequestMulti, -}; - //------------ Configuration Constants ---------------------------------------- /// Default response timeout. From 0a1c866401af3aa184353b0b58d5b3586d88c82b Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:23:09 +0200 Subject: [PATCH 099/333] Undo reformatting in otherwise untouched file. --- src/net/client/stream.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index f6d8a8c1b..b216e7fc6 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -27,7 +27,6 @@ use crate::utils::config::DefMinMax; use bytes::{Bytes, BytesMut}; use core::cmp; use octseq::Octets; - use std::boxed::Box; use std::fmt::Debug; use std::future::ready; From 5720e2bf670253d4695aab09135b8a7065c4b7ec Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:23:29 +0200 Subject: [PATCH 100/333] Undo reformatting in otherwise untouched file. --- src/net/client/stream.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index b216e7fc6..19714babd 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -13,6 +13,7 @@ // - timeouts // - request timeout // - create new connection after end/failure of previous one + use super::request::{ ComposeRequest, ComposeRequestMulti, Error, GetResponse, GetResponseMulti, SendRequest, SendRequestMulti, From 31a083c47c1fbdbe8f09727839b92aa85d267d34 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 2 Sep 2024 23:10:49 +0200 Subject: [PATCH 101/333] Add RustDoc for the multiple response support. --- src/net/client/mod.rs | 41 ++++++++++++++++++++++++++++++++++++++++- 1 file changed, 40 insertions(+), 1 deletion(-) diff --git a/src/net/client/mod.rs b/src/net/client/mod.rs index 91dcfed02..7d868f204 100644 --- a/src/net/client/mod.rs +++ b/src/net/client/mod.rs @@ -38,7 +38,7 @@ //! 1) Creating a request message, //! 2) Creating a DNS transport, //! 3) Sending the request, and -//! 4) Receiving the reply. +//! 4) Receiving the reply or replies. //! //! The first and second step are independent and can happen in any order. //! The third step uses the resuts of the first and second step. @@ -150,6 +150,45 @@ //! let reply = request.get_response().await; //! # } //! ``` +//! +//!
+//! +//! **Support for multiple responses:** +//! +//! [RequestMessage][request::RequestMessage] is designed for the most common +//! use case: single request, single response. +//! +//! However, zone transfers (e.g. using the `AXFR` or `IXFR` query types) can +//! result in multiple responses. Attempting to create a +//! [RequestMessage][request::RequestMessage] for such a query will result in +//! [Error::FormError][request::Error::FormError]. +//! +//! For zone transfers you should use +//! [RequestMessageMulti][request::RequestMessageMulti] instead which can be +//! used like so: +//! +//! ```no_run +//! # use crate::domain::net::client::request::{RequestMessageMulti, SendRequest}; +//! # use std::net::{IpAddr, SocketAddr}; +//! # use std::str::FromStr; +//! # async fn _test() { +//! # let (conn, _) = domain::net::client::stream::Connection::<_, RequestMessageMulti>>::new( +//! # domain::net::client::protocol::TcpConnect::new( +//! # SocketAddr::new(IpAddr::from_str("::1").unwrap(), 53) +//! # ) +//! # ); +//! # let req = domain::net::client::request::RequestMessageMulti::new( +//! # domain::base::MessageBuilder::new_vec() +//! # ).unwrap(); +//! # let mut request = tls_conn.send_request(req); +//! while let Some(reply) = request.get_response().await { +//! // ... +//! } +//! # } +//! ``` +//! +//!
+//! //! # Limitations //! From d2c0d5125e77728c98ccbbd56d03831c86428cf9 Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Tue, 3 Sep 2024 15:59:05 +0200 Subject: [PATCH 102/333] Fix example. --- src/net/client/mod.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/net/client/mod.rs b/src/net/client/mod.rs index 7d868f204..da48b8206 100644 --- a/src/net/client/mod.rs +++ b/src/net/client/mod.rs @@ -168,11 +168,11 @@ //! used like so: //! //! ```no_run -//! # use crate::domain::net::client::request::{RequestMessageMulti, SendRequest}; +//! # use crate::domain::net::client::request::{RequestMessage, SendRequestMulti}; //! # use std::net::{IpAddr, SocketAddr}; //! # use std::str::FromStr; //! # async fn _test() { -//! # let (conn, _) = domain::net::client::stream::Connection::<_, RequestMessageMulti>>::new( +//! # let (conn, _) = domain::net::client::stream::Connection::>, _>::new( //! # domain::net::client::protocol::TcpConnect::new( //! # SocketAddr::new(IpAddr::from_str("::1").unwrap(), 53) //! # ) @@ -180,8 +180,8 @@ //! # let req = domain::net::client::request::RequestMessageMulti::new( //! # domain::base::MessageBuilder::new_vec() //! # ).unwrap(); -//! # let mut request = tls_conn.send_request(req); -//! while let Some(reply) = request.get_response().await { +//! # let mut request = conn.send_request(req); +//! while let Ok(reply) = request.get_response().await { //! // ... //! } //! # } From 04533e7cf2bb82b7e3f8c8f76d64251209372968 Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Tue, 3 Sep 2024 16:09:13 +0200 Subject: [PATCH 103/333] Rename Message::is_stream to Message::is_xfr. Remove is_stream from ComposeRequest and ComposeRequestMulti. --- src/base/message.rs | 11 +++-------- src/net/client/request.rs | 16 +--------------- src/net/client/stream.rs | 30 ++++++++---------------------- 3 files changed, 12 insertions(+), 45 deletions(-) diff --git a/src/base/message.rs b/src/base/message.rs index 0dfac4d9c..7f52ed42c 100644 --- a/src/base/message.rs +++ b/src/base/message.rs @@ -439,14 +439,9 @@ impl Message { } } - /// Could this message result in a stream of responses? - /// - /// Most DNS queries result in a single response, but some (only AXFR and - /// IXFR at the time of writing) can result in a stream of responses. - /// - /// Returns true if the first question is of a type that might result in a - /// stream of responses. - pub fn is_streaming(&self) -> bool { + /// Returns whether the message has a question that is either AXFR or + /// IXFR. + pub fn is_xfr(&self) -> bool { self.first_question() .map(|q| matches!(q.qtype(), Rtype::AXFR | Rtype::IXFR)) .unwrap_or_default() diff --git a/src/net/client/request.rs b/src/net/client/request.rs index 7ab82020c..9eb67b963 100644 --- a/src/net/client/request.rs +++ b/src/net/client/request.rs @@ -60,9 +60,6 @@ pub trait ComposeRequest: Debug + Send + Sync { /// Returns whether a message is an answer to the request. fn is_answer(&self, answer: &Message<[u8]>) -> bool; - /// Returns whether a message results in a response stream or not. - fn is_streaming(&self) -> bool; - /// Return the status of the DNSSEC OK flag. fn dnssec_ok(&self) -> bool; } @@ -104,9 +101,6 @@ pub trait ComposeRequestMulti: Debug + Send + Sync { /// Returns whether a message is an answer to the request. fn is_answer(&self, answer: &Message<[u8]>) -> bool; - /// Returns whether a message results in a response stream or not. - fn is_streaming(&self) -> bool; - /// Return the status of the DNSSEC OK flag. fn dnssec_ok(&self) -> bool; } @@ -386,10 +380,6 @@ impl + Debug + Octets + Send + Sync> ComposeRequest } } - fn is_streaming(&self) -> bool { - self.msg.is_streaming() - } - fn dnssec_ok(&self) -> bool { match &self.opt { None => false, @@ -422,7 +412,7 @@ impl + Debug + Octets> RequestMessageMulti { let msg = msg.into(); // Only accept the streaming types (IXFR and AXFR). - if !msg.is_streaming() { + if !msg.is_xfr() { return Err(Error::FormError); } let header = msg.header(); @@ -606,10 +596,6 @@ impl + Debug + Octets + Send + Sync> ComposeRequestMulti } } - fn is_streaming(&self) -> bool { - self.msg.is_streaming() - } - fn dnssec_ok(&self) -> bool { match &self.opt { None => false, diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index 19714babd..9441a22fa 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -30,7 +30,6 @@ use core::cmp; use octseq::Octets; use std::boxed::Box; use std::fmt::Debug; -use std::future::ready; use std::future::Future; use std::pin::Pin; use std::sync::Arc; @@ -261,14 +260,8 @@ where /// Returns a request handler for this connection. pub fn get_request(&self, request_msg: Req) -> Request { - if request_msg.is_streaming() { - Request { - fut: Box::pin(ready(Err(Error::FormError))), - } - } else { - Request { - fut: Box::pin(self.clone().handle_request_impl(request_msg)), - } + Request { + fut: Box::pin(self.clone().handle_request_impl(request_msg)), } } @@ -278,19 +271,12 @@ where request_msg: ReqMulti, ) -> RequestMulti { let (sender, receiver) = mpsc::channel(DEF_CHAN_CAP); - if !request_msg.is_streaming() { - RequestMulti { - stream: receiver, - fut: Some(Box::pin(ready(Err(Error::FormError)))), - } - } else { - RequestMulti { - stream: receiver, - fut: Some(Box::pin( - self.clone() - .handle_streaming_request_impl(request_msg, sender), - )), - } + RequestMulti { + stream: receiver, + fut: Some(Box::pin( + self.clone() + .handle_streaming_request_impl(request_msg, sender), + )), } } } From 33193958ccf1ccffffd4a4c216ec1952406c3fb2 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 3 Sep 2024 22:47:53 +0200 Subject: [PATCH 104/333] Remove is_streaming() fn as it has been removed in upstream branches. --- src/net/client/tsig.rs | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index 39949728f..9bc2c681b 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -571,10 +571,6 @@ where self.request.set_udp_payload_size(value) } - fn is_streaming(&self) -> bool { - self.request.is_streaming() - } - fn set_dnssec_ok(&mut self, value: bool) { self.request.set_dnssec_ok(value) } @@ -703,10 +699,6 @@ where self.request.is_answer(answer) } - fn is_streaming(&self) -> bool { - self.request.is_streaming() - } - fn dnssec_ok(&self) -> bool { self.request.dnssec_ok() } From dcd1f1ad83ccea33654078cfa5afde58b16e464a Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 3 Sep 2024 23:23:25 +0200 Subject: [PATCH 105/333] Make Time48 obey mock (predictable and controllable) time so that TSIG signing uses mock time during Stelline tests. --- src/rdata/tsig.rs | 21 +++++++++++++-------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/src/rdata/tsig.rs b/src/rdata/tsig.rs index f0f4bcfc9..e9730fe6f 100644 --- a/src/rdata/tsig.rs +++ b/src/rdata/tsig.rs @@ -4,21 +4,26 @@ //! //! [RFC 2845]: https://tools.ietf.org/html/rfc2845 +use core::cmp::Ordering; +use core::{fmt, hash}; + +#[cfg(all(feature = "std", not(test)))] +use std::time::SystemTime; + +#[cfg(all(feature = "std", test))] +use mock_instant::thread_local::SystemTime; +use octseq::builder::OctetsBuilder; +use octseq::octets::{Octets, OctetsFrom, OctetsInto}; +use octseq::parse::Parser; + use crate::base::cmp::CanonicalOrd; use crate::base::iana::{Rtype, TsigRcode}; use crate::base::name::{FlattenInto, ParsedName, ToName}; use crate::base::rdata::{ - ComposeRecordData, LongRecordData, ParseRecordData, RecordData + ComposeRecordData, LongRecordData, ParseRecordData, RecordData, }; use crate::base::wire::{Compose, Composer, Parse, ParseError}; use crate::utils::base64; -use core::cmp::Ordering; -use core::{fmt, hash}; -use octseq::builder::OctetsBuilder; -use octseq::octets::{Octets, OctetsFrom, OctetsInto}; -use octseq::parse::Parser; -#[cfg(feature = "std")] -use std::time::SystemTime; //------------ Tsig ---------------------------------------------------------- From 9e78310f310edc9b16dc5a2193053228b04500ff Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 3 Sep 2024 23:24:46 +0200 Subject: [PATCH 106/333] Don't set the AA flag on test service responses as (a) actual zone serving doesn't do this yet and this still needs fixing, and (b) it violates the expectations of the TSIG Stelline test that verifies at a byte level the TSIG response signature. --- src/net/server/tests/integration.rs | 3 --- test-data/server/edns_downstream_cookies.rpl | 10 +++++----- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index d455f1a85..582e3f1d0 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -372,9 +372,6 @@ fn test_service( let mut answer = mk_builder_for_target() .start_answer(request.message(), Rcode::NOERROR) .unwrap(); - // As we serve all answers from our own zones we are the - // authority for the domain in question. - answer.header_mut().set_aa(true); answer.push(record).unwrap(); answer }, diff --git a/test-data/server/edns_downstream_cookies.rpl b/test-data/server/edns_downstream_cookies.rpl index a86bdf1e9..f20ed4096 100644 --- a/test-data/server/edns_downstream_cookies.rpl +++ b/test-data/server/edns_downstream_cookies.rpl @@ -44,7 +44,7 @@ ENTRY_END STEP 11 CHECK_ANSWER ENTRY_BEGIN MATCH all -REPLY QR RD AA NOERROR +REPLY QR RD NOERROR ; AA SECTION QUESTION test. IN TXT SECTION ANSWER @@ -118,7 +118,7 @@ ENTRY_END STEP 41 CHECK_ANSWER ENTRY_BEGIN MATCH all server_cookie -REPLY QR RD AA DO NOERROR +REPLY QR RD DO NOERROR ; AA SECTION QUESTION test. IN TXT SECTION ANSWER @@ -145,7 +145,7 @@ ENTRY_END STEP 51 CHECK_ANSWER ENTRY_BEGIN MATCH all server_cookie -REPLY QR RD AA DO NOERROR +REPLY QR RD DO NOERROR ; AA SECTION QUESTION test. IN TXT SECTION ANSWER @@ -174,7 +174,7 @@ ENTRY_END STEP 61 CHECK_ANSWER ENTRY_BEGIN MATCH all server_cookie -REPLY QR RD AA DO NOERROR +REPLY QR RD DO NOERROR ; AA SECTION QUESTION test. IN TXT SECTION ANSWER @@ -227,7 +227,7 @@ ENTRY_END STEP 81 CHECK_ANSWER ENTRY_BEGIN MATCH all server_cookie -REPLY QR RD AA DO NOERROR +REPLY QR RD DO NOERROR ; AA SECTION QUESTION test. IN TXT SECTION ANSWER From 27533bd59418bbebe8b4ef04c49ebe654daa552a Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 3 Sep 2024 23:28:39 +0200 Subject: [PATCH 107/333] Strip the TSIG Stelline server test down to what can be tested without the upcoming XFR support. --- test-data/server/tsig.rpl | 90 ++------------------------------------- 1 file changed, 4 insertions(+), 86 deletions(-) diff --git a/test-data/server/tsig.rpl b/test-data/server/tsig.rpl index 7adaf1284..61bf065c5 100644 --- a/test-data/server/tsig.rpl +++ b/test-data/server/tsig.rpl @@ -1,29 +1,6 @@ ;------------ Server configuration -------------------------------------------- server: - ; Permit the server to respond to clients that send XFR requests from - ; 127.0.0.1 (the default). No TSIG signing is expected. - provide-xfr: 127.0.0.1 NOKEY - - ; Permit the server to respond to clients that send XFR requests from - ; 127.0.0.2. The requests must be TSIG signed using the specified key. - provide-xfr: 127.0.0.2 TESTKEY - - ; Permit the server to respond to clients that send XFR requests from - ; 127.0.0.3. The requests must be TSIG signed using the specified key. - provide-xfr: 127.0.0.3 TESTKEY - - ; Permit the server to respond to clients that send XFR requests from - ; 127.0.0.5. The requests must be TSIG signed using the specified key. - ; Responses will be sent in "backward compatible" mode as defined by - ; RFC 5936. - ; See: https://datatracker.ietf.org/doc/html/rfc5936#section-7.1. - provide-xfr: 127.0.0.4 TESTKEY COMPATIBLE - - ; Permit the server to respond to clients that send XFR requests from - ; 127.0.0.4. The requests must be TSIG signed using the specified key. - provide-xfr: 127.0.0.5 TESTKEY - ; Define an in-memory zone to be served by the server. local-data: "example.com. 3600 IN SOA ns.example.com. hostmaster.example.com. 1 3600 900 86400 3600" local-data: "example.com. 3600 IN NS ns.example.net." @@ -32,7 +9,7 @@ CONFIG_END ;------------ Test definition ------------------------------------------------ -SCENARIO_BEGIN Test TSIG signed queries and TSIG signed AXFR replies. +SCENARIO_BEGIN Test TSIG signed SOA query receives correctly signed reply. ;--- Mock replies @@ -40,68 +17,9 @@ SCENARIO_BEGIN Test TSIG signed queries and TSIG signed AXFR replies. ;--- Test steps -; Verify that when a client that is required to TSIG sign requests (127.0.0.2) -; submits an unsigned AXFR request over TCP that it is rejected (because while -; AXFR over TCP is allowed, the request is required to be correctly signed). -STEP 20 QUERY ADDRESS 127.0.0.2 -ENTRY_BEGIN -MATCH TCP -SECTION QUESTION - example.com. IN AXFR -ENTRY_END - -STEP 21 CHECK_ANSWER -ENTRY_BEGIN -MATCH all -REPLY QR AA REFUSED -ENTRY_END - -; Retrieve the zone via AXFR over TCP using the expected TSIG key. -STEP 30 QUERY ADDRESS 127.0.0.3 KEY TESTKEY -ENTRY_BEGIN -MATCH TCP -SECTION QUESTION - example.com. IN AXFR -ENTRY_END - -STEP 31 CHECK_ANSWER -ENTRY_BEGIN -MATCH all -REPLY QR AA NOERROR -SECTION QUESTION - example.com. IN AXFR -SECTION ANSWER - example.com. 3600 IN SOA ns.example.com. hostmaster.example.com. 1 3600 900 86400 3600 - example.com. 3600 IN NS ns.example.net. - www.example.com. 3600 IN A 1.2.3.4 - example.com. 3600 IN SOA ns.example.com. hostmaster.example.com. 1 3600 900 86400 3600 -ENTRY_END - -; Retrieve the zone using a client for which the server is configured to serve -; AXFR in backward compatible mode so that we can test multi-response TSIG. -STEP 40 QUERY ADDRESS 127.0.0.4 KEY TESTKEY -ENTRY_BEGIN -MATCH TCP -SECTION QUESTION - example.com. IN AXFR -ENTRY_END - -STEP 41 CHECK_ANSWER -ENTRY_BEGIN -MATCH all EXTRA_PACKETS -REPLY QR AA NOERROR -SECTION QUESTION - example.com. IN AXFR -SECTION ANSWER - example.com. 3600 IN SOA ns.example.com. hostmaster.example.com. 1 3600 900 86400 3600 - www.example.com. 3600 IN A 1.2.3.4 - example.com. 3600 IN NS ns.example.net. - example.com. 3600 IN SOA ns.example.com. hostmaster.example.com. 1 3600 900 86400 3600 -ENTRY_END - ; NOTE: See test-data/server/README.md regarding the effect of MOCK_CLIENT ; that is used here. -STEP 50 QUERY ADDRESS 127.0.0.5 +STEP 10 QUERY ENTRY_BEGIN MATCH UDP MATCH MOCK_CLIENT @@ -109,7 +27,7 @@ SECTION QUESTION example.com. IN SOA SECTION ADDITIONAL ; Stelline doesn't support parsing zone entries that use the ( multiline ) -; format, otherwise we could use here. +; format, otherwise we could use that here. TESTKEY 0 CLASS255 TYPE250 \# 61 0b686d61632d73686132353600 000000000000 012c 0020 a1c86ced1815d60903129a525a14494516895d99ea94bf0b5b04338126a4d625 0000 0000 0000 ; ^ Other Len ; ^ Error @@ -123,7 +41,7 @@ SECTION ADDITIONAL ; We use this so that we can define the RDATA using HEX bytes. ENTRY_END -STEP 51 CHECK_ANSWER +STEP 11 CHECK_ANSWER ENTRY_BEGIN MATCH all REPLY QR NOERROR From 90f6f210e6def5bbbe6986ec9a1e7e5c22209217 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 3 Sep 2024 23:49:42 +0200 Subject: [PATCH 108/333] Remove unnecessary display impl. --- src/tsig/mod.rs | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index e0f48b44b..6688204a6 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -65,7 +65,6 @@ use crate::base::record::Record; use crate::base::wire::{Composer, ParseError}; use crate::rdata::tsig::{Time48, Tsig}; use bytes::{Bytes, BytesMut}; -use core::fmt::Display; use core::{cmp, fmt, mem, str}; use octseq::octets::Octets; use ring::{constant_time, hkdf::KeyType, hmac, rand}; @@ -388,15 +387,6 @@ impl AsRef for Key { } } -//--- Display - -#[cfg(feature = "std")] -impl Display for Key { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.write_fmt(format_args!("{}", self.name)) - } -} - //------------ KeyStore ------------------------------------------------------ /// A type that stores TSIG secret keys. From b3818bcd09541d34a924002b274161b08002316e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 3 Sep 2024 23:55:50 +0200 Subject: [PATCH 109/333] Add back accidentally removed interop module. --- src/tsig/mod.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index 6688204a6..6a067f0e7 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -54,6 +54,8 @@ #![cfg(feature = "tsig")] #![cfg_attr(docsrs, doc(cfg(feature = "tsig")))] +mod interop; + use crate::base::header::HeaderSection; use crate::base::iana::{Class, Rcode, TsigRcode}; use crate::base::message::Message; From 3a10fc7636fd100a8c633a7683fd78d13554dbed Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 3 Sep 2024 23:59:01 +0200 Subject: [PATCH 110/333] Don't list middleware modules in RustDoc, let the RustDoc module list do that for us. --- src/net/server/middleware/mod.rs | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/src/net/server/middleware/mod.rs b/src/net/server/middleware/mod.rs index 977814d03..b8d8a9100 100644 --- a/src/net/server/middleware/mod.rs +++ b/src/net/server/middleware/mod.rs @@ -13,17 +13,6 @@ //! through the layers to the server. //! //! Currently the following middleware are available: -//! -//! - [`MandatoryMiddlewareSvc`]: Core DNS RFC standards based message -//! processing for MUST requirements. -//! - [`EdnsMiddlewareSvc`]: RFC 6891 and related EDNS message processing. -//! - [`CookiesMiddlewareSvc`]: RFC 7873 DNS Cookies related message -//! processing. -//! -//! [`MandatoryMiddlewareSvc`]: mandatory::MandatoryMiddlewareSvc -//! [`EdnsMiddlewareSvc`]: edns::EdnsMiddlewareSvc -//! [`CookiesMiddlewareSvc`]: cookies::CookiesMiddlewareSvc -//! [`Service`]: crate::net::server::service::Service #[cfg(feature = "siphasher")] pub mod cookies; From 1bc196ad9bc9047fbbee4781c60c55f0972c5414 Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Wed, 4 Sep 2024 11:28:33 +0200 Subject: [PATCH 111/333] Docs --- src/net/client/stream.rs | 30 +++++++++--------------------- 1 file changed, 9 insertions(+), 21 deletions(-) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index 9441a22fa..3c185c78f 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -3,17 +3,6 @@ // RFC 7766 describes DNS over TCP // RFC 7828 describes the edns-tcp-keepalive option -// TODO: -// - errors -// - connect errors? Retry after connection refused? -// - server errors -// - ID out of range -// - ID not in use -// - reply for wrong query -// - timeouts -// - request timeout -// - create new connection after end/failure of previous one - use super::request::{ ComposeRequest, ComposeRequestMulti, Error, GetResponse, GetResponseMulti, SendRequest, SendRequestMulti, @@ -238,7 +227,6 @@ where receiver.await.map_err(|_| Error::StreamReceiveError)? } - /// TODO: Document me. async fn handle_streaming_request_impl( self, msg: ReqMulti, @@ -258,15 +246,15 @@ where Ok(()) } - /// Returns a request handler for this connection. + /// Returns a request handler for a request. pub fn get_request(&self, request_msg: Req) -> Request { Request { fut: Box::pin(self.clone().handle_request_impl(request_msg)), } } - /// TODO - pub fn get_streaming_request( + /// Return a multiple-response request handler for a request. + fn get_streaming_request( &self, request_msg: ReqMulti, ) -> RequestMulti { @@ -359,7 +347,7 @@ impl Debug for Request { /// An active request. pub struct RequestMulti { - /// TODO + /// Receiver for a stream of responses. stream: mpsc::Receiver>, Error>>, /// The underlying future. @@ -430,15 +418,15 @@ pub struct Transport { /// This is the type of sender in [ChanReq]. #[derive(Debug)] enum ReplySender { - /// TODO + /// Return channel for a single response. Single(Option>), - /// TODO + /// Return channel for a stream of responses. Stream(mpsc::Sender>, Error>>), } impl ReplySender { - /// TODO + /// Send a response. async fn send(&mut self, resp: ChanResp) -> Result<(), ()> { match self { ReplySender::Single(sender) => match sender.take() { @@ -463,8 +451,8 @@ impl ReplySender { } } - /// TODO - pub fn is_stream(&self) -> bool { + /// Report whether in stream mode or not. + fn is_stream(&self) -> bool { matches!(self, Self::Stream(_)) } } From e98bfa4b1cd8f85e667ac4166c73ae31324cb36d Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Wed, 4 Sep 2024 11:34:43 +0200 Subject: [PATCH 112/333] Cleanup. --- src/net/client/stream.rs | 16 +--------------- 1 file changed, 1 insertion(+), 15 deletions(-) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index 3c185c78f..65b840a20 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -254,10 +254,7 @@ where } /// Return a multiple-response request handler for a request. - fn get_streaming_request( - &self, - request_msg: ReqMulti, - ) -> RequestMulti { + fn get_streaming_request(&self, request_msg: ReqMulti) -> RequestMulti { let (sender, receiver) = mpsc::channel(DEF_CHAN_CAP); RequestMulti { stream: receiver, @@ -578,17 +575,6 @@ enum XFRState { Error, } -/* -#[derive(Debug)] -struct XFRData { - /// State needed for AXFR and IXFR. - state: XFRState, - - /// - serial: Serial, -} -*/ - impl Transport { /// Creates a new transport. fn new( From 0b89d1ef47c5b35cc0c2f919a356e2789e785553 Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Wed, 4 Sep 2024 11:39:00 +0200 Subject: [PATCH 113/333] Remove comment abou TcpKeepalive. --- src/net/client/stream.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index 65b840a20..fe72cba1d 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -1042,7 +1042,6 @@ where ReqSingleMulti::Single(msg) => { msg.add_opt(&TcpKeepalive::new(None)).is_ok() } - // Do we need to set TcpKeepalive for XFR? ReqSingleMulti::Multi(msg) => { msg.add_opt(&TcpKeepalive::new(None)).is_ok() } From 523fe1d18c4c6c9d7fa78a60f86c6f84ec372dcf Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Wed, 4 Sep 2024 11:41:59 +0200 Subject: [PATCH 114/333] Docs --- src/net/client/stream.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index fe72cba1d..c686d06a3 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -227,6 +227,7 @@ where receiver.await.map_err(|_| Error::StreamReceiveError)? } + /// Start a streaming request. async fn handle_streaming_request_impl( self, msg: ReqMulti, From dc7e3c24f0665dfaac13741563178fce9efaa481 Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Wed, 4 Sep 2024 11:58:52 +0200 Subject: [PATCH 115/333] Docs --- src/net/client/stream.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index c686d06a3..bef76ad17 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -1180,7 +1180,6 @@ where return (false, xfr_state, false); } XFRState::AXFRFirstSoa(serial) => { - // Find the SOA at the end. if let AllRecordData::Soa(soa) = rr.data() { if serial == soa.serial() { // We found a match. @@ -1379,8 +1378,8 @@ impl Queries { Ok((idx, req)) } - /// Inserts the given query at a specified position. The slot has to be - /// empty. + /// Inserts the given query at a specified position. A pre-condition is + /// is that the slot has to be empty. fn insert_at(&mut self, id: u16, req: T) { let id = id as usize; self.vec[id] = Some(req); From 33b5089086581d760173ce93743a4ced974f363d Mon Sep 17 00:00:00 2001 From: Philip-NLnetLabs <93709748+Philip-NLnetLabs@users.noreply.github.com> Date: Wed, 4 Sep 2024 12:07:56 +0200 Subject: [PATCH 116/333] Update src/net/client/request.rs Co-authored-by: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> --- src/net/client/request.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/client/request.rs b/src/net/client/request.rs index 793f3f132..dd45a058f 100644 --- a/src/net/client/request.rs +++ b/src/net/client/request.rs @@ -197,7 +197,7 @@ impl + Debug + Octets> RequestMessage { pub fn new(msg: impl Into>) -> Result { let msg = msg.into(); - // On UDP, IXFR results in a single responses, so we need to accept it. + // On UDP, IXFR results in a single response, so we need to accept it. // We can reject AXFR because it always requires support for multiple // responses. if msg.header().opcode() == Opcode::QUERY From f02f773c8e405e865b0979a4d825ab3e31208f2e Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Wed, 4 Sep 2024 12:09:24 +0200 Subject: [PATCH 117/333] Use unreachable! instead of panic! --- src/net/client/stream.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/client/stream.rs b/src/net/client/stream.rs index bef76ad17..18ed58d38 100644 --- a/src/net/client/stream.rs +++ b/src/net/client/stream.rs @@ -1284,7 +1284,7 @@ where return (true, xfr_state, true); } XFRState::Done => return (true, xfr_state, true), - XFRState::Error => panic!("should not be here"), + XFRState::Error => unreachable!(), } // (eof, xfr_data, is_answer) From c1de8acfd3e17fa56c231bb6bc094620860a58c8 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 4 Sep 2024 12:52:09 +0200 Subject: [PATCH 118/333] Remove left behind trace statement. --- src/net/client/tsig.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index 9bc2c681b..6ae2acb70 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -547,7 +547,6 @@ where fn to_message(&self) -> Result>, Error> { let mut target = StaticCompressor::new(Vec::new()); - trace!("to_message"); self.append_message(&mut target)?; // It would be nice to use .builder() here. But that one deletes all From 8811b43290f2bf2ac096cfb10e6bd8fa6ebb774f Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 4 Sep 2024 14:25:31 +0200 Subject: [PATCH 119/333] Add RFC 1996 NOTIFY middleware. --- src/net/server/middleware/mod.rs | 2 +- src/net/server/middleware/notify.rs | 349 ++++++++++++++++++++++++++++ src/net/server/tests/integration.rs | 40 +++- test-data/server/notify.rpl | 49 ++++ 4 files changed, 437 insertions(+), 3 deletions(-) create mode 100644 src/net/server/middleware/notify.rs create mode 100644 test-data/server/notify.rpl diff --git a/src/net/server/middleware/mod.rs b/src/net/server/middleware/mod.rs index b8d8a9100..680389b3b 100644 --- a/src/net/server/middleware/mod.rs +++ b/src/net/server/middleware/mod.rs @@ -13,11 +13,11 @@ //! through the layers to the server. //! //! Currently the following middleware are available: - #[cfg(feature = "siphasher")] pub mod cookies; pub mod edns; pub mod mandatory; +pub mod notify; pub mod stream; #[cfg(feature = "tsig")] pub mod tsig; diff --git a/src/net/server/middleware/notify.rs b/src/net/server/middleware/notify.rs new file mode 100644 index 000000000..4bfc5a348 --- /dev/null +++ b/src/net/server/middleware/notify.rs @@ -0,0 +1,349 @@ +//! DNS NOTIFY related message processing. +use core::future::{ready, Future, Ready}; +use core::marker::PhantomData; +use core::ops::ControlFlow; +use core::pin::Pin; + +use std::boxed::Box; +use std::fmt::Debug; +use std::sync::Arc; + +use futures::stream::{once, Once, Stream}; +use octseq::Octets; +use tracing::{error, info, warn}; + +use crate::base::iana::{Class, Opcode, OptRcode, Rcode}; +use crate::base::message::CopyRecordsError; +use crate::base::message_builder::AdditionalBuilder; +use crate::base::net::IpAddr; +use crate::base::wire::Composer; +use crate::base::{ + Message, ParsedName, Question, Rtype, StreamTarget, ToName, +}; +use crate::net::server::message::Request; +use crate::net::server::middleware::stream::MiddlewareStream; +use crate::net::server::service::{CallResult, Service}; +use crate::net::server::util::{mk_builder_for_target, mk_error_response}; +use crate::rdata::AllRecordData; +use crate::zonetree::StoredName; + +/// A DNS NOTIFY middleware service. +/// +/// Standards covered by ths implementation: +/// +/// | RFC | Status | +/// |--------|---------| +/// | [1996] | TBD | +/// +/// [1996]: https://datatracker.ietf.org/doc/html/rfc1996 +#[derive(Clone, Debug)] +pub struct NotifyMiddlewareSvc { + next_svc: NextSvc, + + notify_target: N, + + _phantom: PhantomData<(RequestOctets, RequestMeta)>, +} + +impl + NotifyMiddlewareSvc +{ + #[must_use] + pub fn new(next_svc: NextSvc, notify_target: N) -> Self { + Self { + next_svc, + notify_target, + _phantom: PhantomData, + } + } +} + +impl + NotifyMiddlewareSvc +where + RequestOctets: Octets + Send + Sync, + RequestMeta: Clone + Default, + NextSvc: Service, + NextSvc::Target: Composer + Default, + N: Clone + Notifiable + Sync + Send, +{ + async fn preprocess( + req: &Request, + notify_target: N, + ) -> ControlFlow::Item>>> { + let msg = req.message(); + + let Some(q) = Self::get_relevant_question(msg) else { + return ControlFlow::Continue(()); + }; + + let class = q.qclass(); + let apex_name = q.qname().to_name(); + let source = req.client_addr().ip(); + + // https://datatracker.ietf.org/doc/html/rfc1996#section-3 + // "3.1. When a master has updated one or more RRs in which slave + // servers may be interested, the master may send the changed RR's + // name, class, type, and optionally, new RDATA(s), to each known + // slave server using a best efforts protocol based on the NOTIFY + // opcode." + // + // So, we have received a notification from a server that an RR + // changed that we may be interested in. + info!( + "NOTIFY received from {} for zone '{}'", + req.client_addr(), + q.qname() + ); + + // https://datatracker.ietf.org/doc/html/rfc1996#section-3 + // "3.7. A NOTIFY request has QDCOUNT>0, ANCOUNT>=0, AUCOUNT>=0, + // ADCOUNT>=0. If ANCOUNT>0, then the answer section represents an + // unsecure hint at the new RRset for this . A + // slave receiving such a hint is free to treat equivilence of this + // answer section with its local data as a "no further work needs + // to be done" indication. If ANCOUNT=0, or ANCOUNT>0 and the + // answer section differs from the slave's local data, then the + // slave should query its known masters to retrieve the new data." + // + // Note: At the time of writing any answers present in the request are + // ignored and thus we do not examine the equivalence or otherwise + // compared to local data. + + // https://datatracker.ietf.org/doc/html/rfc1996 + // "3.10. If a slave receives a NOTIFY request from a host that is + // not a known master for the zone containing the QNAME, it should + // ignore the request and produce an error message in its operations + // log." + // + // "Note: This implies that slaves of a multihomed master must + // either know their master by the "closest" of the master's + // interface addresses, or must know all of the master's + // interface addresses. Otherwise, a valid NOTIFY request might + // come from an address that is not on the slave's state list of + // masters for the zone, which would be an error." + // + // Announce this notification for processing. + match notify_target + .notify_zone_changed(class, &apex_name, source) + .await + { + Err(NotifyError::NotAuthForZone) => { + warn!("Ignoring NOTIFY from {} for zone '{}': Not authoritative for zone", + req.client_addr(), + q.qname() + ); + ControlFlow::Break(Self::to_stream_compatible( + mk_error_response(msg, OptRcode::NOTAUTH), + )) + } + + Err(NotifyError::Other) => { + error!( + "Error while processing NOTIFY from {} for zone '{}'.", + req.client_addr(), + q.qname() + ); + ControlFlow::Break(Self::to_stream_compatible( + mk_error_response(msg, OptRcode::SERVFAIL), + )) + } + + Ok(()) => { + // https://datatracker.ietf.org/doc/html/rfc1996#section-4 + // "4.7 Slave Receives a NOTIFY Request from a Master + // + // When a slave server receives a NOTIFY request from one + // of its locally designated masters for the zone enclosing + // the given QNAME, with QTYPE=SOA and QR=0, it should + // enter the state it would if the zone's refresh timer had + // expired. It will also send a NOTIFY response back to + // the NOTIFY request's source, with the following + // characteristics: + // + // query ID: (same) + // op: NOTIFY (4) + // resp: NOERROR + // flags: QR AA + // qcount: 1 + // qname: (zone name) + // qclass: (zone class) + // qtype: T_SOA + // + // This is intended to be identical to the NOTIFY request, + // except that the QR bit is also set. The query ID of the + // response must be the same as was received in the + // request." + let mut additional = Self::copy_message(msg).unwrap(); + + let response_hdr = additional.header_mut(); + response_hdr.set_opcode(Opcode::NOTIFY); + response_hdr.set_rcode(Rcode::NOERROR); + response_hdr.set_qr(true); + response_hdr.set_aa(true); + + let res = once(ready(Ok(CallResult::new(additional)))); + ControlFlow::Break(res) + } + } + } + + fn to_stream_compatible( + response: AdditionalBuilder>, + ) -> Once::Item>> { + once(ready(Ok(CallResult::new(response)))) + } + + fn get_relevant_question( + msg: &Message, + ) -> Option>>> { + if Opcode::NOTIFY == msg.header().opcode() { + if let Some(q) = msg.first_question() { + if q.qtype() == Rtype::SOA { + return Some(q); + } + } + } + + None + } + + // Based on RequestMessage::append_message_impl(). + fn copy_message( + source: &Message, + ) -> Result< + AdditionalBuilder>, + CopyRecordsError, + > { + let mut builder = mk_builder_for_target(); + *builder.header_mut() = source.header(); + + let source = source.question(); + let mut question = builder.question(); + for rr in source { + question.push(rr?)?; + } + let mut source = source.answer()?; + let mut answer = question.answer(); + for rr in &mut source { + let rr = rr? + .into_record::>>()? + .expect("record expected"); + answer.push(rr)?; + } + + let mut source = + source.next_section()?.expect("section should be present"); + let mut authority = answer.authority(); + for rr in &mut source { + let rr = rr? + .into_record::>>()? + .expect("record expected"); + authority.push(rr)?; + } + + let source = + source.next_section()?.expect("section should be present"); + let mut additional = authority.additional(); + for rr in source { + let rr = rr?; + if rr.rtype() != Rtype::OPT { + let rr = rr + .into_record::>>()? + .expect("record expected"); + additional.push(rr)?; + } + } + + Ok(additional) + } +} + +//--- Service + +impl + Service + for NotifyMiddlewareSvc +where + RequestOctets: Octets + Send + Sync + 'static, + RequestMeta: Clone + Default + Sync + Send + 'static, + for<'a> ::Range<'a>: Send + Sync, + NextSvc: Service + + Clone + + 'static + + Send + + Sync + + Unpin, + NextSvc::Future: Send + Sync + Unpin, + NextSvc::Target: Composer + Default + Send + Sync, + N: Notifiable + Clone + Sync + Send + 'static, +{ + type Target = NextSvc::Target; + type Stream = MiddlewareStream< + NextSvc::Future, + NextSvc::Stream, + NextSvc::Stream, + Once::Item>>, + ::Item, + >; + type Future = Pin + Send + Sync>>; + + fn call( + &self, + request: Request, + ) -> Self::Future { + let request = request.clone(); + let next_svc = self.next_svc.clone(); + let notify_target = self.notify_target.clone(); + Box::pin(async move { + match Self::preprocess(&request, notify_target).await { + ControlFlow::Continue(()) => { + let stream = next_svc.call(request).await; + MiddlewareStream::IdentityStream(stream) + } + ControlFlow::Break(stream) => { + MiddlewareStream::Result(stream) + } + } + }) + } +} + +//------------ Notifiable ----------------------------------------------------- + +#[derive(Clone, Debug, PartialEq, Eq)] +pub enum NotifyError { + /// We are not authoritative for the zone. + NotAuthForZone, + + /// Notify handling failed for some other reason. + Other, +} + +// Note: The fn signatures can be simplified to fn() -> impl Future<...> if +// our MSRV is later increased. +pub trait Notifiable { + fn notify_zone_changed( + &self, + class: Class, + apex_name: &StoredName, + source: IpAddr, + ) -> Pin< + Box> + Sync + Send + '_>, + >; +} + +//--- impl for Arc + +impl Notifiable for Arc { + fn notify_zone_changed( + &self, + class: Class, + apex_name: &StoredName, + source: IpAddr, + ) -> Pin< + Box> + Sync + Send + '_>, + > { + (**self).notify_zone_changed(class, apex_name, source) + } +} diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index 582e3f1d0..f1366802b 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -15,7 +15,7 @@ use rstest::rstest; use tracing::instrument; use tracing::{trace, warn}; -use crate::base::iana::Rcode; +use crate::base::iana::{Class, Rcode}; use crate::base::name::{Name, ToName}; use crate::base::net::IpAddr; use crate::base::wire::Composer; @@ -32,6 +32,9 @@ use crate::net::server::message::Request; use crate::net::server::middleware::cookies::CookiesMiddlewareSvc; use crate::net::server::middleware::edns::EdnsMiddlewareSvc; use crate::net::server::middleware::mandatory::MandatoryMiddlewareSvc; +use crate::net::server::middleware::notify::{ + Notifiable, NotifyError, NotifyMiddlewareSvc, +}; use crate::net::server::service::{CallResult, Service, ServiceResult}; use crate::net::server::stream::StreamServer; use crate::net::server::util::{mk_builder_for_target, service_fn}; @@ -45,6 +48,10 @@ use crate::stelline::simple_dgram_client; use crate::tsig::{Algorithm, Key, KeyName, KeyStore}; use crate::utils::base16; use crate::zonefile::inplace::{Entry, ScannedRecord, Zonefile}; +use crate::zonetree::StoredName; +use core::future::{ready, Future}; +use core::pin::Pin; +use std::string::ToString; //----------- Tests ---------------------------------------------------------- @@ -130,7 +137,10 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { // 4. Mandatory DNS behaviour (e.g. RFC 1034/35 rules). let svc = MandatoryMiddlewareSvc::new(svc); - // 5. TSIG message authentication. + // 5. RFC 1996 NOTIFY support. + let svc = NotifyMiddlewareSvc::new(svc, TestNotifyTarget); + + // 6. TSIG message authentication. let svc = TsigMiddlewareSvc::new(svc, key_store.clone()); // NOTE: TSIG middleware *MUST* be the first middleware in the chain per @@ -490,3 +500,29 @@ fn parse_server_config(config: &Config) -> ServerConfig { // KeyStore is impl'd elsewhere for HashMap<(KeyName, Algorithm), K, S>. type TestKeyStore = HashMap<(KeyName, Algorithm), Key>; + +//------------ NoOpNotifyTarget ----------------------------------------------- + +#[derive(Copy, Clone, Default, Debug)] +struct TestNotifyTarget; + +impl Notifiable for TestNotifyTarget { + fn notify_zone_changed( + &self, + class: Class, + apex_name: &StoredName, + source: IpAddr, + ) -> Pin< + Box> + Sync + Send + '_>, + > { + trace!("Notify received from {source} of change to zone {apex_name} in class {class}"); + + let res = match apex_name.to_string().to_lowercase().as_str() { + "example.com" => Ok(()), + "othererror.com" => Err(NotifyError::Other), + _ => Err(NotifyError::NotAuthForZone), + }; + + Box::pin(ready(res)) + } +} diff --git a/test-data/server/notify.rpl b/test-data/server/notify.rpl new file mode 100644 index 000000000..63434a988 --- /dev/null +++ b/test-data/server/notify.rpl @@ -0,0 +1,49 @@ +;------------ Server configuration -------------------------------------------- + +CONFIG_END + +;------------ Test definition ------------------------------------------------ + +SCENARIO_BEGIN Test acknowledgement of NOTIFY query. + +;--- Mock replies + +; None + +;--- Test steps + +STEP 10 QUERY +ENTRY_BEGIN +OPCODE NOTIFY +MATCH UDP +SECTION QUESTION + EXAMPLE.COM. IN SOA +ENTRY_END + +STEP 11 CHECK_ANSWER +ENTRY_BEGIN +OPCODE NOTIFY +MATCH all +REPLY QR AA NOERROR +SECTION QUESTION + EXAMPLE.COM. IN SOA +ENTRY_END + +STEP 20 QUERY +ENTRY_BEGIN +OPCODE NOTIFY +MATCH UDP +SECTION QUESTION + UNKNOWNZONE.COM. IN SOA +ENTRY_END + +STEP 21 CHECK_ANSWER +ENTRY_BEGIN +OPCODE NOTIFY +MATCH all +REPLY QR NOTAUTH +SECTION QUESTION + UNKNOWNZONE.COM. IN SOA +ENTRY_END + +SCENARIO_END From 430b99462296edfd1ea6ee553ac68ace73b8573e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 4 Sep 2024 16:45:30 +0200 Subject: [PATCH 120/333] Add RR batching functionality. --- src/base/message_builder.rs | 13 ++ src/net/server/batcher.rs | 371 ++++++++++++++++++++++++++++++++++++ src/net/server/mod.rs | 1 + 3 files changed, 385 insertions(+) create mode 100644 src/net/server/batcher.rs diff --git a/src/base/message_builder.rs b/src/base/message_builder.rs index ae1e1a9a5..d9e2c1aa5 100644 --- a/src/base/message_builder.rs +++ b/src/base/message_builder.rs @@ -876,6 +876,19 @@ impl AnswerBuilder { |counts| counts.inc_ancount(), ) } + + /// Appends a record to the answer section without consuming it. + /// + /// See [`push`][Self::push]. + pub fn push_ref( + &mut self, + record: &impl ComposeRecord, + ) -> Result<(), PushError> { + self.builder.push( + |target| record.compose_record(target).map_err(Into::into), + |counts| counts.inc_ancount(), + ) + } } /// # Conversions diff --git a/src/net/server/batcher.rs b/src/net/server/batcher.rs new file mode 100644 index 000000000..366201fc6 --- /dev/null +++ b/src/net/server/batcher.rs @@ -0,0 +1,371 @@ +//! Resource record batching. + +use core::marker::PhantomData; + +use std::sync::Arc; + +use octseq::Octets; +use tracing::trace; + +use crate::base::iana::Rcode; +use crate::base::message_builder::{AnswerBuilder, PushError}; +use crate::base::record::ComposeRecord; +use crate::base::wire::Composer; +use crate::base::{Message, StreamTarget}; + +use super::util::mk_builder_for_target; + +//----------- PushResult ------------------------------------------------------ + +pub enum PushResult { + PushedAndReadyForMore, + PushedAndLimitReached(AnswerBuilder>), + NotPushedMessageFull(AnswerBuilder>), + Retry, +} + +//------------ ResourceRecordBatcher ------------------------------------------ + +pub trait ResourceRecordBatcher +where + RequestOctets: Octets, + Target: Composer + Default, +{ + #[allow(clippy::result_unit_err)] + fn push( + &mut self, + record: impl ComposeRecord, + ) -> Result, ()>; + + #[allow(clippy::result_unit_err)] + fn finish(&mut self) -> Result<(), ()>; + + fn mk_answer_builder( + &self, + msg: &Message, + ) -> Result>, PushError> { + let builder = mk_builder_for_target(); + builder.start_answer(msg, Rcode::NOERROR) + } +} + +//------------ Callbacks ------------------------------------------------------ + +pub trait Callbacks +where + RequestOctets: Octets, + Target: Composer + Default, +{ + /// Prepare a message builder to push records into. + fn batch_started( + _state: &T, + msg: &Message, + ) -> Result>, PushError> { + let builder = mk_builder_for_target(); + let answer = builder.start_answer(msg, Rcode::NOERROR)?; + Ok(answer) + } + + /// A record has been pushed. Is the message now full? + /// + /// Return true if it is full, false if there is still space. + fn record_pushed( + _state: &T, + _answer: &AnswerBuilder>, + ) -> bool { + false + } + + /// Do something with the completed message. + #[allow(clippy::result_unit_err)] + fn batch_ready( + _state: &T, + _answer: AnswerBuilder>, + ) -> Result<(), ()>; +} + +//------------ CallbackBatcher ------------------------------------------------ + +pub struct CallbackBatcher +where + RequestOctets: Octets, + Target: Composer + Default, + C: Callbacks, +{ + req_msg: Arc>, + answer: Option>, PushError>>, + callback_state: T, + _phantom: PhantomData, +} + +impl CallbackBatcher +where + RequestOctets: Octets, + Target: Composer + Default, + C: Callbacks, +{ + pub fn new( + req_msg: Arc>, + callback_state: T, + ) -> Self { + Self { + req_msg, + answer: None, + callback_state, + _phantom: PhantomData, + } + } + + pub fn callback_state(&self) -> &T { + &self.callback_state + } +} + +impl CallbackBatcher +where + RequestOctets: Octets, + Target: Composer + Default, + C: Callbacks, +{ + fn try_push( + &mut self, + record: &impl ComposeRecord, + ) -> Result, ()> { + match self.push_ref(record).map_err(|_| ())? { + PushResult::PushedAndLimitReached(builder) => { + C::batch_ready(&self.callback_state, builder)?; + Ok(PushResult::PushedAndReadyForMore) + } + PushResult::NotPushedMessageFull(builder) => { + C::batch_ready(&self.callback_state, builder)?; + Ok(PushResult::Retry) + } + other => Ok(other), + } + } + + fn push_ref( + &mut self, + record: &impl ComposeRecord, + ) -> Result, PushError> { + let req_msg = &self.req_msg; + + if self.answer.is_none() { + self.answer = + Some(C::batch_started(&self.callback_state, req_msg)); + } + + let mut answer = self.answer.take().unwrap()?; + + let res = answer.push_ref(record); + let ancount = answer.counts().ancount(); + + match res { + Ok(()) if C::record_pushed(&self.callback_state, &answer) => { + // Push succeeded but the message is as full as the caller + // allows, pass it back to the caller to process. + Ok(PushResult::PushedAndLimitReached(answer)) + } + + Err(_) if ancount > 0 => { + // Push failed because the message is full, pass it back to + // the caller to process. + Ok(PushResult::NotPushedMessageFull(answer)) + } + + Err(err) => { + // We expect to be able to add at least one answer to the message. + Err(err) + } + + Ok(()) => { + // Record has been added, keep the answer builder for the next push. + self.answer = Some(Ok(answer)); + Ok(PushResult::PushedAndReadyForMore) + } + } + } +} + +//--- ResourceRecordBatcher + +impl ResourceRecordBatcher + for CallbackBatcher +where + RequestOctets: Octets, + Target: Composer + Default, + C: Callbacks, +{ + fn push( + &mut self, + record: impl ComposeRecord, + ) -> Result, ()> { + match self.try_push(&record) { + Ok(PushResult::Retry) => self.try_push(&record), + other => other, + } + } + + fn finish(&mut self) -> Result<(), ()> { + if let Some(builder) = self.answer.take() { + C::batch_ready(&self.callback_state, builder.unwrap()) + } else { + Ok(()) + } + } + + fn mk_answer_builder( + &self, + msg: &Message, + ) -> Result>, PushError> { + C::batch_started(&self.callback_state, msg) + } +} + +//--- Drop + +impl Drop + for CallbackBatcher +where + RequestOctets: Octets, + Target: Composer + Default, + C: Callbacks, +{ + fn drop(&mut self) { + if self.answer.is_some() { + trace!("Dropping unfinished batcher, was that intentional or did you forget to call finish()?"); + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::base::{MessageBuilder, Name}; + use crate::rdata::Txt; + use core::sync::atomic::{AtomicU64, Ordering}; + use std::vec::Vec; + + #[test] + fn batch_of_zero() { + let mut batcher = mk_counting_batcher(); + batcher.callback_state().assert_eq(0, 0, 0); + batcher.finish().unwrap(); + batcher.callback_state().assert_eq(0, 0, 0); + } + + #[test] + fn batch_of_one() { + let mut batcher = mk_counting_batcher(); + batcher.push(mk_dummy_rr(&[])).unwrap(); + batcher.callback_state().assert_eq(1, 1, 0); + batcher.finish().unwrap(); + batcher.callback_state().assert_eq(0, 1, 1); + } + + #[test] + fn batch_of_one_very_large_rr() { + let mut batcher = mk_counting_batcher(); + batcher.push(mk_dummy_rr(&vec![0; 65000])).unwrap(); + batcher.callback_state().assert_eq(1, 1, 0); + batcher.finish().unwrap(); + batcher.callback_state().assert_eq(0, 1, 1); + } + + #[test] + fn batch_of_many_small_rrs() { + let mut batcher = mk_counting_batcher(); + for _ in 0..1000 { + batcher.push(mk_dummy_rr(&[0; 10])).unwrap(); + } + batcher.callback_state().assert_eq(1000, 1000, 0); + batcher.finish().unwrap(); + batcher.callback_state().assert_eq(0, 1000, 1); + } + + #[test] + fn batch_of_two_too_big_rrs() { + let mut batcher = mk_counting_batcher(); + batcher.push(mk_dummy_rr(&vec![0; 65000])).unwrap(); + batcher.callback_state().assert_eq(1, 1, 0); + batcher.push(mk_dummy_rr(&vec![0; 1000])).unwrap(); + batcher.callback_state().assert_eq(1, 2, 1); + batcher.finish().unwrap(); + batcher.callback_state().assert_eq(0, 2, 2); + } + + fn mk_counting_batcher( + ) -> CallbackBatcher, Vec, BatchCounter, Arc> + { + let req = Arc::new(MessageBuilder::new_vec().into_message()); + let cnt = Arc::new(TestCounters::new()); + CallbackBatcher::new(req, cnt) + } + + fn mk_dummy_rr(text: &[u8]) -> impl ComposeRecord { + ( + Name::root_vec(), + 0, + Txt::>::build_from_slice(text).unwrap(), + ) + } + + //------------ TestCounters ----------------------------------------------- + + #[derive(Default)] + struct TestCounters { + num_rrs_in_last_batch: AtomicU64, + num_total_rrs: AtomicU64, + num_batches: AtomicU64, + } + + impl TestCounters { + fn new() -> Self { + Self::default() + } + + fn assert_eq( + &self, + num_rrs_in_last_batch: u64, + num_total_rrs: u64, + num_batches: u64, + ) { + assert_eq!( + self.num_rrs_in_last_batch.load(Ordering::SeqCst), + num_rrs_in_last_batch + ); + assert_eq!( + self.num_total_rrs.load(Ordering::SeqCst), + num_total_rrs + ); + assert_eq!(self.num_batches.load(Ordering::SeqCst), num_batches); + } + } + + //------------ TestCallbacks ---------------------------------------------- + + struct BatchCounter; + + impl Callbacks, Vec, Arc> for BatchCounter { + fn batch_ready( + counters: &Arc, + answer: AnswerBuilder>>, + ) -> Result<(), ()> { + counters.num_batches.fetch_add(1, Ordering::SeqCst); + counters.num_rrs_in_last_batch.store(0, Ordering::SeqCst); + eprintln!("Answer byte length: {}", answer.as_slice().len()); + Ok(()) + } + + fn record_pushed( + counters: &Arc, + _answer: &AnswerBuilder>>, + ) -> bool { + counters + .num_rrs_in_last_batch + .fetch_add(1, Ordering::SeqCst); + counters.num_total_rrs.fetch_add(1, Ordering::SeqCst); + false + } + } +} diff --git a/src/net/server/mod.rs b/src/net/server/mod.rs index ab7a3b7dc..7a2508104 100644 --- a/src/net/server/mod.rs +++ b/src/net/server/mod.rs @@ -180,6 +180,7 @@ mod connection; pub use connection::Config as ConnectionConfig; +pub mod batcher; pub mod buf; pub mod dgram; pub mod error; From 02d5b19ac4d1e9d26edfcaa27822ef3d09b3bd1f Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 5 Sep 2024 23:35:54 +0200 Subject: [PATCH 121/333] WIP: Initial extraction of XfrMiddlewareSvc from the xfr branch and conversion such that access control checks are not hard-coded into the middleware nor dependent on policy data being stored in a specific form. --- src/net/server/middleware/mod.rs | 2 + src/net/server/middleware/xfr.rs | 1363 ++++++++++++++++++ src/net/server/tests/integration.rs | 165 ++- src/zonetree/in_memory/read.rs | 91 +- src/zonetree/parsed.rs | 2 +- src/zonetree/walk.rs | 19 +- test-data/server/axfr_large_out.rpl | 1055 ++++++++++++++ test-data/server/axfr_out.rpl | 140 ++ test-data/server/edns_downstream_cookies.rpl | 4 +- test-data/server/edns_keepalive.rpl | 9 +- 10 files changed, 2718 insertions(+), 132 deletions(-) create mode 100644 src/net/server/middleware/xfr.rs create mode 100644 test-data/server/axfr_large_out.rpl create mode 100644 test-data/server/axfr_out.rpl diff --git a/src/net/server/middleware/mod.rs b/src/net/server/middleware/mod.rs index 680389b3b..592f6cf7d 100644 --- a/src/net/server/middleware/mod.rs +++ b/src/net/server/middleware/mod.rs @@ -21,3 +21,5 @@ pub mod notify; pub mod stream; #[cfg(feature = "tsig")] pub mod tsig; +#[cfg(feature = "unstable-xfr")] +pub mod xfr; diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs new file mode 100644 index 000000000..31bde13a5 --- /dev/null +++ b/src/net/server/middleware/xfr.rs @@ -0,0 +1,1363 @@ +//! XFR request handling middleware. + +use core::future::{ready, Future, Ready}; +use core::marker::PhantomData; +use core::ops::{ControlFlow, Deref}; + +use std::boxed::Box; +use std::pin::Pin; +use std::sync::Arc; +use std::vec::Vec; + +use bytes::Bytes; +use futures::stream::{once, Once, Stream}; +use octseq::Octets; +use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; +use tokio::sync::Semaphore; +use tokio_stream::wrappers::UnboundedReceiverStream; +use tracing::{error, info, trace, warn}; + +use crate::base::iana::{Class, Opcode, OptRcode, Rcode}; +use crate::base::message_builder::{ + AdditionalBuilder, AnswerBuilder, PushError, +}; +use crate::base::wire::Composer; +use crate::base::{ + Message, Name, ParsedName, Question, Rtype, Serial, StreamTarget, ToName, +}; +use crate::net::server::batcher::{ + CallbackBatcher, Callbacks, ResourceRecordBatcher, +}; +use crate::net::server::message::{Request, TransportSpecificContext}; +use crate::net::server::middleware::stream::MiddlewareStream; +use crate::net::server::service::{ + CallResult, Service, ServiceFeedback, ServiceResult, +}; +use crate::net::server::util::{mk_builder_for_target, mk_error_response}; +use crate::rdata::{Soa, ZoneRecordData}; +use crate::zonetree::error::OutOfZone; +use crate::zonetree::{ + Answer, AnswerContent, ReadableZone, SharedRrset, StoredName, Zone, + ZoneDiff, ZoneTree, +}; + +//------------ XfrMiddlewareSvc ---------------------------------------------- + +/// A [`MiddlewareProcessor`] for responding to XFR requests. +/// +/// Standards covered by ths implementation: +/// +/// | RFC | Status | +/// |--------|---------| +/// | [1034] | TBD | +/// | [1035] | TBD | +/// | [1995] | TBD | +/// | [5936] | TBD | +/// +/// [`MiddlewareProcessor`]: +/// crate::net::server::middleware::processor::MiddlewareProcessor +/// [1034]: https://datatracker.ietf.org/doc/html/rfc1034 +/// [1035]: https://datatracker.ietf.org/doc/html/rfc1035 +/// [1995]: https://datatracker.ietf.org/doc/html/rfc1995 +/// [5936]: https://datatracker.ietf.org/doc/html/rfc5936 +#[derive(Clone, Debug)] +pub struct XfrMiddlewareSvc { + next_svc: NextSvc, + + xfr_data_provider: XDP, + + zone_walking_semaphore: Arc, + + batcher_semaphore: Arc, + + _phantom: PhantomData, +} + +impl + XfrMiddlewareSvc +where + XDP: XfrDataProvider, +{ + /// Creates a new processor instance. + #[must_use] + pub fn new( + next_svc: NextSvc, + xfr_data_provider: XDP, + max_concurrency: usize, + ) -> Self { + let zone_walking_semaphore = + Arc::new(Semaphore::new(max_concurrency)); + let batcher_semaphore = Arc::new(Semaphore::new(max_concurrency)); + + Self { + next_svc, + xfr_data_provider, + zone_walking_semaphore, + batcher_semaphore, + _phantom: PhantomData, + } + } +} + +impl + XfrMiddlewareSvc +where + RequestOctets: Octets + Send + Sync + 'static + Unpin, + for<'a> ::Range<'a>: Send + Sync, + NextSvc: Service + Clone + Send + Sync + 'static, + NextSvc::Future: Send + Sync + Unpin, + NextSvc::Target: Composer + Default + Send + Sync, + NextSvc::Stream: Send + Sync, + XDP: XfrDataProvider, +{ + pub async fn preprocess( + zone_walking_semaphore: Arc, + batcher_semaphore: Arc, + req: &Request, + xfr_data_provider: XDP, + ) -> ControlFlow< + XfrMiddlewareStream< + NextSvc::Future, + NextSvc::Stream, + ::Item, + >, + > { + let msg = req.message(); + + // Do we support this type of request? + let Some(q) = Self::get_relevant_question(msg) else { + return ControlFlow::Continue(()); + }; + + // https://datatracker.ietf.org/doc/html/rfc1995#section-3 + // 3. Query Format + // "The IXFR query packet format is the same as that of a normal DNS + // query, but with the query type being IXFR and the authority + // section containing the SOA record of client's version of the + // zone." + let ixfr_query_serial = if let Ok(Some(Ok(query_soa))) = msg + .authority() + .map(|section| section.limit_to::>>().next()) + { + Some(query_soa.data().serial()) + } else { + None + }; + + if q.qtype() == Rtype::IXFR && ixfr_query_serial.is_none() { + return Self::log_and_break( + &q, + req, + msg, + OptRcode::FORMERR, + "IXFR request lacks authority section SOA", + ); + } + + // Is transfer allowed for the requested zone for this requestor? + let res = xfr_data_provider + .request(req, q.qname(), q.qclass(), ixfr_query_serial) + .await + .map_err(|err| match err { + XfrDataProviderError::UnknownZone => { + // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2.1 + // 2.2.1 Header Values + // "If a server is not authoritative for the queried + // zone, the server SHOULD set the value to NotAuth(9)" + ("unknown zone", OptRcode::NOTAUTH) + } + + XfrDataProviderError::TemporarilyUnavailable => { + // The zone is not yet loaded or has expired, both of + // which are presumably transient conditions and thus + // SERVFAIL is the appropriate response, not NOTAUTH, as + // we know we are supposed to be authoritative for the + // zone but we just don't have the data right now. + ("zone not currently available", OptRcode::SERVFAIL) + } + + XfrDataProviderError::Refused => { + ("access denied", OptRcode::REFUSED) + } + }); + + let (zone, diffs) = res.unwrap(); + + // Read the zone SOA RR + let read = zone.read(); + let Ok(zone_soa_answer) = + Self::read_soa(&read, q.qname().to_name()).await + else { + return Self::log_and_break( + &q, + req, + msg, + OptRcode::SERVFAIL, + "name is outside the zone", + ); + }; + + match q.qtype() { + Rtype::AXFR if req.transport_ctx().is_udp() => { + // https://datatracker.ietf.org/doc/html/rfc5936#section-4.2 + // 4.2. UDP + // "With the addition of EDNS0 and applications that require + // many small zones, such as in web hosting and some ENUM + // scenarios, AXFR sessions on UDP would now seem + // desirable. However, there are still some aspects of + // AXFR sessions that are not easily translated to UDP. + // + // Therefore, this document does not update RFC 1035 in + // this respect: AXFR sessions over UDP transport are not + // defined." + Self::log_and_break( + &q, + req, + msg, + OptRcode::NOTIMP, + "AXFR not suppored over UDP", + ) + } + + Rtype::AXFR | Rtype::IXFR if diffs.is_empty() => { + if q.qtype() == Rtype::IXFR && diffs.is_empty() { + // https://datatracker.ietf.org/doc/html/rfc1995#section-4 + // 4. Response Format + // "If incremental zone transfer is not available, the + // entire zone is returned. The first and the last RR of + // the response is the SOA record of the zone. I.e. the + // behavior is the same as an AXFR response except the + // query type is IXFR." + info!( + "IXFR for {} from {}: diffs not available, falling back to AXFR", + q.qname(), + req.client_addr() + ); + } else { + info!( + "AXFR for {} from {}", + q.qname(), + req.client_addr() + ); + } + let stream = Self::do_axfr( + zone_walking_semaphore, + batcher_semaphore, + req, + q.qname().to_name(), + &zone_soa_answer, + read, + ) + .await + .unwrap_or_else(|rcode| { + Self::to_stream(mk_error_response(msg, rcode)) + }); + + ControlFlow::Break(stream) + } + + Rtype::IXFR => { + let ixfr_query_serial = ixfr_query_serial.unwrap(); + info!( + "IXFR for {} (serial {ixfr_query_serial}) from {}", + q.qname(), + req.client_addr() + ); + + // https://datatracker.ietf.org/doc/html/rfc1995#section-2 + // 2. Brief Description of the Protocol + // "Transport of a query may be by either UDP or TCP. If an + // IXFR query is via UDP, the IXFR server may attempt to + // reply using UDP if the entire response can be contained + // in a single DNS packet. If the UDP reply does not fit, + // the query is responded to with a single SOA record of + // the server's current version to inform the client that a + // TCP query should be initiated." + let stream = Self::do_ixfr( + batcher_semaphore.clone(), + req, + ixfr_query_serial, + &zone_soa_answer, + diffs, + ) + .await + .unwrap_or_else(|rcode| { + Self::to_stream(mk_error_response(msg, rcode)) + }); + + ControlFlow::Break(stream) + } + + _ => ControlFlow::Continue(()), + } + } + + #[allow(clippy::type_complexity)] + fn log_and_break( + q: &Question::Range<'_>>>, + req: &Request, + msg: &Message, + rcode: OptRcode, + reason: &'static str, + ) -> ControlFlow< + XfrMiddlewareStream< + NextSvc::Future, + NextSvc::Stream, + ::Item, + >, + > { + warn!( + "{} for {} from {} refused: {reason}", + q.qtype(), + q.qname(), + req.client_addr() + ); + ControlFlow::Break(Self::to_stream(mk_error_response(msg, rcode))) + } + + #[allow(clippy::too_many_arguments)] + async fn do_axfr( + zone_walk_semaphore: Arc, + batcher_semaphore: Arc, + req: &Request, + qname: StoredName, + zone_soa_answer: &Answer, + read: Box, + ) -> Result< + XfrMiddlewareStream< + NextSvc::Future, + NextSvc::Stream, + ::Item, + >, + OptRcode, + > { + let msg = req.message(); + + let AnswerContent::Data(zone_soa_rrset) = + zone_soa_answer.content().clone() + else { + error!( + "AXFR for {qname} from {} refused: zone lacks SOA RR", + req.client_addr() + ); + return Err(OptRcode::SERVFAIL); + }; + + // TODO + // let compatibility_mode = xfr_config.compatibility_mode + // == CompatibilityMode::BackwardCompatible; + let compatibility_mode = false; + + if compatibility_mode { + trace!( + "Compatibility mode enabled for client with IP address {}", + req.client_addr().ip() + ); + } + + // Return a stream of response messages containing: + // - SOA + // - RRSETs, one or more per response message + // - SOA + // + // Neither RFC 5936 nor RFC 1035 defined AXFR for UDP, only for TCP. + // However, RFC 1995 says that for IXFR if no diffs are available the + // full zone should be served just as with AXFR, and that UDP is + // supported as long as the entire XFR response fits in a single + // datagram. Thus we don't check for UDP or TCP here, except to abort + // if the response is too large to fit in a single UDP datagram, + // instead we let the caller that has the context decide whether AXFR + // is supported or not. + // + // References: + // - https://datatracker.ietf.org/doc/html/rfc1995#section-2 + // - https://datatracker.ietf.org/doc/html/rfc1995#section-4 + // - https://datatracker.ietf.org/doc/html/rfc5936#section-4.2 + + // Create a stream that will be immediately returned to the caller. + // Async tasks will then push DNS response messages into the stream as + // they become available. + let (sender, receiver) = unbounded_channel(); + let stream = UnboundedReceiverStream::new(receiver); + + // Create a bounded queue for passing RRsets found during zone walking + // to a task which will batch the RRs together before pushing them + // into the result stream. + let (batcher_tx, mut batcher_rx) = + tokio::sync::mpsc::channel::<(StoredName, SharedRrset)>(100); + + // Notify the underlying transport to expect a stream of related + // responses. The transport should modify its behaviour to account for + // the potentially slow and long running nature of a transaction. + Self::add_to_stream( + CallResult::feedback_only(ServiceFeedback::BeginTransaction), + &sender, + ); + + // Enqueue the zone SOA RRset for the batcher to process. + if batcher_tx + .send((qname.clone(), zone_soa_rrset.clone())) + .await + .is_err() + { + return Err(OptRcode::SERVFAIL); + } + + // Stream the remaining non-SOA zone RRsets in the background to the + // batcher. + tokio::spawn(async move { + // Limit the number of concurrently running XFR related zone + // walking operations. + if zone_walk_semaphore.acquire().await.is_err() { + error!("Internal error: Failed to acquire XFR zone walking semaphore"); + return; + } + + let cloned_batcher_tx = batcher_tx.clone(); + let op = + Box::new(move |owner: StoredName, rrset: &SharedRrset| { + if rrset.rtype() != Rtype::SOA { + let _ = cloned_batcher_tx + .blocking_send((owner.clone(), rrset.clone())); + // If the blocking send fails it means that he batcher + // is no longer available. This can happen if it was + // no longer able to pass messages back to the + // underlying transport, which can happen if the + // client closed the connection. We don't log this + // because we can't stop the tree walk and so will + // keep hitting this error until the tree walk is + // complete, causing a lot of noise if we were to log + // this. + } + }); + + // Walk the zone tree, invoking our operation for each leaf. + match read.is_async() { + true => { + read.walk_async(op).await; + if let Err(err) = + batcher_tx.send((qname, zone_soa_rrset)).await + { + error!("Internal error: Failed to send final AXFR SOA to batcher: {err}"); + } + } + false => { + tokio::task::spawn_blocking(move || { + read.walk(op); + if let Err(err) = + batcher_tx.blocking_send((qname, zone_soa_rrset)) + { + error!("Internal error: Failed to send final AXFR SOA to batcher: {err}"); + } + }); + } + } + }); + + // Combine RRsets enumerated by zone walking as many as possible per + // DNS response message and pass the created messages downstream to + // the caller. + let msg = msg.clone(); + let soft_byte_limit = Self::calc_msg_bytes_available(req); + + tokio::spawn(async move { + // Limit the number of concurrently running XFR batching + // operations. + if batcher_semaphore.acquire().await.is_err() { + error!( + "Internal error: Failed to acquire XFR batcher semaphore" + ); + return; + } + + let Ok(qclass) = msg.sole_question().map(|q| q.qclass()) else { + unreachable!(); + }; + + // Note: NSD apparently uses name compresson on AXFR responses + // because AXFR responses they typically contain lots of + // alphabetically ordered duplicate names which compress well. NSD + // limits AXFR responses to 16,383 RRs because DNS name + // compression uses a 14-bit offset (2^14-1=16383) from the start + // of the message to the first occurence of a name instead of + // repeating the name, and name compression is less effective + // over 16383 bytes. (Credit: Wouter Wijngaards) + // + // TODO: Once we start supporting name compression in responses decide + // if we want to behave the same way. + + let hard_rr_limit = match compatibility_mode { + true => Some(1), + false => None, + }; + + let mut batcher = XfrRrBatcher::build( + msg.clone(), + sender.clone(), + Some(soft_byte_limit), + hard_rr_limit, + ); + + while let Some((owner, rrset)) = batcher_rx.recv().await { + for rr in rrset.data() { + if batcher + .push((owner.clone(), qclass, rrset.ttl(), rr)) + .is_err() + { + error!("Internal error: Failed to send final AXFR SOA to batcher"); + let resp = + mk_error_response(&msg, OptRcode::SERVFAIL); + Self::add_to_stream(CallResult::new(resp), &sender); + batcher_rx.close(); + return; + } + } + } + + batcher.finish().unwrap(); // TODO + + trace!("Finishing transaction"); + Self::add_to_stream( + CallResult::feedback_only(ServiceFeedback::EndTransaction), + &sender, + ); + + batcher_rx.close(); + }); + + Ok(MiddlewareStream::Result(stream)) + } + + // Returns None if fallback to AXFR should be done. + #[allow(clippy::too_many_arguments)] + async fn do_ixfr( + batcher_semaphore: Arc, + req: &Request, + query_serial: Serial, + zone_soa_answer: &Answer, + diffs: Vec>, + ) -> Result< + XfrMiddlewareStream< + NextSvc::Future, + NextSvc::Stream, + ::Item, + >, + OptRcode, + > { + // https://datatracker.ietf.org/doc/html/rfc1995#section-2 + // 2. Brief Description of the Protocol + // "Transport of a query may be by either UDP or TCP. If an IXFR + // query is via UDP, the IXFR server may attempt to reply using UDP + // if the entire response can be contained in a single DNS packet. + // If the UDP reply does not fit, the query is responded to with a + // single SOA record of the server's current version to inform the + // client that a TCP query should be initiated." + // + // https://datatracker.ietf.org/doc/html/rfc1995#section-3 + // 3. Query Format + // "The IXFR query packet format is the same as that of a normal DNS + // query, but with the query type being IXFR and the authority + // section containing the SOA record of client's version of the + // zone." + // + // https://datatracker.ietf.org/doc/html/rfc1995#section-4 + // 4. Response Format + // "If incremental zone transfer is not available, the entire zone + // is returned. The first and the last RR of the response is the + // SOA record of the zone. I.e. the behavior is the same as an + // AXFR response except the query type is IXFR." + // + // https://datatracker.ietf.org/doc/html/rfc1995#section-2 + // 2. Brief Description of the Protocol + // "To ensure integrity, servers should use UDP checksums for all + // UDP responses. A cautious client which receives a UDP packet + // with a checksum value of zero should ignore the result and try a + // TCP IXFR instead." + if let AnswerContent::Data(rrset) = zone_soa_answer.content() { + if rrset.data().len() == 1 { + if let ZoneRecordData::Soa(soa) = + rrset.first().unwrap().data() + { + let zone_serial = soa.serial(); + + // TODO: if cached then return cached IXFR response + return Self::compute_ixfr( + batcher_semaphore, + req, + query_serial, + zone_serial, + zone_soa_answer, + diffs, + ) + .await; + } + } + } + + Err(OptRcode::SERVFAIL) + } + + #[allow(clippy::too_many_arguments)] + async fn compute_ixfr( + batcher_semaphore: Arc, + req: &Request, + query_serial: Serial, + zone_serial: Serial, + zone_soa_answer: &Answer, + diffs: Vec>, + ) -> Result< + XfrMiddlewareStream< + NextSvc::Future, + NextSvc::Stream, + ::Item, + >, + OptRcode, + > { + let msg = req.message(); + + let AnswerContent::Data(zone_soa_rrset) = + zone_soa_answer.content().clone() + else { + unreachable!() + }; + + // Note: Unlike RFC 5936 for AXFR, neither RFC 1995 nor RFC 9103 say + // anything about whether an IXFR response can consist of more than + // one response message, but given the 2^16 byte maximum response size + // of a TCP DNS message and the 2^16 maximum number of ANSWER RRs + // allowed per DNS response, large zones may not fit in a single + // response message and will have to be split into multiple response + // messages. + + // https://datatracker.ietf.org/doc/html/rfc1995#section-2 + // 2. Brief Description of the Protocol + // "If an IXFR query with the same or newer version number than that + // of the server is received, it is replied to with a single SOA + // record of the server's current version, just as in AXFR." + // ^^^^^^^^^^^^^^^ + // Errata https://www.rfc-editor.org/errata/eid3196 points out that + // this is NOT "just as in AXFR" as AXFR does not do that. + if query_serial >= zone_serial { + let builder = mk_builder_for_target(); + let response = zone_soa_answer.to_message(msg, builder); + trace!("IXFR finished because query_serial >= zone_serial"); + return Ok(Self::to_stream(response)); + } + + // Get the necessary diffs, if available + // let start_serial = query_serial; + // let end_serial = zone_serial; + // let diffs = zone_info.diffs_for_range(start_serial, end_serial).await; + // if diffs.is_empty() { + // trace!("No diff available for IXFR"); + // return IxfrResult::FallbackToAxfr; + // }; + + // TODO: Add something like the Bind `max-ixfr-ratio` option that + // "sets the size threshold (expressed as a percentage of the size of + // the full zone) beyond which named chooses to use an AXFR response + // rather than IXFR when answering zone transfer requests"? + + // Create a stream that will be immediately returned to the caller. + // Async tasks will then push DNS response messages into the stream as + // they become available. + let (sender, receiver) = unbounded_channel(); + let stream = UnboundedReceiverStream::new(receiver); + + // Notify the underlying transport to expect a stream of related + // responses. The transport should modify its behaviour to account for + // the potentially slow and long running nature of a transaction. + Self::add_to_stream( + CallResult::feedback_only(ServiceFeedback::BeginTransaction), + &sender, + ); + + // Stream the IXFR diffs in the background + let msg = msg.clone(); + let soft_byte_limit = Self::calc_msg_bytes_available(req); + + tokio::spawn(async move { + // Limit the number of concurrently running XFR batching + // operations. + if batcher_semaphore.acquire().await.is_err() { + error!( + "Internal error: Failed to acquire XFR batcher semaphore" + ); + return; + } + + // https://datatracker.ietf.org/doc/html/rfc1995#section-4 + // 4. Response Format + // ... + // "If incremental zone transfer is available, one or more + // difference sequences is returned. The list of difference + // sequences is preceded and followed by a copy of the server's + // current version of the SOA. + // + // Each difference sequence represents one update to the zone + // (one SOA serial change) consisting of deleted RRs and added + // RRs. The first RR of the deleted RRs is the older SOA RR + // and the first RR of the added RRs is the newer SOA RR. + // + // Modification of an RR is performed first by removing the + // original RR and then adding the modified one. + // + // The sequences of differential information are ordered oldest + // first newest last. Thus, the differential sequences are the + // history of changes made since the version known by the IXFR + // client up to the server's current version. + // + // RRs in the incremental transfer messages may be partial. That + // is, if a single RR of multiple RRs of the same RR type changes, + // only the changed RR is transferred." + + let (owner, qclass) = { + let Ok(q) = msg.sole_question() else { + unreachable!(); + }; + (q.qname().to_name::(), q.qclass()) + }; + + let mut batcher = XfrRrBatcher::build( + msg.clone(), + sender.clone(), + Some(soft_byte_limit), + None, + ); + + batcher + .push(( + owner.clone(), + qclass, + zone_soa_rrset.ttl(), + &zone_soa_rrset.data()[0], + )) + .unwrap(); // TODO + + for diff in diffs { + // 4. Response Format + // "Each difference sequence represents one update to the + // zone (one SOA serial change) consisting of deleted RRs + // and added RRs. The first RR of the deleted RRs is the + // older SOA RR and the first RR of the added RRs is the + // newer SOA RR. + let soa_k = &(owner.clone(), Rtype::SOA); + let removed_soa = diff.removed.get(soa_k).unwrap(); // The zone MUST have a SOA record + batcher + .push(( + owner.clone(), + qclass, + removed_soa.ttl(), + &removed_soa.data()[0], + )) + .unwrap(); // TODO + + diff.removed.iter().for_each(|((owner, rtype), rrset)| { + if *rtype != Rtype::SOA { + for rr in rrset.data() { + batcher + .push(( + owner.clone(), + qclass, + rrset.ttl(), + rr, + )) + .unwrap(); // TODO + } + } + }); + + let added_soa = diff.added.get(soa_k).unwrap(); // The zone MUST have a SOA record + batcher + .push(( + owner.clone(), + qclass, + added_soa.ttl(), + &added_soa.data()[0], + )) + .unwrap(); // TODO + + diff.added.iter().for_each(|((owner, rtype), rrset)| { + if *rtype != Rtype::SOA { + for rr in rrset.data() { + batcher + .push(( + owner.clone(), + qclass, + rrset.ttl(), + rr, + )) + .unwrap(); // TODO + } + } + }); + } + + batcher + .push(( + owner, + qclass, + zone_soa_rrset.ttl(), + &zone_soa_rrset.data()[0], + )) + .unwrap(); // TODO + + batcher.finish().unwrap(); // TODO + + trace!("Ending transaction"); + Self::add_to_stream( + CallResult::feedback_only(ServiceFeedback::EndTransaction), + &sender, + ); + }); + + Ok(MiddlewareStream::Result(stream)) + } + + fn add_to_stream( + call_result: CallResult, + sender: &UnboundedSender>, + ) { + sender.send(Ok(call_result)).unwrap(); // TODO: Handle this Result + } + + fn to_stream( + response: AdditionalBuilder>, + ) -> XfrMiddlewareStream< + NextSvc::Future, + NextSvc::Stream, + ::Item, + > { + let res = Ok(CallResult::new(response)); + MiddlewareStream::Map(once(ready(res))) + } + + #[allow(clippy::borrowed_box)] + async fn read_soa( + read: &Box, + qname: Name, + ) -> Result { + match read.is_async() { + true => read.query_async(qname, Rtype::SOA).await, + false => read.query(qname, Rtype::SOA), + } + } + + fn get_relevant_question( + msg: &Message, + ) -> Option>>> { + if Opcode::QUERY == msg.header().opcode() && !msg.header().qr() { + if let Ok(q) = msg.sole_question() { + if matches!(q.qtype(), Rtype::AXFR | Rtype::IXFR) { + return Some(q); + } + } + } + + None + } + + fn calc_msg_bytes_available(req: &Request) -> usize { + let bytes_available = match req.transport_ctx() { + TransportSpecificContext::Udp(ctx) => { + let max_msg_size = + ctx.max_response_size_hint().unwrap_or(512); + max_msg_size - req.num_reserved_bytes() + } + TransportSpecificContext::NonUdp(_) => { + 65535 - req.num_reserved_bytes() + } + }; + + bytes_available as usize + } +} + +//--- impl Service + +impl Service + for XfrMiddlewareSvc +where + RequestOctets: Octets + Send + Sync + Unpin + 'static, + for<'a> ::Range<'a>: Send + Sync, + NextSvc: Service + Clone + Send + Sync + 'static, + NextSvc::Future: Send + Sync + Unpin, + NextSvc::Target: Composer + Default + Send + Sync, + NextSvc::Stream: Send + Sync, + XDP: XfrDataProvider + Clone + Sync + Send + 'static, + Metadata: Clone + Default + Sync + Send + 'static, +{ + type Target = NextSvc::Target; + type Stream = XfrMiddlewareStream< + NextSvc::Future, + NextSvc::Stream, + ::Item, + >; + type Future = Pin + Send + Sync>>; + + fn call( + &self, + request: Request, + ) -> Self::Future { + let request = request.clone(); + let next_svc = self.next_svc.clone(); + let xfr_data_provider = self.xfr_data_provider.clone(); + let zone_walking_semaphore = self.zone_walking_semaphore.clone(); + let batcher_semaphore = self.batcher_semaphore.clone(); + Box::pin(async move { + match Self::preprocess( + zone_walking_semaphore, + batcher_semaphore, + &request, + xfr_data_provider, + ) + .await + { + ControlFlow::Continue(()) => { + let request = request.with_new_metadata(()); + let stream = next_svc.call(request).await; + MiddlewareStream::IdentityStream(stream) + } + ControlFlow::Break(stream) => stream, + } + }) + } +} + +//------------ XfrMapStream --------------------------------------------------- + +pub type XfrResultStream = UnboundedReceiverStream; + +//------------ XfrMiddlewareStream -------------------------------------------- + +pub type XfrMiddlewareStream = MiddlewareStream< + Future, + Stream, + Once>, + XfrResultStream, + StreamItem, +>; + +//------------ XfrMode -------------------------------------------------------- + +#[derive(Copy, Clone, Debug, PartialEq)] +pub enum XfrMode { + AxfrAndIxfr, + AxfrOnly, +} + +//------------ XfrDataProviderError ------------------------------------------- + +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +pub enum XfrDataProviderError { + UnknownZone, + + Refused, + + TemporarilyUnavailable, +} + +//------------ Transferable --------------------------------------------------- + +/// A provider of data needed for responding to XFR requests. +pub trait XfrDataProvider { + /// Request data needed to respond to an XFR request. + /// + /// Returns Ok if the request is allowed and the requested data is + /// available. + /// + /// Returns Err otherwise. + #[allow(clippy::type_complexity)] + fn request( + &self, + req: &Request, + apex_name: &impl ToName, + class: Class, + diff_to: Option, + ) -> Pin< + Box< + dyn Future< + Output = Result< + (Zone, Vec>), + XfrDataProviderError, + >, + > + Sync + + Send, + >, + > + where + Octs: AsRef<[u8]> + Send + Sync; +} + +//--- impl for AsRef + +impl XfrDataProvider for U +where + T: XfrDataProvider, + U: Deref, +{ + fn request( + &self, + req: &Request, + apex_name: &impl ToName, + class: Class, + diff_to: Option, + ) -> Pin< + Box< + dyn Future< + Output = Result< + (Zone, Vec>), + XfrDataProviderError, + >, + > + Sync + + Send, + >, + > + where + Octs: AsRef<[u8]> + Send + Sync, + { + (**self).request(req, apex_name, class, diff_to) + } +} + +//--- impl for Zone + +impl XfrDataProvider for Zone { + /// Request data needed to respond to an XFR request. + /// + /// Returns Ok(Self, vec![]) if the given apex name and class match this + /// zone, irrespective of the given request or diff range. + /// + /// Returns Err if the requested zone is not this zone. + fn request( + &self, + _req: &Request, + apex_name: &impl ToName, + class: Class, + _diff_to: Option, + ) -> Pin< + Box< + dyn Future< + Output = Result< + (Zone, Vec>), + XfrDataProviderError, + >, + > + Sync + + Send, + >, + > + where + Octs: AsRef<[u8]> + Send + Sync, + { + let res = if apex_name.to_name::() == self.apex_name() + && class == self.class() + { + Ok((self.clone(), vec![])) + } else { + Err(XfrDataProviderError::UnknownZone) + }; + + Box::pin(ready(res)) + } +} + +//--- impl for ZoneTree + +impl XfrDataProvider for ZoneTree { + /// Request data needed to respond to an XFR request. + /// + /// Returns Ok(zone, vec![]) if the given apex name and class match a zone + /// in this zone tree, irrespective of the given request or diff range. + /// + /// Returns Err if the requested zone is not this zone tree. + fn request( + &self, + _req: &Request, + apex_name: &impl ToName, + class: Class, + _diff_to: Option, + ) -> Pin< + Box< + dyn Future< + Output = Result< + (Zone, Vec>), + XfrDataProviderError, + >, + > + Sync + + Send, + >, + > + where + Octs: AsRef<[u8]> + Send + Sync, + { + let res = if let Some(zone) = self.get_zone(apex_name, class) { + Ok((zone.clone(), vec![])) + } else { + Err(XfrDataProviderError::UnknownZone) + }; + + Box::pin(ready(res)) + } +} + +//------------ XfrRrBatcher --------------------------------------------------- + +pub struct XfrRrBatcher { + _phantom: PhantomData<(RequestOctets, Target)>, +} + +impl XfrRrBatcher +where + RequestOctets: Octets + Sync + Send + 'static, + Target: Composer + Default + Send + 'static, +{ + pub fn build( + req_msg: Arc>, + sender: UnboundedSender>, + soft_byte_limit: Option, + hard_rr_limit: Option, + ) -> impl ResourceRecordBatcher { + let cb_state = CallbackState::new( + req_msg.clone(), + sender, + soft_byte_limit, + hard_rr_limit, + ); + + CallbackBatcher::< + RequestOctets, + Target, + Self, + CallbackState, + >::new(req_msg, cb_state) + } +} + +impl XfrRrBatcher +where + RequestOctets: Octets, + Target: Composer + Default, +{ + fn set_axfr_header( + msg: &Message, + additional: &mut AdditionalBuilder>, + ) { + // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2.1 + // 2.2.1: Header Values + // + // "These are the DNS message header values for AXFR responses. + // + // ID MUST be copied from request -- see Note a) + // + // QR MUST be 1 (Response) + // + // OPCODE MUST be 0 (Standard Query) + // + // Flags: + // AA normally 1 -- see Note b) + // TC MUST be 0 (Not truncated) + // RD RECOMMENDED: copy request's value; MAY be set to 0 + // RA SHOULD be 0 -- see Note c) + // Z "mbz" -- see Note d) + // AD "mbz" -- see Note d) + // CD "mbz" -- see Note d)" + let header = additional.header_mut(); + + // Note: MandatoryMiddlewareSvc will also "fix" ID and QR, so strictly + // speaking this isn't necessary, but as a caller might not use + // MandatoryMiddlewareSvc we do it anyway to try harder to conform to + // the RFC. + header.set_id(msg.header().id()); + header.set_qr(true); + + header.set_opcode(Opcode::QUERY); + header.set_aa(true); + header.set_tc(false); + header.set_rd(msg.header().rd()); + header.set_ra(false); + header.set_z(false); + header.set_ad(false); + header.set_cd(false); + } +} + +//--- Callbacks + +impl + Callbacks> + for XfrRrBatcher +where + RequestOctets: Octets, + Target: Composer + Default, +{ + fn batch_started( + cb_state: &CallbackState, + msg: &Message, + ) -> Result>, PushError> { + let mut builder = mk_builder_for_target(); + if let Some(limit) = cb_state.soft_byte_limit { + builder.set_push_limit(limit); + } + let answer = builder.start_answer(msg, Rcode::NOERROR)?; + Ok(answer) + } + + fn batch_ready( + cb_state: &CallbackState, + builder: AnswerBuilder>, + ) -> Result<(), ()> { + trace!("Sending RR batch"); + let mut additional = builder.additional(); + Self::set_axfr_header(&cb_state.req_msg, &mut additional); + let call_result = Ok(CallResult::new(additional)); + cb_state.sender.send(call_result).map_err(|err| { + warn!("Internal error: Send from RR batcher failed: {err}"); + }) + } + + fn record_pushed( + cb_state: &CallbackState, + answer: &AnswerBuilder>, + ) -> bool { + if let Some(hard_rr_limit) = cb_state.hard_rr_limit { + let ancount = answer.counts().ancount(); + let limit_reached = ancount == hard_rr_limit; + trace!( + "ancount={ancount}, hard_rr_limit={hard_rr_limit}, limit_reached={limit_reached}"); + limit_reached + } else { + false + } + } +} + +//------------ CallbackState -------------------------------------------------- + +struct CallbackState { + req_msg: Arc>, + sender: UnboundedSender>, + soft_byte_limit: Option, + hard_rr_limit: Option, +} + +impl CallbackState { + fn new( + req_msg: Arc>, + sender: UnboundedSender>, + soft_byte_limit: Option, + hard_rr_limit: Option, + ) -> Self { + Self { + req_msg, + sender, + soft_byte_limit, + hard_rr_limit, + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::base::{MessageBuilder, Ttl}; + use crate::net::server::message::NonUdpTransportContext; + use crate::zonetree::{Rrset, ZoneBuilder}; + use core::str::FromStr; + use futures::StreamExt; + use tokio::time::Instant; + + #[tokio::test] + async fn empty_axfr() { + let zone_walk_semaphore = Arc::new(Semaphore::new(1)); + let batcher_semaphore = Arc::new(Semaphore::new(1)); + + let client_addr = "127.0.0.1:12345".parse().unwrap(); + let received_at = Instant::now(); + let msg = MessageBuilder::new_vec(); + let mut msg = msg.question(); + msg.push((Name::vec_from_str("example.com").unwrap(), Rtype::AXFR)) + .unwrap(); + let msg = msg.into_message(); + let transport_specific = TransportSpecificContext::NonUdp( + NonUdpTransportContext::new(None), + ); + let metadata = (); + let req: Request, ()> = Request::new( + client_addr, + received_at, + msg, + transport_specific, + metadata, + ); + + let qname = Name::from_str("example.com").unwrap(); + let mut zone_soa_answer = Answer::new(Rcode::NOERROR); + let mut soa_rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(3600)); + let mname = Name::from_str("blah").unwrap(); + let rname = Name::from_str("blah").unwrap(); + let serial = Serial::now(); + let refresh = Ttl::from_secs(0); + let retry = Ttl::from_secs(0); + let expire = Ttl::from_secs(0); + let minimum = Ttl::from_secs(0); + let soa = + Soa::new(mname, rname, serial, refresh, retry, expire, minimum); + soa_rrset.push_data(ZoneRecordData::Soa(soa.clone())); + let soa_rrset = SharedRrset::new(soa_rrset); + zone_soa_answer.add_answer(soa_rrset.clone()); + + let mut zone = ZoneBuilder::new(qname.clone(), Class::IN); + zone.insert_rrset(&qname, soa_rrset).unwrap(); + let zone = zone.build(); + let read = zone.read(); + + let mut stream = + XfrMiddlewareSvc::, TestNextSvc, Zone>::do_axfr::<()>( + zone_walk_semaphore, + batcher_semaphore, + &req, + qname, + &zone_soa_answer, + read, + ) + .await + .unwrap(); + + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::BeginTransaction) + )); + + let msg = stream.next().await.unwrap().unwrap(); + let resp_builder = msg.into_inner().0.unwrap(); + let resp = resp_builder.as_message(); + assert!(resp.is_answer(req.message())); + let rec = resp.answer().unwrap().next().unwrap().unwrap(); + let rec = rec + .into_record::>>() + .unwrap() + .unwrap(); + let rcvd_soa = rec.into_data(); + assert_eq!(rcvd_soa, soa); + + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::EndTransaction) + )); + } + + #[derive(Clone)] + struct TestNextSvc; + + impl Service, ()> for TestNextSvc { + type Target = Vec; + type Stream = Once>>; + type Future = Ready; + + fn call(&self, _request: Request, ()>) -> Self::Future { + todo!() + } + } +} diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index f1366802b..657d6ba16 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -1,3 +1,5 @@ +use core::future::{ready, Future}; +use core::pin::Pin; use core::str::FromStr; use std::boxed::Box; @@ -6,6 +8,7 @@ use std::fs::File; use std::net::SocketAddr; use std::path::PathBuf; use std::result::Result; +use std::string::{String, ToString}; use std::sync::Arc; use std::time::Duration; use std::vec::Vec; @@ -16,9 +19,10 @@ use tracing::instrument; use tracing::{trace, warn}; use crate::base::iana::{Class, Rcode}; -use crate::base::name::{Name, ToName}; +use crate::base::name::ToName; use crate::base::net::IpAddr; use crate::base::wire::Composer; +use crate::base::Name; use crate::base::Rtype; use crate::net::client::request::{RequestMessage, RequestMessageMulti}; use crate::net::client::tsig::{ @@ -35,6 +39,8 @@ use crate::net::server::middleware::mandatory::MandatoryMiddlewareSvc; use crate::net::server::middleware::notify::{ Notifiable, NotifyError, NotifyMiddlewareSvc, }; +use crate::net::server::middleware::tsig::TsigMiddlewareSvc; +use crate::net::server::middleware::xfr::XfrMiddlewareSvc; use crate::net::server::service::{CallResult, Service, ServiceResult}; use crate::net::server::stream::StreamServer; use crate::net::server::util::{mk_builder_for_target, service_fn}; @@ -47,11 +53,9 @@ use crate::stelline::parse_stelline::{self, parse_file, Config, Matches}; use crate::stelline::simple_dgram_client; use crate::tsig::{Algorithm, Key, KeyName, KeyStore}; use crate::utils::base16; -use crate::zonefile::inplace::{Entry, ScannedRecord, Zonefile}; -use crate::zonetree::StoredName; -use core::future::{ready, Future}; -use core::pin::Pin; -use std::string::ToString; +use crate::zonefile::inplace::Zonefile; +use crate::zonetree::{Answer, Zone}; +use crate::zonetree::{StoredName, ZoneBuilder, ZoneTree}; //----------- Tests ---------------------------------------------------------- @@ -60,8 +64,6 @@ use std::string::ToString; /// /// Note: Adding or removing .rpl files on disk won't be detected until the /// test is re-compiled. -// #[cfg(feature = "mock-time")] # Needed for the cookies test but that is -// currently disabled by renaming it to .rpl.not. #[instrument(skip_all, fields(rpl = rpl_file.file_name().unwrap().to_str()))] #[rstest] #[tokio::test(start_paused = true)] @@ -73,8 +75,6 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { // Initialize tracing based logging. Override with env var RUST_LOG, e.g. // RUST_LOG=trace. DEBUG level will show the .rpl file name, Stelline step // numbers and types as they are being executed. - - use crate::net::server::middleware::tsig::TsigMiddlewareSvc; tracing_subscriber::fmt() .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) .with_thread_ids(true) @@ -103,7 +103,34 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { let dgram_server_conn = ClientServerChannel::new_dgram(); let stream_server_conn = ClientServerChannel::new_stream(); - let zonefile = server_config.zonefile.clone(); + // Build the test defined zone, if any. + let mut zones = ZoneTree::new(); + let zone = match &server_config.zone { + ServerZone { + zone_file: Some(zone_file), + .. + } => { + // This is a primary zone with content already defined. + Some(Zone::try_from(zone_file.clone()).unwrap()) + } + ServerZone { + zone_name: Some(zone_name), + zone_file: None, + } => { + // This is a secondary zone with content to be received via + // XFR. + let builder = ZoneBuilder::new( + Name::from_str(zone_name).unwrap(), + Class::IN, + ); + Some(builder.build()) + } + _ => None, + }; + if let Some(zone) = zone { + zones.insert_zone(zone).unwrap(); + } + let zones = Arc::new(zones); let with_cookies = server_config.cookies.enabled && server_config.cookies.secret.is_some(); @@ -123,7 +150,7 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { // it and without it having to know or do anything about it. // 1. Application logic service - let svc = service_fn(test_service, zonefile); + let svc = service_fn(test_service, zones.clone()); // 2. DNS COOKIES middleware service let svc = CookiesMiddlewareSvc::new(svc, secret) @@ -134,13 +161,17 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { let svc = EdnsMiddlewareSvc::new(svc).enable(server_config.edns_tcp_keepalive); - // 4. Mandatory DNS behaviour (e.g. RFC 1034/35 rules). - let svc = MandatoryMiddlewareSvc::new(svc); + // 4. XFR(-in) middleware service (XFR-out is handled by the + // ZoneMaintainer). + let svc = XfrMiddlewareSvc::, _, _>::new(svc, zones, 1); // 5. RFC 1996 NOTIFY support. let svc = NotifyMiddlewareSvc::new(svc, TestNotifyTarget); - // 6. TSIG message authentication. + // 6. Mandatory DNS behaviour (e.g. RFC 1034/35 rules). + let svc = MandatoryMiddlewareSvc::new(svc); + + // 7. TSIG message authentication. let svc = TsigMiddlewareSvc::new(svc, key_store.clone()); // NOTE: TSIG middleware *MUST* be the first middleware in the chain per @@ -148,7 +179,8 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { // in order to verify the signature, and has to sign outgoing messages in // their final state without any modification occuring thereafter. - // Create dgram and stream servers for answering requests + // 8. The dgram and stream servers that receive DNS queries and dispatch + // them to the service layers above. let (dgram_srv, stream_srv) = mk_servers( svc, &server_config, @@ -156,7 +188,11 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { stream_server_conn.clone(), ); - // Create a client factory for sending requests + // Create a client factory for creating DNS clients per Stelline STEP with + // the appropriate configuration (as defined by the .rpl content) to + // submit requests to our DNS servers. No actual network communication + // takes place, these clients and servers use a direct in-memory channel + // to exchange messages instead of actual network sockets. let client_factory = mk_client_factory(dgram_server_conn, stream_server_conn, key_store); @@ -334,70 +370,40 @@ fn mk_server_configs( #[allow(clippy::type_complexity)] fn test_service( request: Request, RequestMeta>, - zonefile: Zonefile, + zones: Arc, ) -> ServiceResult> { - fn as_record_and_dname( - r: ScannedRecord, - ) -> Option<(ScannedRecord, Name>)> { - let dname = r.owner().to_name(); - Some((r, dname)) - } - - fn as_records( - e: Result, - ) -> Option { - match e { - Ok(Entry::Record(r)) => Some(r), - Ok(_) => None, - Err(err) => panic!( - "Error while extracting records from the zonefile: {err}" - ), + let question = request.message().sole_question().unwrap(); + + let answer = match zones.find_zone(question.qname(), question.qclass()) { + Some(zone) => { + let readable_zone = zone.read(); + let qname = question.qname().to_bytes(); + let qtype = question.qtype(); + readable_zone.query(qname, qtype).unwrap() } - } + None => Answer::new(Rcode::NXDOMAIN), + }; - trace!("Service received request"); - trace!("Service is constructing a single response"); - // If given a single question: - let answer = request - .message() - .sole_question() - .ok() - .and_then(|q| { - // Walk the zone to find the queried name - zonefile - .clone() - .filter_map(as_records) - .filter_map(as_record_and_dname) - .find(|(_record, dname)| dname == q.qname()) - }) - .map_or_else( - || { - // The Qname was not found in the zone: - mk_builder_for_target() - .start_answer(request.message(), Rcode::NXDOMAIN) - .unwrap() - }, - |(record, _)| { - // Respond with the found record: - let mut answer = mk_builder_for_target() - .start_answer(request.message(), Rcode::NOERROR) - .unwrap(); - answer.push(record).unwrap(); - answer - }, - ); - - Ok(CallResult::new(answer.additional())) + let builder = mk_builder_for_target(); + let additional = answer.to_message(request.message(), builder); + Ok(CallResult::new(additional)) } //----------- Stelline config block parsing ----------------------------------- +#[derive(Default)] +struct ServerZone { + /// Used for an empty secondary zone. Ignored if zone_file is Some. + zone_name: Option, + zone_file: Option, +} + #[derive(Default)] struct ServerConfig<'a> { cookies: CookieConfig<'a>, edns_tcp_keepalive: bool, idle_timeout: Option, - zonefile: Zonefile, + zone: ServerZone, } #[derive(Default)] @@ -409,8 +415,9 @@ struct CookieConfig<'a> { fn parse_server_config(config: &Config) -> ServerConfig { let mut parsed_config = ServerConfig::default(); - let mut zone_file_bytes = VecDeque::::new(); let mut in_server_block = false; + let mut zone_name = None; + let mut zone_file_bytes = VecDeque::::new(); for line in config.lines() { if line.starts_with("server:") { @@ -463,9 +470,6 @@ fn parse_server_config(config: &Config) -> ServerConfig { } } ("local-data", v) => { - if !zone_file_bytes.is_empty() { - zone_file_bytes.push_back(b'\n'); - } zone_file_bytes .extend(v.trim_matches('"').as_bytes().iter()); zone_file_bytes.push_back(b'\n'); @@ -480,6 +484,10 @@ fn parse_server_config(config: &Config) -> ServerConfig { ); } } + ("zone", v) => { + // zone: + zone_name = Some(v.to_string()); + } _ => { eprintln!("Ignoring unknown server setting '{setting}' with value: {value}"); } @@ -488,10 +496,13 @@ fn parse_server_config(config: &Config) -> ServerConfig { } } - if !zone_file_bytes.is_empty() { - parsed_config.zonefile = - Zonefile::load(&mut zone_file_bytes).unwrap(); - } + let zone_file = (!zone_file_bytes.is_empty()) + .then(|| Zonefile::load(&mut zone_file_bytes).unwrap()); + + parsed_config.zone = ServerZone { + zone_name, + zone_file, + }; parsed_config } diff --git a/src/zonetree/in_memory/read.rs b/src/zonetree/in_memory/read.rs index de71e11d2..4a8548440 100644 --- a/src/zonetree/in_memory/read.rs +++ b/src/zonetree/in_memory/read.rs @@ -9,6 +9,7 @@ use crate::base::iana::{Rcode, Rtype}; use crate::base::name::Label; use crate::base::Name; use crate::zonetree::answer::{Answer, AnswerAuthority}; +use crate::zonetree::error::OutOfZone; use crate::zonetree::types::ZoneCut; use crate::zonetree::walk::WalkState; use crate::zonetree::{ReadableZone, Rrset, SharedRr, SharedRrset, WalkOp}; @@ -16,7 +17,6 @@ use crate::zonetree::{ReadableZone, Rrset, SharedRr, SharedRrset, WalkOp}; use super::nodes::{NodeChildren, NodeRrsets, Special, ZoneApex, ZoneNode}; use super::versioned::Version; use super::versioned::VersionMarker; -use crate::zonetree::error::OutOfZone; //------------ ReadZone ------------------------------------------------------ @@ -86,22 +86,44 @@ impl ReadZone { ) -> NodeAnswer { node.with_special(self.version, |special| match special { Some(Special::Cut(ref cut)) => { - let answer = NodeAnswer::authority(AnswerAuthority::new( - cut.name.clone(), - None, - Some(cut.ns.clone()), - cut.ds.as_ref().cloned(), - )); - - walk.op(&cut.ns); - if let Some(ds) = &cut.ds { - walk.op(ds); + if walk.enabled() { + walk.op(&cut.ns); + if let Some(ds) = &cut.ds { + walk.op(ds); + } + NodeAnswer::no_data() + } else { + // There is nothing more in this zone, only a cut here. + // Respond with NODATA and an authority section referring the + // client to the nameserver that should know more. + NodeAnswer::authority(AnswerAuthority::new( + cut.name.clone(), + None, + Some(cut.ns.clone()), + cut.ds.as_ref().cloned(), + )) } - - answer } Some(Special::NxDomain) => NodeAnswer::nx_domain(), - Some(Special::Cname(_)) | None => self.query_children( + Some(Special::Cname(cname)) => { + if walk.enabled() { + let mut rrset = Rrset::new(Rtype::CNAME, cname.ttl()); + rrset.push_data(cname.data().clone()); + walk.op(&SharedRrset::new(rrset)); + } + + // As we are querying for a qname below this node this + // means that the CNAME at this node cannot be a match, so + // ignore it and look for a match in the children. + self.query_children( + node.children(), + label, + qname, + qtype, + walk, + ) + } + None => self.query_children( node.children(), label, qname, @@ -118,25 +140,8 @@ impl ReadZone { walk: WalkState, ) -> NodeAnswer { node.with_special(self.version, |special| match special { - Some(Special::Cut(cut)) => { - let answer = self.query_at_cut(cut, qtype); - if walk.enabled() { - walk.op(&cut.ns); - if let Some(ds) = &cut.ds { - walk.op(ds); - } - } - answer - } - Some(Special::Cname(cname)) => { - let answer = NodeAnswer::cname(cname.clone()); - if walk.enabled() { - let mut rrset = Rrset::new(Rtype::CNAME, cname.ttl()); - rrset.push_data(cname.data().clone()); - walk.op(&rrset); - } - answer - } + Some(Special::Cut(cut)) => self.query_at_cut(cut, qtype), + Some(Special::Cname(cname)) => NodeAnswer::cname(cname.clone()), Some(Special::NxDomain) => NodeAnswer::nx_domain(), None => self.query_rrsets(node.rrsets(), qtype, walk), }) @@ -182,9 +187,9 @@ impl ReadZone { // response." // // We choose for option 1 because option 2 would create lots of - // extra work in the offline signing case (because lots of HFINO + // extra work in the offline signing case (because lots of HINFO // records would need to be synthesized prior to signing) and - // option 3 as stated may still result in a large response. + // option 3, as stated, may still result in a large response. let guard = rrsets.iter(); guard .iter() @@ -284,15 +289,19 @@ impl ReadableZone for ReadZone { } fn walk(&self, op: WalkOp) { - // https://datatracker.ietf.org/doc/html/rfc8482 notes that the ANY - // query type is problematic and should be answered as minimally as - // possible. Rather than use ANY internally here to achieve a walk, as - // specific behaviour may actually be wanted for ANY we instead use - // the presence of a callback `op` to indicate that walking mode is + // The presence of a callback `op` indicates that walking mode is // requested. We still have to pass an Rtype but it won't be used for // matching when in walk mode, so we set it to Any as it most closely // matches our intent and will be ignored anyway. - let walk = WalkState::new(op); + // + // The walk is single threaded. With an empty callback function on a + // "13th Gen Intel(R) Core(TM) i9-13900K" over 43,347,447 resource + // records the walk took ~6 seconds, compared to 47 seconds for the + // callback function to emit the same records as DNS messages and for + // dig to receive the entire zone via AXFR: + // + // dig -4 @127.0.0.1 -p 8053 +noanswer +tries=1 +noidnout AXFR de. + let walk = WalkState::new(op, self.apex.name().clone()); self.query_rrsets(self.apex.rrsets(), Rtype::ANY, walk.clone()); self.query_below_apex(Label::root(), iter::empty(), Rtype::ANY, walk); } diff --git a/src/zonetree/parsed.rs b/src/zonetree/parsed.rs index 765725b63..dc4cab13f 100644 --- a/src/zonetree/parsed.rs +++ b/src/zonetree/parsed.rs @@ -3,7 +3,6 @@ use std::collections::{BTreeMap, HashMap}; use std::vec::Vec; -use super::error::{ContextError, RecordError, ZoneErrors}; use crate::base::iana::{Class, Rtype}; use crate::base::name::{FlattenInto, ToName}; use crate::base::Name; @@ -12,6 +11,7 @@ use crate::zonefile::inplace::{self, Entry}; use crate::zonetree::ZoneBuilder; use crate::zonetree::{Rrset, SharedRr}; +use super::error::{ContextError, RecordError, ZoneErrors}; use super::types::{StoredName, StoredRecord}; //------------ Zonefile ------------------------------------------------------ diff --git a/src/zonetree/walk.rs b/src/zonetree/walk.rs index 3c350abe2..6ba2ec4df 100644 --- a/src/zonetree/walk.rs +++ b/src/zonetree/walk.rs @@ -2,28 +2,29 @@ use std::boxed::Box; use std::sync::{Arc, Mutex}; use std::vec::Vec; -use bytes::Bytes; +use super::{SharedRrset, StoredName}; -use super::Rrset; use crate::base::name::OwnedLabel; -use crate::base::{Name, NameBuilder}; +use crate::base::NameBuilder; /// A callback function invoked for each leaf node visited while walking a /// [`Zone`]. /// /// [`Zone`]: super::Zone -pub type WalkOp = Box, &Rrset) + Send + Sync>; +pub type WalkOp = Box; struct WalkStateInner { op: WalkOp, label_stack: Mutex>, + apex_name: StoredName, } impl WalkStateInner { - fn new(op: WalkOp) -> Self { + fn new(op: WalkOp, apex_name: StoredName) -> Self { Self { op, label_stack: Default::default(), + apex_name, } } } @@ -36,9 +37,9 @@ pub(super) struct WalkState { impl WalkState { pub(super) const DISABLED: WalkState = WalkState { inner: None }; - pub(super) fn new(op: WalkOp) -> Self { + pub(super) fn new(op: WalkOp, apex_name: StoredName) -> Self { Self { - inner: Some(Arc::new(WalkStateInner::new(op))), + inner: Some(Arc::new(WalkStateInner::new(op, apex_name))), } } @@ -46,14 +47,14 @@ impl WalkState { self.inner.is_some() } - pub(super) fn op(&self, rrset: &Rrset) { + pub(super) fn op(&self, rrset: &SharedRrset) { if let Some(inner) = &self.inner { let labels = inner.label_stack.lock().unwrap(); let mut dname = NameBuilder::new_bytes(); for label in labels.iter().rev() { dname.append_label(label.as_slice()).unwrap(); } - let owner = dname.into_name().unwrap(); + let owner = dname.append_origin(&inner.apex_name).unwrap(); (inner.op)(owner, rrset); } } diff --git a/test-data/server/axfr_large_out.rpl b/test-data/server/axfr_large_out.rpl new file mode 100644 index 000000000..f28ed5045 --- /dev/null +++ b/test-data/server/axfr_large_out.rpl @@ -0,0 +1,1055 @@ +;------------ Server configuration -------------------------------------------- + +server: + ; Permit the server to respond to clients that send XFR requests from + ; 127.0.0.1 (the default). No TSIG signing is expected. + provide-xfr: 127.0.0.1 NOKEY + + ; Permit the server to respond to clients that send XFR requests from + ; 127.0.0.2. Responses will be sent in "backward compatible" mode as + ; defined by RFC 5936. + ; See: https://datatracker.ietf.org/doc/html/rfc5936#section-7.1. + provide-xfr: 127.0.0.2 NOKEY COMPATIBLE + + ; Define an in-memory zone to be served by the server. The zone is large + ; enough that it cannot be fit into a single TCP AXFR response and so must + ; be split by the server across multiple responses. + local-data: "$ORIGIN example.com." + local-data: "example.com. 3600 IN SOA ns.example.com. hostmaster.example.com. 1 3600 900 86400 3600" + local-data: "example.com. 3600 IN NS ns.example.net." + local-data: "txt0000 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0001 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0002 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0003 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0004 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0005 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0006 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0007 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0008 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0009 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0010 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0011 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0012 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0013 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0014 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0015 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0016 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0017 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0018 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0019 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0020 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0021 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0022 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0023 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0024 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0025 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0026 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0027 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0028 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0029 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0030 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0031 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0032 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0033 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0034 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0035 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0036 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0037 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0038 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0039 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0040 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0041 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0042 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0043 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0044 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0045 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0046 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0047 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0048 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0049 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0050 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0051 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0052 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0053 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0054 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0055 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0056 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0057 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0058 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0059 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0060 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0061 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0062 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0063 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0064 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0065 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0066 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0067 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0068 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0069 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0070 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0071 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0072 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0073 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0074 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0075 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0076 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0077 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0078 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0079 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0080 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0081 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0082 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0083 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0084 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0085 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0086 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0087 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0088 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0089 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0090 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0091 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0092 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0093 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0094 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0095 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0096 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0097 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0098 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0099 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0100 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0101 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0102 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0103 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0104 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0105 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0106 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0107 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0108 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0109 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0110 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0111 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0112 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0113 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0114 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0115 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0116 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0117 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0118 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0119 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0120 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0121 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0122 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0123 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0124 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0125 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0126 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0127 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0128 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0129 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0130 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0131 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0132 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0133 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0134 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0135 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0136 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0137 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0138 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0139 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0140 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0141 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0142 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0143 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0144 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0145 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0146 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0147 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0148 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0149 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0150 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0151 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0152 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0153 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0154 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0155 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0156 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0157 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0158 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0159 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0160 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0161 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0162 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0163 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0164 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0165 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0166 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0167 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0168 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0169 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0170 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0171 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0172 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0173 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0174 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0175 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0176 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0177 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0178 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0179 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0180 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0181 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0182 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0183 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0184 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0185 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0186 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0187 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0188 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0189 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0190 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0191 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0192 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0193 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0194 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0195 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0196 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0197 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0198 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0199 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0200 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0201 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0202 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0203 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0204 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0205 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0206 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0207 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0208 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0209 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0210 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0211 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0212 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0213 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0214 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0215 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0216 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0217 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0218 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0219 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0220 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0221 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0222 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0223 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0224 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0225 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0226 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0227 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0228 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0229 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0230 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0231 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0232 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0233 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0234 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0235 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0236 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0237 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0238 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0239 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0240 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0241 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0242 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0243 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0244 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0245 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0246 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0247 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0248 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0249 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0250 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0251 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0252 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0253 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0254 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0255 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0256 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0257 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0258 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0259 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0260 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0261 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0262 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0263 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0264 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0265 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0266 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0267 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0268 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0269 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0270 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0271 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0272 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0273 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0274 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0275 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0276 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0277 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0278 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0279 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0280 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0281 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0282 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0283 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0284 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0285 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0286 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0287 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0288 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0289 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0290 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0291 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0292 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0293 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0294 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0295 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0296 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0297 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0298 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0299 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0300 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0301 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0302 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0303 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0304 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0305 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0306 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0307 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0308 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0309 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0310 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0311 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0312 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0313 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0314 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0315 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0316 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0317 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0318 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0319 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0320 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0321 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0322 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0323 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0324 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0325 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0326 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0327 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0328 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0329 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0330 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0331 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0332 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0333 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0334 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0335 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0336 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0337 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0338 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0339 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0340 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0341 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0342 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0343 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0344 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0345 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0346 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0347 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0348 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0349 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0350 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0351 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0352 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0353 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0354 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0355 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0356 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0357 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0358 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0359 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0360 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0361 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0362 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0363 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0364 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0365 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0366 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0367 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0368 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0369 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0370 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0371 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0372 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0373 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0374 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0375 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0376 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0377 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0378 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0379 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0380 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0381 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0382 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0383 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0384 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0385 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0386 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0387 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0388 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0389 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0390 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0391 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0392 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0393 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0394 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0395 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0396 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0397 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0398 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0399 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0400 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0401 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0402 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0403 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0404 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0405 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0406 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0407 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0408 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0409 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0410 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0411 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0412 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0413 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0414 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0415 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0416 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0417 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0418 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0419 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0420 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0421 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0422 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0423 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0424 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0425 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0426 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0427 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0428 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0429 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0430 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0431 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0432 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0433 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0434 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0435 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0436 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0437 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0438 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0439 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0440 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0441 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0442 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0443 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0444 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0445 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0446 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0447 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0448 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0449 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0450 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0451 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0452 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0453 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0454 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0455 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0456 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0457 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0458 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0459 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0460 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0461 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0462 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0463 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0464 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0465 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0466 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0467 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0468 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0469 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0470 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0471 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0472 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0473 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0474 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0475 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0476 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0477 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0478 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0479 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0480 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0481 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0482 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0483 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0484 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0485 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0486 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0487 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0488 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0489 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0490 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0491 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0492 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0493 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0494 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0495 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0496 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0497 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0498 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." + local-data: "txt0499 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible." +CONFIG_END + +;------------ Test definition ------------------------------------------------ + +SCENARIO_BEGIN Test large AXFR out. + +;--- Mock replies + +; None + +;--- Test steps + +; Retrieve the zone via AXFR over TCP from the server. +STEP 10 QUERY +ENTRY_BEGIN +MATCH TCP +SECTION QUESTION + example.com. IN AXFR +ENTRY_END + +; NOTE: See test-data/server/README.md regarding order of RRs across multiple +; response packets. +STEP 11 CHECK_ANSWER +ENTRY_BEGIN +MATCH all EXTRA_PACKETS +REPLY QR AA NOERROR +SECTION QUESTION + example.com. IN AXFR +SECTION ANSWER + $ORIGIN example.com. + example.com. 3600 IN SOA ns.example.com. hostmaster.example.com. 1 3600 900 86400 3600 + example.com. IN NS ns.example.net. + txt0000 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0001 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0002 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0003 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0004 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0005 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0006 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0007 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0008 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0009 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0010 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0011 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0012 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0013 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0014 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0015 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0016 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0017 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0018 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0019 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0020 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0021 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0022 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0023 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0024 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0025 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0026 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0027 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0028 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0029 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0030 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0031 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0032 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0033 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0034 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0035 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0036 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0037 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0038 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0039 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0040 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0041 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0042 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0043 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0044 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0045 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0046 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0047 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0048 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0049 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0050 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0051 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0052 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0053 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0054 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0055 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0056 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0057 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0058 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0059 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0060 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0061 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0062 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0063 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0064 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0065 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0066 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0067 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0068 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0069 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0070 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0071 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0072 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0073 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0074 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0075 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0076 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0077 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0078 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0079 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0080 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0081 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0082 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0083 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0084 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0085 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0086 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0087 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0088 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0089 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0090 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0091 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0092 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0093 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0094 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0095 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0096 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0097 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0098 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0099 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0100 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0101 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0102 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0103 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0104 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0105 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0106 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0107 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0108 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0109 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0110 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0111 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0112 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0113 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0114 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0115 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0116 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0117 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0118 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0119 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0120 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0121 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0122 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0123 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0124 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0125 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0126 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0127 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0128 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0129 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0130 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0131 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0132 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0133 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0134 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0135 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0136 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0137 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0138 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0139 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0140 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0141 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0142 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0143 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0144 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0145 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0146 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0147 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0148 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0149 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0150 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0151 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0152 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0153 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0154 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0155 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0156 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0157 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0158 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0159 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0160 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0161 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0162 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0163 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0164 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0165 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0166 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0167 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0168 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0169 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0170 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0171 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0172 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0173 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0174 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0175 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0176 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0177 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0178 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0179 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0180 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0181 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0182 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0183 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0184 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0185 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0186 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0187 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0188 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0189 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0190 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0191 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0192 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0193 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0194 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0195 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0196 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0197 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0198 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0199 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0200 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0201 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0202 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0203 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0204 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0205 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0206 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0207 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0208 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0209 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0210 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0211 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0212 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0213 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0214 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0215 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0216 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0217 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0218 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0219 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0220 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0221 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0222 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0223 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0224 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0225 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0226 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0227 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0228 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0229 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0230 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0231 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0232 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0233 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0234 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0235 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0236 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0237 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0238 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0239 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0240 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0241 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0242 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0243 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0244 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0245 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0246 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0247 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0248 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0249 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0250 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0251 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0252 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0253 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0254 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0255 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0256 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0257 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0258 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0259 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0260 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0261 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0262 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0263 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0264 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0265 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0266 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0267 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0268 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0269 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0270 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0271 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0272 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0273 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0274 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0275 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0276 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0277 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0278 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0279 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0280 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0281 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0282 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0283 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0284 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0285 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0286 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0287 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0288 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0289 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0290 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0291 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0292 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0293 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0294 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0295 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0296 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0297 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0298 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0299 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0300 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0301 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0302 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0303 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0304 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0305 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0306 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0307 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0308 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0309 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0310 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0311 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0312 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0313 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0314 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0315 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0316 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0317 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0318 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0319 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0320 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0321 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0322 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0323 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0324 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0325 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0326 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0327 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0328 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0329 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0330 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0331 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0332 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0333 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0334 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0335 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0336 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0337 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0338 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0339 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0340 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0341 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0342 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0343 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0344 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0345 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0346 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0347 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0348 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0349 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0350 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0351 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0352 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0353 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0354 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0355 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0356 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0357 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0358 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0359 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0360 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0361 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0362 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0363 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0364 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0365 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0366 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0367 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0368 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0369 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0370 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0371 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0372 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0373 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0374 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0375 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0376 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0377 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0378 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0379 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0380 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0381 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0382 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0383 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0384 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0385 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0386 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0387 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0388 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0389 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0390 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0391 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0392 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0393 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0394 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0395 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0396 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0397 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0398 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0399 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0400 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0401 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0402 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0403 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0404 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0405 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0406 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0407 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0408 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0409 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0410 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0411 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0412 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0413 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0414 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0415 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0416 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0417 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0418 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0419 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0420 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0421 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0422 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0423 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0424 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0425 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0426 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0427 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0428 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0429 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0430 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0431 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0432 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0433 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0434 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0435 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0436 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0437 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0438 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0439 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0440 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0441 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0442 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0443 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0444 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0445 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0446 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0447 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0448 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0449 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0450 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0451 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0452 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0453 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0454 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0455 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0456 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0457 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0458 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0459 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0460 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0461 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0462 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0463 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0464 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0465 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0466 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0467 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0468 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0469 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0470 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0471 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0472 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0473 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0474 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0475 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0476 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0477 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0478 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0479 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0480 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0481 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0482 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0483 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0484 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0485 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0486 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0487 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0488 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0489 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0490 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0491 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0492 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0493 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0494 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0495 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0496 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0497 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0498 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + txt0499 3600 IN CNAME some.very.long.domain.name.that.takes.up.a.lot.of.bytes.so.as.to.fill.the.dns.response.with.as.few.rrs.as.possible. + example.com. 3600 IN SOA ns.example.com. hostmaster.example.com. 1 3600 900 86400 3600 +ENTRY_END + +SCENARIO_END \ No newline at end of file diff --git a/test-data/server/axfr_out.rpl b/test-data/server/axfr_out.rpl new file mode 100644 index 000000000..01efe368d --- /dev/null +++ b/test-data/server/axfr_out.rpl @@ -0,0 +1,140 @@ +; Based on https://github.com/NLnetLabs/unbound/blob/172b84f7ce6507e96fe51bd94448222a5a47274b/testdata/auth_xfr.rpl + +;------------ Server configuration -------------------------------------------- + +server: + provide-xfr: 127.0.0.1 NOKEY + provide-xfr: 127.0.0.2 NOKEY COMPATIBLE + + ; Define an in-memory zone to be served by the server. + local-data: "com. 900 IN SOA a.gtld-servers.net. nstld.verisign-grs.com. 1720688795 1800 900 604800 86400" + local-data: "com. 166972 IN NS a.gtld-servers.net." + local-data: "com. 166972 IN NS b.gtld-servers.net." + local-data: "com. 166972 IN NS c.gtld-servers.net." + local-data: "com. 166972 IN NS d.gtld-servers.net." + local-data: "com. 166972 IN NS e.gtld-servers.net." + local-data: "com. 166972 IN NS f.gtld-servers.net." + local-data: "com. 166972 IN NS g.gtld-servers.net." + local-data: "com. 166972 IN NS h.gtld-servers.net." + local-data: "com. 166972 IN NS i.gtld-servers.net." + local-data: "com. 166972 IN NS j.gtld-servers.net." + local-data: "com. 166972 IN NS k.gtld-servers.net." + local-data: "com. 166972 IN NS l.gtld-servers.net." + local-data: "com. 166972 IN NS m.gtld-servers.net." + local-data: "com. 86400 IN DNSKEY 257 3 13 tx8EZRAd2+K/DJRV0S+hbBzaRPS/G6JVNBitHzqpsGlz8huE61Ms9ANe 6NSDLKJtiTBqfTJWDAywEp1FCsEINQ==" + local-data: "com. 86400 IN DNSKEY 256 3 13 Nps5nxuQHRbY3e9hcbH36kxiELJH5wil+6dC4K1keQI9ci1nqyCP4k1X oXBBn2aeSK4KxwPEs0Opqc0dicuujg==" + local-data: "com. 86400 IN DNSKEY 256 3 13 cCRwZIITlPXwDm0yKpGVYSmWLL4OpEHxA7+Rt3jS0W1N4EMOaF8doSzr JuM7aDbgAR7jtQ9SNCvYZCH2xSyfaQ==" + local-data: "alt.com. 3600 IN CNAME example.com." + local-data: "a.alt.com. 3600 IN A 1.2.3.4" + local-data: "example.com. 172800 IN NS a.iana-servers.net." + local-data: "example.com. 172800 IN NS b.iana-servers.net." + local-data: "example.com. 86400 IN DS 370 13 2 BE74359954660069D5C63D200C39F5603827D7DD02B56F120EE9F3A8 6764247C" + local-data: "www.terminal.com. 3600 IN A 1.2.3.4" + local-data: "alt.terminal.com. 3600 IN CNAME www.example.com." +CONFIG_END + +;------------ Test definition ------------------------------------------------ + +SCENARIO_BEGIN Test AXFR out. + +; Note: It is not currently possible to construct a UDP AXFR query so we cannot +; test that the server refuses the request. This instead results in FORMERR +; during request construction rather than REFUSED from the server. +;STEP 10 QUERY +;ENTRY_BEGIN +;MATCH UDP +;SECTION QUESTION +; com. IN AXFR +;ENTRY_END +; +;STEP 11 CHECK_ANSWER +;ENTRY_BEGIN +;MATCH all +;REPLY QR AA REFUSED +;ENTRY_END + +; Retrieve the zone via AXFR from the server +STEP 20 QUERY +ENTRY_BEGIN +MATCH TCP +SECTION QUESTION + com. IN AXFR +ENTRY_END + +STEP 21 CHECK_ANSWER +ENTRY_BEGIN +MATCH all +REPLY QR AA NOERROR +SECTION QUESTION + com. IN AXFR +SECTION ANSWER + com. 900 IN SOA a.gtld-servers.net. nstld.verisign-grs.com. 1720688795 1800 900 604800 86400 + com. 166972 IN NS a.gtld-servers.net. + com. 166972 IN NS b.gtld-servers.net. + com. 166972 IN NS c.gtld-servers.net. + com. 166972 IN NS d.gtld-servers.net. + com. 166972 IN NS e.gtld-servers.net. + com. 166972 IN NS f.gtld-servers.net. + com. 166972 IN NS g.gtld-servers.net. + com. 166972 IN NS h.gtld-servers.net. + com. 166972 IN NS i.gtld-servers.net. + com. 166972 IN NS j.gtld-servers.net. + com. 166972 IN NS k.gtld-servers.net. + com. 166972 IN NS l.gtld-servers.net. + com. 166972 IN NS m.gtld-servers.net. + com. 86400 IN DNSKEY 257 3 13 tx8EZRAd2+K/DJRV0S+hbBzaRPS/G6JVNBitHzqpsGlz8huE61Ms9ANe 6NSDLKJtiTBqfTJWDAywEp1FCsEINQ== + com. 86400 IN DNSKEY 256 3 13 Nps5nxuQHRbY3e9hcbH36kxiELJH5wil+6dC4K1keQI9ci1nqyCP4k1X oXBBn2aeSK4KxwPEs0Opqc0dicuujg== + com. 86400 IN DNSKEY 256 3 13 cCRwZIITlPXwDm0yKpGVYSmWLL4OpEHxA7+Rt3jS0W1N4EMOaF8doSzr JuM7aDbgAR7jtQ9SNCvYZCH2xSyfaQ== + alt.com. 3600 IN CNAME example.com. + a.alt.com. 3600 IN A 1.2.3.4 + example.com. 172800 IN NS a.iana-servers.net. + example.com. 172800 IN NS b.iana-servers.net. + example.com. 86400 IN DS 370 13 2 BE74359954660069D5C63D200C39F5603827D7DD02B56F120EE9F3A8 6764247C + www.terminal.com. 3600 IN A 1.2.3.4 + alt.terminal.com. 3600 IN CNAME www.example.com. + com. 900 IN SOA a.gtld-servers.net. nstld.verisign-grs.com. 1720688795 1800 900 604800 86400 +ENTRY_END + +; Retrieve the zone via backward compatible AXFR from the server +STEP 30 QUERY ADDRESS 127.0.0.2 +ENTRY_BEGIN +MATCH TCP +SECTION QUESTION + com. IN AXFR +ENTRY_END + +STEP 31 CHECK_ANSWER +ENTRY_BEGIN +MATCH all EXTRA_PACKETS +REPLY QR AA NOERROR +SECTION QUESTION + com. IN AXFR +SECTION ANSWER + com. 900 IN SOA a.gtld-servers.net. nstld.verisign-grs.com. 1720688795 1800 900 604800 86400 + com. 166972 IN NS a.gtld-servers.net. + com. 166972 IN NS b.gtld-servers.net. + com. 166972 IN NS c.gtld-servers.net. + com. 166972 IN NS d.gtld-servers.net. + com. 166972 IN NS e.gtld-servers.net. + com. 166972 IN NS f.gtld-servers.net. + com. 166972 IN NS g.gtld-servers.net. + com. 166972 IN NS h.gtld-servers.net. + com. 166972 IN NS i.gtld-servers.net. + com. 166972 IN NS j.gtld-servers.net. + com. 166972 IN NS k.gtld-servers.net. + com. 166972 IN NS l.gtld-servers.net. + com. 166972 IN NS m.gtld-servers.net. + com. 86400 IN DNSKEY 256 3 13 Nps5nxuQHRbY3e9hcbH36kxiELJH5wil+6dC4K1keQI9ci1nqyCP4k1X oXBBn2aeSK4KxwPEs0Opqc0dicuujg== + com. 86400 IN DNSKEY 256 3 13 cCRwZIITlPXwDm0yKpGVYSmWLL4OpEHxA7+Rt3jS0W1N4EMOaF8doSzr JuM7aDbgAR7jtQ9SNCvYZCH2xSyfaQ== + com. 86400 IN DNSKEY 257 3 13 tx8EZRAd2+K/DJRV0S+hbBzaRPS/G6JVNBitHzqpsGlz8huE61Ms9ANe 6NSDLKJtiTBqfTJWDAywEp1FCsEINQ== + alt.terminal.com. 3600 IN CNAME www.example.com. + a.alt.com. 3600 IN A 1.2.3.4 + example.com. 172800 IN NS a.iana-servers.net. + example.com. 172800 IN NS b.iana-servers.net. + example.com. 86400 IN DS 370 13 2 BE74359954660069D5C63D200C39F5603827D7DD02B56F120EE9F3A8 6764247C + www.terminal.com. 3600 IN A 1.2.3.4 + alt.com. 3600 IN CNAME example.com. + com. 900 IN SOA a.gtld-servers.net. nstld.verisign-grs.com. 1720688795 1800 900 604800 86400 +ENTRY_END + +SCENARIO_END \ No newline at end of file diff --git a/test-data/server/edns_downstream_cookies.rpl b/test-data/server/edns_downstream_cookies.rpl index f20ed4096..a1a7bb1a7 100644 --- a/test-data/server/edns_downstream_cookies.rpl +++ b/test-data/server/edns_downstream_cookies.rpl @@ -6,8 +6,10 @@ server: cookie-secret: "000102030405060708090a0b0c0d0e0f" access-control: 127.0.0.1 allow_cookie access-control: 1.2.3.4 allow - local-data: "test. TXT test" + ; Define an in-memory zone to be served by the server. + local-data: "test. 3600 IN SOA ns.test. hostmaster.test. 1 3600 900 86400 3600" + local-data: "test. TXT test" CONFIG_END SCENARIO_BEGIN Test downstream DNS Cookies diff --git a/test-data/server/edns_keepalive.rpl b/test-data/server/edns_keepalive.rpl index e9f24e0c8..51153227f 100644 --- a/test-data/server/edns_keepalive.rpl +++ b/test-data/server/edns_keepalive.rpl @@ -1,8 +1,11 @@ ; Based on: https://github.com/NLnetLabs/unbound/blob/49e425810275917e7fd09a24bae3b97d83b55c13/testdata/edns_keepalive.rpl server: - edns-tcp-keepalive: yes - edns-tcp-keepalive-timeout: 30000 - local-data: "test. TXT test" + edns-tcp-keepalive: yes + edns-tcp-keepalive-timeout: 30000 + + ; Define an in-memory zone to be served by the server. + local-data: "test. 3600 IN SOA ns.test. hostmaster.test. 1 3600 900 86400 3600" + local-data: "test. TXT test" CONFIG_END SCENARIO_BEGIN TCP Keepalive From c4ac96ed55f5dbca6a42045c0e35b3a12d1c52e4 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 6 Sep 2024 00:09:02 +0200 Subject: [PATCH 122/333] Add accidentally removed error handling. --- src/net/server/middleware/xfr.rs | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 31bde13a5..2decbab31 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -179,9 +179,15 @@ where XfrDataProviderError::Refused => { ("access denied", OptRcode::REFUSED) } + }) + .map_err(|(reason, rcode)| { + Self::log_and_break(&q, req, msg, rcode, reason) }); - let (zone, diffs) = res.unwrap(); + let (zone, diffs) = match res { + Ok(res) => res, + Err(control_flow) => return control_flow, + }; // Read the zone SOA RR let read = zone.read(); From 18239008b6474ff54fa1a791dbec5be41300f1ef Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 6 Sep 2024 00:16:17 +0200 Subject: [PATCH 123/333] Slightly neater. --- src/net/server/middleware/xfr.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 2decbab31..34adfb8b6 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -184,9 +184,8 @@ where Self::log_and_break(&q, req, msg, rcode, reason) }); - let (zone, diffs) = match res { - Ok(res) => res, - Err(control_flow) => return control_flow, + let Ok((zone, diffs)) = res else { + return res.unwrap_err(); }; // Read the zone SOA RR From 60d16bb5ec1eb102a90abc22a0e96dd9c37413da Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 6 Sep 2024 00:27:58 +0200 Subject: [PATCH 124/333] Show use of more interesting metadata. --- src/net/server/middleware/xfr.rs | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 34adfb8b6..1377c7dbc 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -1266,6 +1266,7 @@ mod tests { use super::*; use crate::base::{MessageBuilder, Ttl}; use crate::net::server::message::NonUdpTransportContext; + use crate::net::server::middleware::tsig::Authentication; use crate::zonetree::{Rrset, ZoneBuilder}; use core::str::FromStr; use futures::StreamExt; @@ -1286,8 +1287,10 @@ mod tests { let transport_specific = TransportSpecificContext::NonUdp( NonUdpTransportContext::new(None), ); - let metadata = (); - let req: Request, ()> = Request::new( + let metadata = Authentication(Some( + crate::tsig::KeyName::from_str("blah").unwrap(), + )); + let req = Request::new( client_addr, received_at, msg, @@ -1317,7 +1320,9 @@ mod tests { let read = zone.read(); let mut stream = - XfrMiddlewareSvc::, TestNextSvc, Zone>::do_axfr::<()>( + XfrMiddlewareSvc::, TestNextSvc, Zone>::do_axfr::< + Authentication, + >( zone_walk_semaphore, batcher_semaphore, &req, From 3337ee27d4ebb190284d38f0ef0abff7ed43aee7 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 6 Sep 2024 00:51:58 +0200 Subject: [PATCH 125/333] Fix incorrect error message. --- src/net/server/middleware/xfr.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 1377c7dbc..080abf80b 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -509,7 +509,7 @@ where .push((owner.clone(), qclass, rrset.ttl(), rr)) .is_err() { - error!("Internal error: Failed to send final AXFR SOA to batcher"); + error!("Internal error: Failed to send RR to batcher"); let resp = mk_error_response(&msg, OptRcode::SERVFAIL); Self::add_to_stream(CallResult::new(resp), &sender); From 3c1dc2a61c84ef98b961bee61cd861b3b996efd7 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 6 Sep 2024 14:06:13 +0200 Subject: [PATCH 126/333] Make the XFR unit test intent easier to see and make it easier to make more such tests. --- src/net/server/middleware/xfr.rs | 229 ++++++++++++++++++++++++------- 1 file changed, 179 insertions(+), 50 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 080abf80b..2b0322501 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -1263,75 +1263,161 @@ impl CallbackState { #[cfg(test)] mod tests { - use super::*; - use crate::base::{MessageBuilder, Ttl}; - use crate::net::server::message::NonUdpTransportContext; - use crate::net::server::middleware::tsig::Authentication; - use crate::zonetree::{Rrset, ZoneBuilder}; use core::str::FromStr; + + use std::borrow::ToOwned; + use futures::StreamExt; use tokio::time::Instant; + use crate::base::{MessageBuilder, RecordData, Ttl}; + use crate::net::server::message::NonUdpTransportContext; + use crate::net::server::service::ServiceError; + use crate::rdata::{Aaaa, AllRecordData, Cname, Mx, Ns, A}; + use crate::zonefile::inplace::Zonefile; + + use super::*; + + type ExpectedRecords = + Vec<(Name, AllRecordData>)>; + #[tokio::test] - async fn empty_axfr() { - let zone_walk_semaphore = Arc::new(Semaphore::new(1)); - let batcher_semaphore = Arc::new(Semaphore::new(1)); + async fn axfr_with_example_zone() { + let mut expected_records: ExpectedRecords = vec![ + (n("example.com"), Ns::new(n("example.com")).into()), + (n("example.com"), A::new(p("192.0.2.1")).into()), + (n("example.com"), A::new(p("192.0.2.1")).into()), + (n("example.com"), A::new(p("192.0.2.1")).into()), + (n("example.com"), Aaaa::new(p("2001:db8::3")).into()), + (n("www.example.com"), Cname::new(n("example.com")).into()), + (n("mail.example.com"), Mx::new(10, n("example.com")).into()), + ]; + + let zone = load_zone(include_bytes!( + "../../../../test-data/zonefiles/nsd-example.txt" + )); + + let req = mk_axfr_request(zone.apex_name(), ()); + + let mut stream = do_axfr_for_zone(&zone, &req).await.unwrap(); + + assert_xfr_stream_eq( + req.message(), + &zone, + &mut stream, + &mut expected_records, + ) + .await; + } + + // #[tokio::test] + // async fn axfr_with_tsig() { + // let metadata = Authentication(Some( + // KeyName::from_str("blah").unwrap(), + // )) + // let req = mk_axfr_request("example.com", metadata); + // } + + //------------ Helper functions ------------------------------------------- + + fn n(name: &str) -> Name { + Name::from_str(name).unwrap() + } + + fn p(txt: &str) -> T + where + ::Err: std::fmt::Debug, + { + txt.parse().unwrap() + } + + fn load_zone(bytes: &[u8]) -> Zone { + let mut zone_bytes = std::io::BufReader::new(bytes); + let reader = Zonefile::load(&mut zone_bytes).unwrap(); + Zone::try_from(reader).unwrap() + } + + fn mk_axfr_request( + qname: impl ToName, + metadata: T, + ) -> Request, T> { let client_addr = "127.0.0.1:12345".parse().unwrap(); let received_at = Instant::now(); let msg = MessageBuilder::new_vec(); let mut msg = msg.question(); - msg.push((Name::vec_from_str("example.com").unwrap(), Rtype::AXFR)) - .unwrap(); + msg.push((qname, Rtype::AXFR)).unwrap(); let msg = msg.into_message(); + let transport_specific = TransportSpecificContext::NonUdp( NonUdpTransportContext::new(None), ); - let metadata = Authentication(Some( - crate::tsig::KeyName::from_str("blah").unwrap(), - )); - let req = Request::new( + + Request::new( client_addr, received_at, msg, transport_specific, metadata, - ); + ) + } - let qname = Name::from_str("example.com").unwrap(); - let mut zone_soa_answer = Answer::new(Rcode::NOERROR); - let mut soa_rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(3600)); - let mname = Name::from_str("blah").unwrap(); - let rname = Name::from_str("blah").unwrap(); - let serial = Serial::now(); - let refresh = Ttl::from_secs(0); - let retry = Ttl::from_secs(0); - let expire = Ttl::from_secs(0); - let minimum = Ttl::from_secs(0); - let soa = - Soa::new(mname, rname, serial, refresh, retry, expire, minimum); - soa_rrset.push_data(ZoneRecordData::Soa(soa.clone())); - let soa_rrset = SharedRrset::new(soa_rrset); - zone_soa_answer.add_answer(soa_rrset.clone()); - - let mut zone = ZoneBuilder::new(qname.clone(), Class::IN); - zone.insert_rrset(&qname, soa_rrset).unwrap(); - let zone = zone.build(); + async fn do_axfr_for_zone( + zone: &Zone, + req: &Request, T>, + ) -> Result< + XfrMiddlewareStream< + ::Future, + ::Stream, + <::Stream as Stream>::Item, + >, + OptRcode, + > { + let qname = zone.apex_name(); let read = zone.read(); + let zone_soa_answer = + XfrMiddlewareSvc::<_, TestNextSvc, Zone>::read_soa( + &read, + qname.to_owned(), + ) + .await + .unwrap(); + XfrMiddlewareSvc::<_, TestNextSvc, Zone>::do_axfr( + Arc::new(Semaphore::new(1)), + Arc::new(Semaphore::new(1)), + req, + qname.to_owned(), + &zone_soa_answer, + read, + ) + .await + } - let mut stream = - XfrMiddlewareSvc::, TestNextSvc, Zone>::do_axfr::< - Authentication, - >( - zone_walk_semaphore, - batcher_semaphore, - &req, - qname, - &zone_soa_answer, - read, + async fn assert_xfr_stream_eq( + req: &Message, + zone: &Zone, + mut stream: impl Stream>, ServiceError>> + + Unpin, + expected_records: &mut ExpectedRecords, + ) { + let read = zone.read(); + let q = req.first_question().unwrap(); + let zone_soa_answer = + XfrMiddlewareSvc::<_, TestNextSvc, Zone>::read_soa( + &read, + q.qname().to_name(), ) .await .unwrap(); + let AnswerContent::Data(zone_soa_rrset) = + zone_soa_answer.content().clone() + else { + unreachable!() + }; + let first_rr = zone_soa_rrset.first().unwrap(); + let ZoneRecordData::Soa(expected_soa) = first_rr.data() else { + unreachable!() + }; let msg = stream.next().await.unwrap().unwrap(); assert!(matches!( @@ -1342,20 +1428,63 @@ mod tests { let msg = stream.next().await.unwrap().unwrap(); let resp_builder = msg.into_inner().0.unwrap(); let resp = resp_builder.as_message(); - assert!(resp.is_answer(req.message())); - let rec = resp.answer().unwrap().next().unwrap().unwrap(); - let rec = rec + assert!(resp.is_answer(req)); + let mut records = resp.answer().unwrap(); + + let rec = records.next().unwrap().unwrap(); + assert_eq!(rec.owner(), zone.apex_name()); + assert_eq!(rec.rtype(), Rtype::SOA); + assert_eq!(rec.ttl(), Ttl::from_secs(86400)); + let soa = rec .into_record::>>() .unwrap() - .unwrap(); - let rcvd_soa = rec.into_data(); - assert_eq!(rcvd_soa, soa); + .unwrap() + .into_data(); + assert_eq!(&soa, expected_soa); + + for rec in records.by_ref() { + let rec = rec.unwrap(); + if rec.rtype() == Rtype::SOA { + let soa = rec + .into_record::>>() + .unwrap() + .unwrap() + .into_data(); + assert_eq!(&soa, expected_soa); + break; + } else { + let pos = expected_records + .iter() + .position(|(name, data)| { + name == &rec.owner() && data.rtype() == rec.rtype() + }) + .unwrap_or_else(|| { + panic!( + "XFR record {} {} {} was not expected", + rec.owner(), + rec.class(), + rec.rtype() + ) + }); + let (_, data) = expected_records.remove(pos); + let rec = rec + .into_record::>>() + .unwrap() + .unwrap(); + assert_eq!(&data, rec.data()); + } + } + + assert!(records.next().is_none()); + assert!(expected_records.is_empty()); let msg = stream.next().await.unwrap().unwrap(); assert!(matches!( msg.feedback(), Some(ServiceFeedback::EndTransaction) )); + + assert!(stream.next().await.is_none()); } #[derive(Clone)] From 34d292b94de00a40a87a88dee9171314097c4db7 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 6 Sep 2024 22:20:17 +0200 Subject: [PATCH 127/333] Test that AXFR over UDP is denied. --- src/net/server/middleware/xfr.rs | 137 +++++++++++++++++++++++++------ 1 file changed, 112 insertions(+), 25 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 2b0322501..25a62c0c7 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -509,7 +509,9 @@ where .push((owner.clone(), qclass, rrset.ttl(), rr)) .is_err() { - error!("Internal error: Failed to send RR to batcher"); + error!( + "Internal error: Failed to send RR to batcher" + ); let resp = mk_error_response(&msg, OptRcode::SERVFAIL); Self::add_to_stream(CallResult::new(resp), &sender); @@ -1265,13 +1267,13 @@ impl CallbackState { mod tests { use core::str::FromStr; - use std::borrow::ToOwned; - use futures::StreamExt; use tokio::time::Instant; use crate::base::{MessageBuilder, RecordData, Ttl}; - use crate::net::server::message::NonUdpTransportContext; + use crate::net::server::message::{ + NonUdpTransportContext, UdpTransportContext, + }; use crate::net::server::service::ServiceError; use crate::rdata::{Aaaa, AllRecordData, Cname, Mx, Ns, A}; use crate::zonefile::inplace::Zonefile; @@ -1281,6 +1283,9 @@ mod tests { type ExpectedRecords = Vec<(Name, AllRecordData>)>; + #[tokio::test] + async fn axfr_minimal() {} + #[tokio::test] async fn axfr_with_example_zone() { let mut expected_records: ExpectedRecords = vec![ @@ -1299,7 +1304,11 @@ mod tests { let req = mk_axfr_request(zone.apex_name(), ()); - let mut stream = do_axfr_for_zone(&zone, &req).await.unwrap(); + let res = do_axfr_for_zone(zone.clone(), &req).await; + + let ControlFlow::Break(mut stream) = res else { + panic!("AXFR failed"); + }; assert_xfr_stream_eq( req.message(), @@ -1310,6 +1319,66 @@ mod tests { .await; } + #[tokio::test] + async fn axfr_multi_response() {} + + #[tokio::test] + async fn axfr_delegation_records() { + // https://datatracker.ietf.org/doc/html/rfc5936#section-3.2 + } + + #[tokio::test] + async fn axfr_glue_records() { + // https://datatracker.ietf.org/doc/html/rfc5936#section-3.3 + } + + #[tokio::test] + async fn axfr_name_compression_not_yet_supported() { + // https://datatracker.ietf.org/doc/html/rfc5936#section-3.4 + } + + #[tokio::test] + async fn axfr_occluded_names() { + // https://datatracker.ietf.org/doc/html/rfc5936#section-3.5 + } + + #[tokio::test] + async fn axfr_not_allowed_over_udp() { + // https://datatracker.ietf.org/doc/html/rfc5936#section-4.2 + let zone = load_zone(include_bytes!( + "../../../../test-data/zonefiles/nsd-example.txt" + )); + + let req = mk_udp_axfr_request(zone.apex_name(), ()); + + let res = do_axfr_for_zone(zone, &req).await; + + let ControlFlow::Break(mut stream) = res else { + panic!("AXFR failed"); + }; + + let msg = stream.next().await.unwrap().unwrap(); + let resp_builder = msg.into_inner().0.unwrap(); + let resp = resp_builder.as_message(); + + assert_eq!(resp.header().rcode(), Rcode::NOTIMP); + } + + #[tokio::test] + async fn ixfr_minimal() {} + + #[tokio::test] + async fn ixfr_single_response_udp() {} + + #[tokio::test] + async fn ixfr_too_large_response_udp() {} + + #[tokio::test] + async fn ixfr_single_response_tcp() {} + + #[tokio::test] + async fn ixfr_multi_response_tcp() {} + // #[tokio::test] // async fn axfr_with_tsig() { // let metadata = Authentication(Some( @@ -1341,6 +1410,31 @@ mod tests { fn mk_axfr_request( qname: impl ToName, metadata: T, + ) -> Request, T> { + mk_axfr_request_for_transport( + qname, + metadata, + TransportSpecificContext::NonUdp(NonUdpTransportContext::new( + None, + )), + ) + } + + fn mk_udp_axfr_request( + qname: impl ToName, + metadata: T, + ) -> Request, T> { + mk_axfr_request_for_transport( + qname, + metadata, + TransportSpecificContext::Udp(UdpTransportContext::new(None)), + ) + } + + fn mk_axfr_request_for_transport( + qname: impl ToName, + metadata: T, + transport_specific: TransportSpecificContext, ) -> Request, T> { let client_addr = "127.0.0.1:12345".parse().unwrap(); let received_at = Instant::now(); @@ -1349,10 +1443,6 @@ mod tests { msg.push((qname, Rtype::AXFR)).unwrap(); let msg = msg.into_message(); - let transport_specific = TransportSpecificContext::NonUdp( - NonUdpTransportContext::new(None), - ); - Request::new( client_addr, received_at, @@ -1363,32 +1453,29 @@ mod tests { } async fn do_axfr_for_zone( - zone: &Zone, + zone: Zone, req: &Request, T>, - ) -> Result< + ) -> ControlFlow< XfrMiddlewareStream< ::Future, ::Stream, <::Stream as Stream>::Item, >, - OptRcode, > { - let qname = zone.apex_name(); - let read = zone.read(); - let zone_soa_answer = - XfrMiddlewareSvc::<_, TestNextSvc, Zone>::read_soa( - &read, - qname.to_owned(), - ) - .await - .unwrap(); - XfrMiddlewareSvc::<_, TestNextSvc, Zone>::do_axfr( + // let qname = zone.apex_name(); + // let read = zone.read(); + // let zone_soa_answer = + // XfrMiddlewareSvc::<_, TestNextSvc, Zone>::read_soa( + // &read, + // qname.to_owned(), + // ) + // .await + // .unwrap(); + XfrMiddlewareSvc::<_, TestNextSvc, Zone>::preprocess( Arc::new(Semaphore::new(1)), Arc::new(Semaphore::new(1)), req, - qname.to_owned(), - &zone_soa_answer, - read, + zone, ) .await } From 95621fb7a199007db8c0c35d8e4f6435cc4cb96d Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 6 Sep 2024 22:34:36 +0200 Subject: [PATCH 128/333] Handle multiple responses in assert_xfr_stream_eq(). --- src/net/server/middleware/xfr.rs | 106 ++++++++++++++++--------------- 1 file changed, 55 insertions(+), 51 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 25a62c0c7..b12644fc0 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -1512,60 +1512,64 @@ mod tests { Some(ServiceFeedback::BeginTransaction) )); - let msg = stream.next().await.unwrap().unwrap(); - let resp_builder = msg.into_inner().0.unwrap(); - let resp = resp_builder.as_message(); - assert!(resp.is_answer(req)); - let mut records = resp.answer().unwrap(); - - let rec = records.next().unwrap().unwrap(); - assert_eq!(rec.owner(), zone.apex_name()); - assert_eq!(rec.rtype(), Rtype::SOA); - assert_eq!(rec.ttl(), Ttl::from_secs(86400)); - let soa = rec - .into_record::>>() - .unwrap() - .unwrap() - .into_data(); - assert_eq!(&soa, expected_soa); - - for rec in records.by_ref() { - let rec = rec.unwrap(); - if rec.rtype() == Rtype::SOA { - let soa = rec - .into_record::>>() - .unwrap() - .unwrap() - .into_data(); - assert_eq!(&soa, expected_soa); - break; - } else { - let pos = expected_records - .iter() - .position(|(name, data)| { - name == &rec.owner() && data.rtype() == rec.rtype() - }) - .unwrap_or_else(|| { - panic!( - "XFR record {} {} {} was not expected", - rec.owner(), - rec.class(), - rec.rtype() - ) - }); - let (_, data) = expected_records.remove(pos); - let rec = rec - .into_record::>>() - .unwrap() - .unwrap(); - assert_eq!(&data, rec.data()); + let mut msg = stream.next().await.unwrap().unwrap(); + while msg.response().is_some() { + let resp_builder = msg.into_inner().0.unwrap(); + let resp = resp_builder.as_message(); + assert!(resp.is_answer(req)); + let mut records = resp.answer().unwrap(); + + let rec = records.next().unwrap().unwrap(); + assert_eq!(rec.owner(), zone.apex_name()); + assert_eq!(rec.rtype(), Rtype::SOA); + assert_eq!(rec.ttl(), Ttl::from_secs(86400)); + let soa = rec + .into_record::>>() + .unwrap() + .unwrap() + .into_data(); + assert_eq!(&soa, expected_soa); + + for rec in records.by_ref() { + let rec = rec.unwrap(); + if rec.rtype() == Rtype::SOA { + let soa = rec + .into_record::>>() + .unwrap() + .unwrap() + .into_data(); + assert_eq!(&soa, expected_soa); + break; + } else { + let pos = expected_records + .iter() + .position(|(name, data)| { + name == &rec.owner() + && data.rtype() == rec.rtype() + }) + .unwrap_or_else(|| { + panic!( + "XFR record {} {} {} was not expected", + rec.owner(), + rec.class(), + rec.rtype() + ) + }); + let (_, data) = expected_records.remove(pos); + let rec = rec + .into_record::>>() + .unwrap() + .unwrap(); + assert_eq!(&data, rec.data()); + } } - } - assert!(records.next().is_none()); - assert!(expected_records.is_empty()); + assert!(records.next().is_none()); + assert!(expected_records.is_empty()); + + msg = stream.next().await.unwrap().unwrap(); + } - let msg = stream.next().await.unwrap().unwrap(); assert!(matches!( msg.feedback(), Some(ServiceFeedback::EndTransaction) From e6c478dabfdaf8fc539a77718a8776a5acc6814a Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 6 Sep 2024 22:35:00 +0200 Subject: [PATCH 129/333] Remove commented out code. --- src/net/server/middleware/xfr.rs | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index b12644fc0..ac3ab9ae8 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -1462,15 +1462,6 @@ mod tests { <::Stream as Stream>::Item, >, > { - // let qname = zone.apex_name(); - // let read = zone.read(); - // let zone_soa_answer = - // XfrMiddlewareSvc::<_, TestNextSvc, Zone>::read_soa( - // &read, - // qname.to_owned(), - // ) - // .await - // .unwrap(); XfrMiddlewareSvc::<_, TestNextSvc, Zone>::preprocess( Arc::new(Semaphore::new(1)), Arc::new(Semaphore::new(1)), From 9fd3b6a55a0ab712ba78fafad5633b790522162d Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 6 Sep 2024 22:52:42 +0200 Subject: [PATCH 130/333] Really test multi-response AXFR. --- src/net/server/middleware/xfr.rs | 82 ++++++++++++++++++++++++++------ 1 file changed, 68 insertions(+), 14 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index ac3ab9ae8..f7a44ba24 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -1270,12 +1270,12 @@ mod tests { use futures::StreamExt; use tokio::time::Instant; - use crate::base::{MessageBuilder, RecordData, Ttl}; + use crate::base::{MessageBuilder, RecordData}; use crate::net::server::message::{ NonUdpTransportContext, UdpTransportContext, }; use crate::net::server::service::ServiceError; - use crate::rdata::{Aaaa, AllRecordData, Cname, Mx, Ns, A}; + use crate::rdata::{Aaaa, AllRecordData, Cname, Mx, Ns, Txt, A}; use crate::zonefile::inplace::Zonefile; use super::*; @@ -1320,7 +1320,48 @@ mod tests { } #[tokio::test] - async fn axfr_multi_response() {} + async fn axfr_multi_response() { + let mut expected_records: ExpectedRecords = vec![ + (n("example.com"), Ns::new(n("ns1.example.com")).into()), + (n("example.com"), Ns::new(n("ns2.example.com")).into()), + (n("example.com"), Mx::new(10, n("mail.example.com")).into()), + (n("example.com"), A::new(p("192.0.2.1")).into()), + (n("example.com"), Aaaa::new(p("2001:db8:10::1")).into()), + (n("ns1.example.com"), A::new(p("192.0.2.2")).into()), + (n("ns1.example.com"), Aaaa::new(p("2001:db8:10::2")).into()), + (n("ns2.example.com"), A::new(p("192.0.2.3")).into()), + (n("ns2.example.com"), Aaaa::new(p("2001:db8:10::3")).into()), + (n("mail.example.com"), A::new(p("192.0.2.4")).into()), + (n("mail.example.com"), Aaaa::new(p("2001:db8:10::4")).into()), + ]; + + for i in 1..=10000 { + expected_records.push(( + n(&format!("host-{i}.example.com")), + Txt::build_from_slice(b"text").unwrap().into(), + )); + } + + let zone = load_zone(include_bytes!( + "../../../../test-data/zonefiles/big.example.com.txt" + )); + + let req = mk_axfr_request(zone.apex_name(), ()); + + let res = do_axfr_for_zone(zone.clone(), &req).await; + + let ControlFlow::Break(mut stream) = res else { + panic!("AXFR failed"); + }; + + assert_xfr_stream_eq( + req.message(), + &zone, + &mut stream, + &mut expected_records, + ) + .await; + } #[tokio::test] async fn axfr_delegation_records() { @@ -1504,22 +1545,26 @@ mod tests { )); let mut msg = stream.next().await.unwrap().unwrap(); + let mut first = true; + let mut last = false; while msg.response().is_some() { let resp_builder = msg.into_inner().0.unwrap(); let resp = resp_builder.as_message(); assert!(resp.is_answer(req)); let mut records = resp.answer().unwrap(); - let rec = records.next().unwrap().unwrap(); - assert_eq!(rec.owner(), zone.apex_name()); - assert_eq!(rec.rtype(), Rtype::SOA); - assert_eq!(rec.ttl(), Ttl::from_secs(86400)); - let soa = rec - .into_record::>>() - .unwrap() - .unwrap() - .into_data(); - assert_eq!(&soa, expected_soa); + if first { + let rec = records.next().unwrap().unwrap(); + assert_eq!(rec.owner(), zone.apex_name()); + assert_eq!(rec.rtype(), Rtype::SOA); + let soa = rec + .into_record::>>() + .unwrap() + .unwrap() + .into_data(); + assert_eq!(&soa, expected_soa); + first = false; + } for rec in records.by_ref() { let rec = rec.unwrap(); @@ -1530,6 +1575,7 @@ mod tests { .unwrap() .into_data(); assert_eq!(&soa, expected_soa); + last = true; break; } else { let pos = expected_records @@ -1552,15 +1598,23 @@ mod tests { .unwrap() .unwrap(); assert_eq!(&data, rec.data()); + eprintln!( + "Found {} {} {}", + rec.owner(), + rec.class(), + rec.rtype() + ) } } assert!(records.next().is_none()); - assert!(expected_records.is_empty()); msg = stream.next().await.unwrap().unwrap(); } + assert!(last); + assert!(expected_records.is_empty()); + assert!(matches!( msg.feedback(), Some(ServiceFeedback::EndTransaction) From 2cca443ed8c7623163b2b602c2c7ad5f80a62908 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 9 Sep 2024 11:25:10 +0200 Subject: [PATCH 131/333] Support custom errors and different final batch handling. These extra features will be used by the XFR specific batcher impl. --- src/net/server/batcher.rs | 35 +++++++++++++++++++++++++---------- 1 file changed, 25 insertions(+), 10 deletions(-) diff --git a/src/net/server/batcher.rs b/src/net/server/batcher.rs index 366201fc6..1e1e416fe 100644 --- a/src/net/server/batcher.rs +++ b/src/net/server/batcher.rs @@ -2,6 +2,7 @@ use core::marker::PhantomData; +use std::fmt::Debug; use std::sync::Arc; use octseq::Octets; @@ -31,14 +32,16 @@ where RequestOctets: Octets, Target: Composer + Default, { + type Error: From + Debug; + #[allow(clippy::result_unit_err)] fn push( &mut self, record: impl ComposeRecord, - ) -> Result, ()>; + ) -> Result, Self::Error>; #[allow(clippy::result_unit_err)] - fn finish(&mut self) -> Result<(), ()>; + fn finish(&mut self) -> Result<(), Self::Error>; fn mk_answer_builder( &self, @@ -56,6 +59,8 @@ where RequestOctets: Octets, Target: Composer + Default, { + type Error: From + Debug; + /// Prepare a message builder to push records into. fn batch_started( _state: &T, @@ -81,7 +86,8 @@ where fn batch_ready( _state: &T, _answer: AnswerBuilder>, - ) -> Result<(), ()>; + _finished: bool, + ) -> Result<(), Self::Error>; } //------------ CallbackBatcher ------------------------------------------------ @@ -130,14 +136,14 @@ where fn try_push( &mut self, record: &impl ComposeRecord, - ) -> Result, ()> { - match self.push_ref(record).map_err(|_| ())? { + ) -> Result, C::Error> { + match self.push_ref(record)? { PushResult::PushedAndLimitReached(builder) => { - C::batch_ready(&self.callback_state, builder)?; + C::batch_ready(&self.callback_state, builder, false)?; Ok(PushResult::PushedAndReadyForMore) } PushResult::NotPushedMessageFull(builder) => { - C::batch_ready(&self.callback_state, builder)?; + C::batch_ready(&self.callback_state, builder, false)?; Ok(PushResult::Retry) } other => Ok(other), @@ -196,19 +202,21 @@ where Target: Composer + Default, C: Callbacks, { + type Error = C::Error; + fn push( &mut self, record: impl ComposeRecord, - ) -> Result, ()> { + ) -> Result, Self::Error> { match self.try_push(&record) { Ok(PushResult::Retry) => self.try_push(&record), other => other, } } - fn finish(&mut self) -> Result<(), ()> { + fn finish(&mut self) -> Result<(), Self::Error> { if let Some(builder) = self.answer.take() { - C::batch_ready(&self.callback_state, builder.unwrap()) + C::batch_ready(&self.callback_state, builder.unwrap(), true) } else { Ok(()) } @@ -346,10 +354,17 @@ mod tests { struct BatchCounter; + impl From for () { + fn from(_: PushError) -> Self {} + } + impl Callbacks, Vec, Arc> for BatchCounter { + type Error = (); + fn batch_ready( counters: &Arc, answer: AnswerBuilder>>, + _finished: bool, ) -> Result<(), ()> { counters.num_batches.fetch_add(1, Ordering::SeqCst); counters.num_rrs_in_last_batch.store(0, Ordering::SeqCst); From c43c84e0ab0c68455f7d220c2645790c864099bb Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 9 Sep 2024 12:50:16 +0200 Subject: [PATCH 132/333] Add support for serving the SOA record to trigger fallback from UDP IXFR to TCP IXFR when the response does NOT fit in a single UDP message. --- src/net/server/middleware/xfr.rs | 323 +++++++++++++++++++++++++------ 1 file changed, 263 insertions(+), 60 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index f7a44ba24..49a439d46 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -15,7 +15,7 @@ use octseq::Octets; use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; use tokio::sync::Semaphore; use tokio_stream::wrappers::UnboundedReceiverStream; -use tracing::{error, info, trace, warn}; +use tracing::{debug, error, info, trace, warn}; use crate::base::iana::{Class, Opcode, OptRcode, Rcode}; use crate::base::message_builder::{ @@ -245,7 +245,7 @@ where req.client_addr() ); } - let stream = Self::do_axfr( + let stream = Self::send_axfr_response( zone_walking_semaphore, batcher_semaphore, req, @@ -278,7 +278,7 @@ where // the query is responded to with a single SOA record of // the server's current version to inform the client that a // TCP query should be initiated." - let stream = Self::do_ixfr( + let stream = Self::send_ixfr_response( batcher_semaphore.clone(), req, ixfr_query_serial, @@ -321,7 +321,7 @@ where } #[allow(clippy::too_many_arguments)] - async fn do_axfr( + async fn send_axfr_response( zone_walk_semaphore: Arc, batcher_semaphore: Arc, req: &Request, @@ -391,13 +391,19 @@ where let (batcher_tx, mut batcher_rx) = tokio::sync::mpsc::channel::<(StoredName, SharedRrset)>(100); - // Notify the underlying transport to expect a stream of related - // responses. The transport should modify its behaviour to account for - // the potentially slow and long running nature of a transaction. - Self::add_to_stream( - CallResult::feedback_only(ServiceFeedback::BeginTransaction), - &sender, - ); + let must_fit_in_single_message = + matches!(req.transport_ctx(), TransportSpecificContext::Udp(_)); + + if !must_fit_in_single_message { + // Notify the underlying transport to expect a stream of related + // responses. The transport should modify its behaviour to account + // for the potentially slow and long running nature of a + // transaction. + Self::add_to_stream( + CallResult::feedback_only(ServiceFeedback::BeginTransaction), + &sender, + ); + } // Enqueue the zone SOA RRset for the batcher to process. if batcher_tx @@ -424,9 +430,9 @@ where if rrset.rtype() != Rtype::SOA { let _ = cloned_batcher_tx .blocking_send((owner.clone(), rrset.clone())); - // If the blocking send fails it means that he batcher - // is no longer available. This can happen if it was - // no longer able to pass messages back to the + // If the blocking send fails it means that the + // batcher is no longer available. This can happen if + // it was no longer able to pass messages back to the // underlying transport, which can happen if the // client closed the connection. We don't log this // because we can't stop the tree walk and so will @@ -464,6 +470,7 @@ where // the caller. let msg = msg.clone(); let soft_byte_limit = Self::calc_msg_bytes_available(req); + let zone_soa_answer = zone_soa_answer.clone(); tokio::spawn(async move { // Limit the number of concurrently running XFR batching @@ -501,33 +508,74 @@ where sender.clone(), Some(soft_byte_limit), hard_rr_limit, + must_fit_in_single_message, ); while let Some((owner, rrset)) = batcher_rx.recv().await { for rr in rrset.data() { - if batcher - .push((owner.clone(), qclass, rrset.ttl(), rr)) - .is_err() + if let Err(err) = + batcher.push((owner.clone(), qclass, rrset.ttl(), rr)) { - error!( - "Internal error: Failed to send RR to batcher" - ); - let resp = - mk_error_response(&msg, OptRcode::SERVFAIL); - Self::add_to_stream(CallResult::new(resp), &sender); - batcher_rx.close(); - return; + match err { + BatchReadyError::MustFitInSingleMessage => { + // https://datatracker.ietf.org/doc/html/rfc1995#section-2 + // 2. Brief Description of the Protocol + // .. + // "If the UDP reply does not fit, the + // query is responded to with a single SOA + // record of the server's current version + // to inform the client that a TCP query + // should be initiated." + debug_assert!(must_fit_in_single_message); + let builder = mk_builder_for_target(); + let resp = + zone_soa_answer.to_message(&msg, builder); + debug!("IXFR aborted because response does not fit in a single UDP reply"); + Self::add_to_stream( + CallResult::new(resp), + &sender, + ); + batcher_rx.close(); + return; + } + + BatchReadyError::PushError(err) => { + error!( + "Internal error: Failed to send RR to batcher: {err}" + ); + let resp = mk_error_response( + &msg, + OptRcode::SERVFAIL, + ); + Self::add_to_stream( + CallResult::new(resp), + &sender, + ); + batcher_rx.close(); + return; + } + + BatchReadyError::SendError => { + debug!("Batcher was unable to send completed batch, presumably because the receiver was dropped"); + batcher_rx.close(); + return; + } + } } } } batcher.finish().unwrap(); // TODO - trace!("Finishing transaction"); - Self::add_to_stream( - CallResult::feedback_only(ServiceFeedback::EndTransaction), - &sender, - ); + if !must_fit_in_single_message { + trace!("Finishing transaction"); + Self::add_to_stream( + CallResult::feedback_only( + ServiceFeedback::EndTransaction, + ), + &sender, + ); + } batcher_rx.close(); }); @@ -537,7 +585,7 @@ where // Returns None if fallback to AXFR should be done. #[allow(clippy::too_many_arguments)] - async fn do_ixfr( + async fn send_ixfr_response( batcher_semaphore: Arc, req: &Request, query_serial: Serial, @@ -671,13 +719,19 @@ where let (sender, receiver) = unbounded_channel(); let stream = UnboundedReceiverStream::new(receiver); - // Notify the underlying transport to expect a stream of related - // responses. The transport should modify its behaviour to account for - // the potentially slow and long running nature of a transaction. - Self::add_to_stream( - CallResult::feedback_only(ServiceFeedback::BeginTransaction), - &sender, - ); + let must_fit_in_single_message = + matches!(req.transport_ctx(), TransportSpecificContext::Udp(_)); + + if !must_fit_in_single_message { + // Notify the underlying transport to expect a stream of related + // responses. The transport should modify its behaviour to account + // for the potentially slow and long running nature of a + // transaction. + Self::add_to_stream( + CallResult::feedback_only(ServiceFeedback::BeginTransaction), + &sender, + ); + } // Stream the IXFR diffs in the background let msg = msg.clone(); @@ -730,6 +784,7 @@ where sender.clone(), Some(soft_byte_limit), None, + must_fit_in_single_message, ); batcher @@ -811,11 +866,15 @@ where batcher.finish().unwrap(); // TODO - trace!("Ending transaction"); - Self::add_to_stream( - CallResult::feedback_only(ServiceFeedback::EndTransaction), - &sender, - ); + if !must_fit_in_single_message { + trace!("Ending transaction"); + Self::add_to_stream( + CallResult::feedback_only( + ServiceFeedback::EndTransaction, + ), + &sender, + ); + } }); Ok(MiddlewareStream::Result(stream)) @@ -1123,12 +1182,15 @@ where sender: UnboundedSender>, soft_byte_limit: Option, hard_rr_limit: Option, - ) -> impl ResourceRecordBatcher { + must_fit_in_single_message: bool, + ) -> impl ResourceRecordBatcher + { let cb_state = CallbackState::new( req_msg.clone(), sender, soft_byte_limit, hard_rr_limit, + must_fit_in_single_message, ); CallbackBatcher::< @@ -1190,6 +1252,21 @@ where //--- Callbacks +#[derive(Clone, Copy, Debug)] +pub enum BatchReadyError { + PushError(PushError), + + SendError, + + MustFitInSingleMessage, +} + +impl From for BatchReadyError { + fn from(err: PushError) -> Self { + Self::PushError(err) + } +} + impl Callbacks> for XfrRrBatcher @@ -1197,6 +1274,8 @@ where RequestOctets: Octets, Target: Composer + Default, { + type Error = BatchReadyError; + fn batch_started( cb_state: &CallbackState, msg: &Message, @@ -1212,14 +1291,20 @@ where fn batch_ready( cb_state: &CallbackState, builder: AnswerBuilder>, - ) -> Result<(), ()> { + finished: bool, + ) -> Result<(), Self::Error> { + if !finished && cb_state.must_fit_in_single_message { + return Err(BatchReadyError::MustFitInSingleMessage); + } + trace!("Sending RR batch"); let mut additional = builder.additional(); Self::set_axfr_header(&cb_state.req_msg, &mut additional); let call_result = Ok(CallResult::new(additional)); - cb_state.sender.send(call_result).map_err(|err| { - warn!("Internal error: Send from RR batcher failed: {err}"); - }) + cb_state + .sender + .send(call_result) + .map_err(|_unsent_msg| BatchReadyError::SendError) } fn record_pushed( @@ -1245,6 +1330,7 @@ struct CallbackState { sender: UnboundedSender>, soft_byte_limit: Option, hard_rr_limit: Option, + must_fit_in_single_message: bool, } impl CallbackState { @@ -1253,12 +1339,14 @@ impl CallbackState { sender: UnboundedSender>, soft_byte_limit: Option, hard_rr_limit: Option, + must_fit_in_single_message: bool, ) -> Self { Self { req_msg, sender, soft_byte_limit, hard_rr_limit, + must_fit_in_single_message, } } } @@ -1270,7 +1358,7 @@ mod tests { use futures::StreamExt; use tokio::time::Instant; - use crate::base::{MessageBuilder, RecordData}; + use crate::base::{MessageBuilder, RecordData, Ttl}; use crate::net::server::message::{ NonUdpTransportContext, UdpTransportContext, }; @@ -1304,7 +1392,7 @@ mod tests { let req = mk_axfr_request(zone.apex_name(), ()); - let res = do_axfr_for_zone(zone.clone(), &req).await; + let res = do_preprocess(zone.clone(), &req).await; let ControlFlow::Break(mut stream) = res else { panic!("AXFR failed"); @@ -1348,7 +1436,7 @@ mod tests { let req = mk_axfr_request(zone.apex_name(), ()); - let res = do_axfr_for_zone(zone.clone(), &req).await; + let res = do_preprocess(zone.clone(), &req).await; let ControlFlow::Break(mut stream) = res else { panic!("AXFR failed"); @@ -1392,7 +1480,7 @@ mod tests { let req = mk_udp_axfr_request(zone.apex_name(), ()); - let res = do_axfr_for_zone(zone, &req).await; + let res = do_preprocess(zone, &req).await; let ControlFlow::Break(mut stream) = res else { panic!("AXFR failed"); @@ -1412,7 +1500,29 @@ mod tests { async fn ixfr_single_response_udp() {} #[tokio::test] - async fn ixfr_too_large_response_udp() {} + async fn ixfr_too_large_response_udp() { + let mut expected_records: ExpectedRecords = vec![]; + + let zone = load_zone(include_bytes!( + "../../../../test-data/zonefiles/big.example.com.txt" + )); + + let req = mk_udp_ixfr_request(zone.apex_name(), Serial(0), ()); + + let res = do_preprocess(zone.clone(), &req).await; + + let ControlFlow::Break(mut stream) = res else { + panic!("AXFR failed"); + }; + + assert_udp_xfr_stream_eq( + req.message(), + &zone, + &mut stream, + &mut expected_records, + ) + .await; + } #[tokio::test] async fn ixfr_single_response_tcp() {} @@ -1493,7 +1603,50 @@ mod tests { ) } - async fn do_axfr_for_zone( + fn mk_udp_ixfr_request( + qname: impl ToName + Clone, + serial: Serial, + metadata: T, + ) -> Request, T> { + mk_ixfr_request_for_transport( + qname, + serial, + metadata, + TransportSpecificContext::Udp(UdpTransportContext::new(None)), + ) + } + + fn mk_ixfr_request_for_transport( + qname: impl ToName + Clone, + serial: Serial, + metadata: T, + transport_specific: TransportSpecificContext, + ) -> Request, T> { + let client_addr = "127.0.0.1:12345".parse().unwrap(); + let received_at = Instant::now(); + let msg = MessageBuilder::new_vec(); + let mut msg = msg.question(); + msg.push((qname.clone(), Rtype::IXFR)).unwrap(); + + let mut msg = msg.authority(); + let mname = Name::::from_str("mname").unwrap(); + let rname = Name::::from_str("rname").unwrap(); + let ttl = Ttl::from_secs(0); + let soa = Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl); + msg.push((qname, Class::IN, Ttl::from_secs(0), soa)) + .unwrap(); + let msg = msg.into_message(); + + Request::new( + client_addr, + received_at, + msg, + transport_specific, + metadata, + ) + } + + async fn do_preprocess( zone: Zone, req: &Request, T>, ) -> ControlFlow< @@ -1513,11 +1666,46 @@ mod tests { } async fn assert_xfr_stream_eq( + req: &Message, + zone: &Zone, + stream: impl Stream>, ServiceError>> + + Unpin, + expected_records: &mut ExpectedRecords, + ) { + assert_transport_specific_xfr_stream_eq( + req, + zone, + stream, + expected_records, + false, + ) + .await + } + + async fn assert_udp_xfr_stream_eq( + req: &Message, + zone: &Zone, + stream: impl Stream>, ServiceError>> + + Unpin, + expected_records: &mut ExpectedRecords, + ) { + assert_transport_specific_xfr_stream_eq( + req, + zone, + stream, + expected_records, + true, + ) + .await + } + + async fn assert_transport_specific_xfr_stream_eq( req: &Message, zone: &Zone, mut stream: impl Stream>, ServiceError>> + Unpin, expected_records: &mut ExpectedRecords, + udp: bool, ) { let read = zone.read(); let q = req.first_question().unwrap(); @@ -1538,11 +1726,15 @@ mod tests { unreachable!() }; - let msg = stream.next().await.unwrap().unwrap(); - assert!(matches!( - msg.feedback(), - Some(ServiceFeedback::BeginTransaction) - )); + // Allow for the RFC 1995 UDP IXFR case where the response doesn't fit + // in a single UDP message and so a single SOA reply is sent instead. + if !udp { + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::BeginTransaction) + )); + } let mut msg = stream.next().await.unwrap().unwrap(); let mut first = true; @@ -1551,7 +1743,7 @@ mod tests { let resp_builder = msg.into_inner().0.unwrap(); let resp = resp_builder.as_message(); assert!(resp.is_answer(req)); - let mut records = resp.answer().unwrap(); + let mut records = resp.answer().unwrap().peekable(); if first { let rec = records.next().unwrap().unwrap(); @@ -1564,6 +1756,17 @@ mod tests { .into_data(); assert_eq!(&soa, expected_soa); first = false; + + if udp + && records.peek().is_none() + && expected_records.is_empty() + { + // Stop. This is the IXFR 1995 UDP case where the response + // didn't fit in a single UDP packet and so a single SOA + // reply is sent back instead. + assert!(stream.next().await.is_none()); + return; + } } for rec in records.by_ref() { From 0fa6c2ce9b67fa950ba36b901a4c28358baa1f22 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 9 Sep 2024 16:23:52 +0200 Subject: [PATCH 133/333] Added IXFR tests. --- src/net/server/middleware/xfr.rs | 667 ++++++++++++++++++++++--------- 1 file changed, 477 insertions(+), 190 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 49a439d46..50ef49f40 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -1039,7 +1039,7 @@ pub trait XfrDataProvider { req: &Request, apex_name: &impl ToName, class: Class, - diff_to: Option, + diff_from: Option, ) -> Pin< Box< dyn Future< @@ -1067,7 +1067,7 @@ where req: &Request, apex_name: &impl ToName, class: Class, - diff_to: Option, + diff_from: Option, ) -> Pin< Box< dyn Future< @@ -1082,7 +1082,7 @@ where where Octs: AsRef<[u8]> + Send + Sync, { - (**self).request(req, apex_name, class, diff_to) + (**self).request(req, apex_name, class, diff_from) } } @@ -1100,7 +1100,7 @@ impl XfrDataProvider for Zone { _req: &Request, apex_name: &impl ToName, class: Class, - _diff_to: Option, + _diff_from: Option, ) -> Pin< Box< dyn Future< @@ -1141,7 +1141,7 @@ impl XfrDataProvider for ZoneTree { _req: &Request, apex_name: &impl ToName, class: Class, - _diff_to: Option, + _diff_from: Option, ) -> Pin< Box< dyn Future< @@ -1354,6 +1354,7 @@ impl CallbackState { #[cfg(test)] mod tests { use core::str::FromStr; + use std::borrow::ToOwned; use futures::StreamExt; use tokio::time::Instant; @@ -1365,6 +1366,7 @@ mod tests { use crate::net::server::service::ServiceError; use crate::rdata::{Aaaa, AllRecordData, Cname, Mx, Ns, Txt, A}; use crate::zonefile::inplace::Zonefile; + use crate::zonetree::types::Rrset; use super::*; @@ -1376,7 +1378,22 @@ mod tests { #[tokio::test] async fn axfr_with_example_zone() { + let zone = load_zone(include_bytes!( + "../../../../test-data/zonefiles/nsd-example.txt" + )); + + let req = mk_axfr_request(zone.apex_name(), ()); + + let res = do_preprocess(zone.clone(), &req).await; + + let ControlFlow::Break(mut stream) = res else { + panic!("AXFR failed"); + }; + + let zone_soa = get_zone_soa(&zone).await; + let mut expected_records: ExpectedRecords = vec![ + (n("example.com"), zone_soa.clone().into()), (n("example.com"), Ns::new(n("example.com")).into()), (n("example.com"), A::new(p("192.0.2.1")).into()), (n("example.com"), A::new(p("192.0.2.1")).into()), @@ -1384,10 +1401,33 @@ mod tests { (n("example.com"), Aaaa::new(p("2001:db8::3")).into()), (n("www.example.com"), Cname::new(n("example.com")).into()), (n("mail.example.com"), Mx::new(10, n("example.com")).into()), + (n("example.com"), zone_soa.into()), ]; + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::BeginTransaction) + )); + + let stream = assert_stream_eq( + req.message(), + &mut stream, + &mut expected_records, + ) + .await; + + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::EndTransaction) + )); + } + + #[tokio::test] + async fn axfr_multi_response() { let zone = load_zone(include_bytes!( - "../../../../test-data/zonefiles/nsd-example.txt" + "../../../../test-data/zonefiles/big.example.com.txt" )); let req = mk_axfr_request(zone.apex_name(), ()); @@ -1398,18 +1438,10 @@ mod tests { panic!("AXFR failed"); }; - assert_xfr_stream_eq( - req.message(), - &zone, - &mut stream, - &mut expected_records, - ) - .await; - } + let zone_soa = get_zone_soa(&zone).await; - #[tokio::test] - async fn axfr_multi_response() { let mut expected_records: ExpectedRecords = vec![ + (n("example.com"), zone_soa.clone().into()), (n("example.com"), Ns::new(n("ns1.example.com")).into()), (n("example.com"), Ns::new(n("ns2.example.com")).into()), (n("example.com"), Mx::new(10, n("mail.example.com")).into()), @@ -1430,25 +1462,26 @@ mod tests { )); } - let zone = load_zone(include_bytes!( - "../../../../test-data/zonefiles/big.example.com.txt" - )); - - let req = mk_axfr_request(zone.apex_name(), ()); + expected_records.push((n("example.com"), zone_soa.into())); - let res = do_preprocess(zone.clone(), &req).await; - - let ControlFlow::Break(mut stream) = res else { - panic!("AXFR failed"); - }; + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::BeginTransaction) + )); - assert_xfr_stream_eq( + let stream = assert_stream_eq( req.message(), - &zone, &mut stream, &mut expected_records, ) .await; + + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::EndTransaction) + )); } #[tokio::test] @@ -1494,38 +1527,307 @@ mod tests { } #[tokio::test] - async fn ixfr_minimal() {} + async fn ixfr_rfc1995_section7_full_zone_reply() { + // Based on https://datatracker.ietf.org/doc/html/rfc1995#section-7 + + // initial zone content: + // JAIN.AD.JP. IN SOA NS.JAIN.AD.JP. mohta.jain.ad.jp. ( + // 1 600 600 3600000 604800) + // IN NS NS.JAIN.AD.JP. + // NS.JAIN.AD.JP. IN A 133.69.136.1 + // NEZU.JAIN.AD.JP. IN A 133.69.136.5 + + // Final zone content: + let rfc_1995_zone = r#" +JAIN.AD.JP. IN SOA NS.JAIN.AD.JP. mohta.jain.ad.jp. ( + 3 600 600 3600000 604800) + IN NS NS.JAIN.AD.JP. +NS.JAIN.AD.JP. IN A 133.69.136.1 +JAIN-BB.JAIN.AD.JP. IN A 133.69.136.3 +JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 + "#; + let zone = load_zone(rfc_1995_zone.as_bytes()); + + // Create an object that knows how to provide zone and diff data for + // our zone and no diffs. + let zone_with_diffs = ZoneWithDiffs::new(zone.clone(), vec![]); + + // The following IXFR query + let req = mk_udp_ixfr_request(zone.apex_name(), Serial(1), ()); + + let res = do_preprocess(zone_with_diffs, &req).await; - #[tokio::test] - async fn ixfr_single_response_udp() {} + let ControlFlow::Break(mut stream) = res else { + panic!("IXFR failed"); + }; + + // could be replied to with the following full zone transfer message: + let zone_soa = get_zone_soa(&zone).await; + + let mut expected_records: ExpectedRecords = vec![ + (n("JAIN.AD.JP."), zone_soa.clone().into()), + (n("JAIN.AD.JP."), Ns::new(n("NS.JAIN.AD.JP.")).into()), + (n("NS.JAIN.AD.JP."), A::new(p("133.69.136.1")).into()), + (n("JAIN-BB.JAIN.AD.JP."), A::new(p("133.69.136.3")).into()), + (n("JAIN-BB.JAIN.AD.JP."), A::new(p("192.41.197.2")).into()), + (n("JAIN.AD.JP."), zone_soa.into()), + ]; + + assert_stream_eq(req.message(), &mut stream, &mut expected_records) + .await; + } #[tokio::test] - async fn ixfr_too_large_response_udp() { - let mut expected_records: ExpectedRecords = vec![]; + async fn ixfr_rfc1995_section7_incremental_reply() { + // Based on https://datatracker.ietf.org/doc/html/rfc1995#section-7 + let mut diffs = Vec::new(); + + // initial zone content: + // JAIN.AD.JP. IN SOA NS.JAIN.AD.JP. mohta.jain.ad.jp. ( + // 1 600 600 3600000 604800) + // IN NS NS.JAIN.AD.JP. + // NS.JAIN.AD.JP. IN A 133.69.136.1 + // NEZU.JAIN.AD.JP. IN A 133.69.136.5 + + // Final zone content: + let rfc_1995_zone = r#" +JAIN.AD.JP. IN SOA NS.JAIN.AD.JP. mohta.jain.ad.jp. ( + 3 600 600 3600000 604800) + IN NS NS.JAIN.AD.JP. +NS.JAIN.AD.JP. IN A 133.69.136.1 +JAIN-BB.JAIN.AD.JP. IN A 133.69.136.3 +JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 + "#; + let zone = load_zone(rfc_1995_zone.as_bytes()); + + // Diff 1: NEZU.JAIN.AD.JP. is removed and JAIN-BB.JAIN.AD.JP. is added. + let mut diff = ZoneDiff::new(); + diff.start_serial = Some(Serial(1)); + diff.end_serial = Some(Serial(2)); + + // -- Remove the old SOA. + let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); + let soa = Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(1), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ); + rrset.push_data(soa.into()); + diff.removed + .insert((n("JAIN.AD.JP"), Rtype::SOA), SharedRrset::new(rrset)); + + // -- Remove the A record. + let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); + rrset.push_data(A::new(p("133.69.136.5")).into()); + diff.removed.insert( + (n("NEZU.JAIN.AD.JP"), Rtype::A), + SharedRrset::new(rrset), + ); - let zone = load_zone(include_bytes!( - "../../../../test-data/zonefiles/big.example.com.txt" - )); + // -- Add the new SOA. + let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); + let soa = Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(2), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ); + rrset.push_data(soa.into()); + diff.added + .insert((n("JAIN.AD.JP"), Rtype::SOA), SharedRrset::new(rrset)); + + // -- Add the new A records. + let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); + rrset.push_data(A::new(p("133.69.136.4")).into()); + rrset.push_data(A::new(p("192.41.197.2")).into()); + diff.added.insert( + (n("JAIN-BB.JAIN.AD.JP"), Rtype::A), + SharedRrset::new(rrset), + ); - let req = mk_udp_ixfr_request(zone.apex_name(), Serial(0), ()); + diffs.push(diff); + + // Diff 2: One of the IP addresses of JAIN-BB.JAIN.AD.JP. is changed. + let mut diff = ZoneDiff::new(); + diff.start_serial = Some(Serial(2)); + diff.end_serial = Some(Serial(3)); + + // -- Remove the old SOA. + let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); + let soa = Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(2), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ); + rrset.push_data(soa.into()); + diff.removed + .insert((n("JAIN.AD.JP"), Rtype::SOA), SharedRrset::new(rrset)); + + // Remove the outdated IP address. + let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); + rrset.push_data(A::new(p("133.69.136.4")).into()); + diff.removed.insert( + (n("JAIN-BB.JAIN.AD.JP"), Rtype::A), + SharedRrset::new(rrset), + ); - let res = do_preprocess(zone.clone(), &req).await; + // -- Add the new SOA. + let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); + let soa = Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(3), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ); + rrset.push_data(soa.into()); + diff.added + .insert((n("JAIN.AD.JP"), Rtype::SOA), SharedRrset::new(rrset)); + + // Add the updated IP address. + let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); + rrset.push_data(A::new(p("133.69.136.3")).into()); + diff.added.insert( + (n("JAIN-BB.JAIN.AD.JP"), Rtype::A), + SharedRrset::new(rrset), + ); + + diffs.push(diff); + + // Create an object that knows how to provide zone and diff data for + // our zone and diffs. + let zone_with_diffs = ZoneWithDiffs::new(zone.clone(), diffs); + + // The following IXFR query + let req = mk_ixfr_request(zone.apex_name(), Serial(1), ()); + + let res = do_preprocess(zone_with_diffs, &req).await; let ControlFlow::Break(mut stream) = res else { - panic!("AXFR failed"); + panic!("IXFR failed"); }; - assert_udp_xfr_stream_eq( + let zone_soa = get_zone_soa(&zone).await; + + // could be replied to with the following full zone transfer message: + let mut expected_records: ExpectedRecords = vec![ + (n("JAIN.AD.JP."), zone_soa.clone().into()), + ( + n("JAIN.AD.JP."), + Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(1), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ) + .into(), + ), + (n("NEZU.JAIN.AD.JP."), A::new(p("133.69.136.5")).into()), + ( + n("JAIN.AD.JP."), + Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(2), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ) + .into(), + ), + (n("JAIN-BB.JAIN.AD.JP."), A::new(p("133.69.136.4")).into()), + (n("JAIN-BB.JAIN.AD.JP."), A::new(p("192.41.197.2")).into()), + ( + n("JAIN.AD.JP."), + Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(2), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ) + .into(), + ), + (n("JAIN-BB.JAIN.AD.JP."), A::new(p("133.69.136.4")).into()), + ( + n("JAIN.AD.JP."), + Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(3), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ) + .into(), + ), + (n("JAIN-BB.JAIN.AD.JP."), A::new(p("133.69.136.3")).into()), + (n("JAIN.AD.JP."), zone_soa.into()), + ]; + + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::BeginTransaction) + )); + + let stream = assert_stream_eq( req.message(), - &zone, &mut stream, &mut expected_records, ) .await; + + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::EndTransaction) + )); } #[tokio::test] - async fn ixfr_single_response_tcp() {} + async fn ixfr_rfc1995_section7_udp_packet_overflow() { + // Based on https://datatracker.ietf.org/doc/html/rfc1995#section-7 + let zone = load_zone(include_bytes!( + "../../../../test-data/zonefiles/big.example.com.txt" + )); + + let req = mk_udp_ixfr_request(zone.apex_name(), Serial(0), ()); + + let res = do_preprocess(zone.clone(), &req).await; + + let ControlFlow::Break(mut stream) = res else { + panic!("IXFR failed"); + }; + + let zone_soa = get_zone_soa(&zone).await; + + let mut expected_records: ExpectedRecords = + vec![(n("example.com"), zone_soa.into())]; + + assert_stream_eq(req.message(), &mut stream, &mut expected_records) + .await; + } #[tokio::test] async fn ixfr_multi_response_tcp() {} @@ -1558,6 +1860,27 @@ mod tests { Zone::try_from(reader).unwrap() } + async fn get_zone_soa(zone: &Zone) -> Soa> { + let read = zone.read(); + let zone_soa_answer = + XfrMiddlewareSvc::<_, TestNextSvc, Zone>::read_soa( + &read, + zone.apex_name().to_owned(), + ) + .await + .unwrap(); + let AnswerContent::Data(zone_soa_rrset) = + zone_soa_answer.content().clone() + else { + unreachable!() + }; + let first_rr = zone_soa_rrset.first().unwrap(); + let ZoneRecordData::Soa(soa) = first_rr.data() else { + unreachable!() + }; + soa.clone() + } + fn mk_axfr_request( qname: impl ToName, metadata: T, @@ -1603,6 +1926,21 @@ mod tests { ) } + fn mk_ixfr_request( + qname: impl ToName + Clone, + serial: Serial, + metadata: T, + ) -> Request, T> { + mk_ixfr_request_for_transport( + qname, + serial, + metadata, + TransportSpecificContext::NonUdp(NonUdpTransportContext::new( + None, + )), + ) + } + fn mk_udp_ixfr_request( qname: impl ToName + Clone, serial: Serial, @@ -1629,10 +1967,8 @@ mod tests { msg.push((qname.clone(), Rtype::IXFR)).unwrap(); let mut msg = msg.authority(); - let mname = Name::::from_str("mname").unwrap(); - let rname = Name::::from_str("rname").unwrap(); let ttl = Ttl::from_secs(0); - let soa = Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl); + let soa = Soa::new(n("name"), n("rname"), serial, ttl, ttl, ttl, ttl); msg.push((qname, Class::IN, Ttl::from_secs(0), soa)) .unwrap(); let msg = msg.into_message(); @@ -1646,8 +1982,8 @@ mod tests { ) } - async fn do_preprocess( - zone: Zone, + async fn do_preprocess( + zone: XDP, req: &Request, T>, ) -> ControlFlow< XfrMiddlewareStream< @@ -1656,7 +1992,7 @@ mod tests { <::Stream as Stream>::Item, >, > { - XfrMiddlewareSvc::<_, TestNextSvc, Zone>::preprocess( + XfrMiddlewareSvc::<_, TestNextSvc, XDP>::preprocess( Arc::new(Semaphore::new(1)), Arc::new(Semaphore::new(1)), req, @@ -1665,165 +2001,60 @@ mod tests { .await } - async fn assert_xfr_stream_eq( + async fn assert_stream_eq< + O: octseq::Octets, + S: Stream>, ServiceError>> + Unpin, + >( req: &Message, - zone: &Zone, - stream: impl Stream>, ServiceError>> - + Unpin, + mut stream: S, expected_records: &mut ExpectedRecords, - ) { - assert_transport_specific_xfr_stream_eq( - req, - zone, - stream, - expected_records, - false, - ) - .await - } - - async fn assert_udp_xfr_stream_eq( - req: &Message, - zone: &Zone, - stream: impl Stream>, ServiceError>> - + Unpin, - expected_records: &mut ExpectedRecords, - ) { - assert_transport_specific_xfr_stream_eq( - req, - zone, - stream, - expected_records, - true, - ) - .await - } - - async fn assert_transport_specific_xfr_stream_eq( - req: &Message, - zone: &Zone, - mut stream: impl Stream>, ServiceError>> - + Unpin, - expected_records: &mut ExpectedRecords, - udp: bool, - ) { - let read = zone.read(); - let q = req.first_question().unwrap(); - let zone_soa_answer = - XfrMiddlewareSvc::<_, TestNextSvc, Zone>::read_soa( - &read, - q.qname().to_name(), - ) - .await - .unwrap(); - let AnswerContent::Data(zone_soa_rrset) = - zone_soa_answer.content().clone() - else { - unreachable!() - }; - let first_rr = zone_soa_rrset.first().unwrap(); - let ZoneRecordData::Soa(expected_soa) = first_rr.data() else { - unreachable!() - }; - - // Allow for the RFC 1995 UDP IXFR case where the response doesn't fit - // in a single UDP message and so a single SOA reply is sent instead. - if !udp { + ) -> S { + while !expected_records.is_empty() { let msg = stream.next().await.unwrap().unwrap(); - assert!(matches!( - msg.feedback(), - Some(ServiceFeedback::BeginTransaction) - )); - } - let mut msg = stream.next().await.unwrap().unwrap(); - let mut first = true; - let mut last = false; - while msg.response().is_some() { let resp_builder = msg.into_inner().0.unwrap(); let resp = resp_builder.as_message(); assert!(resp.is_answer(req)); let mut records = resp.answer().unwrap().peekable(); - if first { - let rec = records.next().unwrap().unwrap(); - assert_eq!(rec.owner(), zone.apex_name()); - assert_eq!(rec.rtype(), Rtype::SOA); - let soa = rec - .into_record::>>() - .unwrap() + for rec in records.by_ref() { + let rec = rec.unwrap(); + + let pos = expected_records + .iter() + .position(|(name, data)| { + name == &rec.owner() && data.rtype() == rec.rtype() + }) + .unwrap_or_else(|| { + panic!( + "XFR record {} {} {} was not expected", + rec.owner(), + rec.class(), + rec.rtype() + ) + }); + + let (_, data) = expected_records.remove(pos); + + let rec = rec + .into_record::>>() .unwrap() - .into_data(); - assert_eq!(&soa, expected_soa); - first = false; + .unwrap(); - if udp - && records.peek().is_none() - && expected_records.is_empty() - { - // Stop. This is the IXFR 1995 UDP case where the response - // didn't fit in a single UDP packet and so a single SOA - // reply is sent back instead. - assert!(stream.next().await.is_none()); - return; - } - } + assert_eq!(&data, rec.data()); - for rec in records.by_ref() { - let rec = rec.unwrap(); - if rec.rtype() == Rtype::SOA { - let soa = rec - .into_record::>>() - .unwrap() - .unwrap() - .into_data(); - assert_eq!(&soa, expected_soa); - last = true; - break; - } else { - let pos = expected_records - .iter() - .position(|(name, data)| { - name == &rec.owner() - && data.rtype() == rec.rtype() - }) - .unwrap_or_else(|| { - panic!( - "XFR record {} {} {} was not expected", - rec.owner(), - rec.class(), - rec.rtype() - ) - }); - let (_, data) = expected_records.remove(pos); - let rec = rec - .into_record::>>() - .unwrap() - .unwrap(); - assert_eq!(&data, rec.data()); - eprintln!( - "Found {} {} {}", - rec.owner(), - rec.class(), - rec.rtype() - ) - } + eprintln!( + "Found {} {} {}", + rec.owner(), + rec.class(), + rec.rtype() + ) } assert!(records.next().is_none()); - - msg = stream.next().await.unwrap().unwrap(); } - assert!(last); - assert!(expected_records.is_empty()); - - assert!(matches!( - msg.feedback(), - Some(ServiceFeedback::EndTransaction) - )); - - assert!(stream.next().await.is_none()); + stream } #[derive(Clone)] @@ -1838,4 +2069,60 @@ mod tests { todo!() } } + + struct ZoneWithDiffs { + zone: Zone, + diffs: Vec>, + } + + impl ZoneWithDiffs { + fn new(zone: Zone, diffs: Vec) -> Self { + Self { + zone, + diffs: diffs.into_iter().map(Arc::new).collect(), + } + } + } + + impl XfrDataProvider for ZoneWithDiffs { + fn request( + &self, + _req: &Request, + apex_name: &impl ToName, + class: Class, + diff_from: Option, + ) -> Pin< + Box< + dyn Future< + Output = Result< + (Zone, Vec>), + XfrDataProviderError, + >, + > + Sync + + Send, + >, + > + where + Octs: AsRef<[u8]> + Send + Sync, + { + let res = if apex_name.to_name::() == self.zone.apex_name() + && class == self.zone.class() + { + let diffs = + if self.diffs.first().and_then(|diff| diff.start_serial) + == diff_from + { + self.diffs.clone() + } else { + vec![] + }; + + Ok((self.zone.clone(), diffs)) + } else { + Err(XfrDataProviderError::UnknownZone) + }; + + Box::pin(ready(res)) + } + } } From 87011d5b6ad305eda3ed7c27d2ccfdfd5bb0d74a Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 9 Sep 2024 20:34:51 +0200 Subject: [PATCH 134/333] Minor RustDoc additions. --- src/net/server/middleware/notify.rs | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/src/net/server/middleware/notify.rs b/src/net/server/middleware/notify.rs index 4bfc5a348..097248d4f 100644 --- a/src/net/server/middleware/notify.rs +++ b/src/net/server/middleware/notify.rs @@ -38,8 +38,11 @@ use crate::zonetree::StoredName; /// [1996]: https://datatracker.ietf.org/doc/html/rfc1996 #[derive(Clone, Debug)] pub struct NotifyMiddlewareSvc { + /// The upstream [`Service`] to pass requests to and receive responses + /// from. next_svc: NextSvc, + /// The target to send notifications to. notify_target: N, _phantom: PhantomData<(RequestOctets, RequestMeta)>, @@ -48,6 +51,7 @@ pub struct NotifyMiddlewareSvc { impl NotifyMiddlewareSvc { + /// Creates an instance of this middleware service. #[must_use] pub fn new(next_svc: NextSvc, notify_target: N) -> Self { Self { @@ -309,8 +313,9 @@ where } } -//------------ Notifiable ----------------------------------------------------- +//------------ NotifyError ---------------------------------------------------- +/// Errors reportable by a [`Notifiable`] trait impl. #[derive(Clone, Debug, PartialEq, Eq)] pub enum NotifyError { /// We are not authoritative for the zone. @@ -320,9 +325,15 @@ pub enum NotifyError { Other, } +//------------ Notifiable ----------------------------------------------------- + +/// A target for notifications sent by [`NotifyMiddlewareSvc`]. // Note: The fn signatures can be simplified to fn() -> impl Future<...> if // our MSRV is later increased. pub trait Notifiable { + /// A notification that the content of a zone has changed. + /// + /// The origin of the notification is the passed `source` IP address. fn notify_zone_changed( &self, class: Class, From 53c86e4502eb383eb729e6a0853b89fb9822e059 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 10 Sep 2024 10:47:49 +0200 Subject: [PATCH 135/333] Change support for custom metadata so that the actual type is known, not just some type, so that the XfrDataProvider impl can use functions on the type, and add an axfr_with_tsig_key_name() test showing it in action. --- src/net/server/middleware/xfr.rs | 124 ++++++++++++++++++++-------- src/net/server/tests/integration.rs | 4 +- 2 files changed, 91 insertions(+), 37 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 50ef49f40..8d153afa3 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -61,7 +61,9 @@ use crate::zonetree::{ /// [1995]: https://datatracker.ietf.org/doc/html/rfc1995 /// [5936]: https://datatracker.ietf.org/doc/html/rfc5936 #[derive(Clone, Debug)] -pub struct XfrMiddlewareSvc { +pub struct XfrMiddlewareSvc { + /// The upstream [`Service`] to pass requests to and receive responses + /// from. next_svc: NextSvc, xfr_data_provider: XDP, @@ -70,13 +72,13 @@ pub struct XfrMiddlewareSvc { batcher_semaphore: Arc, - _phantom: PhantomData, + _phantom: PhantomData<(RequestOctets, Metadata)>, } -impl - XfrMiddlewareSvc +impl + XfrMiddlewareSvc where - XDP: XfrDataProvider, + XDP: XfrDataProvider, { /// Creates a new processor instance. #[must_use] @@ -99,8 +101,8 @@ where } } -impl - XfrMiddlewareSvc +impl + XfrMiddlewareSvc where RequestOctets: Octets + Send + Sync + 'static + Unpin, for<'a> ::Range<'a>: Send + Sync, @@ -108,12 +110,12 @@ where NextSvc::Future: Send + Sync + Unpin, NextSvc::Target: Composer + Default + Send + Sync, NextSvc::Stream: Send + Sync, - XDP: XfrDataProvider, + XDP: XfrDataProvider, { - pub async fn preprocess( + pub async fn preprocess( zone_walking_semaphore: Arc, batcher_semaphore: Arc, - req: &Request, + req: &Request, xfr_data_provider: XDP, ) -> ControlFlow< XfrMiddlewareStream< @@ -942,7 +944,7 @@ where //--- impl Service impl Service - for XfrMiddlewareSvc + for XfrMiddlewareSvc where RequestOctets: Octets + Send + Sync + Unpin + 'static, for<'a> ::Range<'a>: Send + Sync, @@ -950,7 +952,7 @@ where NextSvc::Future: Send + Sync + Unpin, NextSvc::Target: Composer + Default + Send + Sync, NextSvc::Stream: Send + Sync, - XDP: XfrDataProvider + Clone + Sync + Send + 'static, + XDP: XfrDataProvider + Clone + Sync + Send + 'static, Metadata: Clone + Default + Sync + Send + 'static, { type Target = NextSvc::Target; @@ -1026,7 +1028,7 @@ pub enum XfrDataProviderError { //------------ Transferable --------------------------------------------------- /// A provider of data needed for responding to XFR requests. -pub trait XfrDataProvider { +pub trait XfrDataProvider { /// Request data needed to respond to an XFR request. /// /// Returns Ok if the request is allowed and the requested data is @@ -1034,7 +1036,7 @@ pub trait XfrDataProvider { /// /// Returns Err otherwise. #[allow(clippy::type_complexity)] - fn request( + fn request( &self, req: &Request, apex_name: &impl ToName, @@ -1057,12 +1059,12 @@ pub trait XfrDataProvider { //--- impl for AsRef -impl XfrDataProvider for U +impl XfrDataProvider for U where - T: XfrDataProvider, + T: XfrDataProvider, U: Deref, { - fn request( + fn request( &self, req: &Request, apex_name: &impl ToName, @@ -1088,14 +1090,14 @@ where //--- impl for Zone -impl XfrDataProvider for Zone { +impl XfrDataProvider for Zone { /// Request data needed to respond to an XFR request. /// /// Returns Ok(Self, vec![]) if the given apex name and class match this /// zone, irrespective of the given request or diff range. /// /// Returns Err if the requested zone is not this zone. - fn request( + fn request( &self, _req: &Request, apex_name: &impl ToName, @@ -1129,14 +1131,14 @@ impl XfrDataProvider for Zone { //--- impl for ZoneTree -impl XfrDataProvider for ZoneTree { +impl XfrDataProvider for ZoneTree { /// Request data needed to respond to an XFR request. /// /// Returns Ok(zone, vec![]) if the given apex name and class match a zone /// in this zone tree, irrespective of the given request or diff range. /// /// Returns Err if the requested zone is not this zone tree. - fn request( + fn request( &self, _req: &Request, apex_name: &impl ToName, @@ -1369,13 +1371,15 @@ mod tests { use crate::zonetree::types::Rrset; use super::*; + use crate::net::server::middleware::tsig::{ + Authentication, MaybeAuthenticated, + }; + use crate::tsig::KeyName; + use core::sync::atomic::{AtomicBool, Ordering}; type ExpectedRecords = Vec<(Name, AllRecordData>)>; - #[tokio::test] - async fn axfr_minimal() {} - #[tokio::test] async fn axfr_with_example_zone() { let zone = load_zone(include_bytes!( @@ -1832,14 +1836,62 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 #[tokio::test] async fn ixfr_multi_response_tcp() {} - // #[tokio::test] - // async fn axfr_with_tsig() { - // let metadata = Authentication(Some( - // KeyName::from_str("blah").unwrap(), - // )) + #[tokio::test] + async fn axfr_with_tsig_key_name() { + // Define an XfrDataProvider that expects to receive a Request that is + // generic over a type that we specify: Authentication. This is the + // type over which the Request produced by TsigMiddlewareSvc is generic. + // When the XfrMiddlewareSvc receives a Request it + // passes it to the XfrDataProvider which in turn can inspect it. + struct KeyReceivingXfrDataProvider { + key_name: KeyName, + checked: Arc, + } + + impl XfrDataProvider for KeyReceivingXfrDataProvider { + #[allow(clippy::type_complexity)] + fn request( + &self, + req: &Request, + _apex_name: &impl ToName, + _class: Class, + _diff_from: Option, + ) -> Pin< + Box< + dyn Future< + Output = Result< + (Zone, Vec>), + XfrDataProviderError, + >, + > + Sync + + Send, + >, + > + where + Octs: AsRef<[u8]> + Send + Sync, + { + assert_eq!(req.metadata().key_name(), Some(&self.key_name)); + self.checked.store(true, Ordering::SeqCst); + Box::pin(ready(Err(XfrDataProviderError::Refused))) + } + } - // let req = mk_axfr_request("example.com", metadata); - // } + let key_name = KeyName::from_str("some_tsig_key_name").unwrap(); + let metadata = Authentication(Some(key_name.clone())); + let req = mk_axfr_request(n("example.com"), metadata); + let checked = Arc::new(AtomicBool::new(false)); + let xdp = KeyReceivingXfrDataProvider { + key_name, + checked: checked.clone(), + }; + + // Invoke XfrMiddlewareSvc with our custom XfrDataProvidedr. + let _ = do_preprocess(xdp, &req).await; + + // Veirfy that our XfrDataProvider was invoked and received the expected + // TSIG key name data. + assert!(checked.load(Ordering::SeqCst)); + } //------------ Helper functions ------------------------------------------- @@ -1982,9 +2034,9 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 ) } - async fn do_preprocess( + async fn do_preprocess>( zone: XDP, - req: &Request, T>, + req: &Request, Metadata>, ) -> ControlFlow< XfrMiddlewareStream< ::Future, @@ -1992,7 +2044,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 <::Stream as Stream>::Item, >, > { - XfrMiddlewareSvc::<_, TestNextSvc, XDP>::preprocess( + XfrMiddlewareSvc::<_, TestNextSvc, XDP, Metadata>::preprocess( Arc::new(Semaphore::new(1)), Arc::new(Semaphore::new(1)), req, @@ -2085,9 +2137,9 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 } impl XfrDataProvider for ZoneWithDiffs { - fn request( + fn request( &self, - _req: &Request, + _req: &Request, apex_name: &impl ToName, class: Class, diff_from: Option, diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index 657d6ba16..2ac673fe1 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -39,6 +39,7 @@ use crate::net::server::middleware::mandatory::MandatoryMiddlewareSvc; use crate::net::server::middleware::notify::{ Notifiable, NotifyError, NotifyMiddlewareSvc, }; +use crate::net::server::middleware::tsig::Authentication; use crate::net::server::middleware::tsig::TsigMiddlewareSvc; use crate::net::server::middleware::xfr::XfrMiddlewareSvc; use crate::net::server::service::{CallResult, Service, ServiceResult}; @@ -163,7 +164,8 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { // 4. XFR(-in) middleware service (XFR-out is handled by the // ZoneMaintainer). - let svc = XfrMiddlewareSvc::, _, _>::new(svc, zones, 1); + let svc = + XfrMiddlewareSvc::, _, _, Authentication>::new(svc, zones, 1); // 5. RFC 1996 NOTIFY support. let svc = NotifyMiddlewareSvc::new(svc, TestNotifyTarget); From eecff8a18177e3e90b86a258836e4ef4180f5c18 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 10 Sep 2024 11:20:03 +0200 Subject: [PATCH 136/333] Fix test compilation issues that occur with Rust 1.68.2 and `cargo +nightly update -Z minimal-versions`. --- src/net/server/middleware/xfr.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 8d153afa3..b13e68d94 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -1915,7 +1915,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 async fn get_zone_soa(zone: &Zone) -> Soa> { let read = zone.read(); let zone_soa_answer = - XfrMiddlewareSvc::<_, TestNextSvc, Zone>::read_soa( + XfrMiddlewareSvc::, TestNextSvc, Zone>::read_soa( &read, zone.apex_name().to_owned(), ) @@ -2044,7 +2044,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 <::Stream as Stream>::Item, >, > { - XfrMiddlewareSvc::<_, TestNextSvc, XDP, Metadata>::preprocess( + XfrMiddlewareSvc::, TestNextSvc, XDP, Metadata>::preprocess( Arc::new(Semaphore::new(1)), Arc::new(Semaphore::new(1)), req, From 360308a62c7922abce60aef244e65327cba35d82 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 10 Sep 2024 15:23:03 +0200 Subject: [PATCH 137/333] Simplify the XfrDataProvider interface and make the trait signature actually usable qua lifetimes (as tested by integrating it in the xfr branch with zone maintainer). Also fix an issue in assert_stream_eq() not comparing properly. --- src/net/server/middleware/xfr.rs | 119 ++++++++++++++++--------------- 1 file changed, 61 insertions(+), 58 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index b13e68d94..30952f92c 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -17,7 +17,7 @@ use tokio::sync::Semaphore; use tokio_stream::wrappers::UnboundedReceiverStream; use tracing::{debug, error, info, trace, warn}; -use crate::base::iana::{Class, Opcode, OptRcode, Rcode}; +use crate::base::iana::{Opcode, OptRcode, Rcode}; use crate::base::message_builder::{ AdditionalBuilder, AnswerBuilder, PushError, }; @@ -158,7 +158,7 @@ where // Is transfer allowed for the requested zone for this requestor? let res = xfr_data_provider - .request(req, q.qname(), q.qclass(), ixfr_query_serial) + .request(req, ixfr_query_serial) .await .map_err(|err| match err { XfrDataProviderError::UnknownZone => { @@ -1039,8 +1039,6 @@ pub trait XfrDataProvider { fn request( &self, req: &Request, - apex_name: &impl ToName, - class: Class, diff_from: Option, ) -> Pin< Box< @@ -1050,25 +1048,24 @@ pub trait XfrDataProvider { XfrDataProviderError, >, > + Sync - + Send, + + Send + + '_, >, > where - Octs: AsRef<[u8]> + Send + Sync; + Octs: Octets + Send + Sync; } //--- impl for AsRef impl XfrDataProvider for U where - T: XfrDataProvider, + T: XfrDataProvider + 'static, U: Deref, { fn request( &self, req: &Request, - apex_name: &impl ToName, - class: Class, diff_from: Option, ) -> Pin< Box< @@ -1078,13 +1075,14 @@ where XfrDataProviderError, >, > + Sync - + Send, + + Send + + '_, >, > where - Octs: AsRef<[u8]> + Send + Sync, + Octs: Octets + Send + Sync, { - (**self).request(req, apex_name, class, diff_from) + (**self).request(req, diff_from) } } @@ -1099,9 +1097,7 @@ impl XfrDataProvider for Zone { /// Returns Err if the requested zone is not this zone. fn request( &self, - _req: &Request, - apex_name: &impl ToName, - class: Class, + req: &Request, _diff_from: Option, ) -> Pin< Box< @@ -1115,12 +1111,14 @@ impl XfrDataProvider for Zone { >, > where - Octs: AsRef<[u8]> + Send + Sync, + Octs: Octets + Send + Sync, { - let res = if apex_name.to_name::() == self.apex_name() - && class == self.class() - { - Ok((self.clone(), vec![])) + let res = if let Ok(q) = req.message().sole_question() { + if q.qname() == self.apex_name() && q.qclass() == self.class() { + Ok((self.clone(), vec![])) + } else { + Err(XfrDataProviderError::UnknownZone) + } } else { Err(XfrDataProviderError::UnknownZone) }; @@ -1140,9 +1138,7 @@ impl XfrDataProvider for ZoneTree { /// Returns Err if the requested zone is not this zone tree. fn request( &self, - _req: &Request, - apex_name: &impl ToName, - class: Class, + req: &Request, _diff_from: Option, ) -> Pin< Box< @@ -1156,10 +1152,14 @@ impl XfrDataProvider for ZoneTree { >, > where - Octs: AsRef<[u8]> + Send + Sync, + Octs: Octets + Send + Sync, { - let res = if let Some(zone) = self.get_zone(apex_name, class) { - Ok((zone.clone(), vec![])) + let res = if let Ok(q) = req.message().sole_question() { + if let Some(zone) = self.find_zone(q.qname(), q.qclass()) { + Ok((zone.clone(), vec![])) + } else { + Err(XfrDataProviderError::UnknownZone) + } } else { Err(XfrDataProviderError::UnknownZone) }; @@ -1356,26 +1356,28 @@ impl CallbackState { #[cfg(test)] mod tests { use core::str::FromStr; + use core::sync::atomic::{AtomicBool, Ordering}; + use std::borrow::ToOwned; use futures::StreamExt; use tokio::time::Instant; - use crate::base::{MessageBuilder, RecordData, Ttl}; + use crate::base::iana::Class; + use crate::base::{MessageBuilder, Ttl}; use crate::net::server::message::{ NonUdpTransportContext, UdpTransportContext, }; + use crate::net::server::middleware::tsig::{ + Authentication, MaybeAuthenticated, + }; use crate::net::server::service::ServiceError; use crate::rdata::{Aaaa, AllRecordData, Cname, Mx, Ns, Txt, A}; + use crate::tsig::KeyName; use crate::zonefile::inplace::Zonefile; use crate::zonetree::types::Rrset; use super::*; - use crate::net::server::middleware::tsig::{ - Authentication, MaybeAuthenticated, - }; - use crate::tsig::KeyName; - use core::sync::atomic::{AtomicBool, Ordering}; type ExpectedRecords = Vec<(Name, AllRecordData>)>; @@ -1400,7 +1402,7 @@ mod tests { (n("example.com"), zone_soa.clone().into()), (n("example.com"), Ns::new(n("example.com")).into()), (n("example.com"), A::new(p("192.0.2.1")).into()), - (n("example.com"), A::new(p("192.0.2.1")).into()), + (n("example.com"), A::new(p("127.0.0.1")).into()), (n("example.com"), A::new(p("192.0.2.1")).into()), (n("example.com"), Aaaa::new(p("2001:db8::3")).into()), (n("www.example.com"), Cname::new(n("example.com")).into()), @@ -1853,8 +1855,6 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 fn request( &self, req: &Request, - _apex_name: &impl ToName, - _class: Class, _diff_from: Option, ) -> Pin< Box< @@ -1868,7 +1868,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 >, > where - Octs: AsRef<[u8]> + Send + Sync, + Octs: Octets + Send + Sync, { assert_eq!(req.metadata().key_name(), Some(&self.key_name)); self.checked.store(true, Ordering::SeqCst); @@ -2072,28 +2072,27 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 for rec in records.by_ref() { let rec = rec.unwrap(); + let rec = rec + .into_record::>>() + .unwrap() + .unwrap(); + let pos = expected_records .iter() .position(|(name, data)| { - name == &rec.owner() && data.rtype() == rec.rtype() + name == rec.owner() && data == rec.data() }) .unwrap_or_else(|| { panic!( - "XFR record {} {} {} was not expected", + "XFR record {} {} {} {} was not expected", rec.owner(), rec.class(), - rec.rtype() + rec.rtype(), + rec.data(), ) }); - let (_, data) = expected_records.remove(pos); - - let rec = rec - .into_record::>>() - .unwrap() - .unwrap(); - - assert_eq!(&data, rec.data()); + let _ = expected_records.remove(pos); eprintln!( "Found {} {} {}", @@ -2139,9 +2138,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 impl XfrDataProvider for ZoneWithDiffs { fn request( &self, - _req: &Request, - apex_name: &impl ToName, - class: Class, + req: &Request, diff_from: Option, ) -> Pin< Box< @@ -2155,13 +2152,16 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 >, > where - Octs: AsRef<[u8]> + Send + Sync, + Octs: Octets + Send + Sync, { - let res = if apex_name.to_name::() == self.zone.apex_name() - && class == self.zone.class() - { - let diffs = - if self.diffs.first().and_then(|diff| diff.start_serial) + let res = if let Ok(q) = req.message().sole_question() { + if q.qname() == self.zone.apex_name() + && q.qclass() == self.zone.class() + { + let diffs = if self + .diffs + .first() + .and_then(|diff| diff.start_serial) == diff_from { self.diffs.clone() @@ -2169,7 +2169,10 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 vec![] }; - Ok((self.zone.clone(), diffs)) + Ok((self.zone.clone(), diffs)) + } else { + Err(XfrDataProviderError::UnknownZone) + } } else { Err(XfrDataProviderError::UnknownZone) }; @@ -2177,4 +2180,4 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 Box::pin(ready(res)) } } -} +} \ No newline at end of file From 1b39cfdcc0d01315081ac49df11c740aa0ff2461 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 10 Sep 2024 15:23:29 +0200 Subject: [PATCH 138/333] Cargo fmt. --- src/net/server/middleware/xfr.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 30952f92c..d35c477d6 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -2180,4 +2180,4 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 Box::pin(ready(res)) } } -} \ No newline at end of file +} From 1071c53b1a699416fcf6de5d7c12f8bc03d251a5 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 10 Sep 2024 15:27:04 +0200 Subject: [PATCH 139/333] Sync nsd-example.txt with the published version. --- src/net/server/middleware/xfr.rs | 2 -- test-data/zonefiles/nsd-example.txt | 16 +++++++--------- 2 files changed, 7 insertions(+), 11 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index d35c477d6..2b5c9ffa5 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -1402,8 +1402,6 @@ mod tests { (n("example.com"), zone_soa.clone().into()), (n("example.com"), Ns::new(n("example.com")).into()), (n("example.com"), A::new(p("192.0.2.1")).into()), - (n("example.com"), A::new(p("127.0.0.1")).into()), - (n("example.com"), A::new(p("192.0.2.1")).into()), (n("example.com"), Aaaa::new(p("2001:db8::3")).into()), (n("www.example.com"), Cname::new(n("example.com")).into()), (n("mail.example.com"), Mx::new(10, n("example.com")).into()), diff --git a/test-data/zonefiles/nsd-example.txt b/test-data/zonefiles/nsd-example.txt index 06ba2b8a6..bedf91ac6 100644 --- a/test-data/zonefiles/nsd-example.txt +++ b/test-data/zonefiles/nsd-example.txt @@ -1,7 +1,8 @@ -$ORIGIN example.com. ; 'default' domain as FQDN for this zone +; From: https://nsd.docs.nlnetlabs.nl/en/latest/zonefile.html#creating-a-zone +$ORIGIN example.com. $TTL 86400 ; default time-to-live for this zone -example.com. IN SOA ns.example.com. noc.dns.icann.org. ( +example.com. IN SOA ns.example.com. noc.dns.example.org. ( 2020080302 ;Serial 7200 ;Refresh 3600 ;Retry @@ -9,17 +10,14 @@ example.com. IN SOA ns.example.com. noc.dns.icann.org. ( 3600 ;Negative response caching TTL ) -; The nameserver that are authoritative for this zone. +; The nameservers that are authoritative for this zone. NS example.com. -; these A records below are equivalent +; A and AAAA records are for IPv4 and IPv6 addresses respectively example.com. A 192.0.2.1 -@ A 192.0.2.1 - A 192.0.2.1 + AAAA 2001:db8::3 -@ AAAA 2001:db8::3 - -; A CNAME redirect from www.exmaple.com to example.com +; A CNAME redirects from www.example.com to example.com www CNAME example.com. mail MX 10 example.com. From fd274f1868bc4afed336ddc7f20d3b292e10498d Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 10 Sep 2024 16:25:54 +0200 Subject: [PATCH 140/333] Review feedback. --- src/net/xfr/processing/iterator.rs | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/net/xfr/processing/iterator.rs b/src/net/xfr/processing/iterator.rs index d18f036c6..fee8cdd0a 100644 --- a/src/net/xfr/processing/iterator.rs +++ b/src/net/xfr/processing/iterator.rs @@ -63,18 +63,19 @@ impl<'a, 'b> Iterator for XfrEventIterator<'a, 'b> { type Item = Result, IterationError>; fn next(&mut self) -> Option { - match self.iter.next() { - Some(Ok(record)) => { + match self.iter.next()? { + Ok(record) => { trace!("XFR record {}: {record:?}", self.state.rr_count); let event = self.state.process_record(record); Some(Ok(event)) } - Some(Err(err)) => Some(Err(IterationError::ParseError(err))), - - None => { - // No more events available: end iteration. - None + Err(err) => { + trace!( + "XFR record {}: parsing error: {err}", + self.state.rr_count + ); + Some(Err(IterationError::ParseError(err))) } } } From ba357b6407208a9ef199ced9b4028db8f1ab18b0 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 10 Sep 2024 16:28:27 +0200 Subject: [PATCH 141/333] Updates backported from the `xfr` branch: - Iterate over `ZoneRecordData` rather than `AllRecordData` to match what ZoneTree supports. - RustDoc and comment fixes/improvements. - Pass the actual record along with more XfrEvent variants (as it is useful for the consumer to recieve them and allows the tests to do more checks). --- src/net/xfr/processing/iterator.rs | 9 +++-- src/net/xfr/processing/processor.rs | 44 ++++++++++----------- src/net/xfr/processing/tests.rs | 59 ++++++++++++++++++++--------- src/net/xfr/processing/types.rs | 32 ++++++++++++---- 4 files changed, 92 insertions(+), 52 deletions(-) diff --git a/src/net/xfr/processing/iterator.rs b/src/net/xfr/processing/iterator.rs index fee8cdd0a..3cbc8b417 100644 --- a/src/net/xfr/processing/iterator.rs +++ b/src/net/xfr/processing/iterator.rs @@ -3,8 +3,9 @@ use bytes::Bytes; use tracing::trace; -use crate::base::{message::AnyRecordIter, Message, ParsedName}; -use crate::rdata::AllRecordData; +use crate::base::message::RecordIter; +use crate::base::{Message, ParsedName}; +use crate::rdata::ZoneRecordData; use super::processor::RecordProcessor; use super::types::{IterationError, ProcessingError, XfrEvent, XfrRecord}; @@ -19,7 +20,7 @@ pub struct XfrEventIterator<'a, 'b> { state: &'a mut RecordProcessor, /// An iterator over the records in the current response. - iter: AnyRecordIter<'b, Bytes, AllRecordData>>, + iter: RecordIter<'b, Bytes, ZoneRecordData>>, } impl<'a, 'b> XfrEventIterator<'a, 'b> { @@ -46,7 +47,7 @@ impl<'a, 'b> XfrEventIterator<'a, 'b> { // that might be expected to exist in a zone (i.e. not just // ZoneRecordData record types). - let mut iter = answer.into_records(); + let mut iter = answer.limit_to(); if state.rr_count == 0 { let Some(Ok(_)) = iter.next() else { diff --git a/src/net/xfr/processing/processor.rs b/src/net/xfr/processing/processor.rs index 90f01539d..2120ef1d7 100644 --- a/src/net/xfr/processing/processor.rs +++ b/src/net/xfr/processing/processor.rs @@ -4,7 +4,7 @@ use bytes::Bytes; use crate::base::iana::Opcode; use crate::base::{Message, ParsedName, Rtype}; -use crate::rdata::{AllRecordData, Soa}; +use crate::rdata::{Soa, ZoneRecordData}; use super::iterator::XfrEventIterator; use super::types::{ @@ -59,6 +59,12 @@ impl XfrResponseProcessor { /// If the returned iterator does not emit an [`XfrEvent::EndOfTransfer`] /// event, call this function with the next outstanding response message /// to continue iterating over the incomplete transfer. + /// + /// Checking that the given response corresponds by ID to the related + /// original XFR query or that the question section of the response, if + /// present (RFC 5936 allows it to be empty for subsequent AXFR responses) + /// matches that of the original query is NOT done here but instead is + /// left to the caller to do. pub fn process_answer( &mut self, resp: Message, @@ -90,11 +96,8 @@ impl XfrResponseProcessor { /// Check if an XFR response header is valid. /// - /// Enforce the rules defined in 2. AXFR Messages of RFC 5936. See: - /// https://www.rfc-editor.org/rfc/rfc5936.html#section-2 - /// - /// Takes a request as well as a response as the response is checked to - /// see if it is in reply to the given request. + /// Enforce the rules defined in 2.2. AXFR Messages of RFC 5936. See: + /// https://www.rfc-editor.org/rfc/rfc5936.html#section-2.2 /// /// Returns Ok on success, Err otherwise. On success the type of XFR that /// was determined is returned as well as the answer section from the XFR @@ -106,9 +109,6 @@ impl XfrResponseProcessor { let resp_header = resp.header(); let resp_counts = resp.header_counts(); - // Note: We don't call Message::is_answer() here because that requires - // the message to have a question but subsequent AXFR responses are - // not required to have a question. if resp.is_error() || !resp_header.qr() || resp_header.opcode() != Opcode::QUERY @@ -173,12 +173,12 @@ impl Inner { // that might be expected to exist in a zone (i.e. not just // ZoneRecordData record types). - let mut records = answer.into_records(); + let mut records = answer.limit_to(); let xfr_type = match resp.qtype() { Some(Rtype::AXFR) => XfrType::Axfr, Some(Rtype::IXFR) => XfrType::Ixfr, - _ => unreachable!(), // Checked already in check_request(). + _ => unreachable!(), }; let Some(Ok(record)) = records.next() else { @@ -186,7 +186,7 @@ impl Inner { }; // The initial record should be a SOA record. - let AllRecordData::Soa(soa) = record.into_data() else { + let ZoneRecordData::Soa(soa) = record.into_data() else { return Err(ProcessingError::NotValidXfrResponse); }; @@ -196,7 +196,7 @@ impl Inner { } } -//------------ State ---------------------------------------------------------- +//------------ RecordProcessor ------------------------------------------------ /// State related to processing the XFR response sequence. #[derive(Debug)] @@ -266,7 +266,7 @@ impl RecordProcessor { // having no effect as it is already present. let soa = match rec.data() { - AllRecordData::Soa(soa) => Some(soa), + ZoneRecordData::Soa(soa) => Some(soa), _ => None, }; @@ -282,7 +282,7 @@ impl RecordProcessor { // MUST conclude with the same SOA resource record. // Intermediate messages MUST NOT contain the SOA resource // record." - XfrEvent::EndOfTransfer + XfrEvent::EndOfTransfer(rec) } XfrType::Axfr => { @@ -294,7 +294,7 @@ impl RecordProcessor { XfrType::Ixfr if self.rr_count == 2 => { if record_matches_initial_soa { // IXFR not available, AXFR of empty zone detected. - XfrEvent::EndOfTransfer + XfrEvent::EndOfTransfer(rec) } else if let Some(soa) = soa { // This SOA record is the start of an IXFR diff sequence. self.current_soa = soa.clone(); @@ -306,7 +306,7 @@ impl RecordProcessor { IxfrUpdateMode::Deleting ); - XfrEvent::BeginBatchDelete(soa.serial()) + XfrEvent::BeginBatchDelete(rec) } else { // https://datatracker.ietf.org/doc/html/rfc1995#section-4 // 4. Response Format @@ -351,18 +351,16 @@ impl RecordProcessor { // Is this the end of the transfer, or the start // of a new diff sequence? if record_matches_initial_soa { - XfrEvent::EndOfTransfer + XfrEvent::EndOfTransfer(rec) } else { - XfrEvent::BeginBatchDelete( - self.current_soa.serial(), - ) + XfrEvent::BeginBatchDelete(rec) } } IxfrUpdateMode::Adding => { // We just switched from the Delete phase of a // diff sequence to the add phase of the diff // sequence. - XfrEvent::BeginBatchAdd(self.current_soa.serial()) + XfrEvent::BeginBatchAdd(rec) } } } else { @@ -380,4 +378,4 @@ impl RecordProcessor { } } } -} +} \ No newline at end of file diff --git a/src/net/xfr/processing/tests.rs b/src/net/xfr/processing/tests.rs index 0067e8af0..c309ec1fd 100644 --- a/src/net/xfr/processing/tests.rs +++ b/src/net/xfr/processing/tests.rs @@ -1,7 +1,7 @@ use core::str::FromStr; use bytes::{Bytes, BytesMut}; -use octseq::Octets; +use octseq::{Octets, Parser}; use crate::base::iana::Rcode; use crate::base::message_builder::{ @@ -13,7 +13,7 @@ use crate::base::{ Message, MessageBuilder, ParsedName, Record, Rtype, Serial, Ttl, }; use crate::base::{Name, ToName}; -use crate::rdata::{AllRecordData, Soa, A}; +use crate::rdata::{Soa, ZoneRecordData, A}; use super::processor::XfrResponseProcessor; use super::types::{ @@ -140,7 +140,7 @@ fn axfr_response_with_only_soas_is_accepted() { let mut it = processor.process_answer(resp).unwrap(); // Verify the events emitted by the XFR processor. - assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); + assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer(_))))); assert!(it.next().is_none()); } @@ -179,7 +179,7 @@ fn axfr_multi_response_with_only_soas_is_accepted() { let mut it = processor.process_answer(resp).unwrap(); // Verify the events emitted by the XFR processor. - assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); + assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer(_))))); assert!(it.next().is_none()); } @@ -210,7 +210,7 @@ fn axfr_response_generates_expected_events() { let s = serial; assert!(matches!(it.next(), Some(Ok(XE::AddRecord(n, _))) if n == s)); assert!(matches!(it.next(), Some(Ok(XE::AddRecord(n, _))) if n == s)); - assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); + assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer(_))))); assert!(it.next().is_none()); } @@ -243,7 +243,7 @@ fn ixfr_response_generates_expected_events() { // (which matches that of the client) followed by records to be // deleted as they were in that version of the zone but are not in the // new version of the zone. - add_answer_record(&req, &mut answer, old_soa); + add_answer_record(&req, &mut answer, old_soa.clone()); add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); // SOA of the servers` new zone version (which is ahead of that of the @@ -253,23 +253,40 @@ fn ixfr_response_generates_expected_events() { add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); // Closing SOA with servers current SOA - add_answer_record(&req, &mut answer, new_soa); + add_answer_record(&req, &mut answer, new_soa.clone()); let resp = answer.into_message(); // Process the response. let it = processor.process_answer(resp).unwrap(); + // Make parsed versions of the old and new SOAs. + let mut buf = BytesMut::new(); + new_soa.compose_rdata(&mut buf).unwrap(); + let buf = buf.freeze(); + let mut parser = Parser::from_ref(&buf); + let expected_new_soa = Soa::parse(&mut parser).unwrap(); + + let mut buf = BytesMut::new(); + old_soa.compose_rdata(&mut buf).unwrap(); + let buf = buf.freeze(); + let mut parser = Parser::from_ref(&buf); + let expected_old_soa = Soa::parse(&mut parser).unwrap(); + // Verify the events emitted by the XFR processor. let owner = ParsedName::::from(Name::from_str("example.com").unwrap()); let expected_events: [Result, IterationError>; 7] = [ - Ok(XfrEvent::BeginBatchDelete(old_serial)), + Ok(XfrEvent::BeginBatchDelete(Record::from(( + owner.clone(), + 0, + ZoneRecordData::Soa(expected_old_soa), + )))), Ok(XfrEvent::DeleteRecord( old_serial, Record::from(( owner.clone(), 0, - AllRecordData::A(A::new(Ipv4Addr::LOCALHOST)), + ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), )), )), Ok(XfrEvent::DeleteRecord( @@ -277,27 +294,35 @@ fn ixfr_response_generates_expected_events() { Record::from(( owner.clone(), 0, - AllRecordData::A(A::new(Ipv4Addr::BROADCAST)), + ZoneRecordData::A(A::new(Ipv4Addr::BROADCAST)), )), )), - Ok(XfrEvent::BeginBatchAdd(new_serial)), + Ok(XfrEvent::BeginBatchAdd(Record::from(( + owner.clone(), + 0, + ZoneRecordData::Soa(expected_new_soa.clone()), + )))), Ok(XfrEvent::AddRecord( new_serial, Record::from(( owner.clone(), 0, - AllRecordData::A(A::new(Ipv4Addr::BROADCAST)), + ZoneRecordData::A(A::new(Ipv4Addr::BROADCAST)), )), )), Ok(XfrEvent::AddRecord( new_serial, Record::from(( - owner, + owner.clone(), 0, - AllRecordData::A(A::new(Ipv4Addr::LOCALHOST)), + ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), )), )), - Ok(XfrEvent::EndOfTransfer), + Ok(XfrEvent::EndOfTransfer(Record::from(( + owner.clone(), + 0, + ZoneRecordData::Soa(expected_new_soa), + )))), ]; assert!(it.eq(expected_events)); @@ -365,7 +390,7 @@ fn multi_ixfr_response_generates_expected_events() { assert!(matches!(it.next(), Some(Ok(XE::BeginBatchAdd(_))))); assert!(matches!(it.next(), Some(Ok(XE::AddRecord(..))))); assert!(matches!(it.next(), Some(Ok(XE::AddRecord(..))))); - assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer)))); + assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer(_))))); assert!(it.next().is_none()); } @@ -432,4 +457,4 @@ fn mk_soa(serial: Serial) -> Soa> { let rname = Name::from_str("rname").unwrap(); let ttl = Ttl::from_secs(0); Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl) -} +} \ No newline at end of file diff --git a/src/net/xfr/processing/types.rs b/src/net/xfr/processing/types.rs index 4f4e46886..e24d0653d 100644 --- a/src/net/xfr/processing/types.rs +++ b/src/net/xfr/processing/types.rs @@ -6,14 +6,14 @@ use bytes::Bytes; use crate::{ base::{wire::ParseError, ParsedName, Record, Rtype, Serial}, - rdata::AllRecordData, + rdata::ZoneRecordData, }; /// The type of record processed by [`XfrResponseProcessor`]. /// /// [`XfrResponseProcessor`]: super::processor::XfrResponseProcessor pub type XfrRecord = - Record, AllRecordData>>; + Record, ZoneRecordData>>; //------------ XfrType -------------------------------------------------------- @@ -51,7 +51,7 @@ impl TryFrom for XfrType { /// An event emitted by [`XfrResponseProcessor`] during transfer processing. /// /// [`XfrResponseProcessor`]: super::processor::XfrResponseProcessor -#[derive(Copy, Clone, Debug, PartialEq, Eq)] +#[derive(Clone, Debug, PartialEq, Eq)] pub enum XfrEvent { /// Delete record R in zone serial S. /// @@ -75,19 +75,19 @@ pub enum XfrEvent { /// /// The transfer signalled that zero or more record deletions will follow, /// all for the zone version with the given serial number. - BeginBatchDelete(Serial), + BeginBatchDelete(R), /// Prepare to add records in zone serial S. /// /// The transfer signalled that zero or more record additions will follow, /// all for the zone version with the given serial number. - BeginBatchAdd(Serial), + BeginBatchAdd(R), /// Transfer completed successfully. /// /// Note: This event is not emitted until the final record of the final /// response in a set of one or more transfer responss has been seen. - EndOfTransfer, + EndOfTransfer(R), /// Transfer processing failed. /// @@ -105,7 +105,7 @@ impl std::fmt::Display for XfrEvent { XfrEvent::AddRecord(_, _) => f.write_str("AddRecord"), XfrEvent::BeginBatchDelete(_) => f.write_str("BeginBatchDelete"), XfrEvent::BeginBatchAdd(_) => f.write_str("BeginBatchAdd"), - XfrEvent::EndOfTransfer => f.write_str("EndOfTransfer"), + XfrEvent::EndOfTransfer(_) => f.write_str("EndOfTransfer"), XfrEvent::ProcessingFailed => f.write_str("ProcessingFailed"), } } @@ -153,6 +153,22 @@ pub enum ProcessingError { Malformed, } +impl std::fmt::Display for ProcessingError { + fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { + match self { + ProcessingError::ParseError(err) => { + f.write_fmt(format_args!("XFR response parsing error: {err}")) + } + ProcessingError::NotValidXfrResponse => { + f.write_str("Not a valid XFR response") + } + ProcessingError::Malformed => { + f.write_str("Malformed XFR response") + } + } + } +} + //------------ IterationError ------------------------------------------------- /// Errors that can occur during [`XfrEventIterator`]` iteration. @@ -162,4 +178,4 @@ pub enum ProcessingError { pub enum IterationError { /// Transfer processing failed. ParseError(ParseError), -} +} \ No newline at end of file From b664e5d61d6a8e71c215c1dbf72831c31feef4d2 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 10 Sep 2024 16:33:15 +0200 Subject: [PATCH 142/333] Updates backported from the `xfr` branch. --- src/zonetree/xfr_event_handler.rs | 208 +++++++++++++++++++----------- 1 file changed, 130 insertions(+), 78 deletions(-) diff --git a/src/zonetree/xfr_event_handler.rs b/src/zonetree/xfr_event_handler.rs index 47e5f814f..2f9be9f69 100644 --- a/src/zonetree/xfr_event_handler.rs +++ b/src/zonetree/xfr_event_handler.rs @@ -8,7 +8,7 @@ use tracing::{error, trace}; use super::error::OutOfZone; use super::{WritableZone, WritableZoneNode, Zone}; use crate::base::name::{FlattenInto, Label, ToLabelIter}; -use crate::base::{Name, Rtype, ToName}; +use crate::base::{Name, ParsedName, Record, Rtype, ToName}; use crate::net::xfr::processing::{XfrEvent, XfrRecord}; use crate::rdata::ZoneRecordData; use crate::zonetree::{Rrset, SharedRrset}; @@ -37,14 +37,6 @@ impl ZoneUpdateEventHandler { }) } - async fn init_batch(&mut self) -> Result<(), ()> { - if self.batching { - self.write = WriteState::new(&self.zone).await.map_err(|_| ())?; - } - - Ok(()) - } - fn mk_relative_name_iterator<'l>( apex_name: &Name, qname: &'l impl ToName, @@ -109,88 +101,39 @@ impl ZoneUpdateEventHandler { &mut self, evt: XfrEvent, ) -> Result<(), ()> { + trace!("Event: {evt}"); match evt { XfrEvent::DeleteRecord(_serial, rec) => { - let (rtype, data, end_node, mut rrset) = - self.prep_add_del(rec).await?; - - let writable = self.write.writable.as_ref().unwrap(); - - trace!("Deleting RR for {rtype}"); - - let node = end_node.as_ref().unwrap_or(writable); - - if let Some(existing_rrset) = - node.get_rrset(rtype).await.map_err(|_| ())? - { - for existing_data in existing_rrset.data() { - if existing_data != &data { - rrset.push_data(existing_data.clone()); - } - } - } - - trace!("Removing single RR of {rtype} so updating RRSET"); - node.update_rrset(SharedRrset::new(rrset)) - .await - .map_err(|_| ())?; + self.delete_record(rec).await? } - XfrEvent::AddRecord(_serial, rec) => { - self.init_batch().await?; - - if !self.first_event_seen && rec.rtype() == Rtype::SOA { - // If the first event is the addition of a SOA record to - // the zone, this must be a complete replacement of the - // zone (as you can't have two SOA records), i.e. - // something like an AXFR transfer. We can't add records - // from a new version of the zone to an existing zone - // because if the old version contained a record which the - // new version does not, it would get left behind. So in - // this case we have to mark all of the existing records - // in the zone as "removed" and then add new records. This - // allows the old records to continue being served to - // current consumers while the zone is being updated. - self.write.remove_all().await.map_err(|_| ())?; - } - - let (rtype, data, end_node, mut rrset) = - self.prep_add_del(rec).await?; - - let writable = self.write.writable.as_ref().unwrap(); - - trace!("Adding RR: {:?}", rrset); - rrset.push_data(data); - - let node = end_node.as_ref().unwrap_or(writable); - - if let Some(existing_rrset) = - node.get_rrset(rtype).await.map_err(|_| ())? - { - for existing_data in existing_rrset.data() { - rrset.push_data(existing_data.clone()); - } - } - - node.update_rrset(SharedRrset::new(rrset)) - .await - .map_err(|_| ())?; - } + XfrEvent::AddRecord(_serial, rec) => self.add_record(rec).await?, - XfrEvent::BeginBatchDelete(_) => { + // Note: Batches first contain deletions then additions, so batch + // deletion signals the start of a batch, and the end of any + // previous batch addition. + XfrEvent::BeginBatchDelete(_old_soa) => { if self.batching { // Commit the previous batch. self.write.commit().await?; + // Open a writer for the new batch. + self.write.reopen().await.map_err(|_| ())?; } self.batching = true; } - XfrEvent::BeginBatchAdd(_) => { + XfrEvent::BeginBatchAdd(new_soa) => { + // Update the SOA record. + self.update_soa(new_soa).await?; self.batching = true; } - XfrEvent::EndOfTransfer => { + XfrEvent::EndOfTransfer(zone_soa) => { + if !self.batching { + // Update the SOA record. + self.update_soa(zone_soa).await?; + } // Commit the previous batch. self.write.commit().await?; } @@ -204,6 +147,110 @@ impl ZoneUpdateEventHandler { Ok(()) } + + async fn update_soa( + &mut self, + new_soa: Record< + ParsedName, + ZoneRecordData>, + >, + ) -> Result<(), ()> { + if new_soa.rtype() != Rtype::SOA { + return Err(()); + } + + let mut rrset = Rrset::new(Rtype::SOA, new_soa.ttl()); + rrset.push_data(new_soa.data().to_owned().flatten_into()); + self.write + .writable + .as_ref() + .unwrap() + .update_rrset(SharedRrset::new(rrset)) + .await + .map_err(|_| ())?; + Ok(()) + } + + async fn delete_record( + &mut self, + rec: Record< + ParsedName, + ZoneRecordData>, + >, + ) -> Result<(), ()> { + let (rtype, data, end_node, mut rrset) = + self.prep_add_del(rec).await?; + + let writable = self.write.writable.as_ref().unwrap(); + + trace!("Deleting RR for {rtype}"); + + let node = end_node.as_ref().unwrap_or(writable); + + if let Some(existing_rrset) = + node.get_rrset(rtype).await.map_err(|_| ())? + { + for existing_data in existing_rrset.data() { + if existing_data != &data { + rrset.push_data(existing_data.clone()); + } + } + } + + trace!("Removing single RR of {rtype} so updating RRSET"); + + node.update_rrset(SharedRrset::new(rrset)) + .await + .map_err(|_| ())?; + + Ok(()) + } + + async fn add_record( + &mut self, + rec: Record< + ParsedName, + ZoneRecordData>, + >, + ) -> Result<(), ()> { + if !self.first_event_seen && rec.rtype() == Rtype::SOA { + // If the first event is the addition of a SOA record to the zone, + // this must be a complete replacement of the zone (as you can't + // have two SOA records), i.e. something like an AXFR transfer. We + // can't add records from a new version of the zone to an existing + // zone because if the old version contained a record which the + // new version does not, it would get left behind. So in this case + // we have to mark all of the existing records in the zone as + // "removed" and then add new records. This allows the old records + // to continue being served to current consumers while the zone is + // being updated. + self.write.remove_all().await.map_err(|_| ())?; + } + + let (rtype, data, end_node, mut rrset) = + self.prep_add_del(rec).await?; + + let writable = self.write.writable.as_ref().unwrap(); + + trace!("Adding RR: {:?}", rrset); + rrset.push_data(data); + + let node = end_node.as_ref().unwrap_or(writable); + + if let Some(existing_rrset) = + node.get_rrset(rtype).await.map_err(|_| ())? + { + for existing_data in existing_rrset.data() { + rrset.push_data(existing_data.clone()); + } + } + + node.update_rrset(SharedRrset::new(rrset)) + .await + .map_err(|_| ())?; + + Ok(()) + } } //------------ WriteState ----------------------------------------------------- @@ -239,6 +286,11 @@ impl WriteState { Ok(()) } + + async fn reopen(&mut self) -> std::io::Result<()> { + self.writable = Some(self.write.open(true).await?); + Ok(()) + } } #[cfg(test)] @@ -281,12 +333,12 @@ mod tests { ); evt_handler - .handle_event(XfrEvent::AddRecord(s, soa)) + .handle_event(XfrEvent::AddRecord(s, soa.clone())) .await .unwrap(); evt_handler - .handle_event(XfrEvent::EndOfTransfer) + .handle_event(XfrEvent::EndOfTransfer(soa)) .await .unwrap(); } @@ -381,4 +433,4 @@ mod tests { .push((qname, qclass, Ttl::from_secs(0), item)) .unwrap(); } -} +} \ No newline at end of file From c58e177a02763944c8fe0b311f9d320a306282d6 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 10 Sep 2024 16:47:40 +0200 Subject: [PATCH 143/333] Cargo fmt. --- src/net/xfr/processing/processor.rs | 4 ++-- src/net/xfr/processing/tests.rs | 2 +- src/net/xfr/processing/types.rs | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/net/xfr/processing/processor.rs b/src/net/xfr/processing/processor.rs index 2120ef1d7..7df7a9303 100644 --- a/src/net/xfr/processing/processor.rs +++ b/src/net/xfr/processing/processor.rs @@ -59,7 +59,7 @@ impl XfrResponseProcessor { /// If the returned iterator does not emit an [`XfrEvent::EndOfTransfer`] /// event, call this function with the next outstanding response message /// to continue iterating over the incomplete transfer. - /// + /// /// Checking that the given response corresponds by ID to the related /// original XFR query or that the question section of the response, if /// present (RFC 5936 allows it to be empty for subsequent AXFR responses) @@ -378,4 +378,4 @@ impl RecordProcessor { } } } -} \ No newline at end of file +} diff --git a/src/net/xfr/processing/tests.rs b/src/net/xfr/processing/tests.rs index c309ec1fd..eaf109933 100644 --- a/src/net/xfr/processing/tests.rs +++ b/src/net/xfr/processing/tests.rs @@ -457,4 +457,4 @@ fn mk_soa(serial: Serial) -> Soa> { let rname = Name::from_str("rname").unwrap(); let ttl = Ttl::from_secs(0); Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl) -} \ No newline at end of file +} diff --git a/src/net/xfr/processing/types.rs b/src/net/xfr/processing/types.rs index e24d0653d..f460589c4 100644 --- a/src/net/xfr/processing/types.rs +++ b/src/net/xfr/processing/types.rs @@ -178,4 +178,4 @@ impl std::fmt::Display for ProcessingError { pub enum IterationError { /// Transfer processing failed. ParseError(ParseError), -} \ No newline at end of file +} From cabedf70d13ff1b33b13eb1fc9d9b36bbacfef69 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 10 Sep 2024 16:48:22 +0200 Subject: [PATCH 144/333] Cargo fmt. --- src/zonetree/xfr_event_handler.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/zonetree/xfr_event_handler.rs b/src/zonetree/xfr_event_handler.rs index 2f9be9f69..771759dc1 100644 --- a/src/zonetree/xfr_event_handler.rs +++ b/src/zonetree/xfr_event_handler.rs @@ -433,4 +433,4 @@ mod tests { .push((qname, qclass, Ttl::from_secs(0), item)) .unwrap(); } -} \ No newline at end of file +} From 76c2b438d97e6ec43d71025b4b65a93ce0d64dee Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 10 Sep 2024 18:30:57 +0200 Subject: [PATCH 145/333] - Merge TsigClient and TsigClientMulti. - Ensure .done() is called. --- src/net/client/tsig.rs | 108 ++++++++++++++++++++++++----------------- 1 file changed, 64 insertions(+), 44 deletions(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index 6ae2acb70..202fa2632 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -35,7 +35,8 @@ use std::sync::Arc; use std::vec::Vec; use bytes::Bytes; -use tracing::{debug, trace, warn}; +use octseq::Octets; +use tracing::trace; use crate::base::message::CopyRecordsError; use crate::base::message_builder::AdditionalBuilder; @@ -54,18 +55,42 @@ use crate::tsig::{ClientSequence, ClientTransaction, Key}; enum TsigClient { /// TODO Transaction(ClientTransaction), - // TODO - //Sequence(ClientSequence), + /// TODO + Sequence(ClientSequence), } -/// TODO -#[derive(Clone, Debug)] -enum TsigClientMulti { +impl TsigClient +where + K: AsRef, +{ /// TODO - //Transaction(ClientTransaction), + pub fn answer( + &mut self, + message: &mut Message, + now: Time48, + ) -> Result<(), Error> + where + Octs: Octets + AsMut<[u8]> + ?Sized, + { + match self { + TsigClient::Transaction(client) => client.answer(message, now), + TsigClient::Sequence(client) => client.answer(message, now), + } + .map_err(Error::Authentication) + } /// TODO - Sequence(ClientSequence), + fn done(self) -> Result<(), Error> { + match self { + TsigClient::Transaction(_) => { + // Nothing to do. + Ok(()) + } + TsigClient::Sequence(client) => { + client.done().map_err(Error::Authentication) + } + } + } } //------------ Connection ----------------------------------------------------- @@ -364,14 +389,20 @@ where RequestStateMulti::GetResponse( ref mut _request, - _tsig_client, + tsig_client, ) => { + let client = tsig_client + .lock() + .unwrap() + .take() + .unwrap(); + client.done()?; self.state = RequestStateMulti::Complete; } RequestStateMulti::Complete => { - debug!("Ignoring attempt to complete TSIG stream that is already complete."); + panic!("Cannot complete an already completed TSIG stream."); } } break Ok(None); @@ -381,25 +412,14 @@ where msg.as_slice().to_vec(), )?; - let mut locked = tsig_client.lock().unwrap(); - match locked.deref_mut() { - Some(TsigClientMulti::Sequence(client)) => { - trace!( - "Validating TSIG for sequence reply" - ); - client - .answer( - &mut modifiable_msg, - Time48::now(), - ) - .map_err(|err| { - Error::Authentication(err) - })?; - } - - _ => { - trace!("Response is not signed, nothing to do"); - } + if let Some(client) = + tsig_client.lock().unwrap().deref_mut() + { + trace!("Validating TSIG for sequence reply"); + client.answer( + &mut modifiable_msg, + Time48::now(), + )?; } let out_vec = modifiable_msg.into_octets(); @@ -457,6 +477,7 @@ where } //------------ RequestState --------------------------------------------------- + /// States of the state machine in get_response_impl enum RequestState { /// Initial state, perform a cache lookup. @@ -470,6 +491,7 @@ enum RequestState { } //------------ RequestStateMulti ---------------------------------------------- + /// States of the state machine in get_response_impl enum RequestStateMulti { /// Initial state, perform a cache lookup. @@ -478,7 +500,7 @@ enum RequestStateMulti { /// Wait for a response and insert the response in the cache. GetResponse( Box, - Arc>>>, + Arc>>>, ), /// TODO @@ -605,7 +627,7 @@ where key: Option, /// TODO - signer: Arc>>>, + signer: Arc>>>, } impl AuthenticatedRequestMessageMulti @@ -644,20 +666,18 @@ where let mut target = self.request.append_message(target)?; if let Some(key) = &self.key { - let client = { - trace!( - "Signing streaming request sequence with key '{}'", - key.as_ref().name() - ); - TsigClientMulti::Sequence( - ClientSequence::request( - key.clone(), - &mut target, - Time48::now(), - ) - .unwrap(), + trace!( + "Signing streaming request sequence with key '{}'", + key.as_ref().name() + ); + let client = TsigClient::Sequence( + ClientSequence::request( + key.clone(), + &mut target, + Time48::now(), ) - }; + .unwrap(), + ); *self.signer.lock().unwrap() = Some(client); } else { From ebe0d2f12a1b60e31c8715f1d325b51529af128c Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 10 Sep 2024 18:31:19 +0200 Subject: [PATCH 146/333] Remove left behind trace logging. --- src/net/client/tsig.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index 202fa2632..09a7f445f 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -662,7 +662,6 @@ where &self, target: Target, ) -> Result, CopyRecordsError> { - trace!("append_message()"); let mut target = self.request.append_message(target)?; if let Some(key) = &self.key { From b309a13907ab170042e3d588ad888fae2b1b3147 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 10 Sep 2024 22:44:07 +0200 Subject: [PATCH 147/333] Remove AuthenticatedRequestMessageMulti, it isn't needed. --- examples/client-transports.rs | 8 +++----- src/net/client/tsig.rs | 38 +++++++++-------------------------- 2 files changed, 12 insertions(+), 34 deletions(-) diff --git a/examples/client-transports.rs b/examples/client-transports.rs index fc4c6bb50..04f4d2b0b 100644 --- a/examples/client-transports.rs +++ b/examples/client-transports.rs @@ -22,9 +22,7 @@ use domain::net::client::stream; #[cfg(feature = "tsig")] use domain::net::client::request::SendRequestMulti; #[cfg(feature = "tsig")] -use domain::net::client::tsig::{ - self, AuthenticatedRequestMessage, AuthenticatedRequestMessageMulti, -}; +use domain::net::client::tsig::{self, AuthenticatedRequestMessage}; #[cfg(feature = "tsig")] use domain::tsig::{Algorithm, Key, KeyName}; @@ -273,7 +271,7 @@ async fn main() { RequestMessage>, Arc, >, - AuthenticatedRequestMessageMulti< + AuthenticatedRequestMessage< RequestMessageMulti>, Arc, >, @@ -342,7 +340,7 @@ where + domain::dep::octseq::Octets + 'static, SR: SendRequestMulti< - tsig::AuthenticatedRequestMessageMulti< + tsig::AuthenticatedRequestMessage< RequestMessageMulti, Arc, >, diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index 09a7f445f..ea01277c5 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -146,7 +146,7 @@ where impl SendRequestMulti for Connection where CR: ComposeRequestMulti + 'static, - Upstream: SendRequestMulti> + Upstream: SendRequestMulti> + Send + Sync + 'static, @@ -330,9 +330,8 @@ where impl RequestMulti where CR: ComposeRequestMulti, - Upstream: SendRequestMulti> - + Send - + Sync, + Upstream: + SendRequestMulti> + Send + Sync, K: Clone + AsRef, Self: GetResponseMulti, { @@ -357,7 +356,7 @@ where RequestStateMulti::Init => { let tsig_client = Arc::new(std::sync::Mutex::new(None)); - let msg = AuthenticatedRequestMessageMulti { + let msg = AuthenticatedRequestMessage { request: self.request_msg.take().unwrap(), key: self.key.clone(), signer: tsig_client.clone(), @@ -447,7 +446,7 @@ where impl Debug for RequestMulti where CR: ComposeRequestMulti, - Upstream: SendRequestMulti>, + Upstream: SendRequestMulti>, { fn fmt(&self, f: &mut Formatter<'_>) -> Result<(), core::fmt::Error> { f.debug_struct("Request").finish() @@ -457,9 +456,8 @@ where impl GetResponseMulti for RequestMulti where CR: ComposeRequestMulti, - Upstream: SendRequestMulti> - + Send - + Sync, + Upstream: + SendRequestMulti> + Send + Sync, K: Clone + AsRef + Send + Sync, { fn get_response( @@ -612,25 +610,7 @@ where } } -//------------ AuthenticatedRequestMessageMulti ------------------------------- - -/// TODO -#[derive(Debug)] -pub struct AuthenticatedRequestMessageMulti -where - CR: Send + Sync, -{ - /// TODO - request: CR, - - /// TODO - key: Option, - - /// TODO - signer: Arc>>>, -} - -impl AuthenticatedRequestMessageMulti +impl AuthenticatedRequestMessage where CR: ComposeRequestMulti + Send + Sync, K: Clone + Debug + Send + Sync + AsRef, @@ -652,7 +632,7 @@ where } } -impl ComposeRequestMulti for AuthenticatedRequestMessageMulti +impl ComposeRequestMulti for AuthenticatedRequestMessage where CR: ComposeRequestMulti, K: Clone + Debug + Send + Sync + AsRef, From 6886079ff1a306b3df2f520098449f988963c635 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 10 Sep 2024 23:59:20 +0200 Subject: [PATCH 148/333] Fold RequestMulti into Request and RequestStateMulti into RequestState. Split common functionality out into handle_result() and relax bounds to make existing code usable in both single and multi contexts. --- src/net/client/tsig.rs | 363 ++++++++++++++++------------------------- 1 file changed, 139 insertions(+), 224 deletions(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index ea01277c5..642999ad6 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -156,7 +156,7 @@ where &self, request_msg: CR, ) -> Box { - Box::new(RequestMulti::::new( + Box::new(Request::::new_multi( request_msg, self.key.clone(), self.upstream.clone(), @@ -164,12 +164,56 @@ where } } +//------------ UpstreamSender ------------------------------------------------- + +/// TODO +type UpstreamSender = fn( + &Upstream, + AuthenticatedRequestMessage, + Arc>>>, +) -> RequestState; + +/// TODO +fn upstream_sender( + upstream: &Upstream, + msg: AuthenticatedRequestMessage, + tsig_client: Arc>>>, +) -> RequestState +where + CR: ComposeRequest, + Upstream: SendRequest> + Send + Sync, +{ + RequestState::GetResponse(upstream.send_request(msg), tsig_client) +} + +/// TODO +fn upstream_sender_multi( + upstream: &Upstream, + msg: AuthenticatedRequestMessage, + tsig_client: Arc>>>, +) -> RequestState +where + CR: ComposeRequestMulti, + Upstream: + SendRequestMulti> + Send + Sync, +{ + RequestState::GetResponseMulti(upstream.send_request(msg), tsig_client) +} + +//------------ HandleResponseResult ------------------------------------------- + +/// TODO +enum HandleResponseResult { + /// TODO + Response(Message), + /// TODO + Complete, +} + //------------ Request -------------------------------------------------------- /// The state of a request that is executed. pub struct Request -where - CR: ComposeRequest, { /// State of the request. state: RequestState, @@ -200,88 +244,9 @@ where upstream, } } - - /// This is the implementation of the get_response method. - /// - /// This function is cancel safe. - async fn get_response_impl(&mut self) -> Result, Error> { - let res = loop { - match &mut self.state { - RequestState::Init => { - let tsig_client = Arc::new(std::sync::Mutex::new(None)); - - // TODO: TSIG sign the request, and send the signed version - // upstream. - let msg = AuthenticatedRequestMessage { - request: self.request_msg.take().unwrap(), - key: self.key.clone(), - signer: tsig_client.clone(), - }; - - trace!("Sending request upstream..."); - let request = self.upstream.send_request(msg); - self.state = - RequestState::GetResponse(request, tsig_client); - continue; - } - - RequestState::GetResponse(request, tsig_client) => { - trace!("Receiving response"); - let response = request.get_response().await; - if self.key.is_some() { - assert!(tsig_client.lock().unwrap().is_some()); - } - - // TSIG validation - match response { - Ok(msg) => { - let mut modifiable_msg = Message::from_octets( - msg.as_slice().to_vec(), - )?; - - let mut locked = tsig_client.lock().unwrap(); - match locked.deref_mut() { - Some(TsigClient::Transaction(client)) => { - trace!( - "Validating TSIG for single reply" - ); - client - .answer( - &mut modifiable_msg, - Time48::now(), - ) - .map_err(|err| { - Error::Authentication(err) - })?; - } - - _ => { - trace!("Response is not signed, nothing to do"); - } - } - - let out_vec = modifiable_msg.into_octets(); - let out_bytes = Bytes::from(out_vec); - let out_msg = - Message::::from_octets(out_bytes)?; - break Ok(out_msg); - } - - Err(err) => break Err(err), - } - } - } - }; - - res - } } -impl Debug for Request -where - CR: ComposeRequest, - Upstream: SendRequest>, -{ +impl Debug for Request { fn fmt(&self, f: &mut Formatter<'_>) -> Result<(), core::fmt::Error> { f.debug_struct("Request").finish() } @@ -303,42 +268,27 @@ where + '_, >, > { - Box::pin(self.get_response_impl()) + Box::pin(async move { + self.get_response_impl(upstream_sender) + .await + .map(|v| v.unwrap()) + }) } } -//------------ RequestMulti --------------------------------------------------- - -/// The state of a request that is executed. -pub struct RequestMulti -where - CR: ComposeRequestMulti, -{ - /// State of the request. - state: RequestStateMulti, - - /// The request message. - request_msg: Option, - - /// TODO - key: Option, - - /// The upstream transport of the connection. - upstream: Arc, -} - -impl RequestMulti +impl Request where - CR: ComposeRequestMulti, - Upstream: - SendRequestMulti> + Send + Sync, + CR: Sync + Send, K: Clone + AsRef, - Self: GetResponseMulti, { /// Create a new Request object. - fn new(request_msg: CR, key: Option, upstream: Arc) -> Self { + fn new_multi( + request_msg: CR, + key: Option, + upstream: Arc, + ) -> Self { Self { - state: RequestStateMulti::Init, + state: RequestState::Init, request_msg: Some(request_msg), key, upstream, @@ -350,10 +300,11 @@ where /// This function is cancel safe. async fn get_response_impl( &mut self, + upstream_sender: UpstreamSender, ) -> Result>, Error> { - let res = loop { + let (response, tsig_client) = loop { match &mut self.state { - RequestStateMulti::Init => { + RequestState::Init => { let tsig_client = Arc::new(std::sync::Mutex::new(None)); let msg = AuthenticatedRequestMessage { @@ -363,97 +314,81 @@ where }; trace!("Sending request upstream..."); - let request = self.upstream.send_request(msg); self.state = - RequestStateMulti::GetResponse(request, tsig_client); + upstream_sender(&self.upstream, msg, tsig_client); continue; } - RequestStateMulti::GetResponse(request, tsig_client) => { + RequestState::GetResponse(request, tsig_client) => { + let response = request.get_response().await.map(Some); + break (response, tsig_client); + } + + RequestState::GetResponseMulti(request, tsig_client) => { let response = request.get_response().await; - if response.is_ok() && self.key.is_some() { - assert!(tsig_client.lock().unwrap().is_some()); - } - - // TSIG validation - match response { - Ok(msg) => { - let msg = match msg { - Some(msg) => msg, - None => { - match &mut self.state { - RequestStateMulti::Init => { - unreachable!() - } - - RequestStateMulti::GetResponse( - ref mut _request, - tsig_client, - ) => { - let client = tsig_client - .lock() - .unwrap() - .take() - .unwrap(); - client.done()?; - self.state = - RequestStateMulti::Complete; - } - - RequestStateMulti::Complete => { - panic!("Cannot complete an already completed TSIG stream."); - } - } - break Ok(None); - } - }; - let mut modifiable_msg = Message::from_octets( - msg.as_slice().to_vec(), - )?; - - if let Some(client) = - tsig_client.lock().unwrap().deref_mut() - { - trace!("Validating TSIG for sequence reply"); - client.answer( - &mut modifiable_msg, - Time48::now(), - )?; - } - - let out_vec = modifiable_msg.into_octets(); - let out_bytes = Bytes::from(out_vec); - let out_msg = - Message::::from_octets(out_bytes)?; - break Ok(Some(out_msg)); - } - - Err(err) => break Err(err), - } + break (response, tsig_client); } - RequestStateMulti::Complete => { - break Err(Error::StreamReceiveError); + RequestState::Complete => { + return Err(Error::StreamReceiveError); } } }; - trace!("Leaving"); - res + match Self::handle_response(&self.key, response, tsig_client) { + Ok(HandleResponseResult::Complete) => { + self.state = RequestState::Complete; + Ok(None) + } + + Ok(HandleResponseResult::Response(response)) => { + Ok(Some(response)) + } + + Err(err) => Err(err), + } } -} -impl Debug for RequestMulti -where - CR: ComposeRequestMulti, - Upstream: SendRequestMulti>, -{ - fn fmt(&self, f: &mut Formatter<'_>) -> Result<(), core::fmt::Error> { - f.debug_struct("Request").finish() + /// TODO + fn handle_response( + key: &Option, + response: Result>, Error>, + tsig_client: &mut Arc>>>, + ) -> Result { + if response.is_ok() && key.is_some() { + assert!(tsig_client.lock().unwrap().is_some()); + } + + // TSIG validation + match response { + Ok(None) => { + let client = tsig_client.lock().unwrap().take().unwrap(); + client.done()?; + Ok(HandleResponseResult::Complete) + } + + Ok(Some(msg)) => { + let mut modifiable_msg = + Message::from_octets(msg.as_slice().to_vec())?; + + if let Some(client) = tsig_client.lock().unwrap().deref_mut() + { + trace!("Validating TSIG for sequence reply"); + client.answer(&mut modifiable_msg, Time48::now())?; + } + + let out_vec = modifiable_msg.into_octets(); + let out_bytes = Bytes::from(out_vec); + let out_msg = Message::::from_octets(out_bytes)?; + Ok(HandleResponseResult::Response(out_msg)) + } + + Err(err) => Err(err), + } } } -impl GetResponseMulti for RequestMulti +impl GetResponseMulti for Request where CR: ComposeRequestMulti, Upstream: @@ -470,7 +405,7 @@ where + '_, >, > { - Box::pin(self.get_response_impl()) + Box::pin(self.get_response_impl(upstream_sender_multi)) } } @@ -481,27 +416,19 @@ enum RequestState { /// Initial state, perform a cache lookup. Init, - /// Wait for a response and insert the response in the cache. + /// Wait for a response and verify it. GetResponse( Box, Arc>>>, ), -} - -//------------ RequestStateMulti ---------------------------------------------- -/// States of the state machine in get_response_impl -enum RequestStateMulti { - /// Initial state, perform a cache lookup. - Init, - - /// Wait for a response and insert the response in the cache. - GetResponse( + /// Wait for multiple responses and verify them. + GetResponseMulti( Box, Arc>>>, ), - /// TODO + /// The last response in a sequence was already received. Complete, } @@ -610,28 +537,6 @@ where } } -impl AuthenticatedRequestMessage -where - CR: ComposeRequestMulti + Send + Sync, - K: Clone + Debug + Send + Sync + AsRef, -{ - /// Create new message based on the changes to the base message. - fn to_message_impl(&self) -> Result>, Error> { - let target = StaticCompressor::new(Vec::new()); - - let target = self.append_message(target)?; - - // It would be nice to use .builder() here. But that one deletes all - // sections. We have to resort to .as_builder() which gives a - // reference and then .clone() - let result = target.as_builder().clone(); - let msg = Message::from_octets(result.finish().into_target()).expect( - "Message should be able to parse output from MessageBuilder", - ); - Ok(msg) - } -} - impl ComposeRequestMulti for AuthenticatedRequestMessage where CR: ComposeRequestMulti, @@ -667,7 +572,17 @@ where } fn to_message(&self) -> Result>, Error> { - self.to_message_impl() + let mut target = StaticCompressor::new(Vec::new()); + + self.append_message(&mut target)?; + + // It would be nice to use .builder() here. But that one deletes all + // sections. We have to resort to .as_builder() which gives a + // reference and then .clone() + let msg = Message::from_octets(target.into_target()).expect( + "Message should be able to parse output from MessageBuilder", + ); + Ok(msg) } fn header(&self) -> &crate::base::Header { From aec32d4b1ef773f8d36d3e43063e125595b7f7b9 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 11 Sep 2024 00:05:33 +0200 Subject: [PATCH 149/333] Cargo fmt. --- src/net/client/tsig.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index 642999ad6..ecf936bc7 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -213,8 +213,7 @@ enum HandleResponseResult { //------------ Request -------------------------------------------------------- /// The state of a request that is executed. -pub struct Request -{ +pub struct Request { /// State of the request. state: RequestState, From a033102d6488b649d515401b1ce6346b00d6fa2e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 11 Sep 2024 00:12:05 +0200 Subject: [PATCH 150/333] Small simplification. --- src/net/client/tsig.rs | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index ecf936bc7..e17cabc7c 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -334,18 +334,15 @@ where } }; - match Self::handle_response(&self.key, response, tsig_client) { - Ok(HandleResponseResult::Complete) => { - self.state = RequestState::Complete; - Ok(None) - } - - Ok(HandleResponseResult::Response(response)) => { - Ok(Some(response)) + Self::handle_response(&self.key, response, tsig_client).map(|res| { + match res { + HandleResponseResult::Complete => { + self.state = RequestState::Complete; + None + } + HandleResponseResult::Response(res) => Some(res), } - - Err(err) => Err(err), - } + }) } /// TODO From ea5dfda5c3e2bde4c786f264c7865f414f97fb6b Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 11 Sep 2024 00:16:40 +0200 Subject: [PATCH 151/333] Review feedback: Key should not be optional. --- examples/client-transports.rs | 2 +- src/net/client/tsig.rs | 95 ++++++++++++++--------------------- 2 files changed, 39 insertions(+), 58 deletions(-) diff --git a/examples/client-transports.rs b/examples/client-transports.rs index 04f4d2b0b..95359eca5 100644 --- a/examples/client-transports.rs +++ b/examples/client-transports.rs @@ -361,7 +361,7 @@ where // Create a signing transport. This assumes that the server being // connected to is configured with a key with the same name, algorithm and // secret and to allow that key to be used for the request we are making. - let tsig_conn = tsig::Connection::new(Some(key), conn); + let tsig_conn = tsig::Connection::new(key, conn); // Send a query message. let mut request = tsig_conn.send_request(req); diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index e17cabc7c..438c1a73f 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -105,12 +105,12 @@ pub struct Connection { upstream: Arc, /// TODO - key: Option, + key: K, } impl Connection { /// TODO - pub fn new(key: Option, upstream: Upstream) -> Self { + pub fn new(key: K, upstream: Upstream) -> Self { Self { upstream: Arc::new(upstream), key, @@ -221,7 +221,7 @@ pub struct Request { request_msg: Option, /// TODO - key: Option, + key: K, /// The upstream transport of the connection. upstream: Arc, @@ -235,7 +235,7 @@ where Self: GetResponse, { /// Create a new Request object. - fn new(request_msg: CR, key: Option, upstream: Arc) -> Self { + fn new(request_msg: CR, key: K, upstream: Arc) -> Self { Self { state: RequestState::Init, request_msg: Some(request_msg), @@ -281,11 +281,7 @@ where K: Clone + AsRef, { /// Create a new Request object. - fn new_multi( - request_msg: CR, - key: Option, - upstream: Arc, - ) -> Self { + fn new_multi(request_msg: CR, key: K, upstream: Arc) -> Self { Self { state: RequestState::Init, request_msg: Some(request_msg), @@ -334,27 +330,20 @@ where } }; - Self::handle_response(&self.key, response, tsig_client).map(|res| { - match res { - HandleResponseResult::Complete => { - self.state = RequestState::Complete; - None - } - HandleResponseResult::Response(res) => Some(res), + Self::handle_response(response, tsig_client).map(|res| match res { + HandleResponseResult::Complete => { + self.state = RequestState::Complete; + None } + HandleResponseResult::Response(res) => Some(res), }) } /// TODO fn handle_response( - key: &Option, response: Result>, Error>, tsig_client: &mut Arc>>>, ) -> Result { - if response.is_ok() && key.is_some() { - assert!(tsig_client.lock().unwrap().is_some()); - } - // TSIG validation match response { Ok(None) => { @@ -440,7 +429,7 @@ where request: CR, /// TODO - key: Option, + key: K, /// TODO signer: Arc>>>, @@ -458,26 +447,22 @@ where ) -> Result, CopyRecordsError> { let mut target = self.request.append_message(target)?; - if let Some(key) = &self.key { - let client = { - trace!( - "Signing single request transaction with key '{}'", - key.as_ref().name() - ); - TsigClient::Transaction( - ClientTransaction::request( - key.clone(), - &mut target, - Time48::now(), - ) - .unwrap(), + let client = { + trace!( + "Signing single request transaction with key '{}'", + self.key.as_ref().name() + ); + TsigClient::Transaction( + ClientTransaction::request( + self.key.clone(), + &mut target, + Time48::now(), ) - }; + .unwrap(), + ) + }; - *self.signer.lock().unwrap() = Some(client); - } else { - trace!("No signing key was configured for this request, nothing to do"); - } + *self.signer.lock().unwrap() = Some(client); Ok(target) } @@ -545,24 +530,20 @@ where ) -> Result, CopyRecordsError> { let mut target = self.request.append_message(target)?; - if let Some(key) = &self.key { - trace!( - "Signing streaming request sequence with key '{}'", - key.as_ref().name() - ); - let client = TsigClient::Sequence( - ClientSequence::request( - key.clone(), - &mut target, - Time48::now(), - ) - .unwrap(), - ); + trace!( + "Signing streaming request sequence with key '{}'", + self.key.as_ref().name() + ); + let client = TsigClient::Sequence( + ClientSequence::request( + self.key.clone(), + &mut target, + Time48::now(), + ) + .unwrap(), + ); - *self.signer.lock().unwrap() = Some(client); - } else { - trace!("No signing key was configured for this request, nothing to do"); - } + *self.signer.lock().unwrap() = Some(client); Ok(target) } From 59cc56d608264c36b626393c932a8fe47f2938b3 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 11 Sep 2024 00:42:21 +0200 Subject: [PATCH 152/333] Merge branch 'tsig-client' into tsig-middleware --- src/net/server/tests/integration.rs | 94 ++++++++++++++++++++--------- 1 file changed, 65 insertions(+), 29 deletions(-) diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index 582e3f1d0..85acf95aa 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -21,9 +21,7 @@ use crate::base::net::IpAddr; use crate::base::wire::Composer; use crate::base::Rtype; use crate::net::client::request::{RequestMessage, RequestMessageMulti}; -use crate::net::client::tsig::{ - AuthenticatedRequestMessage, AuthenticatedRequestMessageMulti, -}; +use crate::net::client::tsig::AuthenticatedRequestMessage; use crate::net::client::{dgram, stream, tsig}; use crate::net::server; use crate::net::server::buf::VecBufSource; @@ -228,28 +226,53 @@ fn mk_client_factory( let tcp_key_store = key_store.clone(); let tcp_client_factory = PerClientAddressClientFactory::new( move |source_addr, entry| { + let stream = stream_server_conn + .connect(Some(SocketAddr::new(*source_addr, 0))); + let key = entry.key_name.as_ref().and_then(|key_name| { tcp_key_store.get_key(&key_name, Algorithm::Sha256) }); - let stream = stream_server_conn - .connect(Some(SocketAddr::new(*source_addr, 0))); - let (conn, transport) = stream::Connection::< - AuthenticatedRequestMessage>, Key>, - AuthenticatedRequestMessageMulti< + + if let Some(key) = key { + let (conn, transport) = stream::Connection::< + AuthenticatedRequestMessage>, Key>, + AuthenticatedRequestMessage< + RequestMessageMulti>, + Key, + >, + >::new(stream); + + tokio::spawn(transport.run()); + + let conn = Box::new(tsig::Connection::new(key, conn)); + + if let Some(sections) = &entry.sections { + if let Some(q) = sections.question.first() { + if matches!(q.qtype(), Rtype::AXFR | Rtype::IXFR) { + return Client::Multi(conn); + } + } + } + Client::Single(conn) + } else { + let (conn, transport) = stream::Connection::< + RequestMessage>, RequestMessageMulti>, - Key, - >, - >::new(stream); - tokio::spawn(transport.run()); - let conn = Box::new(tsig::Connection::new(key, conn)); - if let Some(sections) = &entry.sections { - if let Some(q) = sections.question.first() { - if matches!(q.qtype(), Rtype::AXFR | Rtype::IXFR) { - return Client::Multi(conn); + >::new(stream); + + tokio::spawn(transport.run()); + + let conn = Box::new(conn); + + if let Some(sections) = &entry.sections { + if let Some(q) = sections.question.first() { + if matches!(q.qtype(), Rtype::AXFR | Rtype::IXFR) { + return Client::Multi(conn); + } } } + Client::Single(conn) } - Client::Single(conn) }, only_for_tcp_queries, ); @@ -261,24 +284,37 @@ fn mk_client_factory( let udp_client_factory = PerClientAddressClientFactory::new( move |source_addr, entry| { + let connect = dgram_server_conn + .new_client(Some(SocketAddr::new(*source_addr, 0))); + let key = entry.key_name.as_ref().and_then(|key_name| { key_store.get_key(&key_name, Algorithm::Sha256) }); - let connect = dgram_server_conn - .new_client(Some(SocketAddr::new(*source_addr, 0))); - match entry.matches.as_ref().map(|v| v.mock_client) { - Some(true) => { - Client::Single(Box::new(tsig::Connection::new( + if let Some(key) = key { + match entry.matches.as_ref().map(|v| v.mock_client) { + Some(true) => { + Client::Single(Box::new(tsig::Connection::new( + key, + simple_dgram_client::Connection::new(connect), + ))) + } + + _ => Client::Single(Box::new(tsig::Connection::new( key, - simple_dgram_client::Connection::new(connect), - ))) + dgram::Connection::new(connect), + ))), } + } else { + match entry.matches.as_ref().map(|v| v.mock_client) { + Some(true) => Client::Single(Box::new( + simple_dgram_client::Connection::new(connect), + )), - _ => Client::Single(Box::new(tsig::Connection::new( - key, - dgram::Connection::new(connect), - ))), + _ => Client::Single(Box::new(dgram::Connection::new( + connect, + ))), + } } }, for_all_other_queries, From c8c577a6c7aab6a30c7f7683c64735b47467ac46 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 11 Sep 2024 12:44:16 +0200 Subject: [PATCH 153/333] - More RustDocs. - Renamed AuthenticatedRequestMessage to RequestMessage to be consitent with src::net::client::request and to shorten the name (as it is scoped by the module so is still unique). - Renamed UpstreamSender to Forwarder as it's shorter and (I thnk) better. - Replaced HandleResponseResult by Option. - Renamed handle_response() to validate_response() and simplified the option/error handling a little. - Moved some impls around in the file to be nearer the type they affect. --- examples/client-transports.rs | 19 +- src/net/client/tsig.rs | 374 ++++++++++++++++++++++------------ 2 files changed, 253 insertions(+), 140 deletions(-) diff --git a/examples/client-transports.rs b/examples/client-transports.rs index 95359eca5..5ae9a4897 100644 --- a/examples/client-transports.rs +++ b/examples/client-transports.rs @@ -22,7 +22,7 @@ use domain::net::client::stream; #[cfg(feature = "tsig")] use domain::net::client::request::SendRequestMulti; #[cfg(feature = "tsig")] -use domain::net::client::tsig::{self, AuthenticatedRequestMessage}; +use domain::net::client::tsig::{self}; #[cfg(feature = "tsig")] use domain::tsig::{Algorithm, Key, KeyName}; @@ -266,16 +266,8 @@ async fn main() { #[cfg(feature = "tsig")] { let tcp_conn = TcpStream::connect(server_addr).await.unwrap(); - let (tcp, transport) = stream::Connection::< - AuthenticatedRequestMessage< - RequestMessage>, - Arc, - >, - AuthenticatedRequestMessage< - RequestMessageMulti>, - Arc, - >, - >::new(tcp_conn); + let (tcp, transport) = + stream::Connection::>, _>::new(tcp_conn); tokio::spawn(async move { transport.run().await; println!("single TSIG TCP run terminated"); @@ -340,10 +332,7 @@ where + domain::dep::octseq::Octets + 'static, SR: SendRequestMulti< - tsig::AuthenticatedRequestMessage< - RequestMessageMulti, - Arc, - >, + tsig::RequestMessage, Arc>, > + Send + Sync + 'static, diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index 438c1a73f..38d4ca2de 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -1,7 +1,8 @@ //! A TSIG signing & verifying passthrough transport. //! //! This module provides a transport that wraps the [high-level support for -//! signing message exchanges with TSIG][crate::tsig]. +//! signing message exchanges with TSIG][crate::tsig], thereby authenticating +//! them. //! //! # Usage //! @@ -13,14 +14,35 @@ //! //! # How it works //! -//! Supplying the key is optional. The transport only affects the request and -//! response if a key is supplied. This allows for optional signing without -//! having to construct a different client stack. -//! -//! When a key is supplied, requests are automatically signed and response +//! Requests are automatically signed with the given key and response //! signatures are automatically verified. On verification failure //! [Error::ValidationError][crate::net::client::request::Error] will be //! returned. +//! +//!
+//! +//! TSIG verification is a destructive process. It will alter the response +//! stripping out the TSIG RR contained within the additional section and +//! decrementing the DNS message header ARCOUNT accordingly. It may also +//! adjust the mesage ID, in conformance with [RFC +//! 8945](https://www.rfc-editor.org/rfc/rfc8945.html#name-dns-message). +//! +//! If you wish to receive the response TSIG RR intact, do **NOT** use this +//! transport. Instead process the response records manually using a normal +//! transport. +//! +//!
+//! +//! # Requirements +//! +//! This transport works with any upstream transports so long as they don’t +//! modify the message once signed nor modify the response before it can be +//! verified. +//! +//! Failing to do so will result in signature verification failure. For +//! requests this will occur at the receiving server. For responses this will +//! result in [`GetResponse`][crate::net::client::request::GetResponse] +//! rerturning [Error::ValidationError][crate::net::client::request::Error]. #![cfg(all(feature = "tsig", feature = "unstable-client-transport"))] #![warn(missing_docs)] #![warn(clippy::missing_docs_in_private_items)] @@ -50,12 +72,21 @@ use crate::net::client::request::{ use crate::rdata::tsig::Time48; use crate::tsig::{ClientSequence, ClientTransaction, Key}; -/// TODO +/// A wrapper around [`ClientTransaction`] and [`ClientSequence`]. +/// +/// This wrapper allows us to write calling code once that invokes methods on +/// the TSIG signer/validator which have the same name and purpose for single +/// response vs multiple response streams, yet have distinct Rust types and so +/// must be called on the correct type, without needing to know at the call +/// site which of the distinct types it actually is. #[derive(Clone, Debug)] enum TsigClient { - /// TODO + /// A [`ClientTransaction`] for signing a request and validating a single + /// response. Transaction(ClientTransaction), - /// TODO + + /// A [`ClientSequence`] for signing a request and validating a single + /// response. Sequence(ClientSequence), } @@ -63,7 +94,9 @@ impl TsigClient where K: AsRef, { - /// TODO + /// A helper wrapper around [`ClientTransaction::answer`] and + /// [`ClientSequence::answer`] that allows the appropriate method to be + /// invoked without needing to know which type it actually is. pub fn answer( &mut self, message: &mut Message, @@ -79,7 +112,12 @@ where .map_err(Error::Authentication) } - /// TODO + /// A helper method that allows [`ClientSequence::done`] to be called + /// without knowing or caring if the underlying type is actually + /// [`ClientTransaction`] instead (which doesn't have a `done()` method). + /// + /// Invoking this method on a [`ClientTransaction`] is harmless and has no + /// effect. fn done(self) -> Result<(), Error> { match self { TsigClient::Transaction(_) => { @@ -95,13 +133,17 @@ where //------------ Connection ----------------------------------------------------- +/// A TSIG signing and verifying transport. +/// +/// This transport signs requests and verifies responses using a provided key +/// and upstream transport. For more information see the [module +/// docs][crate::net::client::tsig]. #[derive(Clone)] -/// TODO pub struct Connection { /// Upstream transport to use for requests. /// - /// This should be the final transport, there should be no further - /// modification to the request before it is sent to the recipient. + /// The upstream transport(s) **MUST NOT** modify the request before it is + /// sent nor modify the response before this transport can verify it. upstream: Arc, /// TODO @@ -109,7 +151,11 @@ pub struct Connection { } impl Connection { - /// TODO + /// Create a new tsig transport. + /// + /// After creating the transport call `send_request` via the + /// [`SendRequest`] or [`SendRequestMulti`] traits to send signed messages + /// and verify signed responses. pub fn new(key: K, upstream: Upstream) -> Self { Self { upstream: Arc::new(upstream), @@ -123,10 +169,7 @@ impl Connection { impl SendRequest for Connection where CR: ComposeRequest + 'static, - Upstream: SendRequest> - + Send - + Sync - + 'static, + Upstream: SendRequest> + Send + Sync + 'static, K: Clone + AsRef + Send + Sync + 'static, { fn send_request( @@ -146,10 +189,7 @@ where impl SendRequestMulti for Connection where CR: ComposeRequestMulti + 'static, - Upstream: SendRequestMulti> - + Send - + Sync - + 'static, + Upstream: SendRequestMulti> + Send + Sync + 'static, K: Clone + AsRef + Send + Sync + 'static, { fn send_request( @@ -164,63 +204,67 @@ where } } -//------------ UpstreamSender ------------------------------------------------- +//------------ Forwarder ------------------------------------------------------ -/// TODO -type UpstreamSender = fn( +/// A function that can forward a request via an upstream transport. +/// +/// This type is generic over whether the [`RequestMessage`] being sent was +/// sent via the [`ComposeRequest`] trait or the [`ComposeRequestMulti`] +/// trait, which allows common logic to be used for both despite the different +/// trait bounds required to work with them. +type Forwarder = fn( &Upstream, - AuthenticatedRequestMessage, + RequestMessage, Arc>>>, ) -> RequestState; -/// TODO -fn upstream_sender( +/// Forward a request that should result in a single response. +/// +/// This function forwards a [`RequestMessage`] to an upstream transport using +/// a client that can only accept a single response, i.e. was sent via the +/// [`ComposeRequest`] trait. +fn forwarder( upstream: &Upstream, - msg: AuthenticatedRequestMessage, + msg: RequestMessage, tsig_client: Arc>>>, ) -> RequestState where CR: ComposeRequest, - Upstream: SendRequest> + Send + Sync, + Upstream: SendRequest> + Send + Sync, { RequestState::GetResponse(upstream.send_request(msg), tsig_client) } -/// TODO -fn upstream_sender_multi( +/// Forward a request that may result in multiple responses. +/// +/// This function forwards a [`RequestMessage`] to an upstream transport using +/// a client that can accept multiple responses, i.e. was sent via the +/// [`ComposeRequestMulti`] trait. +fn forwarder_multi( upstream: &Upstream, - msg: AuthenticatedRequestMessage, + msg: RequestMessage, tsig_client: Arc>>>, ) -> RequestState where CR: ComposeRequestMulti, - Upstream: - SendRequestMulti> + Send + Sync, + Upstream: SendRequestMulti> + Send + Sync, { RequestState::GetResponseMulti(upstream.send_request(msg), tsig_client) } -//------------ HandleResponseResult ------------------------------------------- - -/// TODO -enum HandleResponseResult { - /// TODO - Response(Message), - /// TODO - Complete, -} - //------------ Request -------------------------------------------------------- -/// The state of a request that is executed. -pub struct Request { +/// The state and related properties of an in-progress request. +struct Request { /// State of the request. state: RequestState, /// The request message. + /// + /// Initially Some, consumed when sent. request_msg: Option, - /// TODO + /// The TSIG key used to sign the request. key: K, /// The upstream transport of the connection. @@ -230,7 +274,7 @@ pub struct Request { impl Request where CR: ComposeRequest, - Upstream: SendRequest> + Send + Sync, + Upstream: SendRequest> + Send + Sync, K: Clone + AsRef, Self: GetResponse, { @@ -245,36 +289,6 @@ where } } -impl Debug for Request { - fn fmt(&self, f: &mut Formatter<'_>) -> Result<(), core::fmt::Error> { - f.debug_struct("Request").finish() - } -} - -impl GetResponse for Request -where - CR: ComposeRequest, - Upstream: SendRequest> + Send + Sync, - K: Clone + AsRef + Send + Sync, -{ - fn get_response( - &mut self, - ) -> Pin< - Box< - dyn Future, Error>> - + Send - + Sync - + '_, - >, - > { - Box::pin(async move { - self.get_response_impl(upstream_sender) - .await - .map(|v| v.unwrap()) - }) - } -} - impl Request where CR: Sync + Send, @@ -295,18 +309,18 @@ where /// This function is cancel safe. async fn get_response_impl( &mut self, - upstream_sender: UpstreamSender, + upstream_sender: Forwarder, ) -> Result>, Error> { let (response, tsig_client) = loop { match &mut self.state { RequestState::Init => { let tsig_client = Arc::new(std::sync::Mutex::new(None)); - let msg = AuthenticatedRequestMessage { - request: self.request_msg.take().unwrap(), - key: self.key.clone(), - signer: tsig_client.clone(), - }; + let msg = RequestMessage::new( + self.request_msg.take().unwrap(), + self.key.clone(), + tsig_client.clone(), + ); trace!("Sending request upstream..."); self.state = @@ -315,12 +329,12 @@ where } RequestState::GetResponse(request, tsig_client) => { - let response = request.get_response().await.map(Some); - break (response, tsig_client); + let response = request.get_response().await?; + break (Some(response), tsig_client); } RequestState::GetResponseMulti(request, tsig_client) => { - let response = request.get_response().await; + let response = request.get_response().await?; break (response, tsig_client); } @@ -330,29 +344,47 @@ where } }; - Self::handle_response(response, tsig_client).map(|res| match res { - HandleResponseResult::Complete => { - self.state = RequestState::Complete; - None - } - HandleResponseResult::Response(res) => Some(res), - }) + let res = Self::validate_response(response, tsig_client)?; + + if res.is_none() { + self.state = RequestState::Complete; + } + + Ok(res) } - /// TODO - fn handle_response( - response: Result>, Error>, + /// Perform TSIG validation on the result of receiving a response. + /// + /// If no response were received, validation must still be performed in + /// order to verify that the final message that was received was signed + /// correctly. This cannot be done when receiving the final response as we + /// only know that it is final by trying and failing (which may involve + /// waiting) to receive another response. + /// + /// This function therefore takes an optional response message and a + /// [`TsigClient`]. The process of validating that the final response was + /// valid will consume the given [`TsigClient`]. + /// + /// Note: Validation is a destructive process, as it strips the TSIG RR + /// out of the response. The given response message is consumed, altered + /// and returned. + /// + /// Returns: + /// - `Ok(Some)` when returning a successfully validated response. + /// - `Ok(None)` when the end of a responses stream was successfully validated. + /// - `Err` if validation or some other error occurred. + fn validate_response( + response: Option>, tsig_client: &mut Arc>>>, - ) -> Result { - // TSIG validation - match response { - Ok(None) => { + ) -> Result>, Error> { + let res = match response { + None => { let client = tsig_client.lock().unwrap().take().unwrap(); client.done()?; - Ok(HandleResponseResult::Complete) + None } - Ok(Some(msg)) => { + Some(msg) => { let mut modifiable_msg = Message::from_octets(msg.as_slice().to_vec())?; @@ -365,19 +397,54 @@ where let out_vec = modifiable_msg.into_octets(); let out_bytes = Bytes::from(out_vec); let out_msg = Message::::from_octets(out_bytes)?; - Ok(HandleResponseResult::Response(out_msg)) + Some(out_msg) } + }; - Err(err) => Err(err), - } + Ok(res) + } +} + +//-- Debug + +impl Debug for Request { + fn fmt(&self, f: &mut Formatter<'_>) -> Result<(), core::fmt::Error> { + f.debug_struct("Request").finish() + } +} + +//--- GetResponse + +impl GetResponse for Request +where + CR: ComposeRequest, + Upstream: SendRequest> + Send + Sync, + K: Clone + AsRef + Send + Sync, +{ + fn get_response( + &mut self, + ) -> Pin< + Box< + dyn Future, Error>> + + Send + + Sync + + '_, + >, + > { + Box::pin(async move { + // Unwrap the one and only response, we don't need the multiple + // response handling ability of [`Request::get_response_impl`]. + self.get_response_impl(forwarder).await.map(|v| v.unwrap()) + }) } } +//--- GetResponseMulti + impl GetResponseMulti for Request where CR: ComposeRequestMulti, - Upstream: - SendRequestMulti> + Send + Sync, + Upstream: SendRequestMulti> + Send + Sync, K: Clone + AsRef + Send + Sync, { fn get_response( @@ -390,52 +457,109 @@ where + '_, >, > { - Box::pin(self.get_response_impl(upstream_sender_multi)) + Box::pin(self.get_response_impl(forwarder_multi)) } } //------------ RequestState --------------------------------------------------- -/// States of the state machine in get_response_impl +/// State machine used by [`Request::get_response_impl`]. +/// +/// Possible flows: +/// - Init -> GetResponse +/// - Init -> GetResponseMulti -> Complete enum RequestState { - /// Initial state, perform a cache lookup. + /// Initial state, waiting to sign and send the request. Init, - /// Wait for a response and verify it. + /// Waiting for a response to verify. GetResponse( Box, Arc>>>, ), - /// Wait for multiple responses and verify them. + /// Wait for multiple responses to verify. GetResponseMulti( Box, Arc>>>, ), - /// The last response in a sequence was already received. + /// The last of multiple responses was received and verified. + /// + /// Note: This state can only be entered when processing a sequence of + /// responses, i.e. using [`GetResponseMulti`]. When using [`GetResponse`] + /// this state will not be enetered because it only calls + /// [`Request::get_response_impl`] once. Complete, } -//------------ AuthenticatedRequestMessage ------------------------------------ - -/// TODO +//------------ RequestMessage ------------------------------------------------- + +/// A message that can be sent using a [`Connection`]. +/// +/// This type implements the [`ComposeRequest`] and [`ComposeRequestMulti`] +/// traits and thus is compatible with the [`SendRequest`] and +/// [`SendRequestMulti`] traits implemented by [`Connection`]. +/// +/// This type stores the message to be sent and implements the +/// [`ComposeRequest`] and [`ComposeRequestMulti`] traits so that when the +/// upstream transport accesses the message via the traits that we can at that +/// point sign the request. +/// +/// Signing it earlier is not possible as the upstream transport may modify +/// the request prior to sending it, e.g. to assign a message ID or to add +/// EDNS options, and signing **MUST** be the last modification made to the +/// message prior to sending. #[derive(Debug)] -pub struct AuthenticatedRequestMessage +pub struct RequestMessage where CR: Send + Sync, { - /// TODO + /// The actual request to sign. request: CR, - /// TODO + /// The TSIG key to sign the request with. key: K, - /// TODO + /// The TSIG signer state. + /// + /// This must be kept here as it is created only when signing the request + /// and is needed later when verifying responses. + /// + /// Note: It is wrapped inside an [`Arc>`] because the signing is + /// done in [`Request::get_response_impl`] which returns a [`Future`] and + /// the compiler has no way of knowing whether or not a second call to + /// [`Request::get_response_impl`] could be made concurrently with an + /// earlier invocation which has not yet completed its progression through + /// its async state machine, and could be "woken up" in parallel on a + /// different thread thus requiring that access to the signer be made + /// thread safe via a locking mechanism like [`Mutex`]. signer: Arc>>>, } -impl ComposeRequest for AuthenticatedRequestMessage +impl RequestMessage +where + CR: Send + Sync, +{ + /// Creates a new [`RequestMessage`]. + fn new( + request: CR, + key: K, + signer: Arc>>>, + ) -> Self + where + CR: Sync + Send, + K: Clone + AsRef, + { + Self { + request, + key, + signer, + } + } +} + +impl ComposeRequest for RequestMessage where CR: ComposeRequest, K: Clone + Debug + Send + Sync + AsRef, @@ -518,7 +642,7 @@ where } } -impl ComposeRequestMulti for AuthenticatedRequestMessage +impl ComposeRequestMulti for RequestMessage where CR: ComposeRequestMulti, K: Clone + Debug + Send + Sync + AsRef, From 386b2df75479555631d7792ccf681470bfe0ac6e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 11 Sep 2024 12:49:56 +0200 Subject: [PATCH 154/333] Explain on TSIG failure in client-transport that this is probably expected. --- examples/client-transports.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/examples/client-transports.rs b/examples/client-transports.rs index 5ae9a4897..d94608569 100644 --- a/examples/client-transports.rs +++ b/examples/client-transports.rs @@ -358,7 +358,9 @@ where // Get the reply loop { println!("Waiting for signed reply"); - let reply = request.get_response().await.unwrap(); + let reply = request.get_response() + .await + .expect("Failed while getting a TSIG signed response. This is probably expected as the server will not know the TSIG key we are using unless you have ensured that is the case."); println!("Signed reply: {:?}", reply); if reply.is_none() { break; From e5d86bc1689f34d1d3f1899cf19a2f6cc30dd76c Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 11 Sep 2024 12:58:14 +0200 Subject: [PATCH 155/333] Merge fix. --- src/net/server/tests/integration.rs | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index 85acf95aa..5ce405121 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -21,7 +21,6 @@ use crate::base::net::IpAddr; use crate::base::wire::Composer; use crate::base::Rtype; use crate::net::client::request::{RequestMessage, RequestMessageMulti}; -use crate::net::client::tsig::AuthenticatedRequestMessage; use crate::net::client::{dgram, stream, tsig}; use crate::net::server; use crate::net::server::buf::VecBufSource; @@ -235,11 +234,8 @@ fn mk_client_factory( if let Some(key) = key { let (conn, transport) = stream::Connection::< - AuthenticatedRequestMessage>, Key>, - AuthenticatedRequestMessage< - RequestMessageMulti>, - Key, - >, + tsig::RequestMessage>, Key>, + tsig::RequestMessage>, Key>, >::new(stream); tokio::spawn(transport.run()); From 67bb52128e29da4fbc50cd9049d3bcd6b438932e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 11 Sep 2024 15:56:02 +0200 Subject: [PATCH 156/333] Add TSIG client tests. --- src/net/client/tsig.rs | 464 +++++++++++++++++++++++++++++++++++++++++ 1 file changed, 464 insertions(+) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index 38d4ca2de..204c5968a 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -717,3 +717,467 @@ where self.request.dnssec_ok() } } + +#[cfg(test)] +mod tests { + use super::*; + use crate::base::iana::Rcode; + use crate::base::message_builder::QuestionBuilder; + use crate::base::{MessageBuilder, Name, Rtype}; + use crate::tsig::{ + Algorithm, KeyName, KeyStore, ServerSequence, ServerTransaction, + ValidationError, + }; + use core::future::ready; + use core::str::FromStr; + + #[tokio::test] + async fn single_signed_valid_response() { + do_single_response(false).await; + } + + #[tokio::test] + async fn single_signed_invalid_response() { + do_single_response(true).await; + } + + async fn do_single_response(invalidate_signature: bool) { + // Make a query message that would be expected to result in a single + // reply. + let msg = mk_request_msg(Rtype::A); + + // Wrap that message into a request message compatible with a + // transport capable of receving a single response. + let req = + crate::net::client::request::RequestMessage::new(msg).unwrap(); + + // Make a TSIG key to sign the request with. + let key = mk_tsig_key(); + + // Make a mock upstream that will "send" the request and receives a + // mock TSIG signed response back. + let upstream = + Arc::new(MockUpstream::new(key.clone(), invalidate_signature)); + + // Wrap the request message into a TSIG signing request with a signing + // key and upstream transport. + let mut req = Request::new(req, key, upstream); + + // "Send" the request and receive the validated mock response. + let res = req.get_response().await; + + assert_eq!(res.is_err(), invalidate_signature); + + if let Ok(res) = res { + // Verify that the mock response has had its TSIG RR stripped out + // during validation. + assert_eq!(res.header_counts().arcount(), 0, "TSIG RR should have been removed from the additional section during response processing"); + } + } + + #[tokio::test] + async fn multiple_signed_valid_responses() { + do_multiple_responses(false, false).await + } + + #[tokio::test] + async fn multiple_signed_responses_with_one_invalid() { + do_multiple_responses(true, false).await + } + + #[tokio::test] + async fn multiple_signed_valid_responses_and_a_final_unsigned_response() { + do_multiple_responses(false, true).await + } + + async fn do_multiple_responses( + invalidate_signature: bool, + dont_sign_last_response: bool, + ) { + // Make a query message that would be expected to result in multiple + // replies. + let msg = mk_request_msg(Rtype::AXFR); + + // Wrap that message into a request message compatible with a + // transport capable of receving multiple responses. + let req = crate::net::client::request::RequestMessageMulti::new(msg) + .unwrap(); + + // Make a TSIG key to sign the request with. + let key = mk_tsig_key(); + + // Make a mock upstream that will "send" the request and receive + // multiple mock TSIG signed responses back. + let upstream = Arc::new(MockUpstreamMulti::new( + key.clone(), + invalidate_signature, + dont_sign_last_response, + )); + + // Wrap the request message into a TSIG signing request with a signing + // key and upstream transport. + let mut req = Request::new_multi(req, key, upstream); + + // "Send" the request and receive the first validated mock response. + let res = req + .get_response() + .await + .unwrap() + .expect("First response is missing"); + + // Verify that the mock response has had its TSIG RR stripped out + // during validation. + assert_eq!(res.header_counts().arcount(), 0, "TSIG RR should have been removed from the additional section during response processing"); + + // Receive the second mock response, which may have been deliberately + // invalidated. + let res = req.get_response().await; + + if invalidate_signature { + assert!( + matches!( + res, + Err(Error::Authentication(ValidationError::BadSig)) + ), + "Expected error BadSig but the result was: {res:?}" + ); + } else { + assert!(res.is_ok(), "Unexpected error message: {res:?}"); + } + + if let Ok(res) = res { + let res = res.expect("Second response is missing"); + + // Verify that the mock response has had its TSIG RR stripped out + // during validation. + assert_eq!(res.header_counts().arcount(), 0, "TSIG RR should have been removed from the additional section during response processing"); + + // Receive the third and final mock response, which may have been + // deliberately not signed, in order to test whether or not we + // are correctly calling `ClientSequence::done()` to catch this + // case. This shouldn't fail at this point however as apparently + // it's only caught when .done() is called when no more responses + // are received. + let res = req + .get_response() + .await + .unwrap() + .expect("Third response is missing"); + + // Verify that the mock response has had its TSIG RR stripped out + // during validation, or it was never added during response + // generation. + if dont_sign_last_response { + assert_eq!(res.header_counts().arcount(), 0, "TSIG RR should never have been added to the additional section during response generation"); + } else { + assert_eq!(res.header_counts().arcount(), 0, "TSIG RR should have been removed from the additional section during response processing"); + } + + if dont_sign_last_response { + // Attempt to receive another response but discover that the + // last response was not signed as it should have been. + assert!( + matches!(req.get_response().await, Err(Error::Authentication(ValidationError::TooManyUnsigned))), + "Receiving another response should have failed because the last response should have lacked a signature" + ); + } else { + // Attempt to receive another response but discover that this + // is the end of the response sequence. + assert!( + req.get_response().await.unwrap().is_none(), + "There should not be a fourth response" + ); + } + } + } + + // Make a query for the given RTYPE. + fn mk_request_msg(rtype: Rtype) -> QuestionBuilder> { + let mut msg = MessageBuilder::new_vec(); + msg.header_mut().set_rd(true); + msg.header_mut().set_ad(true); + let mut msg = msg.question(); + msg.push((Name::vec_from_str("example.com").unwrap(), rtype)) + .unwrap(); + msg + } + + // Make a TSIG key for signing test requests and responses with. + fn mk_tsig_key() -> Arc { + // Create a signing key. + let key_name = KeyName::from_str("demo-key").unwrap(); + let secret = crate::utils::base64::decode::>( + "zlCZbVJPIhobIs1gJNQfrsS3xCxxsR9pMUrGwG8OgG8=", + ) + .unwrap(); + Arc::new( + Key::new(Algorithm::Sha256, &secret, key_name, None, None) + .unwrap(), + ) + } + + //------------ MockGetResponse -------------------------------------------- + + #[derive(Debug)] + struct MockGetResponse { + request_msg: CR, + key_store: KS, + invalidate_signature: bool, + } + + impl MockGetResponse { + fn new( + request_msg: CR, + key_store: KS, + invalidate_signature: bool, + ) -> Self { + Self { + request_msg, + key_store, + invalidate_signature, + } + } + } + + //--- GetResponse + + impl GetResponse + for MockGetResponse + { + fn get_response( + &mut self, + ) -> Pin< + Box< + dyn Future, Error>> + + Send + + Sync + + '_, + >, + > { + let mut req = self.request_msg.to_message().unwrap(); + + // Create a TSIG signer for a single response based on the + // received request. + let tsig = ServerTransaction::request( + &self.key_store, + &mut req, + Time48::now(), + ) + .unwrap() + .unwrap(); + + // Generate a mock response to the request. + let builder = MessageBuilder::new_bytes(); + let builder = builder.start_answer(&req, Rcode::NOERROR).unwrap(); + let mut builder = builder.additional(); + + // Sign the response. + tsig.answer(&mut builder, Time48::now()).unwrap(); + + if self.invalidate_signature { + // Invalidate the signature. + builder.header_mut().set_rcode(Rcode::SERVFAIL); + } + + // Generate the wire format response message and sanity check it + // before returning it. + let res = builder.into_message(); + assert_eq!(res.header_counts().arcount(), 1, "Constructed response lacks a TSIG RR in the additional section"); + Box::pin(ready(Ok(res))) + } + } + + //------------ MockGetResponseMulti --------------------------------------- + + #[derive(Debug)] + struct MockGetResponseMulti { + request_msg: CR, + key_store: KS, + sent_request: Option>>, + num_responses_generated: usize, + signer: Option>, + invalidate_signature: bool, + dont_sign_last_response: bool, + } + + impl MockGetResponseMulti { + fn new( + request_msg: CR, + key_store: KS, + invalidate_signature: bool, + dont_sign_last_response: bool, + ) -> Self { + Self { + request_msg, + key_store, + sent_request: None, + num_responses_generated: 0, + signer: None, + invalidate_signature, + dont_sign_last_response, + } + } + } + + //--- GetResponseMulti + + impl GetResponseMulti for MockGetResponseMulti + where + CR: ComposeRequestMulti + Debug, + KS: Debug + KeyStore + AsRef, + { + fn get_response( + &mut self, + ) -> Pin< + Box< + dyn Future>, Error>> + + Send + + Sync + + '_, + >, + > { + // Generate a sequence of at most 3 responses. + if self.num_responses_generated == 3 { + return Box::pin(ready(Ok(None))); + } + + self.num_responses_generated += 1; + + // When first receiving the request, generate a TSIG signer for a + // multiple response sequence based on the received request. + let mut tsig = match self.signer.take() { + Some(tsig) => tsig, + None => { + let mut req = self.request_msg.to_message().unwrap(); + + let tsig = ServerSequence::request( + &self.key_store, + &mut req, + Time48::now(), + ) + .unwrap() + .unwrap(); + + // Store the signer, we'll need it to sign subsequent + // responses. + self.sent_request = Some(req); + + tsig + } + }; + + // Generate a mock response to the request. + let req = self.sent_request.as_ref().unwrap(); + let builder = MessageBuilder::new_bytes(); + let builder = builder.start_answer(req, Rcode::NOERROR).unwrap(); + let mut builder = builder.additional(); + + // Decide whether to sign and/or invalidate the response. + let (sign, invalidate) = match self.num_responses_generated { + 1 => (true, false), + 2 => (true, self.invalidate_signature), + 3 => (!self.dont_sign_last_response, false), + _ => unreachable!(), + }; + + eprintln!( + "Response {}: sign={}, invalidate={}", + self.num_responses_generated, sign, invalidate + ); + + // Sign the response (we might strip the signature out below). + if sign { + tsig.answer(&mut builder, Time48::now()).unwrap(); + } + + // Put the signer back in storage for the next response. + self.signer = Some(tsig); + + if invalidate { + // Invalidate the signature. + builder.header_mut().set_rcode(Rcode::SERVFAIL); + } + + // Generate the wire format response message and sanity check it + // before returning it. + let res = builder.into_message(); + if sign { + assert_eq!(res.header_counts().arcount(), 1, "Constructed response lacks a TSIG RR in the additional section"); + let rec = res.additional().unwrap().next().unwrap().unwrap(); + assert_eq!(rec.rtype(), Rtype::TSIG); + } + Box::pin(ready(Ok(Some(res)))) + } + } + + //------------ MockUpstream ----------------------------------------------- + + struct MockUpstream { + key: Arc, + invalidate_signature: bool, + } + + impl MockUpstream { + fn new(key: Arc, invalidate_signature: bool) -> Self { + Self { + key, + invalidate_signature, + } + } + } + + //--- SendRequest + + impl SendRequest + for MockUpstream + { + fn send_request( + &self, + request_msg: CR, + ) -> Box { + Box::new(MockGetResponse::new( + request_msg, + self.key.clone(), + self.invalidate_signature, + )) + } + } + + //------------ MockUpstreamMulti ------------------------------------------ + + struct MockUpstreamMulti { + key: Arc, + invalidate_signature: bool, + dont_sign_last_response: bool, + } + impl MockUpstreamMulti { + fn new( + key: Arc, + invalidate_signature: bool, + dont_sign_last_response: bool, + ) -> Self { + Self { + key, + invalidate_signature, + dont_sign_last_response, + } + } + } + + impl SendRequestMulti for MockUpstreamMulti + where + CR: ComposeRequestMulti + Debug + Send + Sync + 'static, + { + fn send_request( + &self, + request_msg: CR, + ) -> Box { + Box::new(MockGetResponseMulti::new( + request_msg, + self.key.clone(), + self.invalidate_signature, + self.dont_sign_last_response, + )) + } + } +} From 008574babdddc996f947415b3748e00697e933d0 Mon Sep 17 00:00:00 2001 From: Philip Homburg Date: Wed, 11 Sep 2024 16:06:53 +0200 Subject: [PATCH 157/333] Add docs for validator_test. --- src/net/client/validator_test.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/net/client/validator_test.rs b/src/net/client/validator_test.rs index 6b8a587e9..d02168d52 100644 --- a/src/net/client/validator_test.rs +++ b/src/net/client/validator_test.rs @@ -1,3 +1,5 @@ +//! Module for testing the validator client transport. + #![cfg(test)] use std::fs::File; From 22d8c62ed2d7211760e8dfe89a6caf5846fb211c Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 11 Sep 2024 16:15:42 +0200 Subject: [PATCH 158/333] Resolve cargo doc issues. --- src/net/client/tsig.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index 204c5968a..025337458 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -10,7 +10,7 @@ //! 2. Create a [Connection] that wraps an upstream connection and uses the //! key. //! 3. [Send a request][Connection::send_request] using the connection. -//! 4. [Receive the response][Request::get_response] or responses. +//! 4. [Receive the response][GetResponse] or responses. //! //! # How it works //! @@ -41,8 +41,8 @@ //! //! Failing to do so will result in signature verification failure. For //! requests this will occur at the receiving server. For responses this will -//! result in [`GetResponse`][crate::net::client::request::GetResponse] -//! rerturning [Error::ValidationError][crate::net::client::request::Error]. +//! result in [`GetResponse`] rerturning +//! [Error::ValidationError][crate::net::client::request::Error]. #![cfg(all(feature = "tsig", feature = "unstable-client-transport"))] #![warn(missing_docs)] #![warn(clippy::missing_docs_in_private_items)] From 35937a98fc12b2bff492f150451b56117f8c6404 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 11 Sep 2024 22:02:53 +0200 Subject: [PATCH 159/333] Fix test compilation failure after merging with main including reversion of tsig mod KeyStore changes which is good anyway as they would have been breaking changes to stable interfaces. --- src/net/server/tests/integration.rs | 16 +++++++++++++ src/tsig/mod.rs | 37 +---------------------------- 2 files changed, 17 insertions(+), 36 deletions(-) diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index 5ce405121..ffab73b78 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -522,3 +522,19 @@ fn parse_server_config(config: &Config) -> ServerConfig { // KeyStore is impl'd elsewhere for HashMap<(KeyName, Algorithm), K, S>. type TestKeyStore = HashMap<(KeyName, Algorithm), Key>; + +impl KeyStore for Arc { + type Key = Key; + + fn get_key( + &self, + name: &N, + algorithm: Algorithm, + ) -> Option { + if let Ok(name) = name.try_to_name() { + self.get(&(name, algorithm)).cloned() + } else { + None + } + } +} diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index 89820f622..1845a7cc1 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -423,7 +423,7 @@ pub trait KeyStore { ) -> Option; } -impl KeyStore for Key { +impl + Clone> KeyStore for K { type Key = Self; fn get_key( @@ -441,24 +441,6 @@ impl KeyStore for Key { } } -impl KeyStore for &'_ Key { - type Key = Self; - - fn get_key( - &self, - name: &N, - algorithm: Algorithm, - ) -> Option { - if self.as_ref().name() == name - && self.as_ref().algorithm() == algorithm - { - Some(self) - } else { - None - } - } -} - #[cfg(feature = "std")] impl KeyStore for HashMap<(KeyName, Algorithm), K, S> where @@ -478,23 +460,6 @@ where } } -#[cfg(feature = "std")] -impl KeyStore for std::sync::Arc -where - K: AsRef + Clone, - U: KeyStore, -{ - type Key = K; - - fn get_key( - &self, - name: &N, - algorithm: Algorithm, - ) -> Option { - (**self).get_key(name, algorithm) - } -} - //------------ ClientTransaction --------------------------------------------- /// TSIG Client transaction state. From 8b5d17d30b8d5397a1d8cfb8cdd27e0bc9e2ce5d Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 11 Sep 2024 22:25:08 +0200 Subject: [PATCH 160/333] Merge branch 'tsig-middleware' into add-rfc1996-notify-support. --- Changelog.md | 4 +- src/net/client/tsig.rs | 472 +++++++++++++++++++++++++++- src/net/client/validator_test.rs | 2 + src/net/server/tests/integration.rs | 26 +- src/tsig/mod.rs | 41 +-- 5 files changed, 497 insertions(+), 48 deletions(-) diff --git a/Changelog.md b/Changelog.md index 3e396600e..7d95fd0ee 100644 --- a/Changelog.md +++ b/Changelog.md @@ -38,7 +38,9 @@ Unstable features * Add support for requests that may result in multiple responses. This adds ComposeRequestMulti and other *Multi types. The main change is to the stream transport, which is the only transport that implements - SendRequestMulti. (#377) + SendRequestMulti. (#377) + * Added a TSIG request signing and response validating passthrough + transport in `net::client:tsig`. ([#373]) * `unstable-server-transport` * The cookies middleware now allows requests with invalid cookies to proceed if they are authenticated or not required to authenticate. ([#336]) diff --git a/src/net/client/tsig.rs b/src/net/client/tsig.rs index 38d4ca2de..277690dce 100644 --- a/src/net/client/tsig.rs +++ b/src/net/client/tsig.rs @@ -10,7 +10,7 @@ //! 2. Create a [Connection] that wraps an upstream connection and uses the //! key. //! 3. [Send a request][Connection::send_request] using the connection. -//! 4. [Receive the response][Request::get_response] or responses. +//! 4. [Receive the response][GetResponse] or responses. //! //! # How it works //! @@ -41,8 +41,8 @@ //! //! Failing to do so will result in signature verification failure. For //! requests this will occur at the receiving server. For responses this will -//! result in [`GetResponse`][crate::net::client::request::GetResponse] -//! rerturning [Error::ValidationError][crate::net::client::request::Error]. +//! result in [`GetResponse`] returning +//! [Error::ValidationError][crate::net::client::request::Error]. #![cfg(all(feature = "tsig", feature = "unstable-client-transport"))] #![warn(missing_docs)] #![warn(clippy::missing_docs_in_private_items)] @@ -146,7 +146,7 @@ pub struct Connection { /// sent nor modify the response before this transport can verify it. upstream: Arc, - /// TODO + /// The key to sign requests with. key: K, } @@ -717,3 +717,467 @@ where self.request.dnssec_ok() } } + +#[cfg(test)] +mod tests { + use super::*; + use crate::base::iana::Rcode; + use crate::base::message_builder::QuestionBuilder; + use crate::base::{MessageBuilder, Name, Rtype}; + use crate::tsig::{ + Algorithm, KeyName, KeyStore, ServerSequence, ServerTransaction, + ValidationError, + }; + use core::future::ready; + use core::str::FromStr; + + #[tokio::test] + async fn single_signed_valid_response() { + do_single_response(false).await; + } + + #[tokio::test] + async fn single_signed_invalid_response() { + do_single_response(true).await; + } + + async fn do_single_response(invalidate_signature: bool) { + // Make a query message that would be expected to result in a single + // reply. + let msg = mk_request_msg(Rtype::A); + + // Wrap that message into a request message compatible with a + // transport capable of receving a single response. + let req = + crate::net::client::request::RequestMessage::new(msg).unwrap(); + + // Make a TSIG key to sign the request with. + let key = mk_tsig_key(); + + // Make a mock upstream that will "send" the request and receives a + // mock TSIG signed response back. + let upstream = + Arc::new(MockUpstream::new(key.clone(), invalidate_signature)); + + // Wrap the request message into a TSIG signing request with a signing + // key and upstream transport. + let mut req = Request::new(req, key, upstream); + + // "Send" the request and receive the validated mock response. + let res = req.get_response().await; + + assert_eq!(res.is_err(), invalidate_signature); + + if let Ok(res) = res { + // Verify that the mock response has had its TSIG RR stripped out + // during validation. + assert_eq!(res.header_counts().arcount(), 0, "TSIG RR should have been removed from the additional section during response processing"); + } + } + + #[tokio::test] + async fn multiple_signed_valid_responses() { + do_multiple_responses(false, false).await + } + + #[tokio::test] + async fn multiple_signed_responses_with_one_invalid() { + do_multiple_responses(true, false).await + } + + #[tokio::test] + async fn multiple_signed_valid_responses_and_a_final_unsigned_response() { + do_multiple_responses(false, true).await + } + + async fn do_multiple_responses( + invalidate_signature: bool, + dont_sign_last_response: bool, + ) { + // Make a query message that would be expected to result in multiple + // replies. + let msg = mk_request_msg(Rtype::AXFR); + + // Wrap that message into a request message compatible with a + // transport capable of receving multiple responses. + let req = crate::net::client::request::RequestMessageMulti::new(msg) + .unwrap(); + + // Make a TSIG key to sign the request with. + let key = mk_tsig_key(); + + // Make a mock upstream that will "send" the request and receive + // multiple mock TSIG signed responses back. + let upstream = Arc::new(MockUpstreamMulti::new( + key.clone(), + invalidate_signature, + dont_sign_last_response, + )); + + // Wrap the request message into a TSIG signing request with a signing + // key and upstream transport. + let mut req = Request::new_multi(req, key, upstream); + + // "Send" the request and receive the first validated mock response. + let res = req + .get_response() + .await + .unwrap() + .expect("First response is missing"); + + // Verify that the mock response has had its TSIG RR stripped out + // during validation. + assert_eq!(res.header_counts().arcount(), 0, "TSIG RR should have been removed from the additional section during response processing"); + + // Receive the second mock response, which may have been deliberately + // invalidated. + let res = req.get_response().await; + + if invalidate_signature { + assert!( + matches!( + res, + Err(Error::Authentication(ValidationError::BadSig)) + ), + "Expected error BadSig but the result was: {res:?}" + ); + } else { + assert!(res.is_ok(), "Unexpected error message: {res:?}"); + } + + if let Ok(res) = res { + let res = res.expect("Second response is missing"); + + // Verify that the mock response has had its TSIG RR stripped out + // during validation. + assert_eq!(res.header_counts().arcount(), 0, "TSIG RR should have been removed from the additional section during response processing"); + + // Receive the third and final mock response, which may have been + // deliberately not signed, in order to test whether or not we + // are correctly calling `ClientSequence::done()` to catch this + // case. This shouldn't fail at this point however as apparently + // it's only caught when .done() is called when no more responses + // are received. + let res = req + .get_response() + .await + .unwrap() + .expect("Third response is missing"); + + // Verify that the mock response has had its TSIG RR stripped out + // during validation, or it was never added during response + // generation. + if dont_sign_last_response { + assert_eq!(res.header_counts().arcount(), 0, "TSIG RR should never have been added to the additional section during response generation"); + } else { + assert_eq!(res.header_counts().arcount(), 0, "TSIG RR should have been removed from the additional section during response processing"); + } + + if dont_sign_last_response { + // Attempt to receive another response but discover that the + // last response was not signed as it should have been. + assert!( + matches!(req.get_response().await, Err(Error::Authentication(ValidationError::TooManyUnsigned))), + "Receiving another response should have failed because the last response should have lacked a signature" + ); + } else { + // Attempt to receive another response but discover that this + // is the end of the response sequence. + assert!( + req.get_response().await.unwrap().is_none(), + "There should not be a fourth response" + ); + } + } + } + + // Make a query for the given RTYPE. + fn mk_request_msg(rtype: Rtype) -> QuestionBuilder> { + let mut msg = MessageBuilder::new_vec(); + msg.header_mut().set_rd(true); + msg.header_mut().set_ad(true); + let mut msg = msg.question(); + msg.push((Name::vec_from_str("example.com").unwrap(), rtype)) + .unwrap(); + msg + } + + // Make a TSIG key for signing test requests and responses with. + fn mk_tsig_key() -> Arc { + // Create a signing key. + let key_name = KeyName::from_str("demo-key").unwrap(); + let secret = crate::utils::base64::decode::>( + "zlCZbVJPIhobIs1gJNQfrsS3xCxxsR9pMUrGwG8OgG8=", + ) + .unwrap(); + Arc::new( + Key::new(Algorithm::Sha256, &secret, key_name, None, None) + .unwrap(), + ) + } + + //------------ MockGetResponse -------------------------------------------- + + #[derive(Debug)] + struct MockGetResponse { + request_msg: CR, + key_store: KS, + invalidate_signature: bool, + } + + impl MockGetResponse { + fn new( + request_msg: CR, + key_store: KS, + invalidate_signature: bool, + ) -> Self { + Self { + request_msg, + key_store, + invalidate_signature, + } + } + } + + //--- GetResponse + + impl GetResponse + for MockGetResponse + { + fn get_response( + &mut self, + ) -> Pin< + Box< + dyn Future, Error>> + + Send + + Sync + + '_, + >, + > { + let mut req = self.request_msg.to_message().unwrap(); + + // Create a TSIG signer for a single response based on the + // received request. + let tsig = ServerTransaction::request( + &self.key_store, + &mut req, + Time48::now(), + ) + .unwrap() + .unwrap(); + + // Generate a mock response to the request. + let builder = MessageBuilder::new_bytes(); + let builder = builder.start_answer(&req, Rcode::NOERROR).unwrap(); + let mut builder = builder.additional(); + + // Sign the response. + tsig.answer(&mut builder, Time48::now()).unwrap(); + + if self.invalidate_signature { + // Invalidate the signature. + builder.header_mut().set_rcode(Rcode::SERVFAIL); + } + + // Generate the wire format response message and sanity check it + // before returning it. + let res = builder.into_message(); + assert_eq!(res.header_counts().arcount(), 1, "Constructed response lacks a TSIG RR in the additional section"); + Box::pin(ready(Ok(res))) + } + } + + //------------ MockGetResponseMulti --------------------------------------- + + #[derive(Debug)] + struct MockGetResponseMulti { + request_msg: CR, + key_store: KS, + sent_request: Option>>, + num_responses_generated: usize, + signer: Option>, + invalidate_signature: bool, + dont_sign_last_response: bool, + } + + impl MockGetResponseMulti { + fn new( + request_msg: CR, + key_store: KS, + invalidate_signature: bool, + dont_sign_last_response: bool, + ) -> Self { + Self { + request_msg, + key_store, + sent_request: None, + num_responses_generated: 0, + signer: None, + invalidate_signature, + dont_sign_last_response, + } + } + } + + //--- GetResponseMulti + + impl GetResponseMulti for MockGetResponseMulti + where + CR: ComposeRequestMulti + Debug, + KS: Debug + KeyStore + AsRef, + { + fn get_response( + &mut self, + ) -> Pin< + Box< + dyn Future>, Error>> + + Send + + Sync + + '_, + >, + > { + // Generate a sequence of at most 3 responses. + if self.num_responses_generated == 3 { + return Box::pin(ready(Ok(None))); + } + + self.num_responses_generated += 1; + + // When first receiving the request, generate a TSIG signer for a + // multiple response sequence based on the received request. + let mut tsig = match self.signer.take() { + Some(tsig) => tsig, + None => { + let mut req = self.request_msg.to_message().unwrap(); + + let tsig = ServerSequence::request( + &self.key_store, + &mut req, + Time48::now(), + ) + .unwrap() + .unwrap(); + + // Store the signer, we'll need it to sign subsequent + // responses. + self.sent_request = Some(req); + + tsig + } + }; + + // Generate a mock response to the request. + let req = self.sent_request.as_ref().unwrap(); + let builder = MessageBuilder::new_bytes(); + let builder = builder.start_answer(req, Rcode::NOERROR).unwrap(); + let mut builder = builder.additional(); + + // Decide whether to sign and/or invalidate the response. + let (sign, invalidate) = match self.num_responses_generated { + 1 => (true, false), + 2 => (true, self.invalidate_signature), + 3 => (!self.dont_sign_last_response, false), + _ => unreachable!(), + }; + + eprintln!( + "Response {}: sign={}, invalidate={}", + self.num_responses_generated, sign, invalidate + ); + + // Sign the response (we might strip the signature out below). + if sign { + tsig.answer(&mut builder, Time48::now()).unwrap(); + } + + // Put the signer back in storage for the next response. + self.signer = Some(tsig); + + if invalidate { + // Invalidate the signature. + builder.header_mut().set_rcode(Rcode::SERVFAIL); + } + + // Generate the wire format response message and sanity check it + // before returning it. + let res = builder.into_message(); + if sign { + assert_eq!(res.header_counts().arcount(), 1, "Constructed response lacks a TSIG RR in the additional section"); + let rec = res.additional().unwrap().next().unwrap().unwrap(); + assert_eq!(rec.rtype(), Rtype::TSIG); + } + Box::pin(ready(Ok(Some(res)))) + } + } + + //------------ MockUpstream ----------------------------------------------- + + struct MockUpstream { + key: Arc, + invalidate_signature: bool, + } + + impl MockUpstream { + fn new(key: Arc, invalidate_signature: bool) -> Self { + Self { + key, + invalidate_signature, + } + } + } + + //--- SendRequest + + impl SendRequest + for MockUpstream + { + fn send_request( + &self, + request_msg: CR, + ) -> Box { + Box::new(MockGetResponse::new( + request_msg, + self.key.clone(), + self.invalidate_signature, + )) + } + } + + //------------ MockUpstreamMulti ------------------------------------------ + + struct MockUpstreamMulti { + key: Arc, + invalidate_signature: bool, + dont_sign_last_response: bool, + } + impl MockUpstreamMulti { + fn new( + key: Arc, + invalidate_signature: bool, + dont_sign_last_response: bool, + ) -> Self { + Self { + key, + invalidate_signature, + dont_sign_last_response, + } + } + } + + impl SendRequestMulti for MockUpstreamMulti + where + CR: ComposeRequestMulti + Debug + Send + Sync + 'static, + { + fn send_request( + &self, + request_msg: CR, + ) -> Box { + Box::new(MockGetResponseMulti::new( + request_msg, + self.key.clone(), + self.invalidate_signature, + self.dont_sign_last_response, + )) + } + } +} diff --git a/src/net/client/validator_test.rs b/src/net/client/validator_test.rs index 6b8a587e9..d02168d52 100644 --- a/src/net/client/validator_test.rs +++ b/src/net/client/validator_test.rs @@ -1,3 +1,5 @@ +//! Module for testing the validator client transport. + #![cfg(test)] use std::fs::File; diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index 5d1cc7d88..90e65df56 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -528,11 +528,6 @@ fn parse_server_config(config: &Config) -> ServerConfig { parsed_config } -//------------ TestKeyStore --------------------------------------------------- - -// KeyStore is impl'd elsewhere for HashMap<(KeyName, Algorithm), K, S>. -type TestKeyStore = HashMap<(KeyName, Algorithm), Key>; - //------------ NoOpNotifyTarget ----------------------------------------------- #[derive(Copy, Clone, Default, Debug)] @@ -558,3 +553,24 @@ impl Notifiable for TestNotifyTarget { Box::pin(ready(res)) } } + +//------------ TestKeyStore --------------------------------------------------- + +// KeyStore is impl'd elsewhere for HashMap<(KeyName, Algorithm), K, S>. +type TestKeyStore = HashMap<(KeyName, Algorithm), Key>; + +impl KeyStore for Arc { + type Key = Key; + + fn get_key( + &self, + name: &N, + algorithm: Algorithm, + ) -> Option { + if let Ok(name) = name.try_to_name() { + self.get(&(name, algorithm)).cloned() + } else { + None + } + } +} diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index 6a067f0e7..1845a7cc1 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -1,7 +1,7 @@ //! Support for TSIG. //! //! This module provides high-level support for signing message exchanges with -//! TSIG as defined in [RFC 2845]. +//! TSIG as defined in [RFC 8945]. //! //! TSIG is intended to provide authentication for message exchanges. Messages //! are signed using a secret key shared between the two participants. The @@ -41,8 +41,8 @@ //! mostly for testing). //! //! [RFC 2104]: https://tools.ietf.org/html/rfc2104 -//! [RFC 2845]: https://tools.ietf.org/html/rfc2845 //! [RFC 4635]: https://tools.ietf.org/html/rfc4653 +//! [RFC 8945]: https://tools.ietf.org/html/rfc8945 //! [TSIG]: ../rdata/tsig/struct.Tsig.html //! [`Algorithm`]: enum.Algorithm.html //! [`Key`]: enum.Key.html @@ -423,7 +423,7 @@ pub trait KeyStore { ) -> Option; } -impl KeyStore for Key { +impl + Clone> KeyStore for K { type Key = Self; fn get_key( @@ -441,24 +441,6 @@ impl KeyStore for Key { } } -impl KeyStore for &'_ Key { - type Key = Self; - - fn get_key( - &self, - name: &N, - algorithm: Algorithm, - ) -> Option { - if self.as_ref().name() == name - && self.as_ref().algorithm() == algorithm - { - Some(self) - } else { - None - } - } -} - #[cfg(feature = "std")] impl KeyStore for HashMap<(KeyName, Algorithm), K, S> where @@ -478,23 +460,6 @@ where } } -#[cfg(feature = "std")] -impl KeyStore for std::sync::Arc -where - K: AsRef + Clone, - U: KeyStore, -{ - type Key = K; - - fn get_key( - &self, - name: &N, - algorithm: Algorithm, - ) -> Option { - (**self).get_key(name, algorithm) - } -} - //------------ ClientTransaction --------------------------------------------- /// TSIG Client transaction state. From f7829278d5541b89cda9f91c9580ed5355e1f5cd Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 11 Sep 2024 22:52:29 +0200 Subject: [PATCH 161/333] Revert changes not related to the PR. --- src/base/message_builder.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/base/message_builder.rs b/src/base/message_builder.rs index ae1e1a9a5..9d5a75b1e 100644 --- a/src/base/message_builder.rs +++ b/src/base/message_builder.rs @@ -262,8 +262,7 @@ impl MessageBuilder { /// Starts creating an error for the given message. /// - /// Like [`start_answer()`] but infallible. Questions will be pushed if - /// possible. + /// Like [`start_answer()`] but infallible. Questions will be pushed if possible. pub fn start_error( mut self, msg: &Message, From 88665d9e84afab6aab125537dadc1f8a4f4d936c Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 11 Sep 2024 22:52:40 +0200 Subject: [PATCH 162/333] Revert changes not related to the PR. --- src/net/server/middleware/cookies.rs | 2 +- src/net/server/middleware/edns.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/net/server/middleware/cookies.rs b/src/net/server/middleware/cookies.rs index a864ec942..58708e504 100644 --- a/src/net/server/middleware/cookies.rs +++ b/src/net/server/middleware/cookies.rs @@ -471,7 +471,7 @@ where Once::Item>>, ::Item, >; - type Future = Ready; + type Future = core::future::Ready; fn call( &self, diff --git a/src/net/server/middleware/edns.rs b/src/net/server/middleware/edns.rs index 6eae69350..affe96362 100644 --- a/src/net/server/middleware/edns.rs +++ b/src/net/server/middleware/edns.rs @@ -361,7 +361,7 @@ where Once::Item>>, ::Item, >; - type Future = Ready; + type Future = core::future::Ready; fn call( &self, From c4a0afea24f94bbd9567dc774e8c7c64728d7607 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 12 Sep 2024 12:55:38 +0200 Subject: [PATCH 163/333] More RustDoc. Removed the unused MaybeAuthenticated trait. --- src/net/server/middleware/mod.rs | 2 ++ src/net/server/middleware/tsig.rs | 50 +++++++++++++++++++++---------- 2 files changed, 37 insertions(+), 15 deletions(-) diff --git a/src/net/server/middleware/mod.rs b/src/net/server/middleware/mod.rs index b8d8a9100..ddf639209 100644 --- a/src/net/server/middleware/mod.rs +++ b/src/net/server/middleware/mod.rs @@ -13,6 +13,8 @@ //! through the layers to the server. //! //! Currently the following middleware are available: +//! +//! [`Service`]: crate::net::server::service::Service #[cfg(feature = "siphasher")] pub mod cookies; diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index 5cedac484..c21f15942 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -1,4 +1,18 @@ -//! TSIG message authentication middleware. +//! RFC 8495 TSIG message authentication middleware. +//! +//! This module provides a TSIG request validation and response signing +//! middleware service. The underlying TSIG RR processing is implemented using +//! the [`tsig`] module. +//! +//! # Communicating which key signed a request. +//! +//! This middleware service passes the [`Authentication`] type to upstream +//! [`Service`] impls via request metadata. Upstream services can choose to +//! ignore the metadata by being generic over any kind of metadata, or may +//! offer a [`Service`] impl that specifically accepts the [`Authentication`] +//! metadata type. By specifically accepting the [`Authentication`] metadata +//! type the upstream service is able to use it to learn which key the request +//! was signed with. use core::convert::Infallible; use core::future::{ready, Ready}; @@ -30,7 +44,7 @@ use super::stream::{MiddlewareStream, PostprocessingStream}; //------------ TsigMiddlewareSvc ---------------------------------------------- -/// TSIG message authentication middlware. +/// RFC 8495 TSIG message authentication middleware. /// /// This middleware service validates TSIG signatures on incoming requests, if /// any, and adds TSIG signatures to responses to signed requests. @@ -56,7 +70,10 @@ impl TsigMiddlewareSvc where KS: Clone + KeyStore, { - /// Creates a new processor instance. + /// Creates an instance of this middleware service. + /// + /// Keys in the provided [`KeyStore`] will be used to verify received signed + /// requests and to sign the corresponding responses. #[must_use] pub fn new(next_svc: NextSvc, key_store: KS) -> Self { Self { @@ -355,8 +372,19 @@ where //--- Service -// Note: As the TSIG middleware must be the closest middleware to the server, -// it does not receive any special RequestMeta from the server, only (). +/// This [`Service`] implementation specifies that the upstream service will +/// be passed metadata of type [`Authentication`]. The upstream service can +/// optionally use this to learn which TSIG key signed the request. +/// +/// This service does not accept downstream metadata, explicitly restricting +/// what it accepts to `()`. This is because (a) the service should be the +/// first layer above the network server, or as near as possible, such that it +/// receives unmodified requests and that the responses it generates are sent +/// over the network without prior modification, and thus it is not very +/// likely that the is a downstream layer that has metadata to supply to us, +/// and (b) because this service does not propagate the metadata it receives +/// from downstream but instead outputs [`Authentication`] metadata to +/// upstream services. impl Service for TsigMiddlewareSvc where @@ -440,21 +468,13 @@ enum TsigSigner { Sequence(ServerSequence), } -//------------ MaybeAuthenticated --------------------------------------------- - -pub trait MaybeAuthenticated: - Clone + Default + Sync + Send + 'static -{ - fn key_name(&self) -> Option<&KeyName>; -} - //------------ Authentication -------------------------------------------------- #[derive(Clone, Default)] pub struct Authentication(pub Option); -impl MaybeAuthenticated for Authentication { - fn key_name(&self) -> Option<&KeyName> { +impl Authentication { + pub fn key_name(&self) -> Option<&KeyName> { self.0.as_ref() } } From 5174a487b6fb2afbb9c39ef5bb51612f60ddd0c5 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 12 Sep 2024 15:13:25 +0200 Subject: [PATCH 164/333] - More RustDoc. - Don't pass the key store around, we need only the signer when post-processing. - Get tid of the mutex, we don't need it. - Don't clone metadata when invoking a stream post-processing callback. - Add key_wrapper() fn to ServerTransaction, ServerSequence and SigningContext to allow access to the key by its original type (which might e.g. be Arc allowing the Arc to be cloned rather than a deep copy being made). --- examples/server-transports.rs | 4 +- src/net/server/middleware/edns.rs | 2 +- src/net/server/middleware/mandatory.rs | 4 +- src/net/server/middleware/mod.rs | 5 + src/net/server/middleware/stream.rs | 12 +- src/net/server/middleware/tsig.rs | 156 ++++++++++++++----------- src/net/server/service.rs | 10 +- src/net/server/tests/integration.rs | 13 ++- src/tsig/mod.rs | 26 ++++- 9 files changed, 136 insertions(+), 96 deletions(-) diff --git a/examples/server-transports.rs b/examples/server-transports.rs index ece257ce6..7b665c1c0 100644 --- a/examples/server-transports.rs +++ b/examples/server-transports.rs @@ -489,7 +489,7 @@ impl StatsMiddlewareSvc { fn postprocess( request: &Request, response: &AdditionalBuilder>, - stats: Arc>, + stats: &RwLock, ) where RequestOctets: Octets + Send + Sync + Unpin, Svc: Service, @@ -511,7 +511,7 @@ impl StatsMiddlewareSvc { fn map_stream_item( request: Request, stream_item: ServiceResult, - stats: Arc>, + stats: &mut Arc>, ) -> ServiceResult where RequestOctets: Octets + Send + Sync + Unpin, diff --git a/src/net/server/middleware/edns.rs b/src/net/server/middleware/edns.rs index affe96362..b96f670e1 100644 --- a/src/net/server/middleware/edns.rs +++ b/src/net/server/middleware/edns.rs @@ -325,7 +325,7 @@ where fn map_stream_item( request: Request, mut stream_item: ServiceResult, - _pp_meta: (), + _pp_meta: &mut (), ) -> ServiceResult { if let Ok(cr) = &mut stream_item { if let Some(response) = cr.response_mut() { diff --git a/src/net/server/middleware/mandatory.rs b/src/net/server/middleware/mandatory.rs index 9d1a9f90b..2f2818403 100644 --- a/src/net/server/middleware/mandatory.rs +++ b/src/net/server/middleware/mandatory.rs @@ -268,11 +268,11 @@ where fn map_stream_item( request: Request, mut stream_item: ServiceResult, - strict: bool, + strict: &mut bool, ) -> ServiceResult { if let Ok(cr) = &mut stream_item { if let Some(response) = cr.response_mut() { - Self::postprocess(&request, response, strict); + Self::postprocess(&request, response, *strict); } } stream_item diff --git a/src/net/server/middleware/mod.rs b/src/net/server/middleware/mod.rs index ddf639209..52f172bcc 100644 --- a/src/net/server/middleware/mod.rs +++ b/src/net/server/middleware/mod.rs @@ -12,6 +12,11 @@ //! post-processing the resulting responses and propagating them back down //! through the layers to the server. //! +//! If needed middleware services can pass service specific data to upstream +//! services for consumption, via the `RequestMeta` custom data support of +//! the [`Service`] trait. An example of this can be seen in the +//! [`TsigMiddlewareSvc`][tsig::TsigMiddlewareSvc]. +//! //! Currently the following middleware are available: //! //! [`Service`]: crate::net::server::service::Service diff --git a/src/net/server/middleware/stream.rs b/src/net/server/middleware/stream.rs index 6d4c7fb2a..23c476446 100644 --- a/src/net/server/middleware/stream.rs +++ b/src/net/server/middleware/stream.rs @@ -110,7 +110,7 @@ type PostprocessingStreamCallback< > = fn( Request, StreamItem, - PostProcessingMeta, + &mut PostProcessingMeta, ) -> StreamItem; //------------ PostprocessingStream ------------------------------------------ @@ -153,7 +153,7 @@ where pub fn new( svc_call_fut: Future, request: Request, - metadata: PostProcessingMeta, + pp_meta: PostProcessingMeta, cb: PostprocessingStreamCallback< RequestOctets, Stream::Item, @@ -165,7 +165,7 @@ where state: PostprocessingStreamState::Pending(svc_call_fut), request, cb, - pp_meta: metadata, + pp_meta, } } } @@ -187,7 +187,6 @@ where Stream: futures::stream::Stream + Unpin, Self: Unpin, RequestMeta: Clone, - PostProcessingMeta: Clone, { type Item = Stream::Item; @@ -206,9 +205,8 @@ where let stream_item = ready!(stream.poll_next_unpin(cx)); trace!("Stream item retrieved, mapping to downstream type"); let request = self.request.clone(); - let pp_meta = self.pp_meta.clone(); - let map = - stream_item.map(|item| (self.cb)(request, item, pp_meta)); + let map = stream_item + .map(|item| (self.cb)(request, item, &mut self.pp_meta)); Poll::Ready(map) } } diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index c21f15942..d1536ac86 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -6,20 +6,18 @@ //! //! # Communicating which key signed a request. //! -//! This middleware service passes the [`Authentication`] type to upstream -//! [`Service`] impls via request metadata. Upstream services can choose to -//! ignore the metadata by being generic over any kind of metadata, or may -//! offer a [`Service`] impl that specifically accepts the [`Authentication`] -//! metadata type. By specifically accepting the [`Authentication`] metadata -//! type the upstream service is able to use it to learn which key the request -//! was signed with. +//! For signed requests this middleware service passes the signing key to +//! upstream [`Service`] impls via request metadata. Upstream services can +//! choose to ignore the metadata by being generic over any kind of metadata, +//! or may offer a [`Service`] impl that specifically accepts the +//! [`Option`] metadata type. The upstream service is then able to +//! use the received metadata to learn which key the request was signed with. use core::convert::Infallible; use core::future::{ready, Ready}; use core::marker::PhantomData; -use core::ops::{ControlFlow, DerefMut}; +use core::ops::ControlFlow; -use std::sync::Arc; use std::vec::Vec; use futures::stream::{once, Once, Stream}; @@ -36,9 +34,7 @@ use crate::net::server::service::{ }; use crate::net::server::util::{mk_builder_for_target, mk_error_response}; use crate::rdata::tsig::Time48; -use crate::tsig::{ - self, KeyName, KeyStore, ServerSequence, ServerTransaction, -}; +use crate::tsig::{self, KeyStore, ServerSequence, ServerTransaction}; use super::stream::{MiddlewareStream, PostprocessingStream}; @@ -49,6 +45,9 @@ use super::stream::{MiddlewareStream, PostprocessingStream}; /// This middleware service validates TSIG signatures on incoming requests, if /// any, and adds TSIG signatures to responses to signed requests. /// +/// Upstream services can detect whether a request is signed and with which +/// key by consuming the [`Authentication`] metadata output by this service. +/// /// | RFC | Status | /// |--------|---------| /// | [8945] | TBD | @@ -87,9 +86,10 @@ where impl TsigMiddlewareSvc where RequestOctets: Octets + OctetsFrom> + Send + Sync + Unpin, - NextSvc: Service, + NextSvc: Service>, NextSvc::Target: Composer + Default, KS: Clone + KeyStore, + KS::Key: Clone, Infallible: From<>>::Error>, { #[allow(clippy::type_complexity)] @@ -98,7 +98,10 @@ where key_store: &KS, ) -> ControlFlow< AdditionalBuilder>, - Option<(Request, TsigSigner)>, + Option<( + Request>, + TsigSigner, + )>, > { if let Some(q) = Self::get_relevant_question(req.message()) { let octets = req.message().as_slice().to_vec(); @@ -131,7 +134,7 @@ where req.received_at(), new_msg, req.transport_ctx().clone(), - Authentication(Some(tsig.key().name().clone())), + Some(tsig.key_wrapper().clone()), ); let num_bytes_to_reserve = tsig.key().compose_len(); @@ -170,37 +173,54 @@ where fn postprocess( request: &Request, response: &mut AdditionalBuilder>, - pp_config: PostprocessingConfig, + pp_config: &mut PostprocessingConfig, ) -> Option>> { - // Sign the response. - let mut tsig_signer = pp_config.tsig.lock().unwrap(); - // Remove the limit we should have imposed during pre-processing so // that we can use the space we reserved for the OPT RR. response.clear_push_limit(); - let signing_result = match tsig_signer.as_mut() { + // The variable itself isn't used by a reference to its interior value + // *is* used if the if signing_result.is_err() block below. + #[allow(unused_assignments)] + let mut key_for_err_handling = None; + + let (signing_result, key) = match &mut pp_config.signer { Some(TsigSigner::Transaction(_)) => { // Extract the single response signer and consume it in the // signing process. - let Some(TsigSigner::Transaction(tsig)) = tsig_signer.take() + let Some(TsigSigner::Transaction(signer)) = + pp_config.signer.take() else { unreachable!() }; + trace!( "Signing single response with TSIG key '{}'", - tsig.key().name() + signer.key().name() ); - tsig.answer(response, Time48::now()) + + // We have to clone the key here in case the signer produces + // an error, otherwise we lose access to the key as the signer + // is consumed by calling answer(). The caller has control + // over the key type via KS::Key so if cloning cost is a + // problem the caller can choose to wrap the key in an Arc or + // such to reduce the cloning cost. + key_for_err_handling = Some(signer.key().clone()); + + let res = signer.answer(response, Time48::now()); + + (res, key_for_err_handling.as_ref().unwrap()) } - Some(TsigSigner::Sequence(tsig)) => { + Some(TsigSigner::Sequence(ref mut signer)) => { // Use the multi-response signer to sign the response. trace!( "Signing response stream with TSIG key '{}'", - tsig.key().name() + signer.key().name() ); - tsig.answer(response, Time48::now()) + let res = signer.answer(response, Time48::now()); + + (res, signer.key()) } None => { @@ -225,9 +245,7 @@ where // in the failed attempt to sign the answer, so we have to create a new // TSIG state in order to sign the truncated response. if request.transport_ctx().is_udp() { - return Self::mk_signed_truncated_response( - request, &pp_config, - ); + return Self::mk_signed_truncated_response(request, key); } else { // In the TCP case there's not much we can do. The upstream // service pushes response messages into the stream and we try @@ -253,15 +271,12 @@ where fn mk_signed_truncated_response( request: &Request, - pp_config: &PostprocessingConfig, + key: &tsig::Key, ) -> Option>> { let octets = request.message().as_slice().to_vec(); let mut mut_msg = Message::from_octets(octets).unwrap(); - let res = ServerTransaction::request( - &pp_config.key_store, - &mut mut_msg, - Time48::now(), - ); + let res = + ServerTransaction::request(&key, &mut mut_msg, Time48::now()); match res { Ok(None) => { @@ -323,7 +338,7 @@ where fn map_stream_item( request: Request, stream_item: ServiceResult, - pp_config: PostprocessingConfig, + pp_config: &mut PostprocessingConfig, ) -> ServiceResult { if let Ok(mut call_res) = stream_item { if matches!( @@ -341,15 +356,13 @@ where // ServerTransaction the TSIG code means handling of single // messages only and NOT sequences for which there is a // separate ServerSequence type. Sigh. - let mut locked_tsig = pp_config.tsig.lock().unwrap(); - let mutable_tsig = locked_tsig.deref_mut(); if let Some(TsigSigner::Transaction(tsig_txn)) = - mutable_tsig.take() + pp_config.signer.take() { // Do the conversion and store the result for future // invocations of this function for subsequent items // in the response stream. - *mutable_tsig = Some(TsigSigner::Sequence( + pp_config.signer = Some(TsigSigner::Sequence( ServerSequence::from(tsig_txn), )); } @@ -390,7 +403,7 @@ impl Service where RequestOctets: Octets + OctetsFrom> + Send + Sync + 'static + Unpin, - NextSvc: Service, + NextSvc: Service>, NextSvc::Future: Unpin, NextSvc::Target: Composer + Default, KS: Clone + KeyStore + Unpin, @@ -406,7 +419,7 @@ where NextSvc::Future, NextSvc::Stream, (), - PostprocessingConfig, + PostprocessingConfig, >, Once::Item>>, ::Item, @@ -415,16 +428,11 @@ where fn call(&self, request: Request) -> Self::Future { match Self::preprocess(&request, &self.key_store) { - ControlFlow::Continue(Some((modified_req, tsig_opt))) => { - let tsig = Arc::new(std::sync::Mutex::new(Some(tsig_opt))); + ControlFlow::Continue(Some((modified_req, signer))) => { + let pp_config = PostprocessingConfig::new(signer); let svc_call_fut = self.next_svc.call(modified_req); - let pp_config = PostprocessingConfig { - tsig, - key_store: self.key_store.clone(), - }; - let map = PostprocessingStream::new( svc_call_fut, request, @@ -436,7 +444,7 @@ where } ControlFlow::Continue(None) => { - let request = request.with_new_metadata(Authentication(None)); + let request = request.with_new_metadata(None); let svc_call_fut = self.next_svc.call(request); ready(MiddlewareStream::IdentityFuture(svc_call_fut)) } @@ -450,31 +458,39 @@ where } } -#[derive(Clone)] -pub struct PostprocessingConfig -where - KS: KeyStore + Clone, -{ - tsig: Arc::Key>>>>, - key_store: KS, +/// Data needed to do signing during response post-processing. + +pub struct PostprocessingConfig { + /// The signer used to verify the request. + /// + /// Needed to sign responses. + /// + /// We store it as an Option because ServerTransaction::answer() consumes + /// the signer so have to first take it out of this struct, as a reference + /// is held to the struct so it iself cannot be consumed. + signer: Option>, +} + +impl PostprocessingConfig { + fn new(signer: TsigSigner) -> Self { + Self { + signer: Some(signer), + } + } } +/// A wrapper around [`ServerTransaction`] and [`ServerSequence`]. +/// +/// This wrapper allows us to write calling code once that invokes methods on +/// the TSIG signer/validator which have the same name and purpose for single +/// response vs multiple response streams, yet have distinct Rust types and so +/// must be called on the correct type, without needing to know at the call +/// site which of the distinct types it actually is. #[derive(Clone, Debug)] enum TsigSigner { - /// TODO + /// A [`ServerTransaction`] for signing a single response. Transaction(ServerTransaction), - /// TODO + /// A [`ServerSequence`] for signing multiple responses. Sequence(ServerSequence), } - -//------------ Authentication -------------------------------------------------- - -#[derive(Clone, Default)] -pub struct Authentication(pub Option); - -impl Authentication { - pub fn key_name(&self) -> Option<&KeyName> { - self.0.as_ref() - } -} diff --git a/src/net/server/service.rs b/src/net/server/service.rs index f4b0fa1fc..52129e8c0 100644 --- a/src/net/server/service.rs +++ b/src/net/server/service.rs @@ -153,14 +153,8 @@ pub type ServiceResult = Result, ServiceError>; /// /// For more advanced cases you may need to override these defaults. /// -/// - `RequestMeta`: If implementing a [middleware] `Service` you may need to -/// supply your own `RequestMeta` type. `RequestMeta` is intended to enable -/// middleware `Service` impls to express strongly typed support for -/// middleware specific data that can be consumed by upstream middleware, or -/// even by your application service. For example a middleware `Service` may -/// detect that the request is signed using a particular key and communicate -/// the name of the key to any upstream `Service` that needs to know the -/// name of the key used to sign the request. +/// - `RequestMeta`: Use this to pass additional custom data to your service. +/// [Middleware] services use this to pass data to the next layer. /// /// - `RequestOctets`: By specifying your own `RequestOctets` type you can use /// a type other than `Vec` to transport request bytes through your diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index ffab73b78..a215eb954 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -86,7 +86,7 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { let (key, _) = Key::generate(Algorithm::Sha256, &rng, key_name.clone(), None, None) .unwrap(); - key_store.insert((key_name, Algorithm::Sha256), key); + key_store.insert((key_name, Algorithm::Sha256), key.into()); let key_store = Arc::new(key_store); // Create a connection factory. @@ -234,8 +234,11 @@ fn mk_client_factory( if let Some(key) = key { let (conn, transport) = stream::Connection::< - tsig::RequestMessage>, Key>, - tsig::RequestMessage>, Key>, + tsig::RequestMessage>, Arc>, + tsig::RequestMessage< + RequestMessageMulti>, + Arc, + >, >::new(stream); tokio::spawn(transport.run()); @@ -521,10 +524,10 @@ fn parse_server_config(config: &Config) -> ServerConfig { //------------ TestKeyStore --------------------------------------------------- // KeyStore is impl'd elsewhere for HashMap<(KeyName, Algorithm), K, S>. -type TestKeyStore = HashMap<(KeyName, Algorithm), Key>; +type TestKeyStore = HashMap<(KeyName, Algorithm), Arc>; impl KeyStore for Arc { - type Key = Key; + type Key = Arc; fn get_key( &self, diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index 1845a7cc1..3cb517573 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -689,6 +689,14 @@ impl> ServerTransaction { pub fn key(&self) -> &Key { self.context.key() } + + /// Returns a reference to the transaction's key. + /// + /// This is the same as [`key`] but doesn't lose the original key type + /// information. + pub fn key_wrapper(&self) -> &K { + self.context.key_wrapper() + } } //------------ ClientSequence ------------------------------------------------ @@ -1016,6 +1024,14 @@ impl> ServerSequence { pub fn key(&self) -> &Key { self.context.key() } + + /// Returns a reference to the transaction's key. + /// + /// This is the same as [`key`] but doesn't lose the original key type + /// information. + pub fn key_wrapper(&self) -> &K { + self.context.key_wrapper() + } } //--- From @@ -1244,11 +1260,19 @@ impl> SigningContext { } } - /// Returns a references to the key that was used to create the context. + /// Returns a reference to the key that was used to create the context. fn key(&self) -> &Key { self.key.as_ref() } + /// Returns a reference to the key that was used to create the context. + /// + /// This is the same as [`key`] but doesn't lose the original key type + /// information. + fn key_wrapper(&self) -> &K { + &self.key + } + /// Applies a signature to the signing context. /// /// The `data` argument must be the actual signature that has already been From 226d311fdbae2ef275663715de4ad4fd627f59bf Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 12 Sep 2024 15:20:55 +0200 Subject: [PATCH 165/333] Update references to removed Authentication type. --- src/net/server/middleware/tsig.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index d1536ac86..d98099364 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -46,7 +46,7 @@ use super::stream::{MiddlewareStream, PostprocessingStream}; /// any, and adds TSIG signatures to responses to signed requests. /// /// Upstream services can detect whether a request is signed and with which -/// key by consuming the [`Authentication`] metadata output by this service. +/// key by consuming the [`Option`] metadata output by this service. /// /// | RFC | Status | /// |--------|---------| @@ -386,7 +386,7 @@ where //--- Service /// This [`Service`] implementation specifies that the upstream service will -/// be passed metadata of type [`Authentication`]. The upstream service can +/// be passed metadata of type [`Option`]. The upstream service can /// optionally use this to learn which TSIG key signed the request. /// /// This service does not accept downstream metadata, explicitly restricting @@ -396,7 +396,7 @@ where /// over the network without prior modification, and thus it is not very /// likely that the is a downstream layer that has metadata to supply to us, /// and (b) because this service does not propagate the metadata it receives -/// from downstream but instead outputs [`Authentication`] metadata to +/// from downstream but instead outputs [`Option`] metadata to /// upstream services. impl Service for TsigMiddlewareSvc From bf977f71eae901ede6e0db3b06aeb1fc981e2c3c Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 12 Sep 2024 15:21:10 +0200 Subject: [PATCH 166/333] Fix broken RustDoc references. --- src/tsig/mod.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index 3cb517573..199ca5266 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -692,8 +692,8 @@ impl> ServerTransaction { /// Returns a reference to the transaction's key. /// - /// This is the same as [`key`] but doesn't lose the original key type - /// information. + /// This is the same as [`Self::key`] but doesn't lose the original key + /// type information. pub fn key_wrapper(&self) -> &K { self.context.key_wrapper() } @@ -1027,8 +1027,8 @@ impl> ServerSequence { /// Returns a reference to the transaction's key. /// - /// This is the same as [`key`] but doesn't lose the original key type - /// information. + /// This is the same as [`Self::key`] but doesn't lose the original key + /// type information. pub fn key_wrapper(&self) -> &K { self.context.key_wrapper() } From 92821c71ff1de07f41b60c0278be66e46d5aefa4 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 12 Sep 2024 16:11:05 +0200 Subject: [PATCH 167/333] Update uses of no longer existing Authentication type. --- src/net/server/middleware/xfr.rs | 29 ++++++++++++++++++----------- src/net/server/tests/integration.rs | 6 +++--- 2 files changed, 21 insertions(+), 14 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 2b5c9ffa5..de10e7454 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -1368,12 +1368,9 @@ mod tests { use crate::net::server::message::{ NonUdpTransportContext, UdpTransportContext, }; - use crate::net::server::middleware::tsig::{ - Authentication, MaybeAuthenticated, - }; use crate::net::server::service::ServiceError; use crate::rdata::{Aaaa, AllRecordData, Cname, Mx, Ns, Txt, A}; - use crate::tsig::KeyName; + use crate::tsig::{Algorithm, Key, KeyName}; use crate::zonefile::inplace::Zonefile; use crate::zonetree::types::Rrset; @@ -1837,22 +1834,22 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 async fn ixfr_multi_response_tcp() {} #[tokio::test] - async fn axfr_with_tsig_key_name() { + async fn axfr_with_tsig_key() { // Define an XfrDataProvider that expects to receive a Request that is // generic over a type that we specify: Authentication. This is the // type over which the Request produced by TsigMiddlewareSvc is generic. // When the XfrMiddlewareSvc receives a Request it // passes it to the XfrDataProvider which in turn can inspect it. struct KeyReceivingXfrDataProvider { - key_name: KeyName, + key: Arc, checked: Arc, } - impl XfrDataProvider for KeyReceivingXfrDataProvider { + impl XfrDataProvider>> for KeyReceivingXfrDataProvider { #[allow(clippy::type_complexity)] fn request( &self, - req: &Request, + req: &Request>>, _diff_from: Option, ) -> Pin< Box< @@ -1868,18 +1865,28 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 where Octs: Octets + Send + Sync, { - assert_eq!(req.metadata().key_name(), Some(&self.key_name)); + let key = req.metadata().as_ref().unwrap(); + assert_eq!(key.name(), self.key.name()); self.checked.store(true, Ordering::SeqCst); Box::pin(ready(Err(XfrDataProviderError::Refused))) } } let key_name = KeyName::from_str("some_tsig_key_name").unwrap(); - let metadata = Authentication(Some(key_name.clone())); + let secret = crate::utils::base64::decode::>( + "zlCZbVJPIhobIs1gJNQfrsS3xCxxsR9pMUrGwG8OgG8=", + ) + .unwrap(); + let key = Arc::new( + Key::new(Algorithm::Sha256, &secret, key_name, None, None) + .unwrap(), + ); + + let metadata = Some(key.clone()); let req = mk_axfr_request(n("example.com"), metadata); let checked = Arc::new(AtomicBool::new(false)); let xdp = KeyReceivingXfrDataProvider { - key_name, + key, checked: checked.clone(), }; diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index f9ace7c0a..78ab2d9ca 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -36,7 +36,6 @@ use crate::net::server::middleware::mandatory::MandatoryMiddlewareSvc; use crate::net::server::middleware::notify::{ Notifiable, NotifyError, NotifyMiddlewareSvc, }; -use crate::net::server::middleware::tsig::Authentication; use crate::net::server::middleware::tsig::TsigMiddlewareSvc; use crate::net::server::middleware::xfr::XfrMiddlewareSvc; use crate::net::server::service::{CallResult, Service, ServiceResult}; @@ -161,8 +160,9 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { // 4. XFR(-in) middleware service (XFR-out is handled by the // ZoneMaintainer). - let svc = - XfrMiddlewareSvc::, _, _, Authentication>::new(svc, zones, 1); + let svc = XfrMiddlewareSvc::, _, _, Option>>::new( + svc, zones, 1, + ); // 5. RFC 1996 NOTIFY support. let svc = NotifyMiddlewareSvc::new(svc, TestNotifyTarget); From 1783c8201ce93cd2e5d849c6a20d0a1750170f5b Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 12 Sep 2024 16:39:40 +0200 Subject: [PATCH 168/333] Use the existing error response building functionality rather than ignoring the actual error code. --- src/net/server/middleware/tsig.rs | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index d98099364..d200f2d14 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -24,7 +24,7 @@ use futures::stream::{once, Once, Stream}; use octseq::{Octets, OctetsFrom}; use tracing::{error, trace, warn}; -use crate::base::iana::{Opcode, OptRcode, Rcode, TsigRcode}; +use crate::base::iana::{Opcode, OptRcode, Rcode}; use crate::base::message_builder::AdditionalBuilder; use crate::base::wire::Composer; use crate::base::{Message, ParsedName, Question, Rtype, StreamTarget}; @@ -147,7 +147,6 @@ where } Err(err) => { - // Message is incorrectly signed or signed with an unknown key. warn!( "{} for {} from {} refused: {err}", q.qtype(), @@ -155,11 +154,8 @@ where req.client_addr(), ); let builder = mk_builder_for_target(); - let additional = tsig::ServerError::::unsigned( - TsigRcode::BADKEY, - ) - .build_message(req.message(), builder) - .unwrap(); + let additional = + err.build_message(req.message(), builder).unwrap(); return ControlFlow::Break(additional); } } From f29b8d42f5ed383b737acdc525b5a5634c414bc6 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 12 Sep 2024 23:06:47 +0200 Subject: [PATCH 169/333] Document known limitation. --- src/net/server/middleware/tsig.rs | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index d98099364..5967c03bd 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -12,6 +12,13 @@ //! or may offer a [`Service`] impl that specifically accepts the //! [`Option`] metadata type. The upstream service is then able to //! use the received metadata to learn which key the request was signed with. +//! +//! # Limitations +//! +//! * RFC 8945 5.2.3 Time Check and Error Handling states: _"The server SHOULD +//! also cache the most recent Time Signed value in a message generated by a +//! key and SHOULD return BADTIME if a message received later has an earlier +//! Time Signed value."_. This is not implemented. use core::convert::Infallible; use core::future::{ready, Ready}; @@ -47,12 +54,6 @@ use super::stream::{MiddlewareStream, PostprocessingStream}; /// /// Upstream services can detect whether a request is signed and with which /// key by consuming the [`Option`] metadata output by this service. -/// -/// | RFC | Status | -/// |--------|---------| -/// | [8945] | TBD | -/// -/// [8945]: https://datatracker.ietf.org/doc/rfc8945/ #[derive(Clone, Debug)] pub struct TsigMiddlewareSvc where From 41cebe08a153eaee2ff1755fce0cf935712fe563 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 13 Sep 2024 01:44:18 +0200 Subject: [PATCH 170/333] - Handle TSIG response truncation using the existing ServerSequence if available, via a new TruncationContext type. - Truncate for TCP as well as UDP. - Return ServiceError::InternalError if truncation fails. - Implement handling of ServiceError in DgramServer and stream::Connection. - Break DgramServer and stream::Connection dispatch to service code out into helper RequestDispatcher types. - Terminate the response stream if ServiceError::InternalError occurs. --- src/net/server/connection.rs | 353 ++++++++++++++++---------- src/net/server/dgram.rs | 399 ++++++++++++++++++++---------- src/net/server/middleware/tsig.rs | 162 ++++++------ 3 files changed, 585 insertions(+), 329 deletions(-) diff --git a/src/net/server/connection.rs b/src/net/server/connection.rs index a805b1b17..f824c2fa6 100644 --- a/src/net/server/connection.rs +++ b/src/net/server/connection.rs @@ -27,10 +27,11 @@ use crate::net::server::buf::BufSource; use crate::net::server::message::Request; use crate::net::server::metrics::ServerMetrics; use crate::net::server::service::{Service, ServiceError, ServiceFeedback}; -use crate::net::server::util::to_pcap_text; +use crate::net::server::util::{mk_error_response, to_pcap_text}; use crate::utils::config::DefMinMax; use super::message::{NonUdpTransportContext, TransportSpecificContext}; +use super::service::ServiceResult; use super::stream::Config as ServerConfig; use super::ServerCommand; @@ -221,9 +222,10 @@ impl Clone for Config { /// A handler for a single stream connection between client and server. pub struct Connection where - Buf: BufSource, - Buf::Output: Send + Sync + Unpin, - Svc: Service + Clone, + Buf: BufSource + Clone + Send + Sync + 'static, + Buf::Output: Octets + Send + Sync + Unpin, + Svc: Service + Clone + Send + Sync + 'static, + Svc::Target: Composer + Default + Send, { /// Flag used by the Drop impl to track if the metric count has to be /// decreased or not. @@ -258,14 +260,14 @@ where /// to be written back the client. result_q_tx: mpsc::Sender>>, - /// A [`Service`] for handling received requests and generating responses. - service: Svc, - /// DNS protocol idle time out tracking. idle_timer: IdleTimer, /// [`ServerMetrics`] describing the status of the server. metrics: Arc, + + /// Dispatches requests to the service and enqueues responses for sending. + request_dispatcher: RequestDispatcher, } /// Creation @@ -273,9 +275,12 @@ where impl Connection where Stream: AsyncRead + AsyncWrite, - Buf: BufSource, + Buf: BufSource + Clone + Send + Sync, Buf::Output: Octets + Send + Sync + Unpin, - Svc: Service + Clone, + Svc: Service + Clone + Send + Sync, + Svc::Target: Composer + Default + Send, + Svc::Stream: Send, + Svc::Future: Send, { /// Creates a new handler for an accepted stream connection. #[must_use] @@ -322,6 +327,13 @@ where // uses of self we have to do while running. let stream_rx = Some(stream_rx); + let request_dispatcher = RequestDispatcher::new( + config.clone(), + service, + result_q_tx.clone(), + metrics.clone(), + ); + Self { active: false, buf, @@ -331,9 +343,9 @@ where stream_tx, result_q_rx, result_q_tx, - service, idle_timer, metrics, + request_dispatcher, } } } @@ -346,8 +358,9 @@ where Buf: BufSource + Send + Sync + Clone + 'static, Buf::Output: Octets + Send + Sync + Unpin, Svc: Service + Clone + Send + Sync + 'static, - Svc::Target: Composer + Send, + Svc::Target: Composer + Default + Send, Svc::Stream: Send, + Svc::Future: Send, { /// Start reading requests and writing responses to the stream. /// @@ -363,9 +376,7 @@ where pub async fn run( mut self, command_rx: watch::Receiver>, - ) where - Svc::Future: Send, - { + ) { self.metrics.inc_num_connections(); // Flag that we have to decrease the metric count on Drop. @@ -383,7 +394,7 @@ where Buf: BufSource + Send + Sync + Clone + 'static, Buf::Output: Octets + Send + Sync + Unpin, Svc: Service + Clone + Send + Sync + 'static, - Svc::Target: Composer + Send, + Svc::Target: Composer + Default + Send, Svc::Future: Send, Svc::Stream: Send, { @@ -536,10 +547,7 @@ where } /// Stop queueing new responses and process those already in the queue. - async fn flush_write_queue(&mut self) - // where - // Target: Composer, - { + async fn flush_write_queue(&mut self) { debug!("Flushing connection write queue."); // Stop accepting new response messages (should we check for in-flight // messages that haven't generated a response yet but should be @@ -564,10 +572,7 @@ where async fn process_queued_result( &mut self, response: Option>>, - ) -> Result<(), ConnectionEvent> -// where - // Target: Composer, - { + ) -> Result<(), ConnectionEvent> { // If we failed to read the results of requests processed by the // service because the queue holding those results is empty and can no // longer be read from, then there is no point continuing to read from @@ -592,10 +597,7 @@ where async fn write_response_to_stream( &mut self, msg: StreamTarget, - ) - // where - // Target: AsRef<[u8]>, - { + ) { if enabled!(Level::TRACE) { let bytes = msg.as_dgram_slice(); let pcap_text = to_pcap_text(bytes, bytes.len()); @@ -653,6 +655,7 @@ where ) -> Result<(), ConnectionEvent> where Svc::Stream: Send, + Svc::Target: Default, { match res { Ok(buf) => { @@ -682,112 +685,23 @@ where let ctx = NonUdpTransportContext::new(Some( self.config.load().idle_timeout, )); - let ctx = TransportSpecificContext::NonUdp(ctx); + let request = Request::new( self.addr, received_at, msg, - ctx, + TransportSpecificContext::NonUdp(ctx), (), ); - let svc = self.service.clone(); - let result_q_tx = self.result_q_tx.clone(); - let metrics = self.metrics.clone(); - let config = self.config.clone(); - trace!( "Spawning task to handle new message with id {}", request.message().header().id() ); + + let mut dispatcher = self.request_dispatcher.clone(); tokio::spawn(async move { - let request_id = request.message().header().id(); - trace!( - "Calling service for request id {request_id}" - ); - let mut stream = svc.call(request).await; - let mut in_transaction = false; - - trace!("Awaiting service call results for request id {request_id}"); - while let Some(Ok(call_result)) = - stream.next().await - { - trace!("Processing service call result for request id {request_id}"); - let (response, feedback) = - call_result.into_inner(); - - if let Some(feedback) = feedback { - match feedback { - ServiceFeedback::Reconfigure { - idle_timeout, - } => { - if let Some(idle_timeout) = - idle_timeout - { - debug!( - "Reconfigured connection timeout to {idle_timeout:?}" - ); - let guard = config.load(); - let mut new_config = **guard; - new_config.idle_timeout = - idle_timeout; - config.store(Arc::new( - new_config, - )); - } - } - - ServiceFeedback::BeginTransaction => { - in_transaction = true; - } - - ServiceFeedback::EndTransaction => { - in_transaction = false; - } - } - } - - if let Some(mut response) = response { - loop { - match result_q_tx.try_send(response) { - Ok(()) => { - let pending_writes = - result_q_tx - .max_capacity() - - result_q_tx - .capacity(); - trace!("Queued message for sending: # pending writes={pending_writes}"); - metrics - .set_num_pending_writes( - pending_writes, - ); - break; - } - - Err(TrySendError::Closed(_)) => { - error!("Unable to queue message for sending: server is shutting down."); - break; - } - - Err(TrySendError::Full( - unused_response, - )) => { - if in_transaction { - // Wait until there is space in the message queue. - tokio::task::yield_now() - .await; - response = - unused_response; - } else { - error!("Unable to queue message for sending: queue is full."); - break; - } - } - } - } - } - } - trace!("Finished processing service call results for request id {request_id}"); + dispatcher.dispatch(request).await }); } } @@ -804,9 +718,10 @@ where impl Drop for Connection where - Buf: BufSource, - Buf::Output: Send + Sync + Unpin, - Svc: Service + Clone, + Buf: BufSource + Clone + Send + Sync, + Buf::Output: Octets + Send + Sync + Unpin, + Svc: Service + Clone + Send + Sync, + Svc::Target: Composer + Default + Send, { fn drop(&mut self) { if self.active { @@ -1075,3 +990,193 @@ impl IdleTimer { self.reset_idle_timer() } } +//------------ DispatcherStatus ----------------------------------------------- + +enum DispatcherStatus { + Normal, + + InTransaction, + + Aborting, +} + +//------------ RequestDispatcher ---------------------------------------------- + +struct RequestDispatcher +where + RequestOctets: Octets + Send + Sync, + Svc: Service + Clone + Send + Sync + 'static, + Svc::Target: Composer + Default + Send, +{ + /// User supplied settings that influence our behaviour. + /// + /// May updated during request and response processing based on received + /// [`ServiceFeedback`]. + config: Arc>, + + /// A [`Service`] for handling received requests and generating responses. + service: Svc, + + /// The writer for pushing ready responses onto the queue waiting + /// to be written back the client. + result_q_tx: mpsc::Sender>>, + + /// [`ServerMetrics`] describing the status of the server. + metrics: Arc, + + status: DispatcherStatus, +} + +impl Clone for RequestDispatcher +where + RequestOctets: Octets + Send + Sync, + Svc: Service + Clone + Send + Sync + 'static, + Svc::Target: Composer + Default + Send, +{ + fn clone(&self) -> Self { + Self { + config: self.config.clone(), + service: self.service.clone(), + result_q_tx: self.result_q_tx.clone(), + metrics: self.metrics.clone(), + status: DispatcherStatus::Normal, + } + } +} + +impl RequestDispatcher +where + RequestOctets: Octets + Send + Sync, + Svc: Service + Clone + Send + Sync + 'static, + Svc::Target: Composer + Default + Send, + Svc::Future: Send, + Svc::Stream: Send, +{ + fn new( + config: Arc>, + service: Svc, + result_q_tx: mpsc::Sender< + AdditionalBuilder>, + >, + metrics: Arc, + ) -> Self { + Self { + config, + service, + result_q_tx, + metrics, + status: DispatcherStatus::Normal, + } + } + + async fn dispatch(&mut self, request: Request) { + let req_msg = request.message().clone(); + let request_id = request.message().header().id(); + + // Dispatch the request to the service for processing. + trace!("Calling service for request id {request_id}"); + let mut stream = self.service.call(request).await; + + // Handle the resulting stream of responses, most likely just one as + // only XFR requests potentially result in multiple responses. + trace!("Awaiting service call results for request id {request_id}"); + while let Some(item) = stream.next().await { + trace!( + "Processing service call result for request id {request_id}" + ); + + let response = self.process_response_stream_item(item, &req_msg); + + if let Some(response) = response { + self.enqueue_response(response).await; + } + + if matches!(self.status, DispatcherStatus::Aborting) { + trace!("Aborting response stream processing for request id {request_id}"); + break; + } + } + trace!("Finished processing service call results for request id {request_id}"); + } + + fn process_response_stream_item( + &mut self, + stream_item: ServiceResult, + req_msg: &Message, + ) -> Option>> { + match stream_item { + Ok(call_result) => { + let (response, feedback) = call_result.into_inner(); + if let Some(feedback) = feedback { + self.process_feedback(feedback); + } + response + } + + Err(err) => { + self.status = DispatcherStatus::Aborting; + Some(mk_error_response(req_msg, err.rcode().into())) + } + } + } + + fn process_feedback(&mut self, feedback: ServiceFeedback) { + match feedback { + ServiceFeedback::Reconfigure { idle_timeout } => { + self.update_config(idle_timeout); + } + + ServiceFeedback::BeginTransaction => { + self.status = DispatcherStatus::InTransaction + } + + ServiceFeedback::EndTransaction => { + self.status = DispatcherStatus::Normal + } + } + } + + fn update_config(&self, idle_timeout: Option) { + if let Some(idle_timeout) = idle_timeout { + debug!("Reconfigured connection timeout to {idle_timeout:?}"); + let guard = self.config.load(); + let mut new_config = **guard; + new_config.idle_timeout = idle_timeout; + self.config.store(Arc::new(new_config)); + } + } + + async fn enqueue_response( + &self, + mut response: AdditionalBuilder>, + ) { + loop { + match self.result_q_tx.try_send(response) { + Ok(()) => { + let pending_writes = self.result_q_tx.max_capacity() + - self.result_q_tx.capacity(); + trace!("Queued message for sending: # pending writes={pending_writes}"); + self.metrics.set_num_pending_writes(pending_writes); + break; + } + + Err(TrySendError::Closed(_)) => { + error!("Unable to queue message for sending: server is shutting down."); + break; + } + + Err(TrySendError::Full(unused_response)) => { + if matches!(self.status, DispatcherStatus::InTransaction) + { + // Wait until there is space in the message queue. + tokio::task::yield_now().await; + response = unused_response; + } else { + error!("Unable to queue message for sending: queue is full."); + break; + } + } + } + } + } +} diff --git a/src/net/server/dgram.rs b/src/net/server/dgram.rs index 752b8f4ca..152f8172f 100644 --- a/src/net/server/dgram.rs +++ b/src/net/server/dgram.rs @@ -11,6 +11,7 @@ //! [Datagram]: https://en.wikipedia.org/wiki/Datagram use core::fmt::Debug; use core::future::poll_fn; +use core::marker::PhantomData; use core::ops::Deref; use core::time::Duration; @@ -34,8 +35,9 @@ use tracing::warn; use tracing::Level; use tracing::{enabled, error, trace}; +use crate::base::message_builder::AdditionalBuilder; use crate::base::wire::Composer; -use crate::base::Message; +use crate::base::{Message, StreamTarget}; use crate::net::server::buf::BufSource; use crate::net::server::error::Error; use crate::net::server::message::Request; @@ -47,6 +49,8 @@ use crate::utils::config::DefMinMax; use super::buf::VecBufSource; use super::message::{TransportSpecificContext, UdpTransportContext}; +use super::service::ServiceResult; +use super::util::mk_error_response; use super::ServerCommand; /// A UDP transport based DNS server transport. @@ -252,15 +256,11 @@ pub struct DgramServer where Sock: AsyncDgramSock + Send + Sync + 'static, Buf: BufSource + Send + Sync, - ::Output: Octets + Send + Sync + Unpin + 'static, - Svc: Clone - + Service<::Output, ()> - + Send - + Sync - + 'static, - ::Output, ()>>::Future: Send, - ::Output, ()>>::Stream: Send, - ::Output, ()>>::Target: Composer + Send, + Buf::Output: Octets + Send + Sync + Unpin + 'static, + Svc: Service + Clone + Send + Sync + 'static, + Svc::Future: Send, + Svc::Stream: Send, + Svc::Target: Composer + Default + Send, { /// The configuration of the server. config: Arc>, @@ -283,11 +283,11 @@ where /// A [`BufSource`] for creating buffers on demand. buf: Buf, - /// A [`Service`] for handling received requests and generating responses. - service: Svc, - /// [`ServerMetrics`] describing the status of the server. metrics: Arc, + + /// Dispatches requests to the service and enqueues responses for sending. + request_dispatcher: RequestDispatcher, } /// Creation @@ -296,11 +296,11 @@ impl DgramServer where Sock: AsyncDgramSock + Send + Sync, Buf: BufSource + Send + Sync, - ::Output: Octets + Send + Sync + Unpin, - Svc: Clone + Service<::Output, ()> + Send + Sync, - ::Output, ()>>::Future: Send, - ::Output, ()>>::Stream: Send, - ::Output, ()>>::Target: Composer + Send, + Buf::Output: Octets + Send + Sync + Unpin, + Svc: Service + Clone + Send + Sync + 'static, + Svc::Future: Send, + Svc::Stream: Send, + Svc::Target: Composer + Default + Send, { /// Constructs a new [`DgramServer`] with default configuration. /// @@ -334,15 +334,23 @@ where let command_tx = Arc::new(Mutex::new(command_tx)); let metrics = Arc::new(ServerMetrics::connection_less()); let config = Arc::new(ArcSwap::from_pointee(config)); + let sock = Arc::new(sock); + + let request_dispatcher = RequestDispatcher::new( + config.clone(), + service, + sock.clone(), + metrics.clone(), + ); DgramServer { config, command_tx, command_rx, - sock: sock.into(), + sock, buf, - service, metrics, + request_dispatcher, } } } @@ -353,11 +361,11 @@ impl DgramServer where Sock: AsyncDgramSock + Send + Sync, Buf: BufSource + Send + Sync, - ::Output: Octets + Send + Sync + Unpin, - Svc: Clone + Service<::Output, ()> + Send + Sync, - ::Output, ()>>::Future: Send, - ::Output, ()>>::Stream: Send, - ::Output, ()>>::Target: Composer + Send, + Buf::Output: Octets + Send + Sync + Unpin, + Svc: Service + Clone + Send + Sync + 'static, + Svc::Future: Send, + Svc::Stream: Send, + Svc::Target: Composer + Default + Send, { /// Get a reference to the network source being used to receive messages. #[must_use] @@ -378,15 +386,11 @@ impl DgramServer where Sock: AsyncDgramSock + Send + Sync + 'static, Buf: BufSource + Send + Sync, - ::Output: Octets + Send + Sync + 'static + Unpin, - Svc: Clone - + Service<::Output, ()> - + Send - + Sync - + 'static, - ::Output, ()>>::Future: Send, - ::Output, ()>>::Stream: Send, - ::Output, ()>>::Target: Composer + Send, + Buf::Output: Octets + Send + Sync + Unpin + 'static, + Svc: Service + Clone + Send + Sync + 'static, + Svc::Future: Send, + Svc::Stream: Send, + Svc::Target: Composer + Default + Send, { /// Start the server. /// @@ -466,11 +470,11 @@ impl DgramServer where Sock: AsyncDgramSock + Send + Sync, Buf: BufSource + Send + Sync, - ::Output: Octets + Send + Sync + Unpin, - Svc: Clone + Service<::Output, ()> + Send + Sync, - ::Output, ()>>::Future: Send, - ::Output, ()>>::Stream: Send, - ::Output, ()>>::Target: Composer + Send, + Buf::Output: Octets + Send + Sync + Unpin + 'static, + Svc: Service + Clone + Send + Sync + 'static, + Svc::Future: Send, + Svc::Stream: Send, + Svc::Target: Composer + Default + Send, { /// Receive incoming messages until shutdown or fatal error. async fn run_until_error(&self) -> Result<(), String> { @@ -502,74 +506,27 @@ where trace!(%addr, pcap_text, "Received message"); } - let svc = self.service.clone(); - let cfg = self.config.clone(); - let metrics = self.metrics.clone(); - let cloned_sock = self.sock.clone(); - let write_timeout = self.config.load().write_timeout; - - tokio::spawn(async move { - match Message::from_octets(buf) { - Err(err) => { - tracing::warn!("Failed while parsing request message: {err}"); - } - - Ok(msg) => { - let ctx = UdpTransportContext::new(cfg.load().max_response_size); - let ctx = TransportSpecificContext::Udp(ctx); - let request = Request::new(addr, received_at, msg, ctx, ()); - let mut stream = svc.call(request).await; - while let Some(Ok(call_result)) = stream.next().await { - let (response, feedback) = call_result.into_inner(); - - if let Some(feedback) = feedback { - match feedback { - ServiceFeedback::Reconfigure { - idle_timeout: _, // N/A - only applies to connection-oriented transports - } => { - // Nothing to do. - } - - ServiceFeedback::BeginTransaction|ServiceFeedback::EndTransaction => { - // Nothing to do. - } - } - } - - // Process the DNS response message, if any. - if let Some(response) = response { - // Convert the DNS response message into bytes. - let target = response.finish(); - let bytes = target.as_dgram_slice(); - - // Logging - if enabled!(Level::TRACE) { - let pcap_text = to_pcap_text(bytes, bytes.len()); - trace!(%addr, pcap_text, "Sending response"); - } - - metrics.inc_num_pending_writes(); - - // Actually write the DNS response message bytes to the UDP - // socket. - if let Err(err) = Self::send_to( - &cloned_sock, - bytes, - &addr, - write_timeout, - ) - .await - { - warn!(%addr, "Failed to send response: {err}"); - } - - metrics.dec_num_pending_writes(); - metrics.inc_num_sent_responses(); - } - } - } + match Message::from_octets(buf) { + Err(err) => { + tracing::warn!("Failed while parsing request message: {err}"); } - }); + + Ok(msg) => { + let ctx = UdpTransportContext::new(self.config.load().max_response_size); + let ctx = TransportSpecificContext::Udp(ctx); + let request = Request::new(addr, received_at, msg, ctx, ()); + + trace!( + "Spawning task to handle new message with id {}", + request.message().header().id() + ); + + let mut dispatcher = self.request_dispatcher.clone(); + tokio::spawn(async move { + dispatcher.dispatch(request, addr).await + }); + } + } } } } @@ -636,6 +593,212 @@ where .try_recv_buf_from(&mut buf) .map(|(bytes_read, addr)| (msg, addr, bytes_read)) } +} + +//--- Drop + +impl Drop for DgramServer +where + Sock: AsyncDgramSock + Send + Sync + 'static, + Buf: BufSource + Send + Sync, + Buf::Output: Octets + Send + Sync + Unpin + 'static, + Svc: Service + Clone + Send + Sync + 'static, + Svc::Future: Send, + Svc::Stream: Send, + Svc::Target: Composer + Default + Send, +{ + fn drop(&mut self) { + // Shutdown the DgramServer. Don't handle the failure case here as + // I'm not sure if it's safe to log or write to stderr from a Drop + // impl. + let _ = self.shutdown(); + } +} +//------------ DispatcherStatus ----------------------------------------------- + +enum DispatcherStatus { + Normal, + + InTransaction, + + Aborting, +} + +//------------ RequestDispatcher ---------------------------------------------- + +struct RequestDispatcher +where + RequestOctets: Octets + Send + Sync, + Svc: Service + Clone + Send + Sync + 'static, + Svc::Target: Composer + Default + Send, +{ + /// User supplied settings that influence our behaviour. + /// + /// May updated during request and response processing based on received + /// [`ServiceFeedback`]. + config: Arc>, + + /// A [`Service`] for handling received requests and generating responses. + service: Svc, + + /// The network socket to which responses will be sent. + sock: Arc, + + /// [`ServerMetrics`] describing the status of the server. + metrics: Arc, + + status: DispatcherStatus, + + _phantom_data: PhantomData, +} + +impl RequestDispatcher +where + RequestOctets: Octets + Send + Sync, + Svc: Service + Clone + Send + Sync + 'static, + Svc::Target: Composer + Default + Send, +{ + fn clone(&self) -> Self { + Self { + config: self.config.clone(), + service: self.service.clone(), + sock: self.sock.clone(), + metrics: self.metrics.clone(), + status: DispatcherStatus::Normal, + _phantom_data: PhantomData, + } + } +} + +impl RequestDispatcher +where + Sock: AsyncDgramSock + Send + Sync + 'static, + RequestOctets: Octets + Send + Sync, + Svc: Service + Clone + Send + Sync + 'static, + Svc::Target: Composer + Default + Send, + Svc::Future: Send, + Svc::Stream: Send, +{ + fn new( + config: Arc>, + service: Svc, + sock: Arc, + metrics: Arc, + ) -> Self { + Self { + config, + service, + sock, + metrics, + status: DispatcherStatus::Normal, + _phantom_data: PhantomData, + } + } + + async fn dispatch( + &mut self, + request: Request, + addr: SocketAddr, + ) { + let req_msg = request.message().clone(); + let request_id = request.message().header().id(); + + // Dispatch the request to the service for processing. + trace!("Calling service for request id {request_id}"); + let mut stream = self.service.call(request).await; + + // Handle the resulting stream of responses. + // TODO: For UDP does it ever make sense to send multiple responses + // back to the client? + trace!("Awaiting service call results for request id {request_id}"); + while let Some(item) = stream.next().await { + trace!( + "Processing service call result for request id {request_id}" + ); + + let response = self.process_response_stream_item(item, &req_msg); + + if let Some(response) = response { + self.enqueue_response(addr, response).await; + } + + if matches!(self.status, DispatcherStatus::Aborting) { + trace!("Aborting response stream processing for request id {request_id}"); + break; + } + } + trace!("Finished processing service call results for request id {request_id}"); + } + + fn process_response_stream_item( + &mut self, + stream_item: ServiceResult, + req_msg: &Message, + ) -> Option>> { + match stream_item { + Ok(call_result) => { + let (response, feedback) = call_result.into_inner(); + if let Some(feedback) = feedback { + self.process_feedback(feedback); + } + response + } + + Err(err) => { + self.status = DispatcherStatus::Aborting; + Some(mk_error_response(req_msg, err.rcode().into())) + } + } + } + + fn process_feedback(&mut self, feedback: ServiceFeedback) { + match feedback { + ServiceFeedback::Reconfigure { + idle_timeout: _, // N/A - only applies to connection-oriented transports + } => { + // Nothing to do. + } + + ServiceFeedback::BeginTransaction => { + self.status = DispatcherStatus::InTransaction + } + + ServiceFeedback::EndTransaction => { + self.status = DispatcherStatus::Normal + } + } + } + + async fn enqueue_response( + &self, + addr: SocketAddr, + response: AdditionalBuilder>, + ) { + // Convert the DNS response message into bytes. + let target = response.finish(); + let bytes = target.as_dgram_slice(); + + // Logging + if enabled!(Level::TRACE) { + let pcap_text = to_pcap_text(bytes, bytes.len()); + trace!(%addr, pcap_text, "Sending response"); + } + + self.metrics.inc_num_pending_writes(); + + let write_timeout = self.config.load().write_timeout; + + // Actually write the DNS response message bytes to the UDP + // socket. + if let Err(err) = + Self::send_to(&self.sock, bytes, &addr, write_timeout).await + { + warn!(%addr, "Failed to send response: {err}"); + } + + self.metrics.dec_num_pending_writes(); + self.metrics.inc_num_sent_responses(); + } /// Send a single datagram using the user supplied network socket. async fn send_to( @@ -661,27 +824,3 @@ where } } } - -//--- Drop - -impl Drop for DgramServer -where - Sock: AsyncDgramSock + Send + Sync + 'static, - Buf: BufSource + Send + Sync, - ::Output: Octets + Send + Sync + Unpin + 'static, - Svc: Clone - + Service<::Output, ()> - + Send - + Sync - + 'static, - ::Output, ()>>::Future: Send, - ::Output, ()>>::Stream: Send, - ::Output, ()>>::Target: Composer + Send, -{ - fn drop(&mut self) { - // Shutdown the DgramServer. Don't handle the failure case here as - // I'm not sure if it's safe to log or write to stderr from a Drop - // impl. - let _ = self.shutdown(); - } -} diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index 5967c03bd..bc5198595 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -31,15 +31,15 @@ use futures::stream::{once, Once, Stream}; use octseq::{Octets, OctetsFrom}; use tracing::{error, trace, warn}; -use crate::base::iana::{Opcode, OptRcode, Rcode, TsigRcode}; +use crate::base::iana::{Opcode, Rcode, TsigRcode}; use crate::base::message_builder::AdditionalBuilder; use crate::base::wire::Composer; use crate::base::{Message, ParsedName, Question, Rtype, StreamTarget}; use crate::net::server::message::Request; use crate::net::server::service::{ - CallResult, Service, ServiceFeedback, ServiceResult, + CallResult, Service, ServiceError, ServiceFeedback, ServiceResult, }; -use crate::net::server::util::{mk_builder_for_target, mk_error_response}; +use crate::net::server::util::mk_builder_for_target; use crate::rdata::tsig::Time48; use crate::tsig::{self, KeyStore, ServerSequence, ServerTransaction}; @@ -175,17 +175,17 @@ where request: &Request, response: &mut AdditionalBuilder>, pp_config: &mut PostprocessingConfig, - ) -> Option>> { + ) -> Result< + Option>>, + ServiceError, + > { // Remove the limit we should have imposed during pre-processing so // that we can use the space we reserved for the OPT RR. response.clear_push_limit(); - // The variable itself isn't used by a reference to its interior value - // *is* used if the if signing_result.is_err() block below. - #[allow(unused_assignments)] - let mut key_for_err_handling = None; + let truncation_ctx; - let (signing_result, key) = match &mut pp_config.signer { + let res = match &mut pp_config.signer { Some(TsigSigner::Transaction(_)) => { // Extract the single response signer and consume it in the // signing process. @@ -206,11 +206,11 @@ where // over the key type via KS::Key so if cloning cost is a // problem the caller can choose to wrap the key in an Arc or // such to reduce the cloning cost. - key_for_err_handling = Some(signer.key().clone()); - - let res = signer.answer(response, Time48::now()); + truncation_ctx = TruncationContext::NoSignerOnlyTheKey( + signer.key().clone(), + ); - (res, key_for_err_handling.as_ref().unwrap()) + signer.answer(response, Time48::now()) } Some(TsigSigner::Sequence(ref mut signer)) => { @@ -219,21 +219,26 @@ where "Signing response stream with TSIG key '{}'", signer.key().name() ); + let res = signer.answer(response, Time48::now()); - (res, signer.key()) + truncation_ctx = TruncationContext::HaveSigner(signer); + + res } None => { // Nothing to do as unsigned requests don't require response // signing. - return None; + return Ok(None); } }; - // Handle signing failure. This shouldn't happen because we reserve - // space in preprocess() for the TSIG RR that we add when signing. - if signing_result.is_err() { + // Handle signing failure due to push error, i.e. there wasn't enough + // space in the response to add the TSIG RR. This shouldn't happen + // because we reserve space in preprocess() for the TSIG RR that we + // add when signing. + if res.is_err() { // 5.3. Generation of TSIG on Answers // "If addition of the TSIG record will cause the message to be // truncated, the server MUST alter the response so that a TSIG @@ -241,74 +246,73 @@ where // a TSIG record, has the TC bit set, and has an RCODE of 0 // (NOERROR). At this point, the client SHOULD retry the request // using TCP (as per Section 4.2.2 of [RFC1035])." - - // We can't use the TSIG signer state we just had as that was consumed - // in the failed attempt to sign the answer, so we have to create a new - // TSIG state in order to sign the truncated response. - if request.transport_ctx().is_udp() { - return Self::mk_signed_truncated_response(request, key); - } else { - // In the TCP case there's not much we can do. The upstream - // service pushes response messages into the stream and we try - // and sign them. If there isn't enough space to add the TSIG - // signature RR to the response we can't signal the upstream - // to try again to produce a smaller response message as it - // may already have finished pushing into the stream or be - // several messages further on in its processsing. We also - // can't edit the response message content ourselves as we - // know nothing about the content. The only option left to us - // is to try and truncate the TSIG MAC and see if that helps, - // but we don't support that (yet? NSD doesn't support it - // either). - return Some(mk_error_response( - request.message(), - OptRcode::SERVFAIL, - )); - } + Ok(Some(Self::mk_signed_truncated_response( + request, + truncation_ctx, + )?)) + } else { + Ok(None) } - - None } fn mk_signed_truncated_response( request: &Request, - key: &tsig::Key, - ) -> Option>> { - let octets = request.message().as_slice().to_vec(); - let mut mut_msg = Message::from_octets(octets).unwrap(); - let res = - ServerTransaction::request(&key, &mut mut_msg, Time48::now()); - - match res { - Ok(None) => { - warn!("Ignoring attempt to create a signed truncated response for an unsigned request."); - None - } - - Ok(Some(tsig)) => { - let builder = mk_builder_for_target(); - let mut new_response = builder - .start_answer(request.message(), Rcode::NOERROR) - .unwrap(); - new_response.header_mut().set_tc(true); - let mut new_response = new_response.additional(); - + truncation_ctx: TruncationContext, + ) -> Result>, ServiceError> + { + let builder = mk_builder_for_target(); + let mut new_response = builder + .start_answer(request.message(), Rcode::NOERROR) + .unwrap(); + new_response.header_mut().set_tc(true); + let mut additional = new_response.additional(); + + match truncation_ctx { + TruncationContext::HaveSigner(signer) => { if let Err(err) = - tsig.answer(&mut new_response, Time48::now()) + signer.answer(&mut additional, Time48::now()) { error!("Unable to sign truncated TSIG response: {err}"); - Some(mk_error_response( - request.message(), - OptRcode::SERVFAIL, - )) + Err(ServiceError::InternalError) } else { - Some(new_response) + Ok(additional) } } - Err(err) => { - error!("Unable to sign truncated TSIG response: {err}"); - Some(mk_error_response(request.message(), OptRcode::SERVFAIL)) + TruncationContext::NoSignerOnlyTheKey(key) => { + // We can't use the TSIG signer state we just had as that was + // consumed in the failed attempt to sign the answer, so we + // have to create a new TSIG state in order to sign the + // truncated response. + let octets = request.message().as_slice().to_vec(); + let mut mut_msg = Message::from_octets(octets).unwrap(); + + match ServerTransaction::request( + &key, + &mut mut_msg, + Time48::now(), + ) { + Ok(None) => { + error!("Unable to create signer for truncated TSIG response: internal error: request is not signed but was expected to be"); + Err(ServiceError::InternalError) + } + + Err(err) => { + error!("Unable to create signer for truncated TSIG response: {err}"); + Err(ServiceError::InternalError) + } + + Ok(Some(signer)) => { + if let Err(err) = + signer.answer(&mut additional, Time48::now()) + { + error!("Unable to sign truncated TSIG response: {err}"); + Err(ServiceError::InternalError) + } else { + Ok(additional) + } + } + } } } } @@ -371,7 +375,7 @@ where if let Some(response) = call_res.response_mut() { if let Some(new_response) = - Self::postprocess(&request, response, pp_config) + Self::postprocess(&request, response, pp_config)? { *response = new_response; } @@ -495,3 +499,11 @@ enum TsigSigner { /// A [`ServerSequence`] for signing multiple responses. Sequence(ServerSequence), } + +//------------ TruncationContext ---------------------------------------------- + +enum TruncationContext<'a, KSeq, KTxn> { + HaveSigner(&'a mut tsig::ServerSequence), + + NoSignerOnlyTheKey(KTxn), +} From 9251700150dd9bacd0fcfa35a76afe44ad6533b4 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 13 Sep 2024 10:24:24 +0200 Subject: [PATCH 171/333] Revert changes to network servers, will move them to a separate PR. --- src/net/server/connection.rs | 353 +++++++++++-------------------- src/net/server/dgram.rs | 399 ++++++++++++----------------------- 2 files changed, 254 insertions(+), 498 deletions(-) diff --git a/src/net/server/connection.rs b/src/net/server/connection.rs index f824c2fa6..a805b1b17 100644 --- a/src/net/server/connection.rs +++ b/src/net/server/connection.rs @@ -27,11 +27,10 @@ use crate::net::server::buf::BufSource; use crate::net::server::message::Request; use crate::net::server::metrics::ServerMetrics; use crate::net::server::service::{Service, ServiceError, ServiceFeedback}; -use crate::net::server::util::{mk_error_response, to_pcap_text}; +use crate::net::server::util::to_pcap_text; use crate::utils::config::DefMinMax; use super::message::{NonUdpTransportContext, TransportSpecificContext}; -use super::service::ServiceResult; use super::stream::Config as ServerConfig; use super::ServerCommand; @@ -222,10 +221,9 @@ impl Clone for Config { /// A handler for a single stream connection between client and server. pub struct Connection where - Buf: BufSource + Clone + Send + Sync + 'static, - Buf::Output: Octets + Send + Sync + Unpin, - Svc: Service + Clone + Send + Sync + 'static, - Svc::Target: Composer + Default + Send, + Buf: BufSource, + Buf::Output: Send + Sync + Unpin, + Svc: Service + Clone, { /// Flag used by the Drop impl to track if the metric count has to be /// decreased or not. @@ -260,14 +258,14 @@ where /// to be written back the client. result_q_tx: mpsc::Sender>>, + /// A [`Service`] for handling received requests and generating responses. + service: Svc, + /// DNS protocol idle time out tracking. idle_timer: IdleTimer, /// [`ServerMetrics`] describing the status of the server. metrics: Arc, - - /// Dispatches requests to the service and enqueues responses for sending. - request_dispatcher: RequestDispatcher, } /// Creation @@ -275,12 +273,9 @@ where impl Connection where Stream: AsyncRead + AsyncWrite, - Buf: BufSource + Clone + Send + Sync, + Buf: BufSource, Buf::Output: Octets + Send + Sync + Unpin, - Svc: Service + Clone + Send + Sync, - Svc::Target: Composer + Default + Send, - Svc::Stream: Send, - Svc::Future: Send, + Svc: Service + Clone, { /// Creates a new handler for an accepted stream connection. #[must_use] @@ -327,13 +322,6 @@ where // uses of self we have to do while running. let stream_rx = Some(stream_rx); - let request_dispatcher = RequestDispatcher::new( - config.clone(), - service, - result_q_tx.clone(), - metrics.clone(), - ); - Self { active: false, buf, @@ -343,9 +331,9 @@ where stream_tx, result_q_rx, result_q_tx, + service, idle_timer, metrics, - request_dispatcher, } } } @@ -358,9 +346,8 @@ where Buf: BufSource + Send + Sync + Clone + 'static, Buf::Output: Octets + Send + Sync + Unpin, Svc: Service + Clone + Send + Sync + 'static, - Svc::Target: Composer + Default + Send, + Svc::Target: Composer + Send, Svc::Stream: Send, - Svc::Future: Send, { /// Start reading requests and writing responses to the stream. /// @@ -376,7 +363,9 @@ where pub async fn run( mut self, command_rx: watch::Receiver>, - ) { + ) where + Svc::Future: Send, + { self.metrics.inc_num_connections(); // Flag that we have to decrease the metric count on Drop. @@ -394,7 +383,7 @@ where Buf: BufSource + Send + Sync + Clone + 'static, Buf::Output: Octets + Send + Sync + Unpin, Svc: Service + Clone + Send + Sync + 'static, - Svc::Target: Composer + Default + Send, + Svc::Target: Composer + Send, Svc::Future: Send, Svc::Stream: Send, { @@ -547,7 +536,10 @@ where } /// Stop queueing new responses and process those already in the queue. - async fn flush_write_queue(&mut self) { + async fn flush_write_queue(&mut self) + // where + // Target: Composer, + { debug!("Flushing connection write queue."); // Stop accepting new response messages (should we check for in-flight // messages that haven't generated a response yet but should be @@ -572,7 +564,10 @@ where async fn process_queued_result( &mut self, response: Option>>, - ) -> Result<(), ConnectionEvent> { + ) -> Result<(), ConnectionEvent> +// where + // Target: Composer, + { // If we failed to read the results of requests processed by the // service because the queue holding those results is empty and can no // longer be read from, then there is no point continuing to read from @@ -597,7 +592,10 @@ where async fn write_response_to_stream( &mut self, msg: StreamTarget, - ) { + ) + // where + // Target: AsRef<[u8]>, + { if enabled!(Level::TRACE) { let bytes = msg.as_dgram_slice(); let pcap_text = to_pcap_text(bytes, bytes.len()); @@ -655,7 +653,6 @@ where ) -> Result<(), ConnectionEvent> where Svc::Stream: Send, - Svc::Target: Default, { match res { Ok(buf) => { @@ -685,23 +682,112 @@ where let ctx = NonUdpTransportContext::new(Some( self.config.load().idle_timeout, )); - + let ctx = TransportSpecificContext::NonUdp(ctx); let request = Request::new( self.addr, received_at, msg, - TransportSpecificContext::NonUdp(ctx), + ctx, (), ); + let svc = self.service.clone(); + let result_q_tx = self.result_q_tx.clone(); + let metrics = self.metrics.clone(); + let config = self.config.clone(); + trace!( "Spawning task to handle new message with id {}", request.message().header().id() ); - - let mut dispatcher = self.request_dispatcher.clone(); tokio::spawn(async move { - dispatcher.dispatch(request).await + let request_id = request.message().header().id(); + trace!( + "Calling service for request id {request_id}" + ); + let mut stream = svc.call(request).await; + let mut in_transaction = false; + + trace!("Awaiting service call results for request id {request_id}"); + while let Some(Ok(call_result)) = + stream.next().await + { + trace!("Processing service call result for request id {request_id}"); + let (response, feedback) = + call_result.into_inner(); + + if let Some(feedback) = feedback { + match feedback { + ServiceFeedback::Reconfigure { + idle_timeout, + } => { + if let Some(idle_timeout) = + idle_timeout + { + debug!( + "Reconfigured connection timeout to {idle_timeout:?}" + ); + let guard = config.load(); + let mut new_config = **guard; + new_config.idle_timeout = + idle_timeout; + config.store(Arc::new( + new_config, + )); + } + } + + ServiceFeedback::BeginTransaction => { + in_transaction = true; + } + + ServiceFeedback::EndTransaction => { + in_transaction = false; + } + } + } + + if let Some(mut response) = response { + loop { + match result_q_tx.try_send(response) { + Ok(()) => { + let pending_writes = + result_q_tx + .max_capacity() + - result_q_tx + .capacity(); + trace!("Queued message for sending: # pending writes={pending_writes}"); + metrics + .set_num_pending_writes( + pending_writes, + ); + break; + } + + Err(TrySendError::Closed(_)) => { + error!("Unable to queue message for sending: server is shutting down."); + break; + } + + Err(TrySendError::Full( + unused_response, + )) => { + if in_transaction { + // Wait until there is space in the message queue. + tokio::task::yield_now() + .await; + response = + unused_response; + } else { + error!("Unable to queue message for sending: queue is full."); + break; + } + } + } + } + } + } + trace!("Finished processing service call results for request id {request_id}"); }); } } @@ -718,10 +804,9 @@ where impl Drop for Connection where - Buf: BufSource + Clone + Send + Sync, - Buf::Output: Octets + Send + Sync + Unpin, - Svc: Service + Clone + Send + Sync, - Svc::Target: Composer + Default + Send, + Buf: BufSource, + Buf::Output: Send + Sync + Unpin, + Svc: Service + Clone, { fn drop(&mut self) { if self.active { @@ -990,193 +1075,3 @@ impl IdleTimer { self.reset_idle_timer() } } -//------------ DispatcherStatus ----------------------------------------------- - -enum DispatcherStatus { - Normal, - - InTransaction, - - Aborting, -} - -//------------ RequestDispatcher ---------------------------------------------- - -struct RequestDispatcher -where - RequestOctets: Octets + Send + Sync, - Svc: Service + Clone + Send + Sync + 'static, - Svc::Target: Composer + Default + Send, -{ - /// User supplied settings that influence our behaviour. - /// - /// May updated during request and response processing based on received - /// [`ServiceFeedback`]. - config: Arc>, - - /// A [`Service`] for handling received requests and generating responses. - service: Svc, - - /// The writer for pushing ready responses onto the queue waiting - /// to be written back the client. - result_q_tx: mpsc::Sender>>, - - /// [`ServerMetrics`] describing the status of the server. - metrics: Arc, - - status: DispatcherStatus, -} - -impl Clone for RequestDispatcher -where - RequestOctets: Octets + Send + Sync, - Svc: Service + Clone + Send + Sync + 'static, - Svc::Target: Composer + Default + Send, -{ - fn clone(&self) -> Self { - Self { - config: self.config.clone(), - service: self.service.clone(), - result_q_tx: self.result_q_tx.clone(), - metrics: self.metrics.clone(), - status: DispatcherStatus::Normal, - } - } -} - -impl RequestDispatcher -where - RequestOctets: Octets + Send + Sync, - Svc: Service + Clone + Send + Sync + 'static, - Svc::Target: Composer + Default + Send, - Svc::Future: Send, - Svc::Stream: Send, -{ - fn new( - config: Arc>, - service: Svc, - result_q_tx: mpsc::Sender< - AdditionalBuilder>, - >, - metrics: Arc, - ) -> Self { - Self { - config, - service, - result_q_tx, - metrics, - status: DispatcherStatus::Normal, - } - } - - async fn dispatch(&mut self, request: Request) { - let req_msg = request.message().clone(); - let request_id = request.message().header().id(); - - // Dispatch the request to the service for processing. - trace!("Calling service for request id {request_id}"); - let mut stream = self.service.call(request).await; - - // Handle the resulting stream of responses, most likely just one as - // only XFR requests potentially result in multiple responses. - trace!("Awaiting service call results for request id {request_id}"); - while let Some(item) = stream.next().await { - trace!( - "Processing service call result for request id {request_id}" - ); - - let response = self.process_response_stream_item(item, &req_msg); - - if let Some(response) = response { - self.enqueue_response(response).await; - } - - if matches!(self.status, DispatcherStatus::Aborting) { - trace!("Aborting response stream processing for request id {request_id}"); - break; - } - } - trace!("Finished processing service call results for request id {request_id}"); - } - - fn process_response_stream_item( - &mut self, - stream_item: ServiceResult, - req_msg: &Message, - ) -> Option>> { - match stream_item { - Ok(call_result) => { - let (response, feedback) = call_result.into_inner(); - if let Some(feedback) = feedback { - self.process_feedback(feedback); - } - response - } - - Err(err) => { - self.status = DispatcherStatus::Aborting; - Some(mk_error_response(req_msg, err.rcode().into())) - } - } - } - - fn process_feedback(&mut self, feedback: ServiceFeedback) { - match feedback { - ServiceFeedback::Reconfigure { idle_timeout } => { - self.update_config(idle_timeout); - } - - ServiceFeedback::BeginTransaction => { - self.status = DispatcherStatus::InTransaction - } - - ServiceFeedback::EndTransaction => { - self.status = DispatcherStatus::Normal - } - } - } - - fn update_config(&self, idle_timeout: Option) { - if let Some(idle_timeout) = idle_timeout { - debug!("Reconfigured connection timeout to {idle_timeout:?}"); - let guard = self.config.load(); - let mut new_config = **guard; - new_config.idle_timeout = idle_timeout; - self.config.store(Arc::new(new_config)); - } - } - - async fn enqueue_response( - &self, - mut response: AdditionalBuilder>, - ) { - loop { - match self.result_q_tx.try_send(response) { - Ok(()) => { - let pending_writes = self.result_q_tx.max_capacity() - - self.result_q_tx.capacity(); - trace!("Queued message for sending: # pending writes={pending_writes}"); - self.metrics.set_num_pending_writes(pending_writes); - break; - } - - Err(TrySendError::Closed(_)) => { - error!("Unable to queue message for sending: server is shutting down."); - break; - } - - Err(TrySendError::Full(unused_response)) => { - if matches!(self.status, DispatcherStatus::InTransaction) - { - // Wait until there is space in the message queue. - tokio::task::yield_now().await; - response = unused_response; - } else { - error!("Unable to queue message for sending: queue is full."); - break; - } - } - } - } - } -} diff --git a/src/net/server/dgram.rs b/src/net/server/dgram.rs index 152f8172f..752b8f4ca 100644 --- a/src/net/server/dgram.rs +++ b/src/net/server/dgram.rs @@ -11,7 +11,6 @@ //! [Datagram]: https://en.wikipedia.org/wiki/Datagram use core::fmt::Debug; use core::future::poll_fn; -use core::marker::PhantomData; use core::ops::Deref; use core::time::Duration; @@ -35,9 +34,8 @@ use tracing::warn; use tracing::Level; use tracing::{enabled, error, trace}; -use crate::base::message_builder::AdditionalBuilder; use crate::base::wire::Composer; -use crate::base::{Message, StreamTarget}; +use crate::base::Message; use crate::net::server::buf::BufSource; use crate::net::server::error::Error; use crate::net::server::message::Request; @@ -49,8 +47,6 @@ use crate::utils::config::DefMinMax; use super::buf::VecBufSource; use super::message::{TransportSpecificContext, UdpTransportContext}; -use super::service::ServiceResult; -use super::util::mk_error_response; use super::ServerCommand; /// A UDP transport based DNS server transport. @@ -256,11 +252,15 @@ pub struct DgramServer where Sock: AsyncDgramSock + Send + Sync + 'static, Buf: BufSource + Send + Sync, - Buf::Output: Octets + Send + Sync + Unpin + 'static, - Svc: Service + Clone + Send + Sync + 'static, - Svc::Future: Send, - Svc::Stream: Send, - Svc::Target: Composer + Default + Send, + ::Output: Octets + Send + Sync + Unpin + 'static, + Svc: Clone + + Service<::Output, ()> + + Send + + Sync + + 'static, + ::Output, ()>>::Future: Send, + ::Output, ()>>::Stream: Send, + ::Output, ()>>::Target: Composer + Send, { /// The configuration of the server. config: Arc>, @@ -283,11 +283,11 @@ where /// A [`BufSource`] for creating buffers on demand. buf: Buf, + /// A [`Service`] for handling received requests and generating responses. + service: Svc, + /// [`ServerMetrics`] describing the status of the server. metrics: Arc, - - /// Dispatches requests to the service and enqueues responses for sending. - request_dispatcher: RequestDispatcher, } /// Creation @@ -296,11 +296,11 @@ impl DgramServer where Sock: AsyncDgramSock + Send + Sync, Buf: BufSource + Send + Sync, - Buf::Output: Octets + Send + Sync + Unpin, - Svc: Service + Clone + Send + Sync + 'static, - Svc::Future: Send, - Svc::Stream: Send, - Svc::Target: Composer + Default + Send, + ::Output: Octets + Send + Sync + Unpin, + Svc: Clone + Service<::Output, ()> + Send + Sync, + ::Output, ()>>::Future: Send, + ::Output, ()>>::Stream: Send, + ::Output, ()>>::Target: Composer + Send, { /// Constructs a new [`DgramServer`] with default configuration. /// @@ -334,23 +334,15 @@ where let command_tx = Arc::new(Mutex::new(command_tx)); let metrics = Arc::new(ServerMetrics::connection_less()); let config = Arc::new(ArcSwap::from_pointee(config)); - let sock = Arc::new(sock); - - let request_dispatcher = RequestDispatcher::new( - config.clone(), - service, - sock.clone(), - metrics.clone(), - ); DgramServer { config, command_tx, command_rx, - sock, + sock: sock.into(), buf, + service, metrics, - request_dispatcher, } } } @@ -361,11 +353,11 @@ impl DgramServer where Sock: AsyncDgramSock + Send + Sync, Buf: BufSource + Send + Sync, - Buf::Output: Octets + Send + Sync + Unpin, - Svc: Service + Clone + Send + Sync + 'static, - Svc::Future: Send, - Svc::Stream: Send, - Svc::Target: Composer + Default + Send, + ::Output: Octets + Send + Sync + Unpin, + Svc: Clone + Service<::Output, ()> + Send + Sync, + ::Output, ()>>::Future: Send, + ::Output, ()>>::Stream: Send, + ::Output, ()>>::Target: Composer + Send, { /// Get a reference to the network source being used to receive messages. #[must_use] @@ -386,11 +378,15 @@ impl DgramServer where Sock: AsyncDgramSock + Send + Sync + 'static, Buf: BufSource + Send + Sync, - Buf::Output: Octets + Send + Sync + Unpin + 'static, - Svc: Service + Clone + Send + Sync + 'static, - Svc::Future: Send, - Svc::Stream: Send, - Svc::Target: Composer + Default + Send, + ::Output: Octets + Send + Sync + 'static + Unpin, + Svc: Clone + + Service<::Output, ()> + + Send + + Sync + + 'static, + ::Output, ()>>::Future: Send, + ::Output, ()>>::Stream: Send, + ::Output, ()>>::Target: Composer + Send, { /// Start the server. /// @@ -470,11 +466,11 @@ impl DgramServer where Sock: AsyncDgramSock + Send + Sync, Buf: BufSource + Send + Sync, - Buf::Output: Octets + Send + Sync + Unpin + 'static, - Svc: Service + Clone + Send + Sync + 'static, - Svc::Future: Send, - Svc::Stream: Send, - Svc::Target: Composer + Default + Send, + ::Output: Octets + Send + Sync + Unpin, + Svc: Clone + Service<::Output, ()> + Send + Sync, + ::Output, ()>>::Future: Send, + ::Output, ()>>::Stream: Send, + ::Output, ()>>::Target: Composer + Send, { /// Receive incoming messages until shutdown or fatal error. async fn run_until_error(&self) -> Result<(), String> { @@ -506,27 +502,74 @@ where trace!(%addr, pcap_text, "Received message"); } - match Message::from_octets(buf) { - Err(err) => { - tracing::warn!("Failed while parsing request message: {err}"); + let svc = self.service.clone(); + let cfg = self.config.clone(); + let metrics = self.metrics.clone(); + let cloned_sock = self.sock.clone(); + let write_timeout = self.config.load().write_timeout; + + tokio::spawn(async move { + match Message::from_octets(buf) { + Err(err) => { + tracing::warn!("Failed while parsing request message: {err}"); + } + + Ok(msg) => { + let ctx = UdpTransportContext::new(cfg.load().max_response_size); + let ctx = TransportSpecificContext::Udp(ctx); + let request = Request::new(addr, received_at, msg, ctx, ()); + let mut stream = svc.call(request).await; + while let Some(Ok(call_result)) = stream.next().await { + let (response, feedback) = call_result.into_inner(); + + if let Some(feedback) = feedback { + match feedback { + ServiceFeedback::Reconfigure { + idle_timeout: _, // N/A - only applies to connection-oriented transports + } => { + // Nothing to do. + } + + ServiceFeedback::BeginTransaction|ServiceFeedback::EndTransaction => { + // Nothing to do. + } + } + } + + // Process the DNS response message, if any. + if let Some(response) = response { + // Convert the DNS response message into bytes. + let target = response.finish(); + let bytes = target.as_dgram_slice(); + + // Logging + if enabled!(Level::TRACE) { + let pcap_text = to_pcap_text(bytes, bytes.len()); + trace!(%addr, pcap_text, "Sending response"); + } + + metrics.inc_num_pending_writes(); + + // Actually write the DNS response message bytes to the UDP + // socket. + if let Err(err) = Self::send_to( + &cloned_sock, + bytes, + &addr, + write_timeout, + ) + .await + { + warn!(%addr, "Failed to send response: {err}"); + } + + metrics.dec_num_pending_writes(); + metrics.inc_num_sent_responses(); + } + } + } } - - Ok(msg) => { - let ctx = UdpTransportContext::new(self.config.load().max_response_size); - let ctx = TransportSpecificContext::Udp(ctx); - let request = Request::new(addr, received_at, msg, ctx, ()); - - trace!( - "Spawning task to handle new message with id {}", - request.message().header().id() - ); - - let mut dispatcher = self.request_dispatcher.clone(); - tokio::spawn(async move { - dispatcher.dispatch(request, addr).await - }); - } - } + }); } } } @@ -593,212 +636,6 @@ where .try_recv_buf_from(&mut buf) .map(|(bytes_read, addr)| (msg, addr, bytes_read)) } -} - -//--- Drop - -impl Drop for DgramServer -where - Sock: AsyncDgramSock + Send + Sync + 'static, - Buf: BufSource + Send + Sync, - Buf::Output: Octets + Send + Sync + Unpin + 'static, - Svc: Service + Clone + Send + Sync + 'static, - Svc::Future: Send, - Svc::Stream: Send, - Svc::Target: Composer + Default + Send, -{ - fn drop(&mut self) { - // Shutdown the DgramServer. Don't handle the failure case here as - // I'm not sure if it's safe to log or write to stderr from a Drop - // impl. - let _ = self.shutdown(); - } -} -//------------ DispatcherStatus ----------------------------------------------- - -enum DispatcherStatus { - Normal, - - InTransaction, - - Aborting, -} - -//------------ RequestDispatcher ---------------------------------------------- - -struct RequestDispatcher -where - RequestOctets: Octets + Send + Sync, - Svc: Service + Clone + Send + Sync + 'static, - Svc::Target: Composer + Default + Send, -{ - /// User supplied settings that influence our behaviour. - /// - /// May updated during request and response processing based on received - /// [`ServiceFeedback`]. - config: Arc>, - - /// A [`Service`] for handling received requests and generating responses. - service: Svc, - - /// The network socket to which responses will be sent. - sock: Arc, - - /// [`ServerMetrics`] describing the status of the server. - metrics: Arc, - - status: DispatcherStatus, - - _phantom_data: PhantomData, -} - -impl RequestDispatcher -where - RequestOctets: Octets + Send + Sync, - Svc: Service + Clone + Send + Sync + 'static, - Svc::Target: Composer + Default + Send, -{ - fn clone(&self) -> Self { - Self { - config: self.config.clone(), - service: self.service.clone(), - sock: self.sock.clone(), - metrics: self.metrics.clone(), - status: DispatcherStatus::Normal, - _phantom_data: PhantomData, - } - } -} - -impl RequestDispatcher -where - Sock: AsyncDgramSock + Send + Sync + 'static, - RequestOctets: Octets + Send + Sync, - Svc: Service + Clone + Send + Sync + 'static, - Svc::Target: Composer + Default + Send, - Svc::Future: Send, - Svc::Stream: Send, -{ - fn new( - config: Arc>, - service: Svc, - sock: Arc, - metrics: Arc, - ) -> Self { - Self { - config, - service, - sock, - metrics, - status: DispatcherStatus::Normal, - _phantom_data: PhantomData, - } - } - - async fn dispatch( - &mut self, - request: Request, - addr: SocketAddr, - ) { - let req_msg = request.message().clone(); - let request_id = request.message().header().id(); - - // Dispatch the request to the service for processing. - trace!("Calling service for request id {request_id}"); - let mut stream = self.service.call(request).await; - - // Handle the resulting stream of responses. - // TODO: For UDP does it ever make sense to send multiple responses - // back to the client? - trace!("Awaiting service call results for request id {request_id}"); - while let Some(item) = stream.next().await { - trace!( - "Processing service call result for request id {request_id}" - ); - - let response = self.process_response_stream_item(item, &req_msg); - - if let Some(response) = response { - self.enqueue_response(addr, response).await; - } - - if matches!(self.status, DispatcherStatus::Aborting) { - trace!("Aborting response stream processing for request id {request_id}"); - break; - } - } - trace!("Finished processing service call results for request id {request_id}"); - } - - fn process_response_stream_item( - &mut self, - stream_item: ServiceResult, - req_msg: &Message, - ) -> Option>> { - match stream_item { - Ok(call_result) => { - let (response, feedback) = call_result.into_inner(); - if let Some(feedback) = feedback { - self.process_feedback(feedback); - } - response - } - - Err(err) => { - self.status = DispatcherStatus::Aborting; - Some(mk_error_response(req_msg, err.rcode().into())) - } - } - } - - fn process_feedback(&mut self, feedback: ServiceFeedback) { - match feedback { - ServiceFeedback::Reconfigure { - idle_timeout: _, // N/A - only applies to connection-oriented transports - } => { - // Nothing to do. - } - - ServiceFeedback::BeginTransaction => { - self.status = DispatcherStatus::InTransaction - } - - ServiceFeedback::EndTransaction => { - self.status = DispatcherStatus::Normal - } - } - } - - async fn enqueue_response( - &self, - addr: SocketAddr, - response: AdditionalBuilder>, - ) { - // Convert the DNS response message into bytes. - let target = response.finish(); - let bytes = target.as_dgram_slice(); - - // Logging - if enabled!(Level::TRACE) { - let pcap_text = to_pcap_text(bytes, bytes.len()); - trace!(%addr, pcap_text, "Sending response"); - } - - self.metrics.inc_num_pending_writes(); - - let write_timeout = self.config.load().write_timeout; - - // Actually write the DNS response message bytes to the UDP - // socket. - if let Err(err) = - Self::send_to(&self.sock, bytes, &addr, write_timeout).await - { - warn!(%addr, "Failed to send response: {err}"); - } - - self.metrics.dec_num_pending_writes(); - self.metrics.inc_num_sent_responses(); - } /// Send a single datagram using the user supplied network socket. async fn send_to( @@ -824,3 +661,27 @@ where } } } + +//--- Drop + +impl Drop for DgramServer +where + Sock: AsyncDgramSock + Send + Sync + 'static, + Buf: BufSource + Send + Sync, + ::Output: Octets + Send + Sync + Unpin + 'static, + Svc: Clone + + Service<::Output, ()> + + Send + + Sync + + 'static, + ::Output, ()>>::Future: Send, + ::Output, ()>>::Stream: Send, + ::Output, ()>>::Target: Composer + Send, +{ + fn drop(&mut self) { + // Shutdown the DgramServer. Don't handle the failure case here as + // I'm not sure if it's safe to log or write to stderr from a Drop + // impl. + let _ = self.shutdown(); + } +} From 9fbcf22f37b94b5a02935b6c12aaa0dcc20c2f92 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Sun, 15 Sep 2024 11:50:00 +0200 Subject: [PATCH 172/333] Make the reference to the other tsig module less confusing. --- src/net/server/middleware/tsig.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index bc5198595..7dc6e281c 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -2,7 +2,7 @@ //! //! This module provides a TSIG request validation and response signing //! middleware service. The underlying TSIG RR processing is implemented using -//! the [`tsig`] module. +//! the [`rdata::tsig`][crate::rdata::tsig] module. //! //! # Communicating which key signed a request. //! From cef732f6951bdf684bb8477e2948399fa12a651e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Sun, 15 Sep 2024 12:09:47 +0200 Subject: [PATCH 173/333] More RustDoc for the TSIG middleware service. --- src/net/server/middleware/tsig.rs | 38 +++++++++++++++++++++++++------ 1 file changed, 31 insertions(+), 7 deletions(-) diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index 7dc6e281c..f18f51565 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -4,14 +4,38 @@ //! middleware service. The underlying TSIG RR processing is implemented using //! the [`rdata::tsig`][crate::rdata::tsig] module. //! -//! # Communicating which key signed a request. +//! # Affected requests //! -//! For signed requests this middleware service passes the signing key to -//! upstream [`Service`] impls via request metadata. Upstream services can -//! choose to ignore the metadata by being generic over any kind of metadata, -//! or may offer a [`Service`] impl that specifically accepts the -//! [`Option`] metadata type. The upstream service is then able to -//! use the received metadata to learn which key the request was signed with. +//! Requests matching the following criteria will be affected by this +//! middleware: +//! +//! - Requests must have `Opcode::QUERY` in the header. +//! - The first question must have QTYPE `SOA`, `AXFR` or `IXFR`. +//! - The request must be signed, i.e. the last record of the additional +//! section of the request be a TSIG RR. +//! +//! If the request matches all of the above criteria it will only be allowed +//! through by the middleware if it has a valid TSIG signature. +//! +//! All other requests pass through this middleware unchanged. +//! +//! # Affected responses +//! +//! For requests which were correctly signed the response will be signed using +//! the same key as the request. +//! +//! All other responses pass through this middleware unchanged. +//! +//! # Determining the key that a request was signed with +//! +//! The key that signed a request is output by this middleware via the request +//! metadata in the form [`Option`], where `KS` denotes the type of +//! [`KeyStore`] that was used to construct this middleware. Upstream services +//! can choose to ignore the metadata by being generic over any kind of +//! metadata, or may offer a [`Service`] impl that specifically accepts the +//! [`Option`] metadata type, enabling the upstream service to use +//! the request metadata to determine the key that the request was signed +//! with. //! //! # Limitations //! From 0b417747e5662ac7f69b2583498cb7429a68a75f Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Sun, 15 Sep 2024 19:55:42 +0200 Subject: [PATCH 174/333] Fix compilation errors caused by merging with main. --- src/net/server/middleware/tsig.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index f18f51565..b457e268f 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -51,7 +51,6 @@ use core::ops::ControlFlow; use std::vec::Vec; -use futures::stream::{once, Once, Stream}; use octseq::{Octets, OctetsFrom}; use tracing::{error, trace, warn}; @@ -68,6 +67,8 @@ use crate::rdata::tsig::Time48; use crate::tsig::{self, KeyStore, ServerSequence, ServerTransaction}; use super::stream::{MiddlewareStream, PostprocessingStream}; +use futures_util::stream::{once, Once}; +use futures_util::Stream; //------------ TsigMiddlewareSvc ---------------------------------------------- From 73d7d7d74d0df7cde7c18c2b66744187138c7ee3 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 16 Sep 2024 10:21:12 +0200 Subject: [PATCH 175/333] Remove opcode/query type TSIG verification restrictions. --- src/net/server/middleware/tsig.rs | 161 +++++++++++------------------- 1 file changed, 60 insertions(+), 101 deletions(-) diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index b457e268f..f91e88413 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -4,27 +4,13 @@ //! middleware service. The underlying TSIG RR processing is implemented using //! the [`rdata::tsig`][crate::rdata::tsig] module. //! -//! # Affected requests +//! Signed requests thta fail signature verification will be rejected. //! -//! Requests matching the following criteria will be affected by this -//! middleware: +//! Unsigned requests and correctly signed requests will pass through this +//! middleware unchanged. //! -//! - Requests must have `Opcode::QUERY` in the header. -//! - The first question must have QTYPE `SOA`, `AXFR` or `IXFR`. -//! - The request must be signed, i.e. the last record of the additional -//! section of the request be a TSIG RR. -//! -//! If the request matches all of the above criteria it will only be allowed -//! through by the middleware if it has a valid TSIG signature. -//! -//! All other requests pass through this middleware unchanged. -//! -//! # Affected responses -//! -//! For requests which were correctly signed the response will be signed using -//! the same key as the request. -//! -//! All other responses pass through this middleware unchanged. +//! For requests which were correctly signed the corresponding response(s) +//! will be signed using the same key as the request. //! //! # Determining the key that a request was signed with //! @@ -54,10 +40,10 @@ use std::vec::Vec; use octseq::{Octets, OctetsFrom}; use tracing::{error, trace, warn}; -use crate::base::iana::{Opcode, Rcode, TsigRcode}; +use crate::base::iana::{Rcode, TsigRcode}; use crate::base::message_builder::AdditionalBuilder; use crate::base::wire::Composer; -use crate::base::{Message, ParsedName, Question, Rtype, StreamTarget}; +use crate::base::{Message, StreamTarget}; use crate::net::server::message::Request; use crate::net::server::service::{ CallResult, Service, ServiceError, ServiceFeedback, ServiceResult, @@ -129,65 +115,61 @@ where TsigSigner, )>, > { - if let Some(q) = Self::get_relevant_question(req.message()) { - let octets = req.message().as_slice().to_vec(); - let mut mut_msg = Message::from_octets(octets).unwrap(); - - match tsig::ServerTransaction::request( - key_store, - &mut mut_msg, - Time48::now(), - ) { - Ok(None) => { - // Message is not TSIG signed. - } + let octets = req.message().as_slice().to_vec(); + let mut mut_msg = Message::from_octets(octets).unwrap(); - Ok(Some(tsig)) => { - // Message is TSIG signed by a known key. - trace!( - "Request is signed with TSIG key '{}'", - tsig.key().name() - ); - - // Convert to RequestOctets so that the non-TSIG signed - // message case can just pass through the RequestOctets. - let source = mut_msg.into_octets(); - let octets = RequestOctets::octets_from(source); - let new_msg = Message::from_octets(octets).unwrap(); - - let mut new_req = Request::new( - req.client_addr(), - req.received_at(), - new_msg, - req.transport_ctx().clone(), - Some(tsig.key_wrapper().clone()), - ); - - let num_bytes_to_reserve = tsig.key().compose_len(); - new_req.reserve_bytes(num_bytes_to_reserve); - - return ControlFlow::Continue(Some(( - new_req, - TsigSigner::Transaction(tsig), - ))); - } + match tsig::ServerTransaction::request( + key_store, + &mut mut_msg, + Time48::now(), + ) { + Ok(None) => { + // Message is not TSIG signed. + } - Err(err) => { - // Message is incorrectly signed or signed with an unknown key. - warn!( - "{} for {} from {} refused: {err}", - q.qtype(), - q.qname(), - req.client_addr(), - ); - let builder = mk_builder_for_target(); - let additional = tsig::ServerError::::unsigned( - TsigRcode::BADKEY, - ) - .build_message(req.message(), builder) - .unwrap(); - return ControlFlow::Break(additional); - } + Ok(Some(tsig)) => { + // Message is TSIG signed by a known key. + trace!( + "Request is signed with TSIG key '{}'", + tsig.key().name() + ); + + // Convert to RequestOctets so that the non-TSIG signed + // message case can just pass through the RequestOctets. + let source = mut_msg.into_octets(); + let octets = RequestOctets::octets_from(source); + let new_msg = Message::from_octets(octets).unwrap(); + + let mut new_req = Request::new( + req.client_addr(), + req.received_at(), + new_msg, + req.transport_ctx().clone(), + Some(tsig.key_wrapper().clone()), + ); + + let num_bytes_to_reserve = tsig.key().compose_len(); + new_req.reserve_bytes(num_bytes_to_reserve); + + return ControlFlow::Continue(Some(( + new_req, + TsigSigner::Transaction(tsig), + ))); + } + + Err(err) => { + // Message is incorrectly signed or signed with an unknown key. + warn!( + "{} from {} refused: {err}", + req.message().header().opcode(), + req.client_addr(), + ); + let builder = mk_builder_for_target(); + let additional = + tsig::ServerError::::unsigned(TsigRcode::BADKEY) + .build_message(req.message(), builder) + .unwrap(); + return ControlFlow::Break(additional); } } @@ -342,29 +324,6 @@ where } } - fn get_relevant_question( - msg: &Message, - ) -> Option>>> { - if Opcode::QUERY == msg.header().opcode() && !msg.header().qr() { - if let Some(q) = msg.first_question() { - if matches!(q.qtype(), Rtype::SOA | Rtype::AXFR | Rtype::IXFR) - { - return Some(q); - } - } - } else if Opcode::NOTIFY == msg.header().opcode() - && !msg.header().qr() - { - if let Some(q) = msg.first_question() { - if matches!(q.qtype(), Rtype::SOA) { - return Some(q); - } - } - } - - None - } - fn map_stream_item( request: Request, stream_item: ServiceResult, From a0c62813dff906d80278c2cccf16bf6155a259c9 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 16 Sep 2024 10:52:47 +0200 Subject: [PATCH 176/333] Leave xfr response processing out of client-transporrts.rs, keep it focused on what it is supposed to show. --- examples/client-transports.rs | 8 -------- 1 file changed, 8 deletions(-) diff --git a/examples/client-transports.rs b/examples/client-transports.rs index 0e1eefd23..29bb0f117 100644 --- a/examples/client-transports.rs +++ b/examples/client-transports.rs @@ -337,8 +337,6 @@ where + Sync + 'static, { - use domain::net::xfr::processing::XfrResponseProcessor; - // Create a signing key. let key_name = KeyName::from_str("demo-key").unwrap(); let secret = domain::utils::base64::decode::>( @@ -358,7 +356,6 @@ where let mut request = tsig_conn.send_request(req); // Get the reply - let mut processor = XfrResponseProcessor::new(); loop { println!("Waiting for signed reply"); let reply = request.get_response() @@ -367,11 +364,6 @@ where match reply { Some(reply) => { println!("Signed reply: {:?}", reply); - let it = processor.process_answer(reply).unwrap(); - for event in it { - let event = event.unwrap(); - println!("XFR event: {event}"); - } } None => break, } From b1c75bc568c4116a0e6491487b7152eee22fec20 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 16 Sep 2024 11:02:19 +0200 Subject: [PATCH 177/333] Minor RustDoc language improvement. --- src/net/xfr/processing/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/xfr/processing/mod.rs b/src/net/xfr/processing/mod.rs index 91227d878..4bee6fea3 100644 --- a/src/net/xfr/processing/mod.rs +++ b/src/net/xfr/processing/mod.rs @@ -1,6 +1,6 @@ //! Parsing of AXFR/IXFR response messages for higher level processing. //! -//! This module provides [`XfrResponseProcessor`] which enables you to process +//! This module provides [`XfrResponseProcessor`] which can be used to process //! one or more AXFR/IXFR response messages in terms of the high level //! [`XfrEvent`]s that they represent without having to deal with the //! AXFR/IXFR protocol details. From f484929efaa4f1854444e1298876468d72617123 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 16 Sep 2024 11:07:09 +0200 Subject: [PATCH 178/333] Rename XfrResponseProcessor to XfrResponseInterpreter, as it doesn't do any processing of the data other than to translate it into a different higher level language, i.e. interpret it. --- src/net/xfr/processing/iterator.rs | 4 ++-- src/net/xfr/processing/mod.rs | 6 +++--- src/net/xfr/processing/processor.rs | 27 +++++++++++++-------------- src/net/xfr/processing/tests.rs | 20 ++++++++++---------- src/net/xfr/processing/types.rs | 12 ++++++------ 5 files changed, 34 insertions(+), 35 deletions(-) diff --git a/src/net/xfr/processing/iterator.rs b/src/net/xfr/processing/iterator.rs index 3cbc8b417..3026ca64b 100644 --- a/src/net/xfr/processing/iterator.rs +++ b/src/net/xfr/processing/iterator.rs @@ -12,9 +12,9 @@ use super::types::{IterationError, ProcessingError, XfrEvent, XfrRecord}; //------------ XfrEventIterator ----------------------------------------------- -/// An iterator over [`XfrResponseProcessor`] generated [`XfrEvent`]s. +/// An iterator over [`XfrResponseInterpreter`] generated [`XfrEvent`]s. /// -/// [`XfrResponseProcessor`]: super::processor::XfrResponseProcessor +/// [`XfrResponseInterpreter`]: super::processor::XfrResponseInterpreter pub struct XfrEventIterator<'a, 'b> { /// The parent processor. state: &'a mut RecordProcessor, diff --git a/src/net/xfr/processing/mod.rs b/src/net/xfr/processing/mod.rs index 4bee6fea3..91180307e 100644 --- a/src/net/xfr/processing/mod.rs +++ b/src/net/xfr/processing/mod.rs @@ -1,7 +1,7 @@ //! Parsing of AXFR/IXFR response messages for higher level processing. //! -//! This module provides [`XfrResponseProcessor`] which can be used to process -//! one or more AXFR/IXFR response messages in terms of the high level +//! This module provides [`XfrResponseInterpreter`] which can be used to +//! process one or more AXFR/IXFR response messages in terms of the high level //! [`XfrEvent`]s that they represent without having to deal with the //! AXFR/IXFR protocol details. mod iterator; @@ -12,5 +12,5 @@ mod types; mod tests; pub use iterator::XfrEventIterator; -pub use processor::XfrResponseProcessor; +pub use processor::XfrResponseInterpreter; pub use types::{IterationError, ProcessingError, XfrEvent, XfrRecord}; diff --git a/src/net/xfr/processing/processor.rs b/src/net/xfr/processing/processor.rs index 7df7a9303..30c75ebc2 100644 --- a/src/net/xfr/processing/processor.rs +++ b/src/net/xfr/processing/processor.rs @@ -11,18 +11,17 @@ use super::types::{ IxfrUpdateMode, ProcessingError, XfrEvent, XfrRecord, XfrType, }; -//------------ XfrResponseProcessor ------------------------------------------- +//------------ XfrResponseInterpreter ----------------------------------------- /// An AXFR/IXFR response processor. /// -/// Use [`XfrResponseProcessor`] to process a sequence of AXFR or IXFR -/// response messages into a corresponding sequence of high level -/// [`XfrEvent`]s. +/// Use [`XfrResponseInterpreter`] to interpret a sequence of AXFR or IXFR +/// response messages as a sequence of high level [`XfrEvent`]s. /// /// # Usage /// -/// For each response stream to be processed, construct an -/// [`XfrResponseProcessor`] for the corresponding XFR request message, then +/// For each response stream to be interpreted, construct an +/// [`XfrResponseInterpreter`] for the corresponding XFR request message, then /// pass each XFR response message to [`process_answer()`]. /// /// Each call to [`process_answer()`] will return an [`XfrEventIterator`] @@ -34,23 +33,23 @@ use super::types::{ /// that the sequence is incomplete and the next response message in the /// sequence should be passed to [`process_answer()`]. /// -/// [`process_answer()`]: XfrResponseProcessor::process_answer() +/// [`process_answer()`]: XfrResponseInterpreter::process_answer() #[derive(Default)] -pub struct XfrResponseProcessor { +pub struct XfrResponseInterpreter { /// Internal state. /// /// None until the first call to [`process_answer()`]. inner: Option, } -impl XfrResponseProcessor { +impl XfrResponseInterpreter { /// Creates a new XFR message processor. pub fn new() -> Self { Self::default() } } -impl XfrResponseProcessor { +impl XfrResponseInterpreter { /// Process a single AXFR/IXFR response message. /// /// Returns an [`XfrEventIterator`] over [`XfrEvent`]s emitted during @@ -84,7 +83,7 @@ impl XfrResponseProcessor { } } -impl XfrResponseProcessor { +impl XfrResponseInterpreter { /// Initialize inner state. fn initialize( &mut self, @@ -136,9 +135,9 @@ impl XfrResponseProcessor { //------------ Inner ---------------------------------------------------------- -/// Internal dynamic state of [`XfrResponseProcessor`]. +/// Internal dynamic state of [`XfrResponseInterpreter`]. /// -/// Separated out from [`XfrResponseProcessor`] to avoid needing multiple +/// Separated out from [`XfrResponseInterpreter`] to avoid needing multiple /// mutable self references in [`process_answer()`]. struct Inner { /// The response message currently being processed. @@ -260,7 +259,7 @@ impl RecordProcessor { // Note: We do NOT implement this MUST here because it would be very // inefficient to actually check that any received non-SOA RR has not // been seen before during the in-progress transfer. Clients of - // XfrResponseProcessor are better placed to enforce this rule if + // XfrResponseInterpreter are better placed to enforce this rule if // needed, e.g. at the moment of insertion into a zone tree checking // that the record is not already present or insertion of a duplicate // having no effect as it is already present. diff --git a/src/net/xfr/processing/tests.rs b/src/net/xfr/processing/tests.rs index eaf109933..b9bf86456 100644 --- a/src/net/xfr/processing/tests.rs +++ b/src/net/xfr/processing/tests.rs @@ -15,7 +15,7 @@ use crate::base::{ use crate::base::{Name, ToName}; use crate::rdata::{Soa, ZoneRecordData, A}; -use super::processor::XfrResponseProcessor; +use super::processor::XfrResponseInterpreter; use super::types::{ IterationError, ProcessingError, XfrEvent, XfrEvent as XE, XfrRecord, }; @@ -28,7 +28,7 @@ fn non_xfr_response_is_rejected() { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(); + let mut processor = XfrResponseInterpreter::new(); // Create a non-XFR response. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); @@ -51,7 +51,7 @@ fn axfr_response_with_no_answers_is_rejected() { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(); + let mut processor = XfrResponseInterpreter::new(); // Create a response that lacks answers. let resp = mk_empty_answer(&req, Rcode::NOERROR).into_message(); @@ -72,7 +72,7 @@ fn error_axfr_response_is_rejected() { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(); + let mut processor = XfrResponseInterpreter::new(); // Create a minimal valid AXFR response, just something that should // not be rejected by the XFR processor due to its content. It should @@ -97,7 +97,7 @@ fn incomplete_axfr_response_is_accepted() { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(); + let mut processor = XfrResponseInterpreter::new(); // Create an incomplete AXFR response. A proper AXFR response has at // least two identical SOA records, one at the start and one at the @@ -123,7 +123,7 @@ fn axfr_response_with_only_soas_is_accepted() { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(); + let mut processor = XfrResponseInterpreter::new(); // Create a complete but minimal AXFR response. A proper AXFR response // has at least two identical SOA records, one at the start and one at @@ -152,7 +152,7 @@ fn axfr_multi_response_with_only_soas_is_accepted() { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(); + let mut processor = XfrResponseInterpreter::new(); // Create a complete but minimal AXFR response. A proper AXFR response // has at least two identical SOA records, one at the start and one at @@ -191,7 +191,7 @@ fn axfr_response_generates_expected_events() { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(); + let mut processor = XfrResponseInterpreter::new(); // Create an AXFR response. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); @@ -227,7 +227,7 @@ fn ixfr_response_generates_expected_events() { let req = authority.into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(); + let mut processor = XfrResponseInterpreter::new(); // Prepare some serial numbers and SOA records to use in the IXFR response. let old_serial = client_serial; @@ -341,7 +341,7 @@ fn multi_ixfr_response_generates_expected_events() { let req = authority.into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(); + let mut processor = XfrResponseInterpreter::new(); // Prepare some serial numbers and SOA records to use in the IXFR response. let old_serial = client_serial; diff --git a/src/net/xfr/processing/types.rs b/src/net/xfr/processing/types.rs index f460589c4..bb6016c97 100644 --- a/src/net/xfr/processing/types.rs +++ b/src/net/xfr/processing/types.rs @@ -9,9 +9,9 @@ use crate::{ rdata::ZoneRecordData, }; -/// The type of record processed by [`XfrResponseProcessor`]. +/// The type of record processed by [`XfrResponseInterpreter`]. /// -/// [`XfrResponseProcessor`]: super::processor::XfrResponseProcessor +/// [`XfrResponseInterpreter`]: super::processor::XfrResponseInterpreter pub type XfrRecord = Record, ZoneRecordData>>; @@ -48,9 +48,9 @@ impl TryFrom for XfrType { //------------ XfrEvent ------------------------------------------------------- -/// An event emitted by [`XfrResponseProcessor`] during transfer processing. +/// An event emitted by [`XfrResponseInterpreter`] during transfer processing. /// -/// [`XfrResponseProcessor`]: super::processor::XfrResponseProcessor +/// [`XfrResponseInterpreter`]: super::processor::XfrResponseInterpreter #[derive(Clone, Debug, PartialEq, Eq)] pub enum XfrEvent { /// Delete record R in zone serial S. @@ -138,9 +138,9 @@ impl IxfrUpdateMode { //------------ ProcessingError ------------------------------------------------ -/// An error reported by [`XfrResponseProcessor`]. +/// An error reported by [`XfrResponseInterpreter`]. /// -/// [`XfrResponseProcessor`]: super::processor::XfrResponseProcessor +/// [`XfrResponseInterpreter`]: super::processor::XfrResponseInterpreter #[derive(Debug)] pub enum ProcessingError { /// The message could not be parsed. From 306dae6e81d6be1e61f5e90aba9209373b78ad2f Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 16 Sep 2024 11:12:46 +0200 Subject: [PATCH 179/333] More renaming. --- src/net/xfr/mod.rs | 4 ++-- .../xfr/{processing/processor.rs => protocol/interpreter.rs} | 0 src/net/xfr/{processing => protocol}/iterator.rs | 2 +- src/net/xfr/{processing => protocol}/mod.rs | 4 ++-- src/net/xfr/{processing => protocol}/tests.rs | 2 +- src/net/xfr/{processing => protocol}/types.rs | 0 6 files changed, 6 insertions(+), 6 deletions(-) rename src/net/xfr/{processing/processor.rs => protocol/interpreter.rs} (100%) rename src/net/xfr/{processing => protocol}/iterator.rs (98%) rename src/net/xfr/{processing => protocol}/mod.rs (88%) rename src/net/xfr/{processing => protocol}/tests.rs (99%) rename src/net/xfr/{processing => protocol}/types.rs (100%) diff --git a/src/net/xfr/mod.rs b/src/net/xfr/mod.rs index 5f6ec8604..22b877ab5 100644 --- a/src/net/xfr/mod.rs +++ b/src/net/xfr/mod.rs @@ -4,5 +4,5 @@ )] // #![warn(missing_docs)] // #![warn(clippy::missing_docs_in_private_items)] -//! Zone transfer related functionality. -pub mod processing; +//! XFR protocol related functionality. +pub mod protocol; diff --git a/src/net/xfr/processing/processor.rs b/src/net/xfr/protocol/interpreter.rs similarity index 100% rename from src/net/xfr/processing/processor.rs rename to src/net/xfr/protocol/interpreter.rs diff --git a/src/net/xfr/processing/iterator.rs b/src/net/xfr/protocol/iterator.rs similarity index 98% rename from src/net/xfr/processing/iterator.rs rename to src/net/xfr/protocol/iterator.rs index 3026ca64b..a28171004 100644 --- a/src/net/xfr/processing/iterator.rs +++ b/src/net/xfr/protocol/iterator.rs @@ -7,7 +7,7 @@ use crate::base::message::RecordIter; use crate::base::{Message, ParsedName}; use crate::rdata::ZoneRecordData; -use super::processor::RecordProcessor; +use super::interpreter::RecordProcessor; use super::types::{IterationError, ProcessingError, XfrEvent, XfrRecord}; //------------ XfrEventIterator ----------------------------------------------- diff --git a/src/net/xfr/processing/mod.rs b/src/net/xfr/protocol/mod.rs similarity index 88% rename from src/net/xfr/processing/mod.rs rename to src/net/xfr/protocol/mod.rs index 91180307e..50ffe82ad 100644 --- a/src/net/xfr/processing/mod.rs +++ b/src/net/xfr/protocol/mod.rs @@ -5,12 +5,12 @@ //! [`XfrEvent`]s that they represent without having to deal with the //! AXFR/IXFR protocol details. mod iterator; -mod processor; +mod interpreter; mod types; #[cfg(test)] mod tests; pub use iterator::XfrEventIterator; -pub use processor::XfrResponseInterpreter; +pub use interpreter::XfrResponseInterpreter; pub use types::{IterationError, ProcessingError, XfrEvent, XfrRecord}; diff --git a/src/net/xfr/processing/tests.rs b/src/net/xfr/protocol/tests.rs similarity index 99% rename from src/net/xfr/processing/tests.rs rename to src/net/xfr/protocol/tests.rs index b9bf86456..5049a9282 100644 --- a/src/net/xfr/processing/tests.rs +++ b/src/net/xfr/protocol/tests.rs @@ -15,7 +15,7 @@ use crate::base::{ use crate::base::{Name, ToName}; use crate::rdata::{Soa, ZoneRecordData, A}; -use super::processor::XfrResponseInterpreter; +use super::interpreter::XfrResponseInterpreter; use super::types::{ IterationError, ProcessingError, XfrEvent, XfrEvent as XE, XfrRecord, }; diff --git a/src/net/xfr/processing/types.rs b/src/net/xfr/protocol/types.rs similarity index 100% rename from src/net/xfr/processing/types.rs rename to src/net/xfr/protocol/types.rs From d009a5655aa7bccdfb4a31c2dfe6d06faff00d91 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 16 Sep 2024 11:33:51 +0200 Subject: [PATCH 180/333] Cargo fmt. --- src/net/xfr/protocol/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/net/xfr/protocol/mod.rs b/src/net/xfr/protocol/mod.rs index 50ffe82ad..b7855f7ef 100644 --- a/src/net/xfr/protocol/mod.rs +++ b/src/net/xfr/protocol/mod.rs @@ -4,13 +4,13 @@ //! process one or more AXFR/IXFR response messages in terms of the high level //! [`XfrEvent`]s that they represent without having to deal with the //! AXFR/IXFR protocol details. -mod iterator; mod interpreter; +mod iterator; mod types; #[cfg(test)] mod tests; -pub use iterator::XfrEventIterator; pub use interpreter::XfrResponseInterpreter; +pub use iterator::XfrEventIterator; pub use types::{IterationError, ProcessingError, XfrEvent, XfrRecord}; From ef959e1bfa5778fa91f5a7391b1ea95af4dce132 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 16 Sep 2024 11:39:41 +0200 Subject: [PATCH 181/333] Compilation fixes due to changes in upstream code. --- src/zonetree/xfr_event_handler.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/zonetree/xfr_event_handler.rs b/src/zonetree/xfr_event_handler.rs index 771759dc1..d94424303 100644 --- a/src/zonetree/xfr_event_handler.rs +++ b/src/zonetree/xfr_event_handler.rs @@ -9,7 +9,7 @@ use super::error::OutOfZone; use super::{WritableZone, WritableZoneNode, Zone}; use crate::base::name::{FlattenInto, Label, ToLabelIter}; use crate::base::{Name, ParsedName, Record, Rtype, ToName}; -use crate::net::xfr::processing::{XfrEvent, XfrRecord}; +use crate::net::xfr::protocol::{XfrEvent, XfrRecord}; use crate::rdata::ZoneRecordData; use crate::zonetree::{Rrset, SharedRrset}; @@ -307,7 +307,7 @@ mod tests { use crate::base::{ Message, MessageBuilder, ParsedName, Record, Serial, Ttl, }; - use crate::net::xfr::processing::XfrResponseProcessor; + use crate::net::xfr::protocol::XfrResponseInterpreter; use crate::rdata::{Soa, A}; use crate::zonetree::ZoneBuilder; @@ -356,7 +356,7 @@ mod tests { let req = mk_request("example.com", Rtype::AXFR).into_message(); // Create an XFR response processor. - let mut processor = XfrResponseProcessor::new(); + let mut processor = XfrResponseInterpreter::new(); // Create an AXFR response. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); From e21891257502909ae27d4e1808a6bc57bd2a2c36 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 16 Sep 2024 11:40:27 +0200 Subject: [PATCH 182/333] Compilation fixes due to changes in upstream code. --- src/net/server/middleware/notify.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/server/middleware/notify.rs b/src/net/server/middleware/notify.rs index 097248d4f..36be5202f 100644 --- a/src/net/server/middleware/notify.rs +++ b/src/net/server/middleware/notify.rs @@ -8,7 +8,7 @@ use std::boxed::Box; use std::fmt::Debug; use std::sync::Arc; -use futures::stream::{once, Once, Stream}; +use futures_util::stream::{once, Once, Stream}; use octseq::Octets; use tracing::{error, info, warn}; From bf7b77f87cfb407efe7711d2dbd0669d3dbbf822 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 16 Sep 2024 12:07:41 +0200 Subject: [PATCH 183/333] Use the existing TSIG error building code, don't hand create it ourselves. --- src/net/server/middleware/tsig.rs | 51 +++++++++++++++++++------------ 1 file changed, 32 insertions(+), 19 deletions(-) diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index f91e88413..09f8d0728 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -40,7 +40,7 @@ use std::vec::Vec; use octseq::{Octets, OctetsFrom}; use tracing::{error, trace, warn}; -use crate::base::iana::{Rcode, TsigRcode}; +use crate::base::iana::Rcode; use crate::base::message_builder::AdditionalBuilder; use crate::base::wire::Composer; use crate::base::{Message, StreamTarget}; @@ -108,12 +108,15 @@ where fn preprocess( req: &Request, key_store: &KS, - ) -> ControlFlow< - AdditionalBuilder>, - Option<( - Request>, - TsigSigner, - )>, + ) -> Result< + ControlFlow< + AdditionalBuilder>, + Option<( + Request>, + TsigSigner, + )>, + >, + ServiceError, > { let octets = req.message().as_slice().to_vec(); let mut mut_msg = Message::from_octets(octets).unwrap(); @@ -151,10 +154,10 @@ where let num_bytes_to_reserve = tsig.key().compose_len(); new_req.reserve_bytes(num_bytes_to_reserve); - return ControlFlow::Continue(Some(( + return Ok(ControlFlow::Continue(Some(( new_req, TsigSigner::Transaction(tsig), - ))); + )))); } Err(err) => { @@ -164,16 +167,22 @@ where req.message().header().opcode(), req.client_addr(), ); + let builder = mk_builder_for_target(); - let additional = - tsig::ServerError::::unsigned(TsigRcode::BADKEY) - .build_message(req.message(), builder) - .unwrap(); - return ControlFlow::Break(additional); + + let res = match err.build_message(req.message(), builder) { + Ok(additional) => Ok(ControlFlow::Break(additional)), + Err(err) => { + error!("Unable to build TSIG error response: {err}"); + Err(ServiceError::InternalError) + } + }; + + return res; } } - ControlFlow::Continue(None) + Ok(ControlFlow::Continue(None)) } /// Sign the given response, or if necessary construct and return an @@ -410,14 +419,14 @@ where (), PostprocessingConfig, >, - Once::Item>>, + Once>>, ::Item, >; type Future = Ready; fn call(&self, request: Request) -> Self::Future { match Self::preprocess(&request, &self.key_store) { - ControlFlow::Continue(Some((modified_req, signer))) => { + Ok(ControlFlow::Continue(Some((modified_req, signer)))) => { let pp_config = PostprocessingConfig::new(signer); let svc_call_fut = self.next_svc.call(modified_req); @@ -432,17 +441,21 @@ where ready(MiddlewareStream::Map(map)) } - ControlFlow::Continue(None) => { + Ok(ControlFlow::Continue(None)) => { let request = request.with_new_metadata(None); let svc_call_fut = self.next_svc.call(request); ready(MiddlewareStream::IdentityFuture(svc_call_fut)) } - ControlFlow::Break(additional) => { + Ok(ControlFlow::Break(additional)) => { ready(MiddlewareStream::Result(once(ready(Ok( CallResult::new(additional), ))))) } + + Err(err) => { + ready(MiddlewareStream::Result(once(ready(Err(err))))) + } } } } From a3af7e5dfbca69853756d96bc057d5f150f98c2e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 16 Sep 2024 13:24:39 +0200 Subject: [PATCH 184/333] Update src/net/server/middleware/tsig.rs Co-authored-by: Terts Diepraam --- src/net/server/middleware/tsig.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index 09f8d0728..a8d87c01c 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -4,7 +4,7 @@ //! middleware service. The underlying TSIG RR processing is implemented using //! the [`rdata::tsig`][crate::rdata::tsig] module. //! -//! Signed requests thta fail signature verification will be rejected. +//! Signed requests that fail signature verification will be rejected. //! //! Unsigned requests and correctly signed requests will pass through this //! middleware unchanged. From 8509a68fe5fc1bc3a4720310574fa366155ced47 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 16 Sep 2024 13:24:54 +0200 Subject: [PATCH 185/333] Update src/net/server/middleware/tsig.rs Co-authored-by: Terts Diepraam --- src/net/server/middleware/tsig.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index a8d87c01c..b368b8143 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -19,7 +19,7 @@ //! [`KeyStore`] that was used to construct this middleware. Upstream services //! can choose to ignore the metadata by being generic over any kind of //! metadata, or may offer a [`Service`] impl that specifically accepts the -//! [`Option`] metadata type, enabling the upstream service to use +//! `Option` metadata type, enabling the upstream service to use //! the request metadata to determine the key that the request was signed //! with. //! From df0499a7fe68b745237dc7a4dc85337c4b626f42 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 16 Sep 2024 13:30:24 +0200 Subject: [PATCH 186/333] Update src/net/server/middleware/tsig.rs Co-authored-by: Terts Diepraam --- src/net/server/middleware/tsig.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index b368b8143..10d7b2ceb 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -64,7 +64,7 @@ use futures_util::Stream; /// any, and adds TSIG signatures to responses to signed requests. /// /// Upstream services can detect whether a request is signed and with which -/// key by consuming the [`Option`] metadata output by this service. +/// key by consuming the `Option` metadata output by this service. #[derive(Clone, Debug)] pub struct TsigMiddlewareSvc where From d9c04b44927a0376a2e9ad5b3b7306a67156bc68 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 16 Sep 2024 13:31:51 +0200 Subject: [PATCH 187/333] Comment correction. --- src/net/server/middleware/tsig.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index 10d7b2ceb..6966538e7 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -346,10 +346,10 @@ where // Does it need converting from the variant that supports // single messages only (ServerTransaction) to the variant // that supports signing multiple messages (ServerSequence)? - // Note: confusingly BeginTransaction and ServerTransaction - // use the term "transaction" to mean completely the opppsite + // Note: Confusingly BeginTransaction and ServerTransaction + // use the term "transaction" to mean completely the opposite // of each other. With BeginTransaction we mean that the - // caller should instead a sequence of response messages + // caller should expect a sequence of response messages // instead of the usual single response message. With // ServerTransaction the TSIG code means handling of single // messages only and NOT sequences for which there is a From 749721e82ed86badd0142e8aafc6f155931898b3 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 16 Sep 2024 13:43:48 +0200 Subject: [PATCH 188/333] Rename PostprocessingConfig to PostprocessingState to better reflect its mutable nature. --- src/net/server/middleware/tsig.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index 6966538e7..a0efd727f 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -190,7 +190,7 @@ where fn postprocess( request: &Request, response: &mut AdditionalBuilder>, - pp_config: &mut PostprocessingConfig, + state: &mut PostprocessingState, ) -> Result< Option>>, ServiceError, @@ -201,12 +201,12 @@ where let truncation_ctx; - let res = match &mut pp_config.signer { + let res = match &mut state.signer { Some(TsigSigner::Transaction(_)) => { // Extract the single response signer and consume it in the // signing process. let Some(TsigSigner::Transaction(signer)) = - pp_config.signer.take() + state.signer.take() else { unreachable!() }; @@ -336,7 +336,7 @@ where fn map_stream_item( request: Request, stream_item: ServiceResult, - pp_config: &mut PostprocessingConfig, + pp_config: &mut PostprocessingState, ) -> ServiceResult { if let Ok(mut call_res) = stream_item { if matches!( @@ -417,7 +417,7 @@ where NextSvc::Future, NextSvc::Stream, (), - PostprocessingConfig, + PostprocessingState, >, Once>>, ::Item, @@ -427,7 +427,7 @@ where fn call(&self, request: Request) -> Self::Future { match Self::preprocess(&request, &self.key_store) { Ok(ControlFlow::Continue(Some((modified_req, signer)))) => { - let pp_config = PostprocessingConfig::new(signer); + let pp_config = PostprocessingState::new(signer); let svc_call_fut = self.next_svc.call(modified_req); @@ -462,7 +462,7 @@ where /// Data needed to do signing during response post-processing. -pub struct PostprocessingConfig { +pub struct PostprocessingState { /// The signer used to verify the request. /// /// Needed to sign responses. @@ -473,7 +473,7 @@ pub struct PostprocessingConfig { signer: Option>, } -impl PostprocessingConfig { +impl PostprocessingState { fn new(signer: TsigSigner) -> Self { Self { signer: Some(signer), From fb3ed8611318a7e0472a3d068de614820e01dfce Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 17 Sep 2024 10:38:49 +0200 Subject: [PATCH 189/333] Generalize the notion of zone updates, it need not be specific to XFR, e.g. we may be able to use this for DNS UPDATE later as well. --- src/net/xfr/protocol/interpreter.rs | 62 ++++++++++---------- src/net/xfr/protocol/iterator.rs | 21 +++---- src/net/xfr/protocol/mod.rs | 8 ++- src/net/xfr/protocol/tests.rs | 45 ++++++++------- src/net/xfr/protocol/types.rs | 75 ++---------------------- src/zonetree/types.rs | 88 ++++++++++++++++++++++++++++- 6 files changed, 162 insertions(+), 137 deletions(-) diff --git a/src/net/xfr/protocol/interpreter.rs b/src/net/xfr/protocol/interpreter.rs index 30c75ebc2..904305025 100644 --- a/src/net/xfr/protocol/interpreter.rs +++ b/src/net/xfr/protocol/interpreter.rs @@ -5,18 +5,17 @@ use bytes::Bytes; use crate::base::iana::Opcode; use crate::base::{Message, ParsedName, Rtype}; use crate::rdata::{Soa, ZoneRecordData}; +use crate::zonetree::types::ZoneUpdate; -use super::iterator::XfrEventIterator; -use super::types::{ - IxfrUpdateMode, ProcessingError, XfrEvent, XfrRecord, XfrType, -}; +use super::iterator::XfrZoneUpdateIterator; +use super::types::{IxfrUpdateMode, ProcessingError, XfrRecord, XfrType}; //------------ XfrResponseInterpreter ----------------------------------------- -/// An AXFR/IXFR response processor. +/// An AXFR/IXFR response interpreter. /// /// Use [`XfrResponseInterpreter`] to interpret a sequence of AXFR or IXFR -/// response messages as a sequence of high level [`XfrEvent`]s. +/// response messages as a sequence of [`ZoneUpdate`]s. /// /// # Usage /// @@ -24,16 +23,18 @@ use super::types::{ /// [`XfrResponseInterpreter`] for the corresponding XFR request message, then /// pass each XFR response message to [`process_answer()`]. /// -/// Each call to [`process_answer()`] will return an [`XfrEventIterator`] -/// which when iterated over will produce a sequence of [`XfrEvent`]s for a -/// single response message. The iterator emits an [`XfrEvent::EndOfTransfer`] -/// event when the last record in the transfer is reached. +/// Each call to [`process_answer()`] will return an [`XfrZoneUpdateIterator`] +/// which when iterated over will produce a sequence of [`ZoneUpdate`]s for a +/// single response message. The iterator emits [`ZoneUpdate::Complete`] when +/// the last record in the transfer is reached. /// -/// If [`XfrEvent::EndOfTransfer`] event has not yet been emitted it means -/// that the sequence is incomplete and the next response message in the -/// sequence should be passed to [`process_answer()`]. +/// If [`ZoneUpdate::Complete`] has not yet been emitted it means that the +/// sequence is incomplete and the next response message in the sequence +/// should be passed to [`process_answer()`]. /// /// [`process_answer()`]: XfrResponseInterpreter::process_answer() +/// [`ZoneUpdate`]: crate::zonetree::types::ZoneUpdate +/// [`ZoneUpdate::Complete`]: crate::zonetree::types::ZoneUpdate #[derive(Default)] pub struct XfrResponseInterpreter { /// Internal state. @@ -52,12 +53,13 @@ impl XfrResponseInterpreter { impl XfrResponseInterpreter { /// Process a single AXFR/IXFR response message. /// - /// Returns an [`XfrEventIterator`] over [`XfrEvent`]s emitted during + /// Returns an [`XfrZoneUpdateIterator`] over [`ZoneUpdate`]s emitted during /// processing. /// - /// If the returned iterator does not emit an [`XfrEvent::EndOfTransfer`] - /// event, call this function with the next outstanding response message - /// to continue iterating over the incomplete transfer. + /// Call this function with the next outstanding response message to + /// continue iterating over an incomplete transfer (i.e. previous + /// iterators were exhausted without emiting [`ZoneUpdate::Finished`] or + /// [`ZoneUpdate::Corrupt`]). /// /// Checking that the given response corresponds by ID to the related /// original XFR query or that the question section of the response, if @@ -67,7 +69,7 @@ impl XfrResponseInterpreter { pub fn process_answer( &mut self, resp: Message, - ) -> Result { + ) -> Result { // Check that the given message is a DNS XFR response. self.check_response(&resp)?; @@ -79,7 +81,7 @@ impl XfrResponseInterpreter { let inner = self.inner.as_mut().unwrap(); - XfrEventIterator::new(&mut inner.state, &inner.resp) + XfrZoneUpdateIterator::new(&mut inner.state, &inner.resp) } } @@ -247,7 +249,7 @@ impl RecordProcessor { pub(super) fn process_record( &mut self, rec: XfrRecord, - ) -> XfrEvent { + ) -> ZoneUpdate { self.rr_count += 1; // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2 @@ -281,11 +283,11 @@ impl RecordProcessor { // MUST conclude with the same SOA resource record. // Intermediate messages MUST NOT contain the SOA resource // record." - XfrEvent::EndOfTransfer(rec) + ZoneUpdate::Finished(rec) } XfrType::Axfr => { - XfrEvent::AddRecord(self.current_soa.serial(), rec) + ZoneUpdate::AddRecord(self.current_soa.serial(), rec) } XfrType::Ixfr if self.rr_count < 2 => unreachable!(), @@ -293,7 +295,7 @@ impl RecordProcessor { XfrType::Ixfr if self.rr_count == 2 => { if record_matches_initial_soa { // IXFR not available, AXFR of empty zone detected. - XfrEvent::EndOfTransfer(rec) + ZoneUpdate::Finished(rec) } else if let Some(soa) = soa { // This SOA record is the start of an IXFR diff sequence. self.current_soa = soa.clone(); @@ -305,7 +307,7 @@ impl RecordProcessor { IxfrUpdateMode::Deleting ); - XfrEvent::BeginBatchDelete(rec) + ZoneUpdate::BeginBatchDelete(rec) } else { // https://datatracker.ietf.org/doc/html/rfc1995#section-4 // 4. Response Format @@ -335,7 +337,7 @@ impl RecordProcessor { // assume that "incremental zone transfer is not available" // and so "the behaviour is the same as an AXFR response", self.actual_xfr_type = XfrType::Axfr; - XfrEvent::AddRecord(self.current_soa.serial(), rec) + ZoneUpdate::AddRecord(self.current_soa.serial(), rec) } } @@ -350,25 +352,25 @@ impl RecordProcessor { // Is this the end of the transfer, or the start // of a new diff sequence? if record_matches_initial_soa { - XfrEvent::EndOfTransfer(rec) + ZoneUpdate::Finished(rec) } else { - XfrEvent::BeginBatchDelete(rec) + ZoneUpdate::BeginBatchDelete(rec) } } IxfrUpdateMode::Adding => { // We just switched from the Delete phase of a // diff sequence to the add phase of the diff // sequence. - XfrEvent::BeginBatchAdd(rec) + ZoneUpdate::BeginBatchAdd(rec) } } } else { match self.ixfr_update_mode { - IxfrUpdateMode::Deleting => XfrEvent::DeleteRecord( + IxfrUpdateMode::Deleting => ZoneUpdate::DeleteRecord( self.current_soa.serial(), rec, ), - IxfrUpdateMode::Adding => XfrEvent::AddRecord( + IxfrUpdateMode::Adding => ZoneUpdate::AddRecord( self.current_soa.serial(), rec, ), diff --git a/src/net/xfr/protocol/iterator.rs b/src/net/xfr/protocol/iterator.rs index a28171004..e89610a9c 100644 --- a/src/net/xfr/protocol/iterator.rs +++ b/src/net/xfr/protocol/iterator.rs @@ -6,16 +6,17 @@ use tracing::trace; use crate::base::message::RecordIter; use crate::base::{Message, ParsedName}; use crate::rdata::ZoneRecordData; +use crate::zonetree::types::ZoneUpdate; use super::interpreter::RecordProcessor; -use super::types::{IterationError, ProcessingError, XfrEvent, XfrRecord}; +use super::types::{IterationError, ProcessingError, XfrRecord}; -//------------ XfrEventIterator ----------------------------------------------- +//------------ XfrZoneUpdateIterator ------------------------------------------ -/// An iterator over [`XfrResponseInterpreter`] generated [`XfrEvent`]s. +/// An iterator over [`XfrResponseInterpreter`] generated [`ZoneUpdate`]s. /// -/// [`XfrResponseInterpreter`]: super::processor::XfrResponseInterpreter -pub struct XfrEventIterator<'a, 'b> { +/// [`XfrResponseInterpreter`]: super::interpreter::XfrResponseInterpreter +pub struct XfrZoneUpdateIterator<'a, 'b> { /// The parent processor. state: &'a mut RecordProcessor, @@ -23,7 +24,7 @@ pub struct XfrEventIterator<'a, 'b> { iter: RecordIter<'b, Bytes, ZoneRecordData>>, } -impl<'a, 'b> XfrEventIterator<'a, 'b> { +impl<'a, 'b> XfrZoneUpdateIterator<'a, 'b> { pub(super) fn new( state: &'a mut RecordProcessor, resp: &'b Message, @@ -60,15 +61,15 @@ impl<'a, 'b> XfrEventIterator<'a, 'b> { } } -impl<'a, 'b> Iterator for XfrEventIterator<'a, 'b> { - type Item = Result, IterationError>; +impl<'a, 'b> Iterator for XfrZoneUpdateIterator<'a, 'b> { + type Item = Result, IterationError>; fn next(&mut self) -> Option { match self.iter.next()? { Ok(record) => { trace!("XFR record {}: {record:?}", self.state.rr_count); - let event = self.state.process_record(record); - Some(Ok(event)) + let update = self.state.process_record(record); + Some(Ok(update)) } Err(err) => { diff --git a/src/net/xfr/protocol/mod.rs b/src/net/xfr/protocol/mod.rs index b7855f7ef..07ce4c7fe 100644 --- a/src/net/xfr/protocol/mod.rs +++ b/src/net/xfr/protocol/mod.rs @@ -2,8 +2,10 @@ //! //! This module provides [`XfrResponseInterpreter`] which can be used to //! process one or more AXFR/IXFR response messages in terms of the high level -//! [`XfrEvent`]s that they represent without having to deal with the +//! [`ZoneUpdate`]s that they represent without having to deal with the //! AXFR/IXFR protocol details. +//! +//! [`ZoneUpdate`]: crate::zonetree::types::ZoneUpdate mod interpreter; mod iterator; mod types; @@ -12,5 +14,5 @@ mod types; mod tests; pub use interpreter::XfrResponseInterpreter; -pub use iterator::XfrEventIterator; -pub use types::{IterationError, ProcessingError, XfrEvent, XfrRecord}; +pub use iterator::XfrZoneUpdateIterator; +pub use types::{IterationError, ProcessingError, XfrRecord}; diff --git a/src/net/xfr/protocol/tests.rs b/src/net/xfr/protocol/tests.rs index 5049a9282..637573068 100644 --- a/src/net/xfr/protocol/tests.rs +++ b/src/net/xfr/protocol/tests.rs @@ -14,11 +14,10 @@ use crate::base::{ }; use crate::base::{Name, ToName}; use crate::rdata::{Soa, ZoneRecordData, A}; +use crate::zonetree::types::{ZoneUpdate, ZoneUpdate as ZU}; use super::interpreter::XfrResponseInterpreter; -use super::types::{ - IterationError, ProcessingError, XfrEvent, XfrEvent as XE, XfrRecord, -}; +use super::types::{IterationError, ProcessingError, XfrRecord}; #[test] fn non_xfr_response_is_rejected() { @@ -140,7 +139,7 @@ fn axfr_response_with_only_soas_is_accepted() { let mut it = processor.process_answer(resp).unwrap(); // Verify the events emitted by the XFR processor. - assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer(_))))); + assert!(matches!(it.next(), Some(Ok(ZU::Finished(_))))); assert!(it.next().is_none()); } @@ -179,7 +178,7 @@ fn axfr_multi_response_with_only_soas_is_accepted() { let mut it = processor.process_answer(resp).unwrap(); // Verify the events emitted by the XFR processor. - assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer(_))))); + assert!(matches!(it.next(), Some(Ok(ZU::Finished(_))))); assert!(it.next().is_none()); } @@ -208,9 +207,9 @@ fn axfr_response_generates_expected_events() { // Verify the events emitted by the XFR processor. let s = serial; - assert!(matches!(it.next(), Some(Ok(XE::AddRecord(n, _))) if n == s)); - assert!(matches!(it.next(), Some(Ok(XE::AddRecord(n, _))) if n == s)); - assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer(_))))); + assert!(matches!(it.next(), Some(Ok(ZU::AddRecord(n, _))) if n == s)); + assert!(matches!(it.next(), Some(Ok(ZU::AddRecord(n, _))) if n == s)); + assert!(matches!(it.next(), Some(Ok(ZU::Finished(_))))); assert!(it.next().is_none()); } @@ -275,13 +274,13 @@ fn ixfr_response_generates_expected_events() { // Verify the events emitted by the XFR processor. let owner = ParsedName::::from(Name::from_str("example.com").unwrap()); - let expected_events: [Result, IterationError>; 7] = [ - Ok(XfrEvent::BeginBatchDelete(Record::from(( + let expected_events: [Result, IterationError>; 7] = [ + Ok(ZoneUpdate::BeginBatchDelete(Record::from(( owner.clone(), 0, ZoneRecordData::Soa(expected_old_soa), )))), - Ok(XfrEvent::DeleteRecord( + Ok(ZoneUpdate::DeleteRecord( old_serial, Record::from(( owner.clone(), @@ -289,7 +288,7 @@ fn ixfr_response_generates_expected_events() { ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), )), )), - Ok(XfrEvent::DeleteRecord( + Ok(ZoneUpdate::DeleteRecord( old_serial, Record::from(( owner.clone(), @@ -297,12 +296,12 @@ fn ixfr_response_generates_expected_events() { ZoneRecordData::A(A::new(Ipv4Addr::BROADCAST)), )), )), - Ok(XfrEvent::BeginBatchAdd(Record::from(( + Ok(ZoneUpdate::BeginBatchAdd(Record::from(( owner.clone(), 0, ZoneRecordData::Soa(expected_new_soa.clone()), )))), - Ok(XfrEvent::AddRecord( + Ok(ZoneUpdate::AddRecord( new_serial, Record::from(( owner.clone(), @@ -310,7 +309,7 @@ fn ixfr_response_generates_expected_events() { ZoneRecordData::A(A::new(Ipv4Addr::BROADCAST)), )), )), - Ok(XfrEvent::AddRecord( + Ok(ZoneUpdate::AddRecord( new_serial, Record::from(( owner.clone(), @@ -318,7 +317,7 @@ fn ixfr_response_generates_expected_events() { ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), )), )), - Ok(XfrEvent::EndOfTransfer(Record::from(( + Ok(ZoneUpdate::Finished(Record::from(( owner.clone(), 0, ZoneRecordData::Soa(expected_new_soa), @@ -365,8 +364,8 @@ fn multi_ixfr_response_generates_expected_events() { let mut it = processor.process_answer(resp).unwrap(); // Verify the events emitted by the XFR processor. - assert!(matches!(it.next(), Some(Ok(XE::BeginBatchDelete(_))))); - assert!(matches!(it.next(), Some(Ok(XE::DeleteRecord(..))))); + assert!(matches!(it.next(), Some(Ok(ZU::BeginBatchDelete(_))))); + assert!(matches!(it.next(), Some(Ok(ZU::DeleteRecord(..))))); assert!(it.next().is_none()); // Craete a second IXFR response that completes the transfer @@ -386,11 +385,11 @@ fn multi_ixfr_response_generates_expected_events() { let mut it = processor.process_answer(resp).unwrap(); // Verify the events emitted by the XFR processor. - assert!(matches!(it.next(), Some(Ok(XE::DeleteRecord(..))))); - assert!(matches!(it.next(), Some(Ok(XE::BeginBatchAdd(_))))); - assert!(matches!(it.next(), Some(Ok(XE::AddRecord(..))))); - assert!(matches!(it.next(), Some(Ok(XE::AddRecord(..))))); - assert!(matches!(it.next(), Some(Ok(XE::EndOfTransfer(_))))); + assert!(matches!(it.next(), Some(Ok(ZU::DeleteRecord(..))))); + assert!(matches!(it.next(), Some(Ok(ZU::BeginBatchAdd(_))))); + assert!(matches!(it.next(), Some(Ok(ZU::AddRecord(..))))); + assert!(matches!(it.next(), Some(Ok(ZU::AddRecord(..))))); + assert!(matches!(it.next(), Some(Ok(ZU::Finished(_))))); assert!(it.next().is_none()); } diff --git a/src/net/xfr/protocol/types.rs b/src/net/xfr/protocol/types.rs index bb6016c97..1cb0c756a 100644 --- a/src/net/xfr/protocol/types.rs +++ b/src/net/xfr/protocol/types.rs @@ -5,13 +5,13 @@ use bytes::Bytes; use crate::{ - base::{wire::ParseError, ParsedName, Record, Rtype, Serial}, + base::{wire::ParseError, ParsedName, Record, Rtype}, rdata::ZoneRecordData, }; /// The type of record processed by [`XfrResponseInterpreter`]. /// -/// [`XfrResponseInterpreter`]: super::processor::XfrResponseInterpreter +/// [`XfrResponseInterpreter`]: super::interpreter::XfrResponseInterpreter pub type XfrRecord = Record, ZoneRecordData>>; @@ -46,71 +46,6 @@ impl TryFrom for XfrType { } } -//------------ XfrEvent ------------------------------------------------------- - -/// An event emitted by [`XfrResponseInterpreter`] during transfer processing. -/// -/// [`XfrResponseInterpreter`]: super::processor::XfrResponseInterpreter -#[derive(Clone, Debug, PartialEq, Eq)] -pub enum XfrEvent { - /// Delete record R in zone serial S. - /// - /// The transfer signalled that the given record should be deleted from - /// the zone version with the given serial number. - /// - /// Note: If the transfer contains N deletions of fhe same record then - /// this event will occur N times. - DeleteRecord(Serial, R), - - /// Add record R in zone serial S. - /// - /// The transfer signalled that the given record should be added to the - /// zone version with the given serial number. - /// - /// Note: If the transfer contains N additions of fhe same record then - /// this event will occur N times. - AddRecord(Serial, R), - - /// Prepare to delete records in zone serial S. - /// - /// The transfer signalled that zero or more record deletions will follow, - /// all for the zone version with the given serial number. - BeginBatchDelete(R), - - /// Prepare to add records in zone serial S. - /// - /// The transfer signalled that zero or more record additions will follow, - /// all for the zone version with the given serial number. - BeginBatchAdd(R), - - /// Transfer completed successfully. - /// - /// Note: This event is not emitted until the final record of the final - /// response in a set of one or more transfer responss has been seen. - EndOfTransfer(R), - - /// Transfer processing failed. - /// - /// This event indicates that there is a problem with the transfer data - /// and that transfer processing cannot continue. - ProcessingFailed, -} - -//--- Display - -impl std::fmt::Display for XfrEvent { - fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { - match self { - XfrEvent::DeleteRecord(_, _) => f.write_str("DeleteRecord"), - XfrEvent::AddRecord(_, _) => f.write_str("AddRecord"), - XfrEvent::BeginBatchDelete(_) => f.write_str("BeginBatchDelete"), - XfrEvent::BeginBatchAdd(_) => f.write_str("BeginBatchAdd"), - XfrEvent::EndOfTransfer(_) => f.write_str("EndOfTransfer"), - XfrEvent::ProcessingFailed => f.write_str("ProcessingFailed"), - } - } -} - //------------ IxfrUpdateMode ------------------------------------------------- /// The kind of records currently being processed, either adds or deletes. @@ -140,7 +75,7 @@ impl IxfrUpdateMode { /// An error reported by [`XfrResponseInterpreter`]. /// -/// [`XfrResponseInterpreter`]: super::processor::XfrResponseInterpreter +/// [`XfrResponseInterpreter`]: super::interpreter::XfrResponseInterpreter #[derive(Debug)] pub enum ProcessingError { /// The message could not be parsed. @@ -171,9 +106,9 @@ impl std::fmt::Display for ProcessingError { //------------ IterationError ------------------------------------------------- -/// Errors that can occur during [`XfrEventIterator`]` iteration. +/// Errors that can occur during [`XfrZoneUpdateIterator`]` iteration. /// -/// [`XfrEventIterator`]: super::iterator::XfrEventIterator +/// [`XfrZoneUpdateIterator`]: super::iterator::XfrZoneUpdateIterator #[derive(Copy, Clone, Debug, PartialEq, Eq)] pub enum IterationError { /// Transfer processing failed. diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index c66a9256d..dc808457b 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -9,7 +9,7 @@ use serde::{Deserialize, Serialize}; use crate::base::rdata::RecordData; use crate::base::{iana::Rtype, Ttl}; -use crate::base::{Name, Record}; +use crate::base::{Name, Record, Serial}; use crate::rdata::ZoneRecordData; //------------ Type Aliases -------------------------------------------------- @@ -247,3 +247,89 @@ pub struct ZoneCut { /// Zero or more glue records at the zone cut. pub glue: Vec, } + +//------------ ZoneUpdate ----------------------------------------------------- + +/// An update to be applied to a [`Zone`]. +/// +/// Note: This enum is marked as `#[non_exhaustive]` to permit addition of +/// more update operations in future, e.g. to support RFC 2136 Dynamic Updates +/// operations. +/// +/// [`Zone`]: crate::zonetree::zone::Zone +#[derive(Clone, Debug, PartialEq, Eq)] +#[non_exhaustive] +pub enum ZoneUpdate { + /// Delete record R from the specified version (serial) of the zone. + DeleteRecord(Serial, R), + + /// Add record R to the specified version (serial) of the zone. + AddRecord(Serial, R), + + /// Start a batch delete for the specified version (serial) of the zone. + /// + /// If not already in batching mode, this signals the start of batching + /// mode. In batching mode one or more batches of updates will be + /// signalled, each consisting of the sequence: + /// + /// - ZoneUpdate::BeginBatchDelete + /// - ZoneUpdate::DeleteRecord (zero or more) + /// - ZoneUpdate::BeginBatchAdd + /// - ZoneUpdate::AddRecord (zero or more) + /// + /// Batching mode can only be terminated by `UpdateComplete` or + /// `UpdateIncomplete`. + /// + /// Batching mode makes updates more predictable for the receiver to work + /// with by limiting the updates that can be signalled next, enabling + /// receiver logic to be simpler and more efficient. + BeginBatchDelete(R), + + /// Start a batch add for the specified version (serial) of the zone. + /// + /// This can only be signalled when already in batching mode, i.e. when + /// `BeginBatchDelete` has already been signalled. + /// + /// See `BeginBatchDelete` for more information. + BeginBatchAdd(R), + + /// Updates for the specified version (serial) of the zone can now be + /// finalized. + /// + /// This signals the end of a group of related changes to the specified + /// version (serial) of the zone. + /// + /// For example this could be used to trigger an atomic commit of a set of + /// related pending changes. + Finished(R), + + /// A sequence of updates has been deemed to be corrupt. + /// + /// This signals that something is wrong with the sequence of updates + /// being processed and that it will not be possible to complete the + /// sequence. + /// + /// For example this could occur if a sequence of XFR responses from a + /// primary server is fatally interrupted and cannot be completed. Some + /// responses may have already been communicated as [`ZoneUpdate`]s but + /// they should not be used without the remainder that will now not be + /// received. + Corrupt, +} + +//--- Display + +impl std::fmt::Display for ZoneUpdate { + fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { + match self { + ZoneUpdate::DeleteRecord(_, _) => f.write_str("DeleteRecord"), + ZoneUpdate::AddRecord(_, _) => f.write_str("AddRecord"), + ZoneUpdate::BeginBatchDelete(_) => { + f.write_str("BeginBatchDelete") + } + ZoneUpdate::BeginBatchAdd(_) => f.write_str("BeginBatchAdd"), + ZoneUpdate::Finished(_) => f.write_str("Finished"), + ZoneUpdate::Corrupt => f.write_str("Corrupt"), + } + } +} From 4eae169e4a06f3f3825c21f1a52f02af628a9f8f Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 17 Sep 2024 15:06:58 +0200 Subject: [PATCH 190/333] Review feedback: Add `is_finished()` method. Also variaous renames and minor improvements: rename `process_answer()` to `interpret_response()`, remove the unused `ZoneUpdate::Corrupt` enum variant, rename `ProcessingError` to `Error`, add `Error::Finished`. --- src/net/xfr/protocol/interpreter.rs | 75 +++++++---- src/net/xfr/protocol/iterator.rs | 8 +- src/net/xfr/protocol/mod.rs | 2 +- src/net/xfr/protocol/tests.rs | 200 ++++++++++++++++++---------- src/net/xfr/protocol/types.rs | 16 ++- src/zonetree/types.rs | 18 +-- 6 files changed, 192 insertions(+), 127 deletions(-) diff --git a/src/net/xfr/protocol/interpreter.rs b/src/net/xfr/protocol/interpreter.rs index 904305025..35ef7ff83 100644 --- a/src/net/xfr/protocol/interpreter.rs +++ b/src/net/xfr/protocol/interpreter.rs @@ -8,7 +8,7 @@ use crate::rdata::{Soa, ZoneRecordData}; use crate::zonetree::types::ZoneUpdate; use super::iterator::XfrZoneUpdateIterator; -use super::types::{IxfrUpdateMode, ProcessingError, XfrRecord, XfrType}; +use super::types::{Error, IxfrUpdateMode, XfrRecord, XfrType}; //------------ XfrResponseInterpreter ----------------------------------------- @@ -21,25 +21,25 @@ use super::types::{IxfrUpdateMode, ProcessingError, XfrRecord, XfrType}; /// /// For each response stream to be interpreted, construct an /// [`XfrResponseInterpreter`] for the corresponding XFR request message, then -/// pass each XFR response message to [`process_answer()`]. +/// pass each XFR response message to [`intrepret_response()`]. /// -/// Each call to [`process_answer()`] will return an [`XfrZoneUpdateIterator`] +/// Each call to [`intrepret_response()`] will return an [`XfrZoneUpdateIterator`] /// which when iterated over will produce a sequence of [`ZoneUpdate`]s for a /// single response message. The iterator emits [`ZoneUpdate::Complete`] when /// the last record in the transfer is reached. /// /// If [`ZoneUpdate::Complete`] has not yet been emitted it means that the /// sequence is incomplete and the next response message in the sequence -/// should be passed to [`process_answer()`]. +/// should be passed to [`intrepret_response()`]. /// -/// [`process_answer()`]: XfrResponseInterpreter::process_answer() +/// [`intrepret_response()`]: XfrResponseInterpreter::intrepret_response() /// [`ZoneUpdate`]: crate::zonetree::types::ZoneUpdate /// [`ZoneUpdate::Complete`]: crate::zonetree::types::ZoneUpdate #[derive(Default)] pub struct XfrResponseInterpreter { /// Internal state. /// - /// None until the first call to [`process_answer()`]. + /// None until the first call to [`intrepret_response()`]. inner: Option, } @@ -53,23 +53,26 @@ impl XfrResponseInterpreter { impl XfrResponseInterpreter { /// Process a single AXFR/IXFR response message. /// - /// Returns an [`XfrZoneUpdateIterator`] over [`ZoneUpdate`]s emitted during - /// processing. + /// Returns an [`XfrZoneUpdateIterator`] over [`ZoneUpdate`]s emitted + /// during processing. /// /// Call this function with the next outstanding response message to /// continue iterating over an incomplete transfer (i.e. previous - /// iterators were exhausted without emiting [`ZoneUpdate::Finished`] or - /// [`ZoneUpdate::Corrupt`]). + /// iterators were exhausted without emiting [`ZoneUpdate::Finished`]. /// /// Checking that the given response corresponds by ID to the related /// original XFR query or that the question section of the response, if /// present (RFC 5936 allows it to be empty for subsequent AXFR responses) /// matches that of the original query is NOT done here but instead is /// left to the caller to do. - pub fn process_answer( + pub fn intrepret_response( &mut self, resp: Message, - ) -> Result { + ) -> Result { + if self.is_finished() { + return Err(Error::Finished); + } + // Check that the given message is a DNS XFR response. self.check_response(&resp)?; @@ -83,14 +86,21 @@ impl XfrResponseInterpreter { XfrZoneUpdateIterator::new(&mut inner.state, &inner.resp) } + + /// Is the transfer finished? + /// + /// Returns true if the end of the transfer has been detected, false otherwise. + pub fn is_finished(&self) -> bool { + self.inner + .as_ref() + .map(|inner| inner.state.finished) + .unwrap_or_default() + } } impl XfrResponseInterpreter { /// Initialize inner state. - fn initialize( - &mut self, - resp: Message, - ) -> Result<(), ProcessingError> { + fn initialize(&mut self, resp: Message) -> Result<(), Error> { self.inner = Some(Inner::new(resp)?); Ok(()) } @@ -103,10 +113,7 @@ impl XfrResponseInterpreter { /// Returns Ok on success, Err otherwise. On success the type of XFR that /// was determined is returned as well as the answer section from the XFR /// response. - fn check_response( - &self, - resp: &Message, - ) -> Result<(), ProcessingError> { + fn check_response(&self, resp: &Message) -> Result<(), Error> { let resp_header = resp.header(); let resp_counts = resp.header_counts(); @@ -117,7 +124,7 @@ impl XfrResponseInterpreter { || resp_counts.ancount() == 0 || resp_counts.nscount() != 0 { - return Err(ProcessingError::NotValidXfrResponse); + return Err(Error::NotValidXfrResponse); } //https://www.rfc-editor.org/rfc/rfc5936.html#section-2.2.1 @@ -128,7 +135,7 @@ impl XfrResponseInterpreter { let first_message = self.inner.is_none(); if (first_message && qdcount != 1) || (!first_message && qdcount > 1) { - return Err(ProcessingError::NotValidXfrResponse); + return Err(Error::NotValidXfrResponse); } Ok(()) @@ -140,7 +147,7 @@ impl XfrResponseInterpreter { /// Internal dynamic state of [`XfrResponseInterpreter`]. /// /// Separated out from [`XfrResponseInterpreter`] to avoid needing multiple -/// mutable self references in [`process_answer()`]. +/// mutable self references in [`intrepret_response()`]. struct Inner { /// The response message currently being processed. resp: Message, @@ -154,8 +161,8 @@ impl Inner { /// /// Records the initial SOA record and other details will will be used /// while processing the rest of the response. - fn new(resp: Message) -> Result { - let answer = resp.answer().map_err(ProcessingError::ParseError)?; + fn new(resp: Message) -> Result { + let answer = resp.answer().map_err(Error::ParseError)?; // https://datatracker.ietf.org/doc/html/rfc5936#section-3 // 3. Zone Contents @@ -183,12 +190,12 @@ impl Inner { }; let Some(Ok(record)) = records.next() else { - return Err(ProcessingError::Malformed); + return Err(Error::Malformed); }; // The initial record should be a SOA record. let ZoneRecordData::Soa(soa) = record.into_data() else { - return Err(ProcessingError::NotValidXfrResponse); + return Err(Error::NotValidXfrResponse); }; let state = RecordProcessor::new(xfr_type, soa); @@ -225,6 +232,9 @@ pub(super) struct RecordProcessor { /// The number of resource records parsed so far. pub(super) rr_count: usize, + + /// True if the end of the transfer has been detected, false otherwise. + pub(super) finished: bool, } impl RecordProcessor { @@ -239,12 +249,13 @@ impl RecordProcessor { current_soa: initial_soa, rr_count: 0, ixfr_update_mode: Default::default(), + finished: false, } } /// Process a single resource record. /// - /// Returns an [`XfrEvent`] that should be emitted for the processed + /// Returns a [`ZoneUpdate`] that should be emitted for the processed /// record, if any. pub(super) fn process_record( &mut self, @@ -273,7 +284,7 @@ impl RecordProcessor { let record_matches_initial_soa = soa == Some(&self.initial_soa); - match self.actual_xfr_type { + let update = match self.actual_xfr_type { XfrType::Axfr if record_matches_initial_soa => { // https://www.rfc-editor.org/rfc/rfc5936.html#section-2.2 // 2.2. AXFR Response @@ -377,6 +388,12 @@ impl RecordProcessor { } } } + }; + + if matches!(update, ZoneUpdate::Finished(_)) { + self.finished = true; } + + update } } diff --git a/src/net/xfr/protocol/iterator.rs b/src/net/xfr/protocol/iterator.rs index e89610a9c..233191e46 100644 --- a/src/net/xfr/protocol/iterator.rs +++ b/src/net/xfr/protocol/iterator.rs @@ -9,7 +9,7 @@ use crate::rdata::ZoneRecordData; use crate::zonetree::types::ZoneUpdate; use super::interpreter::RecordProcessor; -use super::types::{IterationError, ProcessingError, XfrRecord}; +use super::types::{Error, IterationError, XfrRecord}; //------------ XfrZoneUpdateIterator ------------------------------------------ @@ -28,8 +28,8 @@ impl<'a, 'b> XfrZoneUpdateIterator<'a, 'b> { pub(super) fn new( state: &'a mut RecordProcessor, resp: &'b Message, - ) -> Result { - let answer = resp.answer().map_err(ProcessingError::ParseError)?; + ) -> Result { + let answer = resp.answer().map_err(Error::ParseError)?; // https://datatracker.ietf.org/doc/html/rfc5936#section-3 // 3. Zone Contents @@ -52,7 +52,7 @@ impl<'a, 'b> XfrZoneUpdateIterator<'a, 'b> { if state.rr_count == 0 { let Some(Ok(_)) = iter.next() else { - return Err(ProcessingError::Malformed); + return Err(Error::Malformed); }; state.rr_count += 1; } diff --git a/src/net/xfr/protocol/mod.rs b/src/net/xfr/protocol/mod.rs index 07ce4c7fe..9cff713a8 100644 --- a/src/net/xfr/protocol/mod.rs +++ b/src/net/xfr/protocol/mod.rs @@ -15,4 +15,4 @@ mod tests; pub use interpreter::XfrResponseInterpreter; pub use iterator::XfrZoneUpdateIterator; -pub use types::{IterationError, ProcessingError, XfrRecord}; +pub use types::{Error, IterationError, XfrRecord}; diff --git a/src/net/xfr/protocol/tests.rs b/src/net/xfr/protocol/tests.rs index 637573068..23d0ea425 100644 --- a/src/net/xfr/protocol/tests.rs +++ b/src/net/xfr/protocol/tests.rs @@ -1,5 +1,7 @@ use core::str::FromStr; +use std::collections::VecDeque; + use bytes::{Bytes, BytesMut}; use octseq::{Octets, Parser}; @@ -17,7 +19,7 @@ use crate::rdata::{Soa, ZoneRecordData, A}; use crate::zonetree::types::{ZoneUpdate, ZoneUpdate as ZU}; use super::interpreter::XfrResponseInterpreter; -use super::types::{IterationError, ProcessingError, XfrRecord}; +use super::types::{Error, IterationError, XfrRecord}; #[test] fn non_xfr_response_is_rejected() { @@ -26,8 +28,8 @@ fn non_xfr_response_is_rejected() { // Create an AXFR-like request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); - // Create an XFR response processor. - let mut processor = XfrResponseInterpreter::new(); + // Create an XFR response interpreter. + let mut interpreter = XfrResponseInterpreter::new(); // Create a non-XFR response. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); @@ -35,10 +37,10 @@ fn non_xfr_response_is_rejected() { let resp = answer.into_message(); // Process the response and assert that it is rejected as not being - // a valid XFR response and that no XFR processor events were emitted. + // a valid XFR response and that no XFR interpreter updates were emitted. assert!(matches!( - processor.process_answer(resp), - Err(ProcessingError::NotValidXfrResponse) + interpreter.intrepret_response(resp), + Err(Error::NotValidXfrResponse) )); } @@ -49,17 +51,17 @@ fn axfr_response_with_no_answers_is_rejected() { // Create an AXFR request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); - // Create an XFR response processor. - let mut processor = XfrResponseInterpreter::new(); + // Create an XFR response interpreter. + let mut interpreter = XfrResponseInterpreter::new(); // Create a response that lacks answers. let resp = mk_empty_answer(&req, Rcode::NOERROR).into_message(); // Process the response and assert that it is rejected as not being - // a valid XFR response and that no XFR processor events were emitted. + // a valid XFR response and that no XFR interpreter updates were emitted. assert!(matches!( - processor.process_answer(resp), - Err(ProcessingError::NotValidXfrResponse) + interpreter.intrepret_response(resp), + Err(Error::NotValidXfrResponse) )); } @@ -70,21 +72,21 @@ fn error_axfr_response_is_rejected() { // Create an AXFR request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); - // Create an XFR response processor. - let mut processor = XfrResponseInterpreter::new(); + // Create an XFR response interpreter. + let mut interpreter = XfrResponseInterpreter::new(); // Create a minimal valid AXFR response, just something that should - // not be rejected by the XFR processor due to its content. It should + // not be rejected by the XFR interpreter due to its content. It should // however be rejected due to the non-NOERROR rcode. let mut answer = mk_empty_answer(&req, Rcode::SERVFAIL); add_answer_record(&req, &mut answer, mk_soa(Serial::now())); let resp = answer.into_message(); // Process the response and assert that it is rejected as not being - // a valid XFR response and that no XFR processor events were emitted. + // a valid XFR response and that no XFR interpreter updates were emitted. assert!(matches!( - processor.process_answer(resp), - Err(ProcessingError::NotValidXfrResponse) + interpreter.intrepret_response(resp), + Err(Error::NotValidXfrResponse) )); } @@ -95,8 +97,8 @@ fn incomplete_axfr_response_is_accepted() { // Create an AXFR request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); - // Create an XFR response processor. - let mut processor = XfrResponseInterpreter::new(); + // Create an XFR response interpreter. + let mut interpreter = XfrResponseInterpreter::new(); // Create an incomplete AXFR response. A proper AXFR response has at // least two identical SOA records, one at the start and one at the @@ -108,9 +110,9 @@ fn incomplete_axfr_response_is_accepted() { let resp = answer.into_message(); // Process the response. - let mut it = processor.process_answer(resp).unwrap(); + let mut it = interpreter.intrepret_response(resp).unwrap(); - // Verify that no events are by the XFR processor. + // Verify that no updates are by the XFR interpreter. assert!(it.next().is_none()); } @@ -121,8 +123,8 @@ fn axfr_response_with_only_soas_is_accepted() { // Create an AXFR request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); - // Create an XFR response processor. - let mut processor = XfrResponseInterpreter::new(); + // Create an XFR response interpreter. + let mut interpreter = XfrResponseInterpreter::new(); // Create a complete but minimal AXFR response. A proper AXFR response // has at least two identical SOA records, one at the start and one at @@ -136,9 +138,9 @@ fn axfr_response_with_only_soas_is_accepted() { let resp = answer.into_message(); // Process the response. - let mut it = processor.process_answer(resp).unwrap(); + let mut it = interpreter.intrepret_response(resp).unwrap(); - // Verify the events emitted by the XFR processor. + // Verify the updates emitted by the XFR interpreter. assert!(matches!(it.next(), Some(Ok(ZU::Finished(_))))); assert!(it.next().is_none()); } @@ -150,8 +152,8 @@ fn axfr_multi_response_with_only_soas_is_accepted() { // Create an AXFR request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); - // Create an XFR response processor. - let mut processor = XfrResponseInterpreter::new(); + // Create an XFR response interpreter. + let mut interpreter = XfrResponseInterpreter::new(); // Create a complete but minimal AXFR response. A proper AXFR response // has at least two identical SOA records, one at the start and one at @@ -164,9 +166,9 @@ fn axfr_multi_response_with_only_soas_is_accepted() { let resp = answer.into_message(); // Process the response. - let mut it = processor.process_answer(resp).unwrap(); + let mut it = interpreter.intrepret_response(resp).unwrap(); - // Verify the events emitted by the XFR processor. + // Verify the updates emitted by the XFR interpreter. assert!(it.next().is_none()); // Create another AXFR response to complete the transfer. @@ -175,22 +177,22 @@ fn axfr_multi_response_with_only_soas_is_accepted() { let resp = answer.into_message(); // Process the response. - let mut it = processor.process_answer(resp).unwrap(); + let mut it = interpreter.intrepret_response(resp).unwrap(); - // Verify the events emitted by the XFR processor. + // Verify the updates emitted by the XFR interpreter. assert!(matches!(it.next(), Some(Ok(ZU::Finished(_))))); assert!(it.next().is_none()); } #[test] -fn axfr_response_generates_expected_events() { +fn axfr_response_generates_expected_updates() { init_logging(); // Create an AXFR request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); - // Create an XFR response processor. - let mut processor = XfrResponseInterpreter::new(); + // Create an XFR response interpreter. + let mut interpreter = XfrResponseInterpreter::new(); // Create an AXFR response. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); @@ -203,9 +205,9 @@ fn axfr_response_generates_expected_events() { let resp = answer.into_message(); // Process the response. - let mut it = processor.process_answer(resp).unwrap(); + let mut it = interpreter.intrepret_response(resp).unwrap(); - // Verify the events emitted by the XFR processor. + // Verify the updates emitted by the XFR interpreter. let s = serial; assert!(matches!(it.next(), Some(Ok(ZU::AddRecord(n, _))) if n == s)); assert!(matches!(it.next(), Some(Ok(ZU::AddRecord(n, _))) if n == s)); @@ -214,7 +216,7 @@ fn axfr_response_generates_expected_events() { } #[test] -fn ixfr_response_generates_expected_events() { +fn ixfr_response_generates_expected_updates() { init_logging(); // Create an IXFR request to reply to. @@ -225,8 +227,8 @@ fn ixfr_response_generates_expected_events() { add_authority_record(&mut authority, soa); let req = authority.into_message(); - // Create an XFR response processor. - let mut processor = XfrResponseInterpreter::new(); + // Create an XFR response interpreter. + let mut interpreter = XfrResponseInterpreter::new(); // Prepare some serial numbers and SOA records to use in the IXFR response. let old_serial = client_serial; @@ -256,7 +258,7 @@ fn ixfr_response_generates_expected_events() { let resp = answer.into_message(); // Process the response. - let it = processor.process_answer(resp).unwrap(); + let it = interpreter.intrepret_response(resp).unwrap(); // Make parsed versions of the old and new SOAs. let mut buf = BytesMut::new(); @@ -271,10 +273,10 @@ fn ixfr_response_generates_expected_events() { let mut parser = Parser::from_ref(&buf); let expected_old_soa = Soa::parse(&mut parser).unwrap(); - // Verify the events emitted by the XFR processor. + // Verify the updates emitted by the XFR interpreter. let owner = ParsedName::::from(Name::from_str("example.com").unwrap()); - let expected_events: [Result, IterationError>; 7] = [ + let expected_updates: [Result, IterationError>; 7] = [ Ok(ZoneUpdate::BeginBatchDelete(Record::from(( owner.clone(), 0, @@ -324,11 +326,11 @@ fn ixfr_response_generates_expected_events() { )))), ]; - assert!(it.eq(expected_events)); + assert!(it.eq(expected_updates)); } #[test] -fn multi_ixfr_response_generates_expected_events() { +fn multi_ixfr_response_generates_expected_updates() { init_logging(); // Create an IXFR request to reply to. @@ -339,9 +341,6 @@ fn multi_ixfr_response_generates_expected_events() { add_authority_record(&mut authority, soa); let req = authority.into_message(); - // Create an XFR response processor. - let mut processor = XfrResponseInterpreter::new(); - // Prepare some serial numbers and SOA records to use in the IXFR response. let old_serial = client_serial; let new_serial = client_serial.add(1); @@ -349,26 +348,96 @@ fn multi_ixfr_response_generates_expected_events() { let new_soa = mk_soa(new_serial); // Create a partial IXFR response. - let mut answer = mk_empty_answer(&req, Rcode::NOERROR); - // Outer SOA with servers current SOA - add_answer_record(&req, &mut answer, new_soa.clone()); - // Start of diff sequence: SOA of the servers' previous zone version - // (which matches that of the client) followed by records to be - // deleted as they were in that version of the zone but are not in the - // new version of the zone. - add_answer_record(&req, &mut answer, old_soa); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - let resp = answer.into_message(); + let resp = mk_first_ixfr_response(&req, &new_soa, old_soa); + + // Create an XFR response interpreter. + let mut interpreter = XfrResponseInterpreter::new(); // Process the response. - let mut it = processor.process_answer(resp).unwrap(); + let mut it = interpreter.intrepret_response(resp).unwrap(); - // Verify the events emitted by the XFR processor. + // Verify the updates emitted by the XFR interpreter. assert!(matches!(it.next(), Some(Ok(ZU::BeginBatchDelete(_))))); assert!(matches!(it.next(), Some(Ok(ZU::DeleteRecord(..))))); assert!(it.next().is_none()); // Craete a second IXFR response that completes the transfer + let resp = mk_second_ixfr_response(req, new_soa); + + // Process the response. + let mut it = interpreter.intrepret_response(resp).unwrap(); + + // Verify the updates emitted by the XFR interpreter. + assert!(matches!(it.next(), Some(Ok(ZU::DeleteRecord(..))))); + assert!(matches!(it.next(), Some(Ok(ZU::BeginBatchAdd(_))))); + assert!(matches!(it.next(), Some(Ok(ZU::AddRecord(..))))); + assert!(matches!(it.next(), Some(Ok(ZU::AddRecord(..))))); + assert!(matches!(it.next(), Some(Ok(ZU::Finished(_))))); + assert!(it.next().is_none()); +} + +#[test] +fn is_finished() { + init_logging(); + + // Create an IXFR request to reply to. + let req = mk_request("example.com", Rtype::IXFR); + let mut authority = req.authority(); + let client_serial = Serial::now(); + let soa = mk_soa(client_serial); + add_authority_record(&mut authority, soa); + let req = authority.into_message(); + + // Prepare some serial numbers and SOA records to use in the IXFR response. + let old_serial = client_serial; + let new_serial = client_serial.add(1); + let old_soa = mk_soa(old_serial); + let new_soa = mk_soa(new_serial); + + // Create a partial IXFR response. + let mut responses: VecDeque<_> = vec![ + mk_first_ixfr_response(&req, &new_soa, old_soa), + mk_second_ixfr_response(req, new_soa), + ] + .into(); + + // Create an XFR response interpreter. + let mut interpreter = XfrResponseInterpreter::new(); + + // Process the responses + let mut count = 0; + while !interpreter.is_finished() { + let resp = responses.pop_front().unwrap(); + let it = interpreter.intrepret_response(resp).unwrap(); + count += it.count(); + } + + assert!(interpreter.is_finished()); + assert!(responses.is_empty()); + assert_eq!(count, 7); +} + +fn mk_first_ixfr_response( + req: &Message, + new_soa: &Soa>, + old_soa: Soa>, +) -> Message { + let mut answer = mk_empty_answer(req, Rcode::NOERROR); + // Outer SOA with servers current SOA + add_answer_record(req, &mut answer, new_soa.clone()); + // Start of diff sequence: SOA of the servers' previous zone version + // (which matches that of the client) followed by records to be + // deleted as they were in that version of the zone but are not in the + // new version of the zone. + add_answer_record(req, &mut answer, old_soa); + add_answer_record(req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); + answer.into_message() +} + +fn mk_second_ixfr_response( + req: Message, + new_soa: Soa>, +) -> Message { let mut answer = mk_empty_answer(&req, Rcode::NOERROR); add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); // SOA of the servers` new zone version (which is ahead of that of the @@ -379,18 +448,7 @@ fn multi_ixfr_response_generates_expected_events() { add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); // Closing SOA with servers current SOA add_answer_record(&req, &mut answer, new_soa); - let resp = answer.into_message(); - - // Process the response. - let mut it = processor.process_answer(resp).unwrap(); - - // Verify the events emitted by the XFR processor. - assert!(matches!(it.next(), Some(Ok(ZU::DeleteRecord(..))))); - assert!(matches!(it.next(), Some(Ok(ZU::BeginBatchAdd(_))))); - assert!(matches!(it.next(), Some(Ok(ZU::AddRecord(..))))); - assert!(matches!(it.next(), Some(Ok(ZU::AddRecord(..))))); - assert!(matches!(it.next(), Some(Ok(ZU::Finished(_))))); - assert!(it.next().is_none()); + answer.into_message() } //------------ Helper functions ------------------------------------------- diff --git a/src/net/xfr/protocol/types.rs b/src/net/xfr/protocol/types.rs index 1cb0c756a..5b358a404 100644 --- a/src/net/xfr/protocol/types.rs +++ b/src/net/xfr/protocol/types.rs @@ -77,7 +77,7 @@ impl IxfrUpdateMode { /// /// [`XfrResponseInterpreter`]: super::interpreter::XfrResponseInterpreter #[derive(Debug)] -pub enum ProcessingError { +pub enum Error { /// The message could not be parsed. ParseError(ParseError), @@ -86,20 +86,22 @@ pub enum ProcessingError { /// At least one record in the XFR response sequence is incorrect. Malformed, + + /// A complete transfer was already processed. + Finished, } -impl std::fmt::Display for ProcessingError { +impl std::fmt::Display for Error { fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { match self { - ProcessingError::ParseError(err) => { + Error::ParseError(err) => { f.write_fmt(format_args!("XFR response parsing error: {err}")) } - ProcessingError::NotValidXfrResponse => { + Error::NotValidXfrResponse => { f.write_str("Not a valid XFR response") } - ProcessingError::Malformed => { - f.write_str("Malformed XFR response") - } + Error::Malformed => f.write_str("Malformed XFR response"), + Error::Finished => f.write_str("XFR already finished"), } } } diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index dc808457b..415f043a5 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -7,9 +7,11 @@ use std::vec::Vec; use bytes::Bytes; use serde::{Deserialize, Serialize}; +use crate::base::name::Name; use crate::base::rdata::RecordData; +use crate::base::record::Record; +use crate::base::Serial; use crate::base::{iana::Rtype, Ttl}; -use crate::base::{Name, Record, Serial}; use crate::rdata::ZoneRecordData; //------------ Type Aliases -------------------------------------------------- @@ -302,19 +304,6 @@ pub enum ZoneUpdate { /// For example this could be used to trigger an atomic commit of a set of /// related pending changes. Finished(R), - - /// A sequence of updates has been deemed to be corrupt. - /// - /// This signals that something is wrong with the sequence of updates - /// being processed and that it will not be possible to complete the - /// sequence. - /// - /// For example this could occur if a sequence of XFR responses from a - /// primary server is fatally interrupted and cannot be completed. Some - /// responses may have already been communicated as [`ZoneUpdate`]s but - /// they should not be used without the remainder that will now not be - /// received. - Corrupt, } //--- Display @@ -329,7 +318,6 @@ impl std::fmt::Display for ZoneUpdate { } ZoneUpdate::BeginBatchAdd(_) => f.write_str("BeginBatchAdd"), ZoneUpdate::Finished(_) => f.write_str("Finished"), - ZoneUpdate::Corrupt => f.write_str("Corrupt"), } } } From 34e893b32af25badd023234cb48cf72e24093175 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 17 Sep 2024 15:10:37 +0200 Subject: [PATCH 191/333] Follow upstream changes, some cleanup, and more RustDoc. --- src/net/xfr/protocol/tests.rs | 1 - src/zonetree/update.rs | 179 ++++++++++++++++++++-------------- 2 files changed, 105 insertions(+), 75 deletions(-) diff --git a/src/net/xfr/protocol/tests.rs b/src/net/xfr/protocol/tests.rs index 23d0ea425..c5644e535 100644 --- a/src/net/xfr/protocol/tests.rs +++ b/src/net/xfr/protocol/tests.rs @@ -1,5 +1,4 @@ use core::str::FromStr; - use std::collections::VecDeque; use bytes::{Bytes, BytesMut}; diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index f8bacc0f8..88113d907 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -1,4 +1,38 @@ -//! Support for applying XFR changes to a [`Zone`]. +//! High-level support for applying changes to a [`Zone`]. +//! +//! This module provides a high-level interface for making alterations to the +//! content of zones without requiring knowledge of the low-level details of +//! how the [`WritableZone`] trait implemented by [`Zone`] works. +//! +//! # Applying XFR changes to a zone +//! +//! ```rust +//! // Given a ZoneUpdater +//! let mut updater = ZoneUpdater::new(&zone).await.unwrap(); +//! +//! // And a zone +//! let builder = ZoneBuilder::new(Name::from_str("example.com").unwrap(), Class::IN); +//! let zone = builder.build() +//! +//! // And an XFR response interpreter +//! let mut interpreter = XfrResponseInterpreter::new(); +//! +//! // Iterate over the XFR responses applying the updates to the zone +//! while !updater.finished() { +//! // Get the next XFR response +//! // let next_xfr_response = ... +//! +//! // Convert it to an update iterator +//! let it = interpreter.intrepret_response(next_xfr_response).unwrap(); +//! +//! // Iterate over the updates +//! for update in it { +//! // Apply each update to the zone +//! updater.apply(update.unwrap()).await.unwrap(); +//! } +//! } +//! +//! ``` use std::borrow::ToOwned; use std::boxed::Box; @@ -16,8 +50,8 @@ use super::types::ZoneUpdate; use super::{WritableZone, WritableZoneNode, Zone}; /// TODO -pub struct ZoneUpdater { - zone: Zone, +pub struct ZoneUpdater<'a> { + zone: &'a Zone, write: WriteState, @@ -26,10 +60,10 @@ pub struct ZoneUpdater { first_event_seen: bool, } -impl ZoneUpdater { +impl<'a> ZoneUpdater<'a> { /// TODO - pub async fn new(zone: Zone) -> std::io::Result { - let write = WriteState::new(&zone).await?; + pub async fn new(zone: &'a Zone) -> std::io::Result { + let write = WriteState::new(zone).await?; Ok(Self { zone, @@ -38,7 +72,61 @@ impl ZoneUpdater { first_event_seen: false, }) } +} + +impl<'a> ZoneUpdater<'a> { + /// TODO + pub async fn apply( + &mut self, + evt: ZoneUpdate, + ) -> Result<(), ()> { + trace!("Event: {evt}"); + match evt { + ZoneUpdate::DeleteRecord(_serial, rec) => { + self.delete_record(rec).await? + } + + ZoneUpdate::AddRecord(_serial, rec) => { + self.add_record(rec).await? + } + + // Note: Batches first contain deletions then additions, so batch + // deletion signals the start of a batch, and the end of any + // previous batch addition. + ZoneUpdate::BeginBatchDelete(_old_soa) => { + if self.batching { + // Commit the previous batch. + self.write.commit().await?; + // Open a writer for the new batch. + self.write.reopen().await.map_err(|_| ())?; + } + self.batching = true; + } + + ZoneUpdate::BeginBatchAdd(new_soa) => { + // Update the SOA record. + self.update_soa(new_soa).await?; + self.batching = true; + } + + ZoneUpdate::Finished(zone_soa) => { + if !self.batching { + // Update the SOA record. + self.update_soa(zone_soa).await?; + } + // Commit the previous batch. + self.write.commit().await?; + } + } + + self.first_event_seen = true; + + Ok(()) + } +} + +impl<'a> ZoneUpdater<'a> { fn mk_relative_name_iterator<'l>( apex_name: &Name, qname: &'l impl ToName, @@ -98,60 +186,6 @@ impl ZoneUpdater { Ok((rtype, data, end_node, rrset)) } - /// TODO - pub async fn handle_event( - &mut self, - evt: ZoneUpdate, - ) -> Result<(), ()> { - trace!("Event: {evt}"); - match evt { - ZoneUpdate::DeleteRecord(_serial, rec) => { - self.delete_record(rec).await? - } - - ZoneUpdate::AddRecord(_serial, rec) => { - self.add_record(rec).await? - } - - // Note: Batches first contain deletions then additions, so batch - // deletion signals the start of a batch, and the end of any - // previous batch addition. - ZoneUpdate::BeginBatchDelete(_old_soa) => { - if self.batching { - // Commit the previous batch. - self.write.commit().await?; - // Open a writer for the new batch. - self.write.reopen().await.map_err(|_| ())?; - } - - self.batching = true; - } - - ZoneUpdate::BeginBatchAdd(new_soa) => { - // Update the SOA record. - self.update_soa(new_soa).await?; - self.batching = true; - } - - ZoneUpdate::Finished(zone_soa) => { - if !self.batching { - // Update the SOA record. - self.update_soa(zone_soa).await?; - } - // Commit the previous batch. - self.write.commit().await?; - } - - ZoneUpdate::Corrupt => { - // ??? - } - } - - self.first_event_seen = true; - - Ok(()) - } - async fn update_soa( &mut self, new_soa: Record< @@ -323,7 +357,7 @@ mod tests { let zone = mk_empty_zone("example.com"); - let mut evt_handler = ZoneUpdater::new(zone.clone()).await.unwrap(); + let mut updater = ZoneUpdater::new(&zone).await.unwrap(); let s = Serial::now(); let soa = mk_soa(s); @@ -335,15 +369,12 @@ mod tests { soa, ); - evt_handler - .handle_event(ZoneUpdate::AddRecord(s, soa.clone())) + updater + .apply(ZoneUpdate::AddRecord(s, soa.clone())) .await .unwrap(); - evt_handler - .handle_event(ZoneUpdate::Finished(soa)) - .await - .unwrap(); + updater.apply(ZoneUpdate::Finished(soa)).await.unwrap(); } #[tokio::test] @@ -352,13 +383,13 @@ mod tests { let zone = mk_empty_zone("example.com"); - let mut evt_handler = ZoneUpdater::new(zone.clone()).await.unwrap(); + let mut updater = ZoneUpdater::new(&zone).await.unwrap(); // Create an AXFR request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); - // Create an XFR response processor. - let mut processor = XfrResponseInterpreter::new(); + // Create an XFR response interpreter. + let mut interpreter = XfrResponseInterpreter::new(); // Create an AXFR response. let mut answer = mk_empty_answer(&req, Rcode::NOERROR); @@ -371,11 +402,11 @@ mod tests { let resp = answer.into_message(); // Process the response. - let it = processor.process_answer(resp).unwrap(); + let it = interpreter.intrepret_response(resp).unwrap(); - for evt in it { - let evt = evt.unwrap(); - evt_handler.handle_event(evt).await.unwrap(); + for update in it { + let update = update.unwrap(); + updater.apply(update).await.unwrap(); } dbg!(zone); From 871ce9b53f175c29e0b6303f32523fdd0c5d21a5 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 17 Sep 2024 15:17:15 +0200 Subject: [PATCH 192/333] Remove unnecessary change to exsting public interface. --- src/tsig/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index 2fa14b4e3..d88362f76 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -1726,7 +1726,7 @@ enum ServerErrorInner { } impl ServerError { - pub fn unsigned(error: TsigRcode) -> Self { + fn unsigned(error: TsigRcode) -> Self { ServerError(ServerErrorInner::Unsigned { error }) } From f719bbf4ea4f615173f65e568a7a2ae1ed5b0403 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 17 Sep 2024 15:46:14 +0200 Subject: [PATCH 193/333] Avoid lifetime issues. --- src/zonetree/update.rs | 39 +++++++++++++++++++++++---------------- 1 file changed, 23 insertions(+), 16 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 88113d907..398cebbcb 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -33,6 +33,9 @@ //! } //! //! ``` +use core::future::Future; +use core::pin::Pin; + use std::borrow::ToOwned; use std::boxed::Box; @@ -50,8 +53,8 @@ use super::types::ZoneUpdate; use super::{WritableZone, WritableZoneNode, Zone}; /// TODO -pub struct ZoneUpdater<'a> { - zone: &'a Zone, +pub struct ZoneUpdater { + zone: Zone, write: WriteState, @@ -60,21 +63,25 @@ pub struct ZoneUpdater<'a> { first_event_seen: bool, } -impl<'a> ZoneUpdater<'a> { +impl ZoneUpdater { /// TODO - pub async fn new(zone: &'a Zone) -> std::io::Result { - let write = WriteState::new(zone).await?; - - Ok(Self { - zone, - write, - batching: false, - first_event_seen: false, + pub fn new( + zone: Zone, + ) -> Pin>>> { + Box::pin(async move { + let write = WriteState::new(zone.clone()).await?; + + Ok(Self { + zone, + write, + batching: false, + first_event_seen: false, + }) }) } } -impl<'a> ZoneUpdater<'a> { +impl ZoneUpdater { /// TODO pub async fn apply( &mut self, @@ -126,7 +133,7 @@ impl<'a> ZoneUpdater<'a> { } } -impl<'a> ZoneUpdater<'a> { +impl ZoneUpdater { fn mk_relative_name_iterator<'l>( apex_name: &Name, qname: &'l impl ToName, @@ -299,7 +306,7 @@ struct WriteState { } impl WriteState { - async fn new(zone: &Zone) -> std::io::Result { + async fn new(zone: Zone) -> std::io::Result { let write = zone.write().await; let writable = Some(write.open(true).await?); Ok(Self { write, writable }) @@ -357,7 +364,7 @@ mod tests { let zone = mk_empty_zone("example.com"); - let mut updater = ZoneUpdater::new(&zone).await.unwrap(); + let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); let s = Serial::now(); let soa = mk_soa(s); @@ -383,7 +390,7 @@ mod tests { let zone = mk_empty_zone("example.com"); - let mut updater = ZoneUpdater::new(&zone).await.unwrap(); + let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); // Create an AXFR request to reply to. let req = mk_request("example.com", Rtype::AXFR).into_message(); From a048f5543dd535950392088f2b7fb91190a36a58 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 17 Sep 2024 16:06:38 +0200 Subject: [PATCH 194/333] RustDoc and typo correction. --- src/zonetree/update.rs | 39 ++++++++++++++++++++++++++++----------- 1 file changed, 28 insertions(+), 11 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 398cebbcb..942f82b8f 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -3,27 +3,43 @@ //! This module provides a high-level interface for making alterations to the //! content of zones without requiring knowledge of the low-level details of //! how the [`WritableZone`] trait implemented by [`Zone`] works. +//! +//! It can be used manually, or in combination with a source of +//! [`ZoneUpdate`]s such as +//! [`XfrResponseInterpreter`][crate::net::xfr::protocol::XfrResponseInterpreter]. //! //! # Applying XFR changes to a zone //! -//! ```rust -//! // Given a ZoneUpdater -//! let mut updater = ZoneUpdater::new(&zone).await.unwrap(); -//! -//! // And a zone +//! ```no_run +//! # use std::str::FromStr; +//! # +//! # use domain::base::iana::Class; +//! # use domain::base::MessageBuilder; +//! # use domain::base::Name; +//! # use domain::net::xfr::protocol::XfrResponseInterpreter; +//! # use domain::zonetree::ZoneBuilder; +//! # use domain::zonetree::update::ZoneUpdater; +//! # +//! # #[tokio::main] +//! # async fn main() { +//! # +//! // Given a zone //! let builder = ZoneBuilder::new(Name::from_str("example.com").unwrap(), Class::IN); -//! let zone = builder.build() +//! let zone = builder.build(); +//! +//! // And a ZoneUpdater +//! let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); //! //! // And an XFR response interpreter //! let mut interpreter = XfrResponseInterpreter::new(); //! //! // Iterate over the XFR responses applying the updates to the zone -//! while !updater.finished() { +//! while !interpreter.is_finished() { //! // Get the next XFR response -//! // let next_xfr_response = ... +//! let next_xfr_response = MessageBuilder::new_bytes().into_message(); // A dummy response //! //! // Convert it to an update iterator -//! let it = interpreter.intrepret_response(next_xfr_response).unwrap(); +//! let it = interpreter.interpret_response(next_xfr_response).unwrap(); //! //! // Iterate over the updates //! for update in it { @@ -31,7 +47,8 @@ //! updater.apply(update.unwrap()).await.unwrap(); //! } //! } -//! +//! # +//! # } //! ``` use core::future::Future; use core::pin::Pin; @@ -409,7 +426,7 @@ mod tests { let resp = answer.into_message(); // Process the response. - let it = interpreter.intrepret_response(resp).unwrap(); + let it = interpreter.interpret_response(resp).unwrap(); for update in it { let update = update.unwrap(); From 3f91553d37974da909d51bba88207badacfc85cb Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 17 Sep 2024 16:10:11 +0200 Subject: [PATCH 195/333] Cargo fmt. --- src/zonetree/update.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 942f82b8f..b95982729 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -3,7 +3,7 @@ //! This module provides a high-level interface for making alterations to the //! content of zones without requiring knowledge of the low-level details of //! how the [`WritableZone`] trait implemented by [`Zone`] works. -//! +//! //! It can be used manually, or in combination with a source of //! [`ZoneUpdate`]s such as //! [`XfrResponseInterpreter`][crate::net::xfr::protocol::XfrResponseInterpreter]. @@ -12,21 +12,21 @@ //! //! ```no_run //! # use std::str::FromStr; -//! # +//! # //! # use domain::base::iana::Class; //! # use domain::base::MessageBuilder; //! # use domain::base::Name; //! # use domain::net::xfr::protocol::XfrResponseInterpreter; //! # use domain::zonetree::ZoneBuilder; //! # use domain::zonetree::update::ZoneUpdater; -//! # +//! # //! # #[tokio::main] //! # async fn main() { -//! # +//! # //! // Given a zone //! let builder = ZoneBuilder::new(Name::from_str("example.com").unwrap(), Class::IN); //! let zone = builder.build(); -//! +//! //! // And a ZoneUpdater //! let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); //! @@ -47,7 +47,7 @@ //! updater.apply(update.unwrap()).await.unwrap(); //! } //! } -//! # +//! # //! # } //! ``` use core::future::Future; From 4f67ecbad0e9cbcfbfdd622ec1985e31224d0beb Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 17 Sep 2024 16:18:29 +0200 Subject: [PATCH 196/333] Rename `XfrRecord` to `ParsedRecord`, `evt` to `update` and add more RustDoc. --- src/net/xfr/protocol/interpreter.rs | 6 ++--- src/net/xfr/protocol/iterator.rs | 4 ++-- src/net/xfr/protocol/mod.rs | 2 +- src/net/xfr/protocol/tests.rs | 4 ++-- src/net/xfr/protocol/types.rs | 2 +- src/zonetree/update.rs | 36 +++++++++++++++++++---------- 6 files changed, 33 insertions(+), 21 deletions(-) diff --git a/src/net/xfr/protocol/interpreter.rs b/src/net/xfr/protocol/interpreter.rs index ab39d7a55..8bfe8e049 100644 --- a/src/net/xfr/protocol/interpreter.rs +++ b/src/net/xfr/protocol/interpreter.rs @@ -8,7 +8,7 @@ use crate::rdata::{Soa, ZoneRecordData}; use crate::zonetree::types::ZoneUpdate; use super::iterator::XfrZoneUpdateIterator; -use super::types::{Error, IxfrUpdateMode, XfrRecord, XfrType}; +use super::types::{Error, IxfrUpdateMode, ParsedRecord, XfrType}; //------------ XfrResponseInterpreter ----------------------------------------- @@ -259,8 +259,8 @@ impl RecordProcessor { /// record, if any. pub(super) fn process_record( &mut self, - rec: XfrRecord, - ) -> ZoneUpdate { + rec: ParsedRecord, + ) -> ZoneUpdate { self.rr_count += 1; // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2 diff --git a/src/net/xfr/protocol/iterator.rs b/src/net/xfr/protocol/iterator.rs index 233191e46..21464061f 100644 --- a/src/net/xfr/protocol/iterator.rs +++ b/src/net/xfr/protocol/iterator.rs @@ -9,7 +9,7 @@ use crate::rdata::ZoneRecordData; use crate::zonetree::types::ZoneUpdate; use super::interpreter::RecordProcessor; -use super::types::{Error, IterationError, XfrRecord}; +use super::types::{Error, IterationError, ParsedRecord}; //------------ XfrZoneUpdateIterator ------------------------------------------ @@ -62,7 +62,7 @@ impl<'a, 'b> XfrZoneUpdateIterator<'a, 'b> { } impl<'a, 'b> Iterator for XfrZoneUpdateIterator<'a, 'b> { - type Item = Result, IterationError>; + type Item = Result, IterationError>; fn next(&mut self) -> Option { match self.iter.next()? { diff --git a/src/net/xfr/protocol/mod.rs b/src/net/xfr/protocol/mod.rs index 9cff713a8..e6e5ae201 100644 --- a/src/net/xfr/protocol/mod.rs +++ b/src/net/xfr/protocol/mod.rs @@ -15,4 +15,4 @@ mod tests; pub use interpreter::XfrResponseInterpreter; pub use iterator::XfrZoneUpdateIterator; -pub use types::{Error, IterationError, XfrRecord}; +pub use types::{Error, IterationError, ParsedRecord}; diff --git a/src/net/xfr/protocol/tests.rs b/src/net/xfr/protocol/tests.rs index 2753a5682..05ecd26b6 100644 --- a/src/net/xfr/protocol/tests.rs +++ b/src/net/xfr/protocol/tests.rs @@ -19,7 +19,7 @@ use crate::rdata::{Soa, ZoneRecordData, A}; use crate::zonetree::types::{ZoneUpdate, ZoneUpdate as ZU}; use super::interpreter::XfrResponseInterpreter; -use super::types::{Error, IterationError, XfrRecord}; +use super::types::{Error, IterationError, ParsedRecord}; #[test] fn non_xfr_response_is_rejected() { @@ -276,7 +276,7 @@ fn ixfr_response_generates_expected_updates() { // Verify the updates emitted by the XFR interpreter. let owner = ParsedName::::from(Name::from_str("example.com").unwrap()); - let expected_updates: [Result, IterationError>; 7] = [ + let expected_updates: [Result, IterationError>; 7] = [ Ok(ZoneUpdate::BeginBatchDelete(Record::from(( owner.clone(), 0, diff --git a/src/net/xfr/protocol/types.rs b/src/net/xfr/protocol/types.rs index 5b358a404..0f9ee1e64 100644 --- a/src/net/xfr/protocol/types.rs +++ b/src/net/xfr/protocol/types.rs @@ -12,7 +12,7 @@ use crate::{ /// The type of record processed by [`XfrResponseInterpreter`]. /// /// [`XfrResponseInterpreter`]: super::interpreter::XfrResponseInterpreter -pub type XfrRecord = +pub type ParsedRecord = Record, ZoneRecordData>>; //------------ XfrType -------------------------------------------------------- diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index b95982729..2ffc7dfd6 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -61,7 +61,7 @@ use tracing::{error, trace}; use crate::base::name::{FlattenInto, Label, ToLabelIter}; use crate::base::{Name, ParsedName, Record, Rtype, ToName}; -use crate::net::xfr::protocol::XfrRecord; +use crate::net::xfr::protocol::ParsedRecord; use crate::rdata::ZoneRecordData; use crate::zonetree::{Rrset, SharedRrset}; @@ -69,19 +69,29 @@ use super::error::OutOfZone; use super::types::ZoneUpdate; use super::{WritableZone, WritableZoneNode, Zone}; -/// TODO +/// Apply a sequence of [`ZoneUpdate`]s to alter the content of a [`Zone`]. pub struct ZoneUpdater { + /// The zone to be updated. zone: Zone, + /// The current write handles in use. write: WriteState, + /// Whether or not we entered an IXFR-like batching mode. batching: bool, - first_event_seen: bool, + /// Whether or not at least one update has been applied so far. + first_update_seen: bool, } impl ZoneUpdater { - /// TODO + /// Creates a new [`ZoneUpdater`] that will update the given [`Zone`] + /// content. + /// + /// Returns the new instance on success, or an error if the zone could not + /// be opened for writing. + /// + /// Use [`apply`][Self::apply] to apply changes to the zone. pub fn new( zone: Zone, ) -> Pin>>> { @@ -92,20 +102,22 @@ impl ZoneUpdater { zone, write, batching: false, - first_event_seen: false, + first_update_seen: false, }) }) } } impl ZoneUpdater { - /// TODO + /// Apply the given [`ZoneUpdate`] to the [`Zone`] being updated. + /// + /// Returns `Ok` on success, `Err` otherwise. pub async fn apply( &mut self, - evt: ZoneUpdate, + update: ZoneUpdate, ) -> Result<(), ()> { - trace!("Event: {evt}"); - match evt { + trace!("Event: {update}"); + match update { ZoneUpdate::DeleteRecord(_serial, rec) => { self.delete_record(rec).await? } @@ -144,7 +156,7 @@ impl ZoneUpdater { } } - self.first_event_seen = true; + self.first_update_seen = true; Ok(()) } @@ -170,7 +182,7 @@ impl ZoneUpdater { async fn prep_add_del( &mut self, - rec: XfrRecord, + rec: ParsedRecord, ) -> Result< ( Rtype, @@ -275,7 +287,7 @@ impl ZoneUpdater { ZoneRecordData>, >, ) -> Result<(), ()> { - if !self.first_event_seen && rec.rtype() == Rtype::SOA { + if !self.first_update_seen && rec.rtype() == Rtype::SOA { // If the first event is the addition of a SOA record to the zone, // this must be a complete replacement of the zone (as you can't // have two SOA records), i.e. something like an AXFR transfer. We From 024a980e3336b52bb570109bbe9a85f51cb74470 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 17 Sep 2024 16:19:44 +0200 Subject: [PATCH 197/333] Cargo fmt. --- src/net/xfr/protocol/tests.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/net/xfr/protocol/tests.rs b/src/net/xfr/protocol/tests.rs index 05ecd26b6..73992134e 100644 --- a/src/net/xfr/protocol/tests.rs +++ b/src/net/xfr/protocol/tests.rs @@ -276,7 +276,8 @@ fn ixfr_response_generates_expected_updates() { // Verify the updates emitted by the XFR interpreter. let owner = ParsedName::::from(Name::from_str("example.com").unwrap()); - let expected_updates: [Result, IterationError>; 7] = [ + let expected_updates: [Result, IterationError>; + 7] = [ Ok(ZoneUpdate::BeginBatchDelete(Record::from(( owner.clone(), 0, From 74b129f162b5bf015fa2754c7b987f36527ceb82 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 17 Sep 2024 18:12:40 +0200 Subject: [PATCH 198/333] More RustDoc. --- src/zonetree/types.rs | 33 +++++++++++++++++++++++++++++++-- 1 file changed, 31 insertions(+), 2 deletions(-) diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index 93780abbf..f101b6ba8 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -284,11 +284,40 @@ impl ZoneDiff { /// An update to be applied to a [`Zone`]. /// -/// Note: This enum is marked as `#[non_exhaustive]` to permit addition of -/// more update operations in future, e.g. to support RFC 2136 Dynamic Updates +/// # Design +/// +/// The variants of this enum are modelled after the way the AXFR and IXFR +/// protocols represent updates to zones. +/// +/// AXFR responses can be represented as a sequence of +/// [`ZoneUpdate::AddRecord`]s. +/// +/// IXFR responses can be represented as a sequence of batches, each +/// consisting of: +/// - [`ZoneUpdate::BeginBatchDelete`] +/// - [`ZoneUpdate::DeleteRecord`]s _(zero or more)_ +/// - [`ZoneUpdate::BeginBatchAdd`] +/// - [`ZoneUpdate::AddRecord`]s _(zero or more)_ +/// +/// Both AXFR and IXFR responses encoded using this enum are terminated by a +/// final [`ZoneUpdate::Finished`]. +/// +/// # Use within this crate +/// +/// [`XfrResponseInterpreter`] can convert received XFR responses into +/// sequences of [`ZoneUpdate`]s. These can then be consumed by a +/// [`ZoneUpdater`] to effect changes to an existing [`Zone`]. +/// +/// # Future extensions +/// +/// This enum is marked as `#[non_exhaustive]` to permit addition of more +/// update operations in future, e.g. to support RFC 2136 Dynamic Updates /// operations. /// +/// [`XfrResponseInterpreter`]: +/// crate::net::xfr::protocol::XfrResponseInterpreter /// [`Zone`]: crate::zonetree::zone::Zone +/// [`ZoneUpdater`]: crate::zonetree::update::ZoneUpdater #[derive(Clone, Debug, PartialEq, Eq)] #[non_exhaustive] pub enum ZoneUpdate { From 071c58f7469089bbbead20d6f07dbc3d0789abd0 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 17 Sep 2024 18:22:47 +0200 Subject: [PATCH 199/333] More RustDoc. --- src/zonetree/types.rs | 20 ++++++++++++++------ src/zonetree/update.rs | 8 +++++--- 2 files changed, 19 insertions(+), 9 deletions(-) diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index f101b6ba8..f11c1e265 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -285,20 +285,20 @@ impl ZoneDiff { /// An update to be applied to a [`Zone`]. /// /// # Design -/// +/// /// The variants of this enum are modelled after the way the AXFR and IXFR /// protocols represent updates to zones. -/// +/// /// AXFR responses can be represented as a sequence of /// [`ZoneUpdate::AddRecord`]s. -/// +/// /// IXFR responses can be represented as a sequence of batches, each /// consisting of: /// - [`ZoneUpdate::BeginBatchDelete`] /// - [`ZoneUpdate::DeleteRecord`]s _(zero or more)_ /// - [`ZoneUpdate::BeginBatchAdd`] /// - [`ZoneUpdate::AddRecord`]s _(zero or more)_ -/// +/// /// Both AXFR and IXFR responses encoded using this enum are terminated by a /// final [`ZoneUpdate::Finished`]. /// @@ -307,9 +307,9 @@ impl ZoneDiff { /// [`XfrResponseInterpreter`] can convert received XFR responses into /// sequences of [`ZoneUpdate`]s. These can then be consumed by a /// [`ZoneUpdater`] to effect changes to an existing [`Zone`]. -/// +/// /// # Future extensions -/// +/// /// This enum is marked as `#[non_exhaustive]` to permit addition of more /// update operations in future, e.g. to support RFC 2136 Dynamic Updates /// operations. @@ -344,6 +344,10 @@ pub enum ZoneUpdate { /// Batching mode makes updates more predictable for the receiver to work /// with by limiting the updates that can be signalled next, enabling /// receiver logic to be simpler and more efficient. + /// + /// The record must be a SOA record that matches the SOA record of the + /// zone version in which the subsequent [`ZoneUpdate::DeleteRecord`]s + /// should be deleted. BeginBatchDelete(R), /// Start a batch add for the specified version (serial) of the zone. @@ -351,6 +355,10 @@ pub enum ZoneUpdate { /// This can only be signalled when already in batching mode, i.e. when /// `BeginBatchDelete` has already been signalled. /// + /// The record must be the SOA record to use for the new version of the + /// zone under which the subsequent [`ZoneUpdate::AddRecord`]s will be + /// added. + /// /// See `BeginBatchDelete` for more information. BeginBatchAdd(R), diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 2ffc7dfd6..17785076f 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -180,7 +180,8 @@ impl ZoneUpdater { Ok(qname) } - async fn prep_add_del( + /// Given a zone record, obtain a [`WritableZoneNode`] for the owner. + async fn get_writable_node_for_owner( &mut self, rec: ParsedRecord, ) -> Result< @@ -245,6 +246,7 @@ impl ZoneUpdater { Ok(()) } + /// Find and delete a record in the zone by exact match. async fn delete_record( &mut self, rec: Record< @@ -253,7 +255,7 @@ impl ZoneUpdater { >, ) -> Result<(), ()> { let (rtype, data, end_node, mut rrset) = - self.prep_add_del(rec).await?; + self.get_update_child_write_handle(rec).await?; let writable = self.write.writable.as_ref().unwrap(); @@ -302,7 +304,7 @@ impl ZoneUpdater { } let (rtype, data, end_node, mut rrset) = - self.prep_add_del(rec).await?; + self.get_update_child_write_handle(rec).await?; let writable = self.write.writable.as_ref().unwrap(); From 2d6b949b370322f99378a34ea21775ef0e830ba1 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 17 Sep 2024 23:49:03 +0200 Subject: [PATCH 200/333] More RustDoc. Error handling tweaks. Some simplification. --- src/zonetree/update.rs | 118 ++++++++++++++++++++++------------------- 1 file changed, 64 insertions(+), 54 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 17785076f..5777f2376 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -60,6 +60,7 @@ use bytes::Bytes; use tracing::{error, trace}; use crate::base::name::{FlattenInto, Label, ToLabelIter}; +use crate::base::scan::ScannerError; use crate::base::{Name, ParsedName, Record, Rtype, ToName}; use crate::net::xfr::protocol::ParsedRecord; use crate::rdata::ZoneRecordData; @@ -70,11 +71,23 @@ use super::types::ZoneUpdate; use super::{WritableZone, WritableZoneNode, Zone}; /// Apply a sequence of [`ZoneUpdate`]s to alter the content of a [`Zone`]. +/// +/// Each [`ZoneUpdate::BeginBatchDelete`] starts a new zone version, +/// +/// For each version of the zone that is edited the zone will be opened for +/// writing, edits made and then the changes committed, only then becoming +/// visible for readers of the zone. +/// +/// For each commit of the zone a diff of the changes made will be stored with +/// the zone. pub struct ZoneUpdater { /// The zone to be updated. zone: Zone, /// The current write handles in use. + /// + /// For each new zone version any old write state has to be committed and + /// a new write state opened. write: WriteState, /// Whether or not we entered an IXFR-like batching mode. @@ -115,7 +128,7 @@ impl ZoneUpdater { pub async fn apply( &mut self, update: ZoneUpdate, - ) -> Result<(), ()> { + ) -> std::io::Result<()> { trace!("Event: {update}"); match update { ZoneUpdate::DeleteRecord(_serial, rec) => { @@ -132,9 +145,13 @@ impl ZoneUpdater { ZoneUpdate::BeginBatchDelete(_old_soa) => { if self.batching { // Commit the previous batch. - self.write.commit().await?; + self.write.commit().await.map_err(|()| { + std::io::Error::custom( + "Error commiting changes to zone", + ) + })?; // Open a writer for the new batch. - self.write.reopen().await.map_err(|_| ())?; + self.write.reopen().await?; } self.batching = true; @@ -152,7 +169,9 @@ impl ZoneUpdater { self.update_soa(zone_soa).await?; } // Commit the previous batch. - self.write.commit().await?; + self.write.commit().await.map_err(|()| { + std::io::Error::custom("Error commiting changes to zone") + })?; } } @@ -181,29 +200,35 @@ impl ZoneUpdater { } /// Given a zone record, obtain a [`WritableZoneNode`] for the owner. + /// + /// A [`Zone`] is a tree structure which can be modified by descending the + /// tree from parent to child one (dot separated) label at a time. + /// + /// This function constructs an iterator over the labels of the owner name + /// of the given record then descends the tree one label at a time, + /// creating nodes if needed, until the appropriate end node has been + /// reached. + /// + /// If the owner name of the given record is not overlapping with the apex + /// name of the zone an out of zone error will occur. + /// + /// # Panics + /// + /// This function may panic if it is unable to create new tree nodes for + /// the record owner name. async fn get_writable_node_for_owner( &mut self, - rec: ParsedRecord, - ) -> Result< - ( - Rtype, - ZoneRecordData>, - Option>, - Rrset, - ), - (), - > { + rec: &ParsedRecord, + ) -> std::io::Result>> { let owner = rec.owner().to_owned(); - let ttl = rec.ttl(); - let rtype = rec.rtype(); - let data: ZoneRecordData> = - rec.into_data().flatten_into(); let mut end_node: Option> = None; let name = Self::mk_relative_name_iterator(self.zone.apex_name(), &owner) - .map_err(|_| ())?; + .map_err(|_| { + std::io::Error::custom("Record owner name out of zone") + })?; let writable = self.write.writable.as_ref().unwrap(); @@ -214,13 +239,11 @@ impl ZoneUpdater { Some(new_node) => new_node.update_child(label), None => writable.update_child(label), } - .await - .map_err(|_| ())?, + .await?, ); } - let rrset = Rrset::new(rtype, ttl); - Ok((rtype, data, end_node, rrset)) + Ok(end_node) } async fn update_soa( @@ -229,9 +252,9 @@ impl ZoneUpdater { ParsedName, ZoneRecordData>, >, - ) -> Result<(), ()> { + ) -> std::io::Result<()> { if new_soa.rtype() != Rtype::SOA { - return Err(()); + return Err(std::io::Error::custom("Invalid SOA rtype")); } let mut rrset = Rrset::new(Rtype::SOA, new_soa.ttl()); @@ -242,8 +265,6 @@ impl ZoneUpdater { .unwrap() .update_rrset(SharedRrset::new(rrset)) .await - .map_err(|_| ())?; - Ok(()) } /// Find and delete a record in the zone by exact match. @@ -253,33 +274,27 @@ impl ZoneUpdater { ParsedName, ZoneRecordData>, >, - ) -> Result<(), ()> { - let (rtype, data, end_node, mut rrset) = - self.get_update_child_write_handle(rec).await?; - + ) -> std::io::Result<()> { + let end_node = self.get_writable_node_for_owner(&rec).await?; + let mut rrset = Rrset::new(rec.rtype(), rec.ttl()); + let rtype = rec.rtype(); + let data = rec.data(); let writable = self.write.writable.as_ref().unwrap(); trace!("Deleting RR for {rtype}"); let node = end_node.as_ref().unwrap_or(writable); - if let Some(existing_rrset) = - node.get_rrset(rtype).await.map_err(|_| ())? - { + if let Some(existing_rrset) = node.get_rrset(rtype).await? { for existing_data in existing_rrset.data() { - if existing_data != &data { + if existing_data != data { rrset.push_data(existing_data.clone()); } } } trace!("Removing single RR of {rtype} so updating RRSET"); - - node.update_rrset(SharedRrset::new(rrset)) - .await - .map_err(|_| ())?; - - Ok(()) + node.update_rrset(SharedRrset::new(rrset)).await } async fn add_record( @@ -288,7 +303,7 @@ impl ZoneUpdater { ParsedName, ZoneRecordData>, >, - ) -> Result<(), ()> { + ) -> std::io::Result<()> { if !self.first_update_seen && rec.rtype() == Rtype::SOA { // If the first event is the addition of a SOA record to the zone, // this must be a complete replacement of the zone (as you can't @@ -300,12 +315,13 @@ impl ZoneUpdater { // "removed" and then add new records. This allows the old records // to continue being served to current consumers while the zone is // being updated. - self.write.remove_all().await.map_err(|_| ())?; + self.write.remove_all().await?; } - let (rtype, data, end_node, mut rrset) = - self.get_update_child_write_handle(rec).await?; - + let end_node = self.get_writable_node_for_owner(&rec).await?; + let mut rrset = Rrset::new(rec.rtype(), rec.ttl()); + let rtype = rec.rtype(); + let data = rec.into_data().flatten_into(); let writable = self.write.writable.as_ref().unwrap(); trace!("Adding RR: {:?}", rrset); @@ -313,19 +329,13 @@ impl ZoneUpdater { let node = end_node.as_ref().unwrap_or(writable); - if let Some(existing_rrset) = - node.get_rrset(rtype).await.map_err(|_| ())? - { + if let Some(existing_rrset) = node.get_rrset(rtype).await? { for existing_data in existing_rrset.data() { rrset.push_data(existing_data.clone()); } } - node.update_rrset(SharedRrset::new(rrset)) - .await - .map_err(|_| ())?; - - Ok(()) + node.update_rrset(SharedRrset::new(rrset)).await } } From 9f443dfad9e863e91233d8189ffb94277eb6bf61 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 18 Sep 2024 10:41:05 +0200 Subject: [PATCH 201/333] More RustDocs. --- src/zonetree/mod.rs | 51 ++++++++++++++++++++++++++++++++------------- 1 file changed, 36 insertions(+), 15 deletions(-) diff --git a/src/zonetree/mod.rs b/src/zonetree/mod.rs index 1ea5b18ef..0531ed20c 100644 --- a/src/zonetree/mod.rs +++ b/src/zonetree/mod.rs @@ -1,35 +1,54 @@ #![cfg(feature = "unstable-zonetree")] #![cfg_attr(docsrs, doc(cfg(feature = "unstable-zonetree")))] #![warn(missing_docs)] -//! Experimental storing and querying of zone trees. +//! Experimental storing and querying of zones and zone trees. //! -//! A [`ZoneTree`] is a multi-rooted hierarchy of [`Zone`]s, each root being -//! the apex of a subtree for a distinct [`Class`]. +//! # Zone trees //! -//! Individual `Zone`s within the tree can be looked up by containing or exact -//! name, and then one can [`query`] the found `Zone` by [`Class`], [`Rtype`] and -//! [`Name`] to produce an [`Answer`], which in turn can be used to produce a -//! response [`Message`] for serving to a DNS client. +//! A [`ZoneTree`] is a multi-rooted hierarchy of [`Zone`]s, each root being +//! the apex of a subtree for a distinct [`Class`]. `Zone`s can be inserted +//! and removed from the tree, looked up by containing or exact name, and the +//! set of zones in the tree can be iterated over. //! -//! Trees can also be iterated over to inspect or export their content. +//! # Zones //! //! The `Zone`s that a tree is comprised of can be created by feeding //! zonefiles or individual resource records into [`ZoneBuilder`] and then -//! inserted into a `ZoneTree`. +//! inserted into a `ZoneTree`. `Zone`s can also be used directly without +//! inserting them into a `ZoneTree`. +//! +//! `Zone`s can be queried via their [read interface][traits::ReadableZone] by +//! [`Class`], [`Rtype`] and [`Name`] to produce an [`Answer`], which in turn +//! can be used to produce a response [`Message`] for serving to a DNS client. +//! Entire `Zone`s can also be [`walk`]ed to inspect or export their content. +//! +//! Updating a zone can be done via the low-level [`WritableZone`] interface +//! or using a higher-level helper like the [`ZoneUpdater`]. Updates to a +//! `Zone` can be captured as difference sets which for example can be used to +//! respond to IXFR queries. +//! +//! # Backing stores //! //! By default `Zone`s are stored in memory only. Zones with other types of //! backing store can be created by implementing the [`ZoneStore`] trait and //! passing an instance of the implementing struct to [`Zone::new`]. Zones //! with different backing store types can be mixed and matched within the -//! same tree. +//! same tree. Backing stores can be synchronous or asynchronous, the latter +//! being useful for a remote backing store such as a distributed database. //! -//! The example below shows how to populate a [`ZoneTree`] from a zonefile. For -//! more examples of using [`Zone`]s and [`ZoneTree`]s including implementing an -//! alternate zone backing store for your [`Zone`]s, see the -//! [examples in the GitHub repository](https://github.com/NLnetLabs/domain/tree/main/examples). +//! The default in-memory zone implementation uses an append only write +//! strategy with new zone versions only becoming visible to consumers on +//! commit and existing zone versions remaining readable during write +//! operations. //! //! # Usage //! +//! The example below shows how to populate a [`ZoneTree`] from a zonefile. +//! For more examples of using [`Zone`]s and [`ZoneTree`]s including +//! implementing an alternate zone backing store for your [`Zone`]s, see the +//! [examples in the GitHub +//! repository](https://github.com/NLnetLabs/domain/tree/main/examples). +//! //! The following example builds and queries a [`ZoneTree`] containing a //! single in-memory [`Zone`]. //! @@ -71,7 +90,8 @@ //! assert_eq!(res.rcode(), Rcode::NOERROR); //! ``` //! -//! [`query`]: crate::zonetree::ReadableZone::query +//! [`query`]: ReadableZone::query +//! [`walk`]: ReadableZone::walk //! [`Class`]: crate::base::iana::Class //! [`Rtype`]: crate::base::iana::Rtype //! [`Name`]: crate::base::name::Name @@ -79,6 +99,7 @@ //! [`NoError`]: crate::base::iana::code::Rcode::NOERROR //! [`NxDomain`]: crate::base::iana::code::Rcode::NXDOMAIN //! [`ZoneBuilder`]: in_memory::ZoneBuilder +//! [`ZoneUpdater`]: update::ZoneUpdater mod answer; pub mod error; From 146d4c25e22f85884e379d2d8e75d8575d72b98f Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 18 Sep 2024 11:57:50 +0200 Subject: [PATCH 202/333] Don't assume that an initial ZoneUpdate::AddRecord(soa) when updating a zone means this is a full zone replacement so delete existing records, instead make the intent explcit by having XfrResponseInterpreter output ZoneUpdate::DeleteAllRecords before any other updates. --- src/net/xfr/protocol/interpreter.rs | 18 +++---- src/net/xfr/protocol/iterator.rs | 19 +++++++- src/net/xfr/protocol/tests.rs | 74 ++++++++++++++--------------- src/zonetree/types.rs | 23 +++++---- src/zonetree/update.rs | 41 ++++++---------- 5 files changed, 86 insertions(+), 89 deletions(-) diff --git a/src/net/xfr/protocol/interpreter.rs b/src/net/xfr/protocol/interpreter.rs index 8bfe8e049..1e6f3805d 100644 --- a/src/net/xfr/protocol/interpreter.rs +++ b/src/net/xfr/protocol/interpreter.rs @@ -297,9 +297,7 @@ impl RecordProcessor { ZoneUpdate::Finished(rec) } - XfrType::Axfr => { - ZoneUpdate::AddRecord(self.current_soa.serial(), rec) - } + XfrType::Axfr => ZoneUpdate::AddRecord(rec), XfrType::Ixfr if self.rr_count < 2 => unreachable!(), @@ -348,7 +346,7 @@ impl RecordProcessor { // assume that "incremental zone transfer is not available" // and so "the behaviour is the same as an AXFR response", self.actual_xfr_type = XfrType::Axfr; - ZoneUpdate::AddRecord(self.current_soa.serial(), rec) + ZoneUpdate::AddRecord(rec) } } @@ -377,14 +375,10 @@ impl RecordProcessor { } } else { match self.ixfr_update_mode { - IxfrUpdateMode::Deleting => ZoneUpdate::DeleteRecord( - self.current_soa.serial(), - rec, - ), - IxfrUpdateMode::Adding => ZoneUpdate::AddRecord( - self.current_soa.serial(), - rec, - ), + IxfrUpdateMode::Deleting => { + ZoneUpdate::DeleteRecord(rec) + } + IxfrUpdateMode::Adding => ZoneUpdate::AddRecord(rec), } } } diff --git a/src/net/xfr/protocol/iterator.rs b/src/net/xfr/protocol/iterator.rs index 21464061f..a672e4e9b 100644 --- a/src/net/xfr/protocol/iterator.rs +++ b/src/net/xfr/protocol/iterator.rs @@ -9,7 +9,7 @@ use crate::rdata::ZoneRecordData; use crate::zonetree::types::ZoneUpdate; use super::interpreter::RecordProcessor; -use super::types::{Error, IterationError, ParsedRecord}; +use super::types::{Error, IterationError, ParsedRecord, XfrType}; //------------ XfrZoneUpdateIterator ------------------------------------------ @@ -54,7 +54,6 @@ impl<'a, 'b> XfrZoneUpdateIterator<'a, 'b> { let Some(Ok(_)) = iter.next() else { return Err(Error::Malformed); }; - state.rr_count += 1; } Ok(Self { state, iter }) @@ -65,6 +64,22 @@ impl<'a, 'b> Iterator for XfrZoneUpdateIterator<'a, 'b> { type Item = Result, IterationError>; fn next(&mut self) -> Option { + if self.state.rr_count == 0 { + // We already skipped the first record in new() above by calling + // iter.next(). We didn't reflect that yet in rr_count because we + // wanted to still be able to detect the first call to next() and + // handle it specially for AXFR. + self.state.rr_count += 1; + + if self.state.actual_xfr_type == XfrType::Axfr { + // For AXFR we're not making changes to a zone, we're + // replacing its entire contents, so before returning any + // actual updates to apply, first instruct the consumer to + // "discard" everything it has. + return Some(Ok(ZoneUpdate::DeleteAllRecords)); + } + } + match self.iter.next()? { Ok(record) => { trace!("XFR record {}: {record:?}", self.state.rr_count); diff --git a/src/net/xfr/protocol/tests.rs b/src/net/xfr/protocol/tests.rs index 73992134e..aa3cef5f7 100644 --- a/src/net/xfr/protocol/tests.rs +++ b/src/net/xfr/protocol/tests.rs @@ -15,11 +15,12 @@ use crate::base::{ Message, MessageBuilder, ParsedName, Record, Rtype, Serial, Ttl, }; use crate::base::{Name, ToName}; -use crate::rdata::{Soa, ZoneRecordData, A}; +use crate::rdata::{Aaaa, Soa, ZoneRecordData, A}; use crate::zonetree::types::{ZoneUpdate, ZoneUpdate as ZU}; use super::interpreter::XfrResponseInterpreter; use super::types::{Error, IterationError, ParsedRecord}; +use core::net::Ipv6Addr; #[test] fn non_xfr_response_is_rejected() { @@ -112,7 +113,8 @@ fn incomplete_axfr_response_is_accepted() { // Process the response. let mut it = interpreter.interpret_response(resp).unwrap(); - // Verify that no updates are by the XFR interpreter. + // Verify that no updates are output by the XFR interpreter. + assert_eq!(it.next(), Some(Ok(ZoneUpdate::DeleteAllRecords))); assert!(it.next().is_none()); } @@ -141,6 +143,7 @@ fn axfr_response_with_only_soas_is_accepted() { let mut it = interpreter.interpret_response(resp).unwrap(); // Verify the updates emitted by the XFR interpreter. + assert_eq!(it.next(), Some(Ok(ZoneUpdate::DeleteAllRecords))); assert!(matches!(it.next(), Some(Ok(ZU::Finished(_))))); assert!(it.next().is_none()); } @@ -169,6 +172,7 @@ fn axfr_multi_response_with_only_soas_is_accepted() { let mut it = interpreter.interpret_response(resp).unwrap(); // Verify the updates emitted by the XFR interpreter. + assert_eq!(it.next(), Some(Ok(ZoneUpdate::DeleteAllRecords))); assert!(it.next().is_none()); // Create another AXFR response to complete the transfer. @@ -200,7 +204,7 @@ fn axfr_response_generates_expected_updates() { let soa = mk_soa(serial); add_answer_record(&req, &mut answer, soa.clone()); add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); + add_answer_record(&req, &mut answer, Aaaa::new(Ipv6Addr::LOCALHOST)); add_answer_record(&req, &mut answer, soa); let resp = answer.into_message(); @@ -208,10 +212,14 @@ fn axfr_response_generates_expected_updates() { let mut it = interpreter.interpret_response(resp).unwrap(); // Verify the updates emitted by the XFR interpreter. - let s = serial; - assert!(matches!(it.next(), Some(Ok(ZU::AddRecord(n, _))) if n == s)); - assert!(matches!(it.next(), Some(Ok(ZU::AddRecord(n, _))) if n == s)); - assert!(matches!(it.next(), Some(Ok(ZU::Finished(_))))); + assert_eq!(it.next(), Some(Ok(ZoneUpdate::DeleteAllRecords))); + assert!( + matches!(it.next(), Some(Ok(ZoneUpdate::AddRecord(r))) if r.rtype() == Rtype::A) + ); + assert!( + matches!(it.next(), Some(Ok(ZoneUpdate::AddRecord(r))) if r.rtype() == Rtype::AAAA) + ); + assert!(matches!(it.next(), Some(Ok(ZoneUpdate::Finished(_))))); assert!(it.next().is_none()); } @@ -283,43 +291,31 @@ fn ixfr_response_generates_expected_updates() { 0, ZoneRecordData::Soa(expected_old_soa), )))), - Ok(ZoneUpdate::DeleteRecord( - old_serial, - Record::from(( - owner.clone(), - 0, - ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), - )), - )), - Ok(ZoneUpdate::DeleteRecord( - old_serial, - Record::from(( - owner.clone(), - 0, - ZoneRecordData::A(A::new(Ipv4Addr::BROADCAST)), - )), - )), + Ok(ZoneUpdate::DeleteRecord(Record::from(( + owner.clone(), + 0, + ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), + )))), + Ok(ZoneUpdate::DeleteRecord(Record::from(( + owner.clone(), + 0, + ZoneRecordData::A(A::new(Ipv4Addr::BROADCAST)), + )))), Ok(ZoneUpdate::BeginBatchAdd(Record::from(( owner.clone(), 0, ZoneRecordData::Soa(expected_new_soa.clone()), )))), - Ok(ZoneUpdate::AddRecord( - new_serial, - Record::from(( - owner.clone(), - 0, - ZoneRecordData::A(A::new(Ipv4Addr::BROADCAST)), - )), - )), - Ok(ZoneUpdate::AddRecord( - new_serial, - Record::from(( - owner.clone(), - 0, - ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), - )), - )), + Ok(ZoneUpdate::AddRecord(Record::from(( + owner.clone(), + 0, + ZoneRecordData::A(A::new(Ipv4Addr::BROADCAST)), + )))), + Ok(ZoneUpdate::AddRecord(Record::from(( + owner.clone(), + 0, + ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), + )))), Ok(ZoneUpdate::Finished(Record::from(( owner.clone(), 0, diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index f11c1e265..5fa2023b2 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -321,11 +321,14 @@ impl ZoneDiff { #[derive(Clone, Debug, PartialEq, Eq)] #[non_exhaustive] pub enum ZoneUpdate { - /// Delete record R from the specified version (serial) of the zone. - DeleteRecord(Serial, R), + /// Delete all records in the zone. + DeleteAllRecords, - /// Add record R to the specified version (serial) of the zone. - AddRecord(Serial, R), + /// Delete record R from the zone. + DeleteRecord(R), + + /// Add record R to the zone. + AddRecord(R), /// Start a batch delete for the specified version (serial) of the zone. /// @@ -362,11 +365,10 @@ pub enum ZoneUpdate { /// See `BeginBatchDelete` for more information. BeginBatchAdd(R), - /// Updates for the specified version (serial) of the zone can now be - /// finalized. + /// In progress updates for the zone can now be finalized. /// - /// This signals the end of a group of related changes to the specified - /// version (serial) of the zone. + /// This signals the end of a group of related changes for the given SOA + /// record of the zone. /// /// For example this could be used to trigger an atomic commit of a set of /// related pending changes. @@ -378,8 +380,9 @@ pub enum ZoneUpdate { impl std::fmt::Display for ZoneUpdate { fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { match self { - ZoneUpdate::DeleteRecord(_, _) => f.write_str("DeleteRecord"), - ZoneUpdate::AddRecord(_, _) => f.write_str("AddRecord"), + ZoneUpdate::DeleteAllRecords => f.write_str("DeleteAllRecords"), + ZoneUpdate::DeleteRecord(_) => f.write_str("DeleteRecord"), + ZoneUpdate::AddRecord(_) => f.write_str("AddRecord"), ZoneUpdate::BeginBatchDelete(_) => { f.write_str("BeginBatchDelete") } diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 5777f2376..7582bf070 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -92,9 +92,6 @@ pub struct ZoneUpdater { /// Whether or not we entered an IXFR-like batching mode. batching: bool, - - /// Whether or not at least one update has been applied so far. - first_update_seen: bool, } impl ZoneUpdater { @@ -115,7 +112,6 @@ impl ZoneUpdater { zone, write, batching: false, - first_update_seen: false, }) }) } @@ -131,13 +127,22 @@ impl ZoneUpdater { ) -> std::io::Result<()> { trace!("Event: {update}"); match update { - ZoneUpdate::DeleteRecord(_serial, rec) => { - self.delete_record(rec).await? + ZoneUpdate::DeleteAllRecords => { + // To completely replace the content of the zone, i.e. + // something like an AXFR transfer, we can't add records from + // a new version of the zone to an existing zone because if + // the old version contained a record which the new version + // does not, the record would remain in the zone. So in this + // case we have to mark all of the existing records in the + // zone as "removed" and then add new records. This allows the + // old records to continue being served to current consumers + // while the zone is being updated. + self.write.remove_all().await?; } - ZoneUpdate::AddRecord(_serial, rec) => { - self.add_record(rec).await? - } + ZoneUpdate::DeleteRecord(rec) => self.delete_record(rec).await?, + + ZoneUpdate::AddRecord(rec) => self.add_record(rec).await?, // Note: Batches first contain deletions then additions, so batch // deletion signals the start of a batch, and the end of any @@ -175,8 +180,6 @@ impl ZoneUpdater { } } - self.first_update_seen = true; - Ok(()) } } @@ -304,20 +307,6 @@ impl ZoneUpdater { ZoneRecordData>, >, ) -> std::io::Result<()> { - if !self.first_update_seen && rec.rtype() == Rtype::SOA { - // If the first event is the addition of a SOA record to the zone, - // this must be a complete replacement of the zone (as you can't - // have two SOA records), i.e. something like an AXFR transfer. We - // can't add records from a new version of the zone to an existing - // zone because if the old version contained a record which the - // new version does not, it would get left behind. So in this case - // we have to mark all of the existing records in the zone as - // "removed" and then add new records. This allows the old records - // to continue being served to current consumers while the zone is - // being updated. - self.write.remove_all().await?; - } - let end_node = self.get_writable_node_for_owner(&rec).await?; let mut rrset = Rrset::new(rec.rtype(), rec.ttl()); let rtype = rec.rtype(); @@ -418,7 +407,7 @@ mod tests { ); updater - .apply(ZoneUpdate::AddRecord(s, soa.clone())) + .apply(ZoneUpdate::AddRecord(soa.clone())) .await .unwrap(); From c97d06277db561d69b5d3e5898945d2f54f6a12c Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 18 Sep 2024 12:42:47 +0200 Subject: [PATCH 203/333] More RustDoc. --- src/net/xfr/protocol/interpreter.rs | 4 + src/zonetree/update.rs | 158 +++++++++++++++++++++++++++- 2 files changed, 157 insertions(+), 5 deletions(-) diff --git a/src/net/xfr/protocol/interpreter.rs b/src/net/xfr/protocol/interpreter.rs index 1e6f3805d..9e4d874cf 100644 --- a/src/net/xfr/protocol/interpreter.rs +++ b/src/net/xfr/protocol/interpreter.rs @@ -40,6 +40,8 @@ pub struct XfrResponseInterpreter { /// Internal state. /// /// None until the first call to [`interpret_response()`]. + /// + /// [`interpret_response()`]: XfrResponseInterpreter::interpret_response() inner: Option, } @@ -148,6 +150,8 @@ impl XfrResponseInterpreter { /// /// Separated out from [`XfrResponseInterpreter`] to avoid needing multiple /// mutable self references in [`interpret_response()`]. +/// +/// [`interpret_response()`]: XfrResponseInterpreter::interpret_response() struct Inner { /// The response message currently being processed. resp: Message, diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 7582bf070..c5c189bf2 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -8,6 +8,135 @@ //! [`ZoneUpdate`]s such as //! [`XfrResponseInterpreter`][crate::net::xfr::protocol::XfrResponseInterpreter]. //! +//!
+//! +//! `WritableZone::commit()` is invoked by `ZoneUpdater` when it receives +//! `ZoneUpdate::Finished`. If `ZoneUpdate::Finished` is not passed to +//! `ZoneUpdater::apply()` there is no guarantee that the applied changes to +//! the zone will take effect. +//! +//!
+//! +//! # Replacing the content of a zone +//! +//! ``` +//! # use std::str::FromStr; +//! # +//! # use domain::base::iana::Class; +//! # use domain::base::MessageBuilder; +//! # use domain::base::Name; +//! # use domain::base::ParsedName; +//! # use domain::base::Record; +//! # use domain::base::Serial; +//! # use domain::base::Ttl; +//! # use domain::base::net::Ipv4Addr; +//! # use domain::net::xfr::protocol::XfrResponseInterpreter; +//! # use domain::rdata::A; +//! # use domain::rdata::Soa; +//! # use domain::rdata::ZoneRecordData; +//! # use domain::zonetree::ZoneBuilder; +//! # use domain::zonetree::types::ZoneUpdate; +//! # use domain::zonetree::update::ZoneUpdater; +//! # +//! # #[tokio::main] +//! # async fn main() { +//! # +//! # let builder = ZoneBuilder::new(Name::from_str("example.com").unwrap(), Class::IN); +//! # let zone = builder.build(); +//! # +//! # // Prepare some records to pass to ZoneUpdater +//! # let serial = Serial::now(); +//! # let mname = ParsedName::from(Name::from_str("mname").unwrap()); +//! # let rname = ParsedName::from(Name::from_str("rname").unwrap()); +//! # let ttl = Ttl::from_secs(0); +//! # let new_soa_rec = Record::new( +//! # ParsedName::from(Name::from_str("example.com").unwrap()), +//! # Class::IN, +//! # Ttl::from_secs(0), +//! # ZoneRecordData::Soa(Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl)), +//! # ); +//! # +//! # let a_data = A::new(Ipv4Addr::LOCALHOST); +//! # let a_rec = Record::new( +//! # ParsedName::from(Name::from_str("a.example.com").unwrap()), +//! # Class::IN, +//! # Ttl::from_secs(0), +//! # ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), +//! # ); +//! # +//! let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); +//! updater.apply(ZoneUpdate::DeleteAllRecords); +//! updater.apply(ZoneUpdate::AddRecord(a_rec)); +//! updater.apply(ZoneUpdate::Finished(new_soa_rec)); +//! # +//! # } +//! ``` +//! +//! # Altering the content of a zone +//! +//! ```rust +//! # use std::str::FromStr; +//! # +//! # use domain::base::iana::Class; +//! # use domain::base::MessageBuilder; +//! # use domain::base::Name; +//! # use domain::base::ParsedName; +//! # use domain::base::Record; +//! # use domain::base::Serial; +//! # use domain::base::Ttl; +//! # use domain::base::net::Ipv4Addr; +//! # use domain::base::net::Ipv6Addr; +//! # use domain::net::xfr::protocol::XfrResponseInterpreter; +//! # use domain::rdata::A; +//! # use domain::rdata::Aaaa; +//! # use domain::rdata::Soa; +//! # use domain::rdata::ZoneRecordData; +//! # use domain::zonetree::ZoneBuilder; +//! # use domain::zonetree::update::ZoneUpdater; +//! # use domain::zonetree::types::ZoneUpdate; +//! # +//! # #[tokio::main] +//! # async fn main() { +//! # +//! # let builder = ZoneBuilder::new(Name::from_str("example.com").unwrap(), Class::IN); +//! # let zone = builder.build(); +//! # +//! # // Prepare some records to pass to ZoneUpdater +//! # let serial = Serial::now(); +//! # let mname = ParsedName::from(Name::from_str("mname").unwrap()); +//! # let rname = ParsedName::from(Name::from_str("rname").unwrap()); +//! # let ttl = Ttl::from_secs(0); +//! # let new_soa_rec = Record::new( +//! # ParsedName::from(Name::from_str("example.com").unwrap()), +//! # Class::IN, +//! # Ttl::from_secs(0), +//! # ZoneRecordData::Soa(Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl)), +//! # ); +//! # +//! # let old_a_data = A::new(Ipv4Addr::LOCALHOST); +//! # let old_a_rec = Record::new( +//! # ParsedName::from(Name::from_str("a.example.com").unwrap()), +//! # Class::IN, +//! # Ttl::from_secs(0), +//! # ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), +//! # ); +//! # +//! # let new_aaaa_data = Aaaa::new(Ipv6Addr::LOCALHOST); +//! # let new_aaaa_rec = Record::new( +//! # ParsedName::from(Name::from_str("a.example.com").unwrap()), +//! # Class::IN, +//! # Ttl::from_secs(0), +//! # ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), +//! # ); +//! # +//! let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); +//! updater.apply(ZoneUpdate::DeleteRecord(old_a_rec)); +//! updater.apply(ZoneUpdate::AddRecord(new_aaaa_rec)); +//! updater.apply(ZoneUpdate::Finished(new_soa_rec)); +//! # +//! # } +//! ``` +//! //! # Applying XFR changes to a zone //! //! ```no_run @@ -16,6 +145,7 @@ //! # use domain::base::iana::Class; //! # use domain::base::MessageBuilder; //! # use domain::base::Name; +//! # use domain::base::Serial; //! # use domain::net::xfr::protocol::XfrResponseInterpreter; //! # use domain::zonetree::ZoneBuilder; //! # use domain::zonetree::update::ZoneUpdater; @@ -35,8 +165,10 @@ //! //! // Iterate over the XFR responses applying the updates to the zone //! while !interpreter.is_finished() { -//! // Get the next XFR response -//! let next_xfr_response = MessageBuilder::new_bytes().into_message(); // A dummy response +//! // Get the next XFR response: +//! // For this example this is just a dummy response, which would cause +//! // Error::NotValidXfrResponse if this code were run. +//! let next_xfr_response = MessageBuilder::new_bytes().into_message(); //! //! // Convert it to an update iterator //! let it = interpreter.interpret_response(next_xfr_response).unwrap(); @@ -70,14 +202,21 @@ use super::error::OutOfZone; use super::types::ZoneUpdate; use super::{WritableZone, WritableZoneNode, Zone}; -/// Apply a sequence of [`ZoneUpdate`]s to alter the content of a [`Zone`]. -/// -/// Each [`ZoneUpdate::BeginBatchDelete`] starts a new zone version, +/// Apply a sequence of [`ZoneUpdate`]s to update the content of a [`Zone`]. /// /// For each version of the zone that is edited the zone will be opened for /// writing, edits made and then the changes committed, only then becoming /// visible for readers of the zone. /// +/// To completely replace the content of a zone pass +/// [`ZoneUpdate::DeleteAllRecords`] to `apply` before any other updates. +/// +/// Changes to the zone are committed when [`ZoneUpdate::Finished`] is +/// received. +/// +/// Passing [`ZoneUpdate::BeginBatchDelete`] commits any edits in progress and +/// starts editing a new zone version. +/// /// For each commit of the zone a diff of the changes made will be stored with /// the zone. pub struct ZoneUpdater { @@ -121,6 +260,15 @@ impl ZoneUpdater { /// Apply the given [`ZoneUpdate`] to the [`Zone`] being updated. /// /// Returns `Ok` on success, `Err` otherwise. + /// + ///
+ /// + /// This method invokes `WritableZone::commit()` when it receives + /// `ZoneUpdate::Finished`. If `ZoneUpdate::Finished` is not passed to + /// `ZoneUpdater::apply()` there is no guarantee that the applied changes + /// to the zone will take effect. + /// + ///
pub async fn apply( &mut self, update: ZoneUpdate, From 1c096dba944d47d5cec3e05aace1dd35c85ccd13 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 18 Sep 2024 12:50:39 +0200 Subject: [PATCH 204/333] Compilation fix. --- src/net/xfr/protocol/tests.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/net/xfr/protocol/tests.rs b/src/net/xfr/protocol/tests.rs index aa3cef5f7..0bcfc5826 100644 --- a/src/net/xfr/protocol/tests.rs +++ b/src/net/xfr/protocol/tests.rs @@ -9,7 +9,7 @@ use crate::base::iana::Rcode; use crate::base::message_builder::{ AnswerBuilder, AuthorityBuilder, QuestionBuilder, }; -use crate::base::net::Ipv4Addr; +use crate::base::net::{Ipv4Addr, Ipv6Addr}; use crate::base::rdata::ComposeRecordData; use crate::base::{ Message, MessageBuilder, ParsedName, Record, Rtype, Serial, Ttl, @@ -20,7 +20,6 @@ use crate::zonetree::types::{ZoneUpdate, ZoneUpdate as ZU}; use super::interpreter::XfrResponseInterpreter; use super::types::{Error, IterationError, ParsedRecord}; -use core::net::Ipv6Addr; #[test] fn non_xfr_response_is_rejected() { From ef41acfea3572b2844267bc863505106a8dcd49a Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 18 Sep 2024 16:57:42 +0200 Subject: [PATCH 205/333] Make ZoneDiff immutable and cheaply clonable by introducing ZoneDiffBuilder, --- src/zonetree/in_memory/write.rs | 63 +++++++++++++++++------------ src/zonetree/types.rs | 70 +++++++++++++++++++++++++-------- src/zonetree/update.rs | 2 +- 3 files changed, 92 insertions(+), 43 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index c33c72f79..74badd99b 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -18,7 +18,7 @@ use tracing::trace; use crate::base::iana::Rtype; use crate::base::name::Label; use crate::base::NameBuilder; -use crate::zonetree::types::{ZoneCut, ZoneDiff}; +use crate::zonetree::types::{ZoneCut, ZoneDiff, ZoneDiffBuilder}; use crate::zonetree::StoredName; use crate::zonetree::{Rrset, SharedRr}; use crate::zonetree::{SharedRrset, WritableZone, WritableZoneNode}; @@ -35,7 +35,7 @@ pub struct WriteZone { version: Version, dirty: bool, zone_versions: Arc>, - diff: Arc>>>>, + diff: Arc>>>>, } impl WriteZone { @@ -167,8 +167,6 @@ impl WritableZone for WriteZone { let diff = arc_into_inner(diff).unwrap(); let mut diff = Mutex::into_inner(diff).unwrap(); - diff.start_serial = Some(old_soa.serial()); - diff.end_serial = Some(new_soa.serial()); if bump_soa_serial { let mut removed_soa_rrset = @@ -183,17 +181,25 @@ impl WritableZone for WriteZone { let new_soa_rrset = SharedRrset::new(new_soa_shared_rrset); - let k = (self.apex.name().clone(), Rtype::SOA); trace!("Diff: recording removal of old SOA: {removed_soa_rrset:#?}"); - diff.removed.insert(k.clone(), removed_soa_rrset); + diff.remove( + self.apex.name().clone(), + Rtype::SOA, + removed_soa_rrset, + ); trace!( "Diff: recording addition of new SOA: {new_soa_rrset:#?}" ); - diff.added.insert(k, new_soa_rrset); + diff.add( + self.apex.name().clone(), + Rtype::SOA, + new_soa_rrset, + ); } - out_diff = Some(diff); + out_diff = + Some(diff.build(old_soa.serial(), new_soa.serial())); } } @@ -216,13 +222,13 @@ impl WritableZone for WriteZone { } #[rustversion::since(1.70.0)] -fn arc_into_inner(this: Arc>) -> Option> { +fn arc_into_inner(this: Arc>) -> Option> { #[allow(clippy::incompatible_msrv)] Arc::into_inner(this) } #[rustversion::before(1.70.0)] -fn arc_into_inner(this: Arc>) -> Option> { +fn arc_into_inner(this: Arc>) -> Option> { // From: https://doc.rust-lang.org/std/sync/struct.Arc.html#method.into_inner // // "If Arc::into_inner is called on every clone of this Arc, it is @@ -250,7 +256,7 @@ pub struct WriteNode { node: Either, Arc>, /// The diff we are building, if enabled. - diff: Option<(StoredName, Arc>)>, + diff: Option<(StoredName, Arc>)>, } impl WriteNode { @@ -263,7 +269,7 @@ impl WriteNode { let diff = if create_diff { Some(( zone.apex.name().clone(), - Arc::new(Mutex::new(ZoneDiff::new())), + Arc::new(Mutex::new(ZoneDiffBuilder::new())), )) } else { None @@ -313,8 +319,6 @@ impl WriteNode { trace!("Updating RRset"); if let Some((owner, diff)) = &self.diff { - let k = (owner.clone(), rrset.rtype()); - let changed = if let Some(removed_rrset) = rrsets.get(rrset.rtype(), self.zone.version.prev()) { @@ -322,10 +326,11 @@ impl WriteNode { if changed && !removed_rrset.is_empty() { trace!("Diff detected: update of existing RRSET - recording removal of the current RRSET: {removed_rrset:#?}"); - diff.lock() - .unwrap() - .removed - .insert(k.clone(), removed_rrset.clone()); + diff.lock().unwrap().remove( + owner.clone(), + rrset.rtype(), + removed_rrset.clone(), + ); } changed @@ -335,7 +340,11 @@ impl WriteNode { if changed && !rrset.is_empty() { trace!("Diff detected: update of existing RRSET - recording addition of the new RRSET: {rrset:#?}"); - diff.lock().unwrap().added.insert(k, rrset.clone()); + diff.lock().unwrap().add( + owner.clone(), + rrset.rtype(), + rrset.clone(), + ); } } @@ -372,11 +381,11 @@ impl WriteNode { trace!( "Diff detected: removal of existing RRSET: {removed:#?}" ); - let k = (owner.clone(), rtype); - diff.lock() - .unwrap() - .removed - .insert(k.clone(), removed.clone()); + diff.lock().unwrap().remove( + owner.clone(), + rtype, + removed.clone(), + ); } } @@ -482,8 +491,10 @@ impl WriteNode { Ok(()) } - fn diff(&self) -> Option>> { - self.diff.as_ref().map(|(_, diff)| diff.clone()) + fn diff(&self) -> Option>> { + self.diff + .as_ref() + .map(|(_, diff_builder)| diff_builder.clone()) } } diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index 5fa2023b2..39b034c84 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -251,33 +251,71 @@ pub struct ZoneCut { pub glue: Vec, } +//------------ ZoneDiffBuilder ----------------------------------------------- + +/// A [`ZoneDiff`] builder. +#[derive(Debug, Default)] +pub struct ZoneDiffBuilder { + /// The records added to the Zone. + added: HashMap<(StoredName, Rtype), SharedRrset>, + + /// The records removed from the Zone. + removed: HashMap<(StoredName, Rtype), SharedRrset>, +} + +impl ZoneDiffBuilder { + /// TODO + pub fn new() -> Self { + Default::default() + } + + /// TODO + pub fn add( + &mut self, + owner: StoredName, + rtype: Rtype, + rrset: SharedRrset, + ) { + self.added.insert((owner, rtype), rrset); + } + + /// TODO + pub fn remove( + &mut self, + owner: StoredName, + rtype: Rtype, + rrset: SharedRrset, + ) { + self.removed.insert((owner, rtype), rrset); + } + + /// TODO + pub fn build(self, start_serial: Serial, end_serial: Serial) -> ZoneDiff { + ZoneDiff { + start_serial, + end_serial, + added: Arc::new(self.added), + removed: Arc::new(self.removed), + } + } +} + //------------ ZoneDiff ------------------------------------------------------ /// The differences between one serial and another for a Zone. -#[derive(Clone, Debug, Default)] +#[derive(Clone, Debug)] pub struct ZoneDiff { /// The serial number of the Zone which was modified. - /// - /// For a completed diff this must be Some. - pub start_serial: Option, + pub start_serial: Serial, /// The serial number of the Zone that resulted from the modifications. - /// - /// For a completed diff this must be Some. - pub end_serial: Option, + pub end_serial: Serial, /// The records added to the Zone. - pub added: HashMap<(StoredName, Rtype), SharedRrset>, + pub added: Arc>, /// The records removed from the Zone. - pub removed: HashMap<(StoredName, Rtype), SharedRrset>, -} - -impl ZoneDiff { - /// Creates a new empty diff. - pub fn new() -> Self { - Self::default() - } + pub removed: Arc>, } //------------ ZoneUpdate ----------------------------------------------------- diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index c5c189bf2..523496ed6 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -243,7 +243,7 @@ impl ZoneUpdater { /// Use [`apply`][Self::apply] to apply changes to the zone. pub fn new( zone: Zone, - ) -> Pin>>> { + ) -> Pin> + Send>> { Box::pin(async move { let write = WriteState::new(zone.clone()).await?; From f98ba5d5134d173947f007ba850eb9e7ab56f9ac Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 18 Sep 2024 17:01:56 +0200 Subject: [PATCH 206/333] Compilation fixes for upstream changes. --- src/net/server/middleware/xfr.rs | 63 ++++++++++++-------------------- src/zonetree/mod.rs | 1 + 2 files changed, 24 insertions(+), 40 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index dd9be7bce..ae9b54574 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -1373,6 +1373,7 @@ mod tests { use crate::tsig::{Algorithm, Key, KeyName}; use crate::zonefile::inplace::Zonefile; use crate::zonetree::types::Rrset; + use crate::zonetree::ZoneDiffBuilder; use super::*; @@ -1602,9 +1603,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 let zone = load_zone(rfc_1995_zone.as_bytes()); // Diff 1: NEZU.JAIN.AD.JP. is removed and JAIN-BB.JAIN.AD.JP. is added. - let mut diff = ZoneDiff::new(); - diff.start_serial = Some(Serial(1)); - diff.end_serial = Some(Serial(2)); + let mut diff = ZoneDiffBuilder::new(); // -- Remove the old SOA. let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); @@ -1618,16 +1617,12 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 Ttl::from_secs(604800), ); rrset.push_data(soa.into()); - diff.removed - .insert((n("JAIN.AD.JP"), Rtype::SOA), SharedRrset::new(rrset)); + diff.remove(n("JAIN.AD.JP"), Rtype::SOA, SharedRrset::new(rrset)); // -- Remove the A record. let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); rrset.push_data(A::new(p("133.69.136.5")).into()); - diff.removed.insert( - (n("NEZU.JAIN.AD.JP"), Rtype::A), - SharedRrset::new(rrset), - ); + diff.remove(n("NEZU.JAIN.AD.JP"), Rtype::A, SharedRrset::new(rrset)); // -- Add the new SOA. let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); @@ -1641,24 +1636,18 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 Ttl::from_secs(604800), ); rrset.push_data(soa.into()); - diff.added - .insert((n("JAIN.AD.JP"), Rtype::SOA), SharedRrset::new(rrset)); + diff.add(n("JAIN.AD.JP"), Rtype::SOA, SharedRrset::new(rrset)); // -- Add the new A records. let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); rrset.push_data(A::new(p("133.69.136.4")).into()); rrset.push_data(A::new(p("192.41.197.2")).into()); - diff.added.insert( - (n("JAIN-BB.JAIN.AD.JP"), Rtype::A), - SharedRrset::new(rrset), - ); + diff.add(n("JAIN-BB.JAIN.AD.JP"), Rtype::A, SharedRrset::new(rrset)); - diffs.push(diff); + diffs.push(diff.build(Serial(1), Serial(2))); // Diff 2: One of the IP addresses of JAIN-BB.JAIN.AD.JP. is changed. - let mut diff = ZoneDiff::new(); - diff.start_serial = Some(Serial(2)); - diff.end_serial = Some(Serial(3)); + let mut diff = ZoneDiffBuilder::new(); // -- Remove the old SOA. let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); @@ -1672,14 +1661,14 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 Ttl::from_secs(604800), ); rrset.push_data(soa.into()); - diff.removed - .insert((n("JAIN.AD.JP"), Rtype::SOA), SharedRrset::new(rrset)); + diff.remove(n("JAIN.AD.JP"), Rtype::SOA, SharedRrset::new(rrset)); // Remove the outdated IP address. let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); rrset.push_data(A::new(p("133.69.136.4")).into()); - diff.removed.insert( - (n("JAIN-BB.JAIN.AD.JP"), Rtype::A), + diff.remove( + n("JAIN-BB.JAIN.AD.JP"), + Rtype::A, SharedRrset::new(rrset), ); @@ -1695,18 +1684,14 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 Ttl::from_secs(604800), ); rrset.push_data(soa.into()); - diff.added - .insert((n("JAIN.AD.JP"), Rtype::SOA), SharedRrset::new(rrset)); + diff.add(n("JAIN.AD.JP"), Rtype::SOA, SharedRrset::new(rrset)); // Add the updated IP address. let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); rrset.push_data(A::new(p("133.69.136.3")).into()); - diff.added.insert( - (n("JAIN-BB.JAIN.AD.JP"), Rtype::A), - SharedRrset::new(rrset), - ); + diff.add(n("JAIN-BB.JAIN.AD.JP"), Rtype::A, SharedRrset::new(rrset)); - diffs.push(diff); + diffs.push(diff.build(Serial(2), Serial(3))); // Create an object that knows how to provide zone and diff data for // our zone and diffs. @@ -2163,16 +2148,14 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 if q.qname() == self.zone.apex_name() && q.qclass() == self.zone.class() { - let diffs = if self - .diffs - .first() - .and_then(|diff| diff.start_serial) - == diff_from - { - self.diffs.clone() - } else { - vec![] - }; + let diffs = + if self.diffs.first().map(|diff| diff.start_serial) + == diff_from + { + self.diffs.clone() + } else { + vec![] + }; Ok((self.zone.clone(), diffs)) } else { diff --git a/src/zonetree/mod.rs b/src/zonetree/mod.rs index 0531ed20c..5472790ce 100644 --- a/src/zonetree/mod.rs +++ b/src/zonetree/mod.rs @@ -120,6 +120,7 @@ pub use self::traits::{ pub use self::tree::{ZoneSetIter, ZoneTree}; pub use self::types::{ Rrset, SharedRr, SharedRrset, StoredName, StoredRecord, ZoneDiff, + ZoneDiffBuilder, }; pub use self::walk::WalkOp; pub use self::zone::{Zone, ZoneKey}; From d060e07b8184203d83945548238af587db980b83 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 18 Sep 2024 17:05:07 +0200 Subject: [PATCH 207/333] Compilation fix. --- src/zonetree/in_memory/write.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index 74badd99b..95540987c 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -228,7 +228,7 @@ fn arc_into_inner(this: Arc>) -> Option> { } #[rustversion::before(1.70.0)] -fn arc_into_inner(this: Arc>) -> Option> { +fn arc_into_inner(this: Arc>) -> Option> { // From: https://doc.rust-lang.org/std/sync/struct.Arc.html#method.into_inner // // "If Arc::into_inner is called on every clone of this Arc, it is From 2eb45076636b002e12592e3fc21c60904824571e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 18 Sep 2024 20:01:30 +0200 Subject: [PATCH 208/333] Don't swallow any created diffs. --- src/zonetree/types.rs | 6 ++++-- src/zonetree/update.rs | 42 +++++++++++++++++++++++------------------- 2 files changed, 27 insertions(+), 21 deletions(-) diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index 39b034c84..640290a05 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -368,7 +368,8 @@ pub enum ZoneUpdate { /// Add record R to the zone. AddRecord(R), - /// Start a batch delete for the specified version (serial) of the zone. + /// Start a batch delete for the version of the zone with the given SOA + /// record. /// /// If not already in batching mode, this signals the start of batching /// mode. In batching mode one or more batches of updates will be @@ -391,7 +392,8 @@ pub enum ZoneUpdate { /// should be deleted. BeginBatchDelete(R), - /// Start a batch add for the specified version (serial) of the zone. + /// Start a batch add for the version of the zone with the given SOA + /// record. /// /// This can only be signalled when already in batching mode, i.e. when /// `BeginBatchDelete` has already been signalled. diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 523496ed6..419f1eead 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -200,7 +200,7 @@ use crate::zonetree::{Rrset, SharedRrset}; use super::error::OutOfZone; use super::types::ZoneUpdate; -use super::{WritableZone, WritableZoneNode, Zone}; +use super::{WritableZone, WritableZoneNode, Zone, ZoneDiff}; /// Apply a sequence of [`ZoneUpdate`]s to update the content of a [`Zone`]. /// @@ -259,7 +259,9 @@ impl ZoneUpdater { impl ZoneUpdater { /// Apply the given [`ZoneUpdate`] to the [`Zone`] being updated. /// - /// Returns `Ok` on success, `Err` otherwise. + /// Returns `Ok` on success, `Err` otherwise. On success, if changes were + /// committed then any diff made by the `Zone` backing store + /// implementation will be returned. /// ///
/// @@ -272,7 +274,7 @@ impl ZoneUpdater { pub async fn apply( &mut self, update: ZoneUpdate, - ) -> std::io::Result<()> { + ) -> std::io::Result> { trace!("Event: {update}"); match update { ZoneUpdate::DeleteAllRecords => { @@ -296,18 +298,21 @@ impl ZoneUpdater { // deletion signals the start of a batch, and the end of any // previous batch addition. ZoneUpdate::BeginBatchDelete(_old_soa) => { - if self.batching { + let diff = if self.batching { // Commit the previous batch. - self.write.commit().await.map_err(|()| { - std::io::Error::custom( - "Error commiting changes to zone", - ) - })?; + let diff = self.write.commit().await?; + // Open a writer for the new batch. self.write.reopen().await?; - } + + diff + } else { + None + }; self.batching = true; + + return Ok(diff); } ZoneUpdate::BeginBatchAdd(new_soa) => { @@ -321,14 +326,13 @@ impl ZoneUpdater { // Update the SOA record. self.update_soa(zone_soa).await?; } - // Commit the previous batch. - self.write.commit().await.map_err(|()| { - std::io::Error::custom("Error commiting changes to zone") - })?; + + // Commit the previous batch and return any diff produced. + return self.write.commit().await; } } - Ok(()) + Ok(None) } } @@ -498,16 +502,16 @@ impl WriteState { Ok(()) } - async fn commit(&mut self) -> Result<(), ()> { + async fn commit(&mut self) -> std::io::Result> { // Commit the deletes and adds that just occurred if let Some(writable) = self.writable.take() { // Ensure that there are no dangling references to the created // diff (otherwise commit() will panic). drop(writable); - self.write.commit(false).await.map_err(|_| ())?; + self.write.commit(false).await + } else { + Ok(None) } - - Ok(()) } async fn reopen(&mut self) -> std::io::Result<()> { From 3f798d51724d5e565f8771600f911009510a90e0 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 09:42:29 +0200 Subject: [PATCH 209/333] Fix cargo doc broken link warnings. --- src/zonetree/traits.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index 716d318d4..e92e62646 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -145,12 +145,14 @@ pub trait WritableZone: Send + Sync { /// Complete a write operation for the zone. /// - /// This function commits the changes accumulated since [`open`] was + /// This function commits the changes accumulated since [`open()`] was /// invoked. Clients who obtain a [`ReadableZone`] interface to this zone /// _before_ this function has been called will not see any of the changes /// made since the last commit. Only clients who obtain a [`ReadableZone`] /// _after_ invoking this function will be able to see the changes made - /// since [`open`] was called. + /// since [`open()`] was called. + /// + /// [`open()`]: Self::open fn commit( &mut self, bump_soa_serial: bool, From f55046bff35fff7dfd0d3389ea6cdbfea99d8094 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 10:23:31 +0200 Subject: [PATCH 210/333] Remove questionable Serial::prev() addition, we don't want some mathematically earlier version anyway, we want the version prior to the one we are creating which is just the last version that was published by whatever number that had. Also improved some names to make it clearer what is being referred to. --- src/base/serial.rs | 23 +++----- src/zonetree/in_memory/versioned.rs | 3 - src/zonetree/in_memory/write.rs | 85 +++++++++++++++++------------ 3 files changed, 57 insertions(+), 54 deletions(-) diff --git a/src/base/serial.rs b/src/base/serial.rs index 9569fdc68..9bb91b46e 100644 --- a/src/base/serial.rs +++ b/src/base/serial.rs @@ -4,21 +4,19 @@ //! viewed as the 32 bit modulus of a larger number space. Because of that, //! special rules apply when processing these values. This module provides //! the type [`Serial`] that implements these rules. -use core::cmp::Ordering; -use core::{cmp, fmt, str}; - -#[cfg(all(feature = "std", not(test)))] -use std::time::{SystemTime, UNIX_EPOCH}; +use super::cmp::CanonicalOrd; +use super::scan::{Scan, Scanner}; +use super::wire::{Compose, Composer, Parse, ParseError}; #[cfg(feature = "chrono")] use chrono::{DateTime, TimeZone}; +use core::cmp::Ordering; +use core::{cmp, fmt, str}; #[cfg(all(feature = "std", test))] use mock_instant::thread_local::{SystemTime, UNIX_EPOCH}; use octseq::parse::Parser; - -use super::cmp::CanonicalOrd; -use super::scan::{Scan, Scanner}; -use super::wire::{Compose, Composer, Parse, ParseError}; +#[cfg(all(feature = "std", not(test)))] +use std::time::{SystemTime, UNIX_EPOCH}; //------------ Serial -------------------------------------------------------- @@ -92,13 +90,6 @@ impl Serial { Serial(self.0.wrapping_add(other)) } - #[allow(clippy::should_implement_trait)] - #[must_use] - pub fn sub(self, other: u32) -> Self { - assert!(other <= 0x7FFF_FFFF); - Serial(self.0.wrapping_sub(other)) - } - pub fn scan(scanner: &mut S) -> Result { u32::scan(scanner).map(Into::into) } diff --git a/src/zonetree/in_memory/versioned.rs b/src/zonetree/in_memory/versioned.rs index 034729444..0b611c363 100644 --- a/src/zonetree/in_memory/versioned.rs +++ b/src/zonetree/in_memory/versioned.rs @@ -21,9 +21,6 @@ impl Version { pub fn next(self) -> Version { Version(self.0.add(1)) } - pub fn prev(self) -> Version { - Version(self.0.sub(1)) - } } impl Default for Version { diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index 95540987c..5e4232a06 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -32,9 +32,9 @@ use crate::rdata::ZoneRecordData; pub struct WriteZone { apex: Arc, _lock: Option>, - version: Version, + new_version: Version, dirty: bool, - zone_versions: Arc>, + published_versions: Arc>, diff: Arc>>>>, } @@ -42,18 +42,22 @@ impl WriteZone { pub(super) fn new( apex: Arc, _lock: OwnedMutexGuard<()>, - version: Version, - zone_versions: Arc>, + new_version: Version, + published_versions: Arc>, ) -> Self { WriteZone { apex, _lock: Some(_lock), - version, + new_version, dirty: false, - zone_versions, + published_versions, diff: Arc::new(Mutex::new(None)), } } + + fn last_published_version(&self) -> Version { + self.published_versions.read().current().0 + } } //--- impl Clone @@ -63,9 +67,9 @@ impl Clone for WriteZone { Self { apex: self.apex.clone(), _lock: None, - version: self.version, + new_version: self.new_version, dirty: self.dirty, - zone_versions: self.zone_versions.clone(), + published_versions: self.published_versions.clone(), diff: self.diff.clone(), } } @@ -76,7 +80,7 @@ impl Clone for WriteZone { impl Drop for WriteZone { fn drop(&mut self) { if self.dirty { - self.apex.rollback(self.version); + self.apex.rollback(self.new_version); self.dirty = false; } } @@ -129,7 +133,9 @@ impl WritableZone for WriteZone { let mut out_diff = None; // An empty zone that is being filled by AXFR won't have an existing SOA. - if let Some(old_soa_rr) = self.apex.get_soa(self.version.prev()) { + if let Some(old_soa_rr) = + self.apex.get_soa(self.last_published_version()) + { let ZoneRecordData::Soa(old_soa) = old_soa_rr.data() else { unreachable!() }; @@ -155,12 +161,12 @@ impl WritableZone for WriteZone { self.apex .rrsets() - .update(new_soa_shared_rrset.clone(), self.version); + .update(new_soa_shared_rrset.clone(), self.new_version); } // Extract the created diff, if any. if let Some(diff) = self.diff.lock().unwrap().take() { - let new_soa_rr = self.apex.get_soa(self.version).unwrap(); + let new_soa_rr = self.apex.get_soa(self.new_version).unwrap(); let ZoneRecordData::Soa(new_soa) = new_soa_rr.data() else { unreachable!() }; @@ -204,17 +210,23 @@ impl WritableZone for WriteZone { } // Make the new version visible. - trace!("Commit: Making zone version '{:#?}' current", self.version); - let marker = self.zone_versions.write().update_current(self.version); - self.zone_versions + trace!( + "Commit: Making zone version '{:#?}' current", + self.new_version + ); + let marker = self + .published_versions + .write() + .update_current(self.new_version); + self.published_versions .write() - .push_version(self.version, marker); + .push_version(self.new_version, marker); - trace!("Commit: zone versions: {:#?}", self.zone_versions); + trace!("Commit: zone versions: {:#?}", self.published_versions); trace!("Commit: zone dump:\n{:#?}", self.apex); // Start the next version. - self.version = self.version.next(); + self.new_version = self.new_version.next(); self.dirty = false; Box::pin(ready(Ok(out_diff))) @@ -320,7 +332,7 @@ impl WriteNode { trace!("Updating RRset"); if let Some((owner, diff)) = &self.diff { let changed = if let Some(removed_rrset) = - rrsets.get(rrset.rtype(), self.zone.version.prev()) + rrsets.get(rrset.rtype(), self.zone.last_published_version()) { let changed = rrset != removed_rrset; @@ -351,7 +363,7 @@ impl WriteNode { // if rrset.is_empty() { // rrsets.remove(rrset.rtype(), self.zone.version.prev()); // } else { - rrsets.update(rrset, self.zone.version); + rrsets.update(rrset, self.zone.new_version); // } self.check_nx_domain()?; Ok(()) @@ -366,7 +378,7 @@ impl WriteNode { Either::Right(ref node) => node.rrsets(), }; - Ok(rrsets.get(rtype, self.zone.version)) + Ok(rrsets.get(rtype, self.zone.new_version)) } fn remove_rrset(&self, rtype: Rtype) -> Result<(), io::Error> { @@ -376,7 +388,8 @@ impl WriteNode { }; if let Some((owner, diff)) = &self.diff { - if let Some(removed) = rrsets.get(rtype, self.zone.version.prev()) + if let Some(removed) = + rrsets.get(rtype, self.zone.last_published_version()) { trace!( "Diff detected: removal of existing RRSET: {removed:#?}" @@ -389,7 +402,7 @@ impl WriteNode { } } - rrsets.remove(rtype, self.zone.version); + rrsets.remove(rtype, self.zone.new_version); self.check_nx_domain()?; Ok(()) @@ -397,7 +410,7 @@ impl WriteNode { fn make_regular(&self) -> Result<(), io::Error> { if let Either::Right(ref node) = self.node { - node.update_special(self.zone.version, None); + node.update_special(self.zone.new_version, None); self.check_nx_domain()?; } Ok(()) @@ -408,7 +421,7 @@ impl WriteNode { Either::Left(_) => Err(WriteApexError::NotAllowed), Either::Right(ref node) => { node.update_special( - self.zone.version, + self.zone.new_version, Some(Special::Cut(cut)), ); Ok(()) @@ -427,7 +440,7 @@ impl WriteNode { Either::Left(_) => Err(WriteApexError::NotAllowed), Either::Right(ref node) => { node.update_special( - self.zone.version, + self.zone.new_version, Some(Special::Cname(cname)), ); Ok(()) @@ -444,10 +457,10 @@ impl WriteNode { fn remove_all(&self) -> Result<(), io::Error> { match self.node { Either::Left(ref apex) => { - apex.remove_all(self.zone.version); + apex.remove_all(self.zone.new_version); } Either::Right(ref node) => { - node.remove_all(self.zone.version); + node.remove_all(self.zone.new_version); } } @@ -460,32 +473,34 @@ impl WriteNode { Either::Left(_) => return Ok(()), Either::Right(ref node) => node, }; - let opt_new_nxdomain = - node.with_special(self.zone.version, |special| match special { + let opt_new_nxdomain = node.with_special( + self.zone.new_version, + |special| match special { Some(Special::NxDomain) => { - if !node.rrsets().is_empty(self.zone.version) { + if !node.rrsets().is_empty(self.zone.new_version) { Some(false) } else { None } } None => { - if node.rrsets().is_empty(self.zone.version) { + if node.rrsets().is_empty(self.zone.new_version) { Some(true) } else { None } } _ => None, - }); + }, + ); if let Some(new_nxdomain) = opt_new_nxdomain { if new_nxdomain { node.update_special( - self.zone.version, + self.zone.new_version, Some(Special::NxDomain), ); } else { - node.update_special(self.zone.version, None); + node.update_special(self.zone.new_version, None); } } Ok(()) From 75fef4e9d8ce4e8b66bdb27b109a508a54efc2bd Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 10:23:42 +0200 Subject: [PATCH 211/333] Cargo fmt. --- src/zonetree/traits.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index e92e62646..8ffc66115 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -151,7 +151,7 @@ pub trait WritableZone: Send + Sync { /// made since the last commit. Only clients who obtain a [`ReadableZone`] /// _after_ invoking this function will be able to see the changes made /// since [`open()`] was called. - /// + /// /// [`open()`]: Self::open fn commit( &mut self, From 54892b01db2b0807b4651fd4766fd2150de85cee Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 10:25:33 +0200 Subject: [PATCH 212/333] Move dependency requirement to correct feature gate. --- Cargo.toml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 15f013ac2..82108c642 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -61,11 +61,11 @@ zonefile = ["bytes", "serde", "std"] # Unstable features unstable-client-transport = ["moka", "net", "tracing"] -unstable-server-transport = ["arc-swap", "chrono/clock", "libc", "net", "rustversion", "siphasher", "tracing"] +unstable-server-transport = ["arc-swap", "chrono/clock", "libc", "net", "siphasher", "tracing"] unstable-stelline = ["tokio/test-util", "tracing", "tracing-subscriber", "unstable-server-transport", "zonefile"] unstable-validator = ["validate", "zonefile", "unstable-client-transport"] unstable-xfr = [] -unstable-zonetree = ["futures-util", "parking_lot", "serde", "tokio", "tracing"] +unstable-zonetree = ["futures-util", "parking_lot", "rustversion", "serde", "tokio", "tracing"] [dev-dependencies] lazy_static = { version = "1.4.0" } From 6dde95198ad1bdc8507dc58a33f2284cc0aa5c48 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 11:29:20 +0200 Subject: [PATCH 213/333] Remove added fn as_any(), this PR doesn't use it. --- src/zonetree/in_memory/nodes.rs | 5 ----- src/zonetree/traits.rs | 3 --- 2 files changed, 8 deletions(-) diff --git a/src/zonetree/in_memory/nodes.rs b/src/zonetree/in_memory/nodes.rs index 73c5ef98e..826bd8150 100644 --- a/src/zonetree/in_memory/nodes.rs +++ b/src/zonetree/in_memory/nodes.rs @@ -1,6 +1,5 @@ //! The nodes in a zone tree. -use core::any::Any; use std::boxed::Box; use std::collections::{hash_map, HashMap}; use std::future::Future; @@ -152,10 +151,6 @@ impl ZoneStore for ZoneApex { as Box }) } - - fn as_any(&self) -> &dyn Any { - self as &dyn Any - } } //--- impl From<&'a ZoneApex> diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index 8ffc66115..37904047f 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -57,9 +57,6 @@ pub trait ZoneStore: Debug + Sync + Send + Any { + 'static), >, >; - - /// TODO - fn as_any(&self) -> &dyn Any; } //------------ ReadableZone -------------------------------------------------- From d003eb44a9c44655d0efa74bb17602e692acf095 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 11:29:36 +0200 Subject: [PATCH 214/333] More accurate name. --- src/zonetree/in_memory/nodes.rs | 2 +- src/zonetree/in_memory/write.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/zonetree/in_memory/nodes.rs b/src/zonetree/in_memory/nodes.rs index 826bd8150..ee2f91a92 100644 --- a/src/zonetree/in_memory/nodes.rs +++ b/src/zonetree/in_memory/nodes.rs @@ -268,7 +268,7 @@ impl NodeRrsets { } /// Removes the RRset for the given type. - pub fn remove(&self, rtype: Rtype, version: Version) { + pub fn remove_rtype(&self, rtype: Rtype, version: Version) { self.rrsets .write() .entry(rtype) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index 5e4232a06..4590759dd 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -402,7 +402,7 @@ impl WriteNode { } } - rrsets.remove(rtype, self.zone.new_version); + rrsets.remove_rtype(rtype, self.zone.new_version); self.check_nx_domain()?; Ok(()) From 8d52037ebbe8e7b56e50b3c6637e5489ad71be25 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 11:29:45 +0200 Subject: [PATCH 215/333] Improved comment. --- src/net/xfr/protocol/iterator.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/net/xfr/protocol/iterator.rs b/src/net/xfr/protocol/iterator.rs index a672e4e9b..ae17771b8 100644 --- a/src/net/xfr/protocol/iterator.rs +++ b/src/net/xfr/protocol/iterator.rs @@ -72,9 +72,9 @@ impl<'a, 'b> Iterator for XfrZoneUpdateIterator<'a, 'b> { self.state.rr_count += 1; if self.state.actual_xfr_type == XfrType::Axfr { - // For AXFR we're not making changes to a zone, we're - // replacing its entire contents, so before returning any - // actual updates to apply, first instruct the consumer to + // For AXFR we're not making incremental changes to a zone, + // we're replacing its entire contents, so before returning + // any actual updates to apply first instruct the consumer to // "discard" everything it has. return Some(Ok(ZoneUpdate::DeleteAllRecords)); } From aaedea4466c97a69739613eac0b758ae87d7d226 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 11:29:55 +0200 Subject: [PATCH 216/333] Remove unused function. --- src/zonetree/in_memory/nodes.rs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/zonetree/in_memory/nodes.rs b/src/zonetree/in_memory/nodes.rs index ee2f91a92..3510c6ae7 100644 --- a/src/zonetree/in_memory/nodes.rs +++ b/src/zonetree/in_memory/nodes.rs @@ -335,11 +335,6 @@ impl NodeRrset { pub fn rollback(&mut self, version: Version) { self.rrsets.rollback(version); } - - #[allow(dead_code)] - pub fn clean(&mut self, version: Version) { - self.rrsets.rollback(version); - } } //------------ Special ------------------------------------------------------- From e836a1a98b48017f8ff3eff61985ce4936270172 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 11:31:16 +0200 Subject: [PATCH 217/333] This isn't the right place for this change. --- src/zonetree/in_memory/versioned.rs | 19 +------------------ 1 file changed, 1 insertion(+), 18 deletions(-) diff --git a/src/zonetree/in_memory/versioned.rs b/src/zonetree/in_memory/versioned.rs index 0b611c363..07bb4c9e5 100644 --- a/src/zonetree/in_memory/versioned.rs +++ b/src/zonetree/in_memory/versioned.rs @@ -69,24 +69,7 @@ impl Versioned { } pub fn remove(&mut self, version: Version) { - // WARNING: This isn't safe to do while updating a zone, e.g. via an - // AXFR that lacks some records that were in the previous version of - // the zone, as the effects are immediately visible to users of the - // zone! - // - // self.data.retain(|item| item.0 >= version) - // - // When updating a Zone via ZoneStore::write(), the new version of the - // zone that is created will be one higher than the highest version of - // data currently in the zone. - // - // So adding an empty value at the new version will cause current - // clients to continue seeing the old version, but clients of the zone - // after it is committed will see the new version, i.e. the empty - // value which will cause get() to return None. - if self.data.last().map(|item| item.0).is_some() { - self.data.push((version, None)); - } + self.data.retain(|item| item.0 >= version) } } From 0ac6a2a38644405df6842129c67e700ddaf5b1e9 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 11:59:31 +0200 Subject: [PATCH 218/333] Hmm, maybe it should be here, but with slightly different behaviour. --- src/zonetree/in_memory/versioned.rs | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/zonetree/in_memory/versioned.rs b/src/zonetree/in_memory/versioned.rs index 07bb4c9e5..59ad09f7f 100644 --- a/src/zonetree/in_memory/versioned.rs +++ b/src/zonetree/in_memory/versioned.rs @@ -69,7 +69,17 @@ impl Versioned { } pub fn remove(&mut self, version: Version) { - self.data.retain(|item| item.0 >= version) + // We can't just remove the value for the specified version because if + // it should be a new version of the zone and a value exists for a + // previous version, then we have to mask the old value so that it + // isn't seen by consumers of the newer version of the zone. + if let Some(last) = self.data.last_mut() { + if last.0 == version { + last.1 = None; + return; + } + } + self.data.push((version, None)) } } From 2ee5a1a4c2d3a9bc6e880dfdf92bc979f2796c79 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 12:11:48 +0200 Subject: [PATCH 219/333] Avoid cluttering the version vec with uneccessary entries. --- src/zonetree/in_memory/versioned.rs | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/src/zonetree/in_memory/versioned.rs b/src/zonetree/in_memory/versioned.rs index 59ad09f7f..7d49329bd 100644 --- a/src/zonetree/in_memory/versioned.rs +++ b/src/zonetree/in_memory/versioned.rs @@ -73,13 +73,30 @@ impl Versioned { // it should be a new version of the zone and a value exists for a // previous version, then we have to mask the old value so that it // isn't seen by consumers of the newer version of the zone. + let len = self.data.len(); if let Some(last) = self.data.last_mut() { + if last.1.is_none() { + // If it was already marked as removed in the last version + // we don't need to mark it removed again. + return; + } if last.0 == version { - last.1 = None; + if len == 1 { + // If this new version is the only version, we can + // remove it entirely rather than mark it as deleted. + let _ = self.data.pop(); + } else { + last.1 = None; + } return; } } - self.data.push((version, None)) + + // If there's nothing here, we don't need to explicitly mark that + // there is nothing here. + if !self.data.is_empty() { + self.data.push((version, None)) + } } } From 570942b9aa818d101538b1b3d4a2c2b34643d290 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 12:23:59 +0200 Subject: [PATCH 220/333] Move and tweak some RustDocs. --- src/zonetree/update.rs | 359 ++++++++++++++++++++--------------------- 1 file changed, 177 insertions(+), 182 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 419f1eead..664e06091 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -3,185 +3,6 @@ //! This module provides a high-level interface for making alterations to the //! content of zones without requiring knowledge of the low-level details of //! how the [`WritableZone`] trait implemented by [`Zone`] works. -//! -//! It can be used manually, or in combination with a source of -//! [`ZoneUpdate`]s such as -//! [`XfrResponseInterpreter`][crate::net::xfr::protocol::XfrResponseInterpreter]. -//! -//!
-//! -//! `WritableZone::commit()` is invoked by `ZoneUpdater` when it receives -//! `ZoneUpdate::Finished`. If `ZoneUpdate::Finished` is not passed to -//! `ZoneUpdater::apply()` there is no guarantee that the applied changes to -//! the zone will take effect. -//! -//!
-//! -//! # Replacing the content of a zone -//! -//! ``` -//! # use std::str::FromStr; -//! # -//! # use domain::base::iana::Class; -//! # use domain::base::MessageBuilder; -//! # use domain::base::Name; -//! # use domain::base::ParsedName; -//! # use domain::base::Record; -//! # use domain::base::Serial; -//! # use domain::base::Ttl; -//! # use domain::base::net::Ipv4Addr; -//! # use domain::net::xfr::protocol::XfrResponseInterpreter; -//! # use domain::rdata::A; -//! # use domain::rdata::Soa; -//! # use domain::rdata::ZoneRecordData; -//! # use domain::zonetree::ZoneBuilder; -//! # use domain::zonetree::types::ZoneUpdate; -//! # use domain::zonetree::update::ZoneUpdater; -//! # -//! # #[tokio::main] -//! # async fn main() { -//! # -//! # let builder = ZoneBuilder::new(Name::from_str("example.com").unwrap(), Class::IN); -//! # let zone = builder.build(); -//! # -//! # // Prepare some records to pass to ZoneUpdater -//! # let serial = Serial::now(); -//! # let mname = ParsedName::from(Name::from_str("mname").unwrap()); -//! # let rname = ParsedName::from(Name::from_str("rname").unwrap()); -//! # let ttl = Ttl::from_secs(0); -//! # let new_soa_rec = Record::new( -//! # ParsedName::from(Name::from_str("example.com").unwrap()), -//! # Class::IN, -//! # Ttl::from_secs(0), -//! # ZoneRecordData::Soa(Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl)), -//! # ); -//! # -//! # let a_data = A::new(Ipv4Addr::LOCALHOST); -//! # let a_rec = Record::new( -//! # ParsedName::from(Name::from_str("a.example.com").unwrap()), -//! # Class::IN, -//! # Ttl::from_secs(0), -//! # ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), -//! # ); -//! # -//! let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); -//! updater.apply(ZoneUpdate::DeleteAllRecords); -//! updater.apply(ZoneUpdate::AddRecord(a_rec)); -//! updater.apply(ZoneUpdate::Finished(new_soa_rec)); -//! # -//! # } -//! ``` -//! -//! # Altering the content of a zone -//! -//! ```rust -//! # use std::str::FromStr; -//! # -//! # use domain::base::iana::Class; -//! # use domain::base::MessageBuilder; -//! # use domain::base::Name; -//! # use domain::base::ParsedName; -//! # use domain::base::Record; -//! # use domain::base::Serial; -//! # use domain::base::Ttl; -//! # use domain::base::net::Ipv4Addr; -//! # use domain::base::net::Ipv6Addr; -//! # use domain::net::xfr::protocol::XfrResponseInterpreter; -//! # use domain::rdata::A; -//! # use domain::rdata::Aaaa; -//! # use domain::rdata::Soa; -//! # use domain::rdata::ZoneRecordData; -//! # use domain::zonetree::ZoneBuilder; -//! # use domain::zonetree::update::ZoneUpdater; -//! # use domain::zonetree::types::ZoneUpdate; -//! # -//! # #[tokio::main] -//! # async fn main() { -//! # -//! # let builder = ZoneBuilder::new(Name::from_str("example.com").unwrap(), Class::IN); -//! # let zone = builder.build(); -//! # -//! # // Prepare some records to pass to ZoneUpdater -//! # let serial = Serial::now(); -//! # let mname = ParsedName::from(Name::from_str("mname").unwrap()); -//! # let rname = ParsedName::from(Name::from_str("rname").unwrap()); -//! # let ttl = Ttl::from_secs(0); -//! # let new_soa_rec = Record::new( -//! # ParsedName::from(Name::from_str("example.com").unwrap()), -//! # Class::IN, -//! # Ttl::from_secs(0), -//! # ZoneRecordData::Soa(Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl)), -//! # ); -//! # -//! # let old_a_data = A::new(Ipv4Addr::LOCALHOST); -//! # let old_a_rec = Record::new( -//! # ParsedName::from(Name::from_str("a.example.com").unwrap()), -//! # Class::IN, -//! # Ttl::from_secs(0), -//! # ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), -//! # ); -//! # -//! # let new_aaaa_data = Aaaa::new(Ipv6Addr::LOCALHOST); -//! # let new_aaaa_rec = Record::new( -//! # ParsedName::from(Name::from_str("a.example.com").unwrap()), -//! # Class::IN, -//! # Ttl::from_secs(0), -//! # ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), -//! # ); -//! # -//! let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); -//! updater.apply(ZoneUpdate::DeleteRecord(old_a_rec)); -//! updater.apply(ZoneUpdate::AddRecord(new_aaaa_rec)); -//! updater.apply(ZoneUpdate::Finished(new_soa_rec)); -//! # -//! # } -//! ``` -//! -//! # Applying XFR changes to a zone -//! -//! ```no_run -//! # use std::str::FromStr; -//! # -//! # use domain::base::iana::Class; -//! # use domain::base::MessageBuilder; -//! # use domain::base::Name; -//! # use domain::base::Serial; -//! # use domain::net::xfr::protocol::XfrResponseInterpreter; -//! # use domain::zonetree::ZoneBuilder; -//! # use domain::zonetree::update::ZoneUpdater; -//! # -//! # #[tokio::main] -//! # async fn main() { -//! # -//! // Given a zone -//! let builder = ZoneBuilder::new(Name::from_str("example.com").unwrap(), Class::IN); -//! let zone = builder.build(); -//! -//! // And a ZoneUpdater -//! let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); -//! -//! // And an XFR response interpreter -//! let mut interpreter = XfrResponseInterpreter::new(); -//! -//! // Iterate over the XFR responses applying the updates to the zone -//! while !interpreter.is_finished() { -//! // Get the next XFR response: -//! // For this example this is just a dummy response, which would cause -//! // Error::NotValidXfrResponse if this code were run. -//! let next_xfr_response = MessageBuilder::new_bytes().into_message(); -//! -//! // Convert it to an update iterator -//! let it = interpreter.interpret_response(next_xfr_response).unwrap(); -//! -//! // Iterate over the updates -//! for update in it { -//! // Apply each update to the zone -//! updater.apply(update.unwrap()).await.unwrap(); -//! } -//! } -//! # -//! # } -//! ``` use core::future::Future; use core::pin::Pin; @@ -212,13 +33,187 @@ use super::{WritableZone, WritableZoneNode, Zone, ZoneDiff}; /// [`ZoneUpdate::DeleteAllRecords`] to `apply` before any other updates. /// /// Changes to the zone are committed when [`ZoneUpdate::Finished`] is -/// received. +/// received, or rolled back if [`ZoneUpdater`] is dropped before receiving +/// [`ZoneUpdate::Finished`]. /// /// Passing [`ZoneUpdate::BeginBatchDelete`] commits any edits in progress and /// starts editing a new zone version. /// -/// For each commit of the zone a diff of the changes made will be stored with -/// the zone. +/// For each commit of the zone a diff of the changes made is requested and, +/// if a diff was actually created, it will be returned by +/// [`ZoneUpdater::apply()`]. +/// +/// # Usage +/// +/// [`ZoneUpdater`] can be used manually, or in combination with a source of +/// [`ZoneUpdate`]s such as +/// [`XfrResponseInterpreter`][crate::net::xfr::protocol::XfrResponseInterpreter]. +/// +/// # Replacing the content of a zone +/// +/// ``` +/// # use std::str::FromStr; +/// # +/// # use domain::base::iana::Class; +/// # use domain::base::MessageBuilder; +/// # use domain::base::Name; +/// # use domain::base::ParsedName; +/// # use domain::base::Record; +/// # use domain::base::Serial; +/// # use domain::base::Ttl; +/// # use domain::base::net::Ipv4Addr; +/// # use domain::net::xfr::protocol::XfrResponseInterpreter; +/// # use domain::rdata::A; +/// # use domain::rdata::Soa; +/// # use domain::rdata::ZoneRecordData; +/// # use domain::zonetree::ZoneBuilder; +/// # use domain::zonetree::types::ZoneUpdate; +/// # use domain::zonetree::update::ZoneUpdater; +/// # +/// # #[tokio::main] +/// # async fn main() { +/// # +/// # let builder = ZoneBuilder::new(Name::from_str("example.com").unwrap(), Class::IN); +/// # let zone = builder.build(); +/// # +/// # // Prepare some records to pass to ZoneUpdater +/// # let serial = Serial::now(); +/// # let mname = ParsedName::from(Name::from_str("mname").unwrap()); +/// # let rname = ParsedName::from(Name::from_str("rname").unwrap()); +/// # let ttl = Ttl::from_secs(0); +/// # let new_soa_rec = Record::new( +/// # ParsedName::from(Name::from_str("example.com").unwrap()), +/// # Class::IN, +/// # Ttl::from_secs(0), +/// # ZoneRecordData::Soa(Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl)), +/// # ); +/// # +/// # let a_data = A::new(Ipv4Addr::LOCALHOST); +/// # let a_rec = Record::new( +/// # ParsedName::from(Name::from_str("a.example.com").unwrap()), +/// # Class::IN, +/// # Ttl::from_secs(0), +/// # ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), +/// # ); +/// # +/// let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); +/// updater.apply(ZoneUpdate::DeleteAllRecords); +/// updater.apply(ZoneUpdate::AddRecord(a_rec)); +/// updater.apply(ZoneUpdate::Finished(new_soa_rec)); +/// # +/// # } +/// ``` +/// +/// # Altering the content of a zone +/// +/// ```rust +/// # use std::str::FromStr; +/// # +/// # use domain::base::iana::Class; +/// # use domain::base::MessageBuilder; +/// # use domain::base::Name; +/// # use domain::base::ParsedName; +/// # use domain::base::Record; +/// # use domain::base::Serial; +/// # use domain::base::Ttl; +/// # use domain::base::net::Ipv4Addr; +/// # use domain::base::net::Ipv6Addr; +/// # use domain::net::xfr::protocol::XfrResponseInterpreter; +/// # use domain::rdata::A; +/// # use domain::rdata::Aaaa; +/// # use domain::rdata::Soa; +/// # use domain::rdata::ZoneRecordData; +/// # use domain::zonetree::ZoneBuilder; +/// # use domain::zonetree::update::ZoneUpdater; +/// # use domain::zonetree::types::ZoneUpdate; +/// # +/// # #[tokio::main] +/// # async fn main() { +/// # +/// # let builder = ZoneBuilder::new(Name::from_str("example.com").unwrap(), Class::IN); +/// # let zone = builder.build(); +/// # +/// # // Prepare some records to pass to ZoneUpdater +/// # let serial = Serial::now(); +/// # let mname = ParsedName::from(Name::from_str("mname").unwrap()); +/// # let rname = ParsedName::from(Name::from_str("rname").unwrap()); +/// # let ttl = Ttl::from_secs(0); +/// # let new_soa_rec = Record::new( +/// # ParsedName::from(Name::from_str("example.com").unwrap()), +/// # Class::IN, +/// # Ttl::from_secs(0), +/// # ZoneRecordData::Soa(Soa::new(mname, rname, serial, ttl, ttl, ttl, ttl)), +/// # ); +/// # +/// # let old_a_data = A::new(Ipv4Addr::LOCALHOST); +/// # let old_a_rec = Record::new( +/// # ParsedName::from(Name::from_str("a.example.com").unwrap()), +/// # Class::IN, +/// # Ttl::from_secs(0), +/// # ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), +/// # ); +/// # +/// # let new_aaaa_data = Aaaa::new(Ipv6Addr::LOCALHOST); +/// # let new_aaaa_rec = Record::new( +/// # ParsedName::from(Name::from_str("a.example.com").unwrap()), +/// # Class::IN, +/// # Ttl::from_secs(0), +/// # ZoneRecordData::A(A::new(Ipv4Addr::LOCALHOST)), +/// # ); +/// # +/// let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); +/// updater.apply(ZoneUpdate::DeleteRecord(old_a_rec)); +/// updater.apply(ZoneUpdate::AddRecord(new_aaaa_rec)); +/// updater.apply(ZoneUpdate::Finished(new_soa_rec)); +/// # +/// # } +/// ``` +/// +/// # Applying XFR changes to a zone +/// +/// ```no_run +/// # use std::str::FromStr; +/// # +/// # use domain::base::iana::Class; +/// # use domain::base::MessageBuilder; +/// # use domain::base::Name; +/// # use domain::base::Serial; +/// # use domain::net::xfr::protocol::XfrResponseInterpreter; +/// # use domain::zonetree::ZoneBuilder; +/// # use domain::zonetree::update::ZoneUpdater; +/// # +/// # #[tokio::main] +/// # async fn main() { +/// # +/// // Given a zone +/// let builder = ZoneBuilder::new(Name::from_str("example.com").unwrap(), Class::IN); +/// let zone = builder.build(); +/// +/// // And a ZoneUpdater +/// let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); +/// +/// // And an XFR response interpreter +/// let mut interpreter = XfrResponseInterpreter::new(); +/// +/// // Iterate over the XFR responses applying the updates to the zone +/// while !interpreter.is_finished() { +/// // Get the next XFR response: +/// // For this example this is just a dummy response, which would cause +/// // Error::NotValidXfrResponse if this code were run. +/// let next_xfr_response = MessageBuilder::new_bytes().into_message(); +/// +/// // Convert it to an update iterator +/// let it = interpreter.interpret_response(next_xfr_response).unwrap(); +/// +/// // Iterate over the updates +/// for update in it { +/// // Apply each update to the zone +/// updater.apply(update.unwrap()).await.unwrap(); +/// } +/// } +/// # +/// # } +/// ``` pub struct ZoneUpdater { /// The zone to be updated. zone: Zone, From 95182218044a5d5c6fa270c19cb689030c9be149 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 12:27:51 +0200 Subject: [PATCH 221/333] More RustDocs. --- src/zonetree/update.rs | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 664e06091..69e7d79a6 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -29,19 +29,12 @@ use super::{WritableZone, WritableZoneNode, Zone, ZoneDiff}; /// writing, edits made and then the changes committed, only then becoming /// visible for readers of the zone. /// -/// To completely replace the content of a zone pass -/// [`ZoneUpdate::DeleteAllRecords`] to `apply` before any other updates. -/// /// Changes to the zone are committed when [`ZoneUpdate::Finished`] is /// received, or rolled back if [`ZoneUpdater`] is dropped before receiving /// [`ZoneUpdate::Finished`]. /// -/// Passing [`ZoneUpdate::BeginBatchDelete`] commits any edits in progress and -/// starts editing a new zone version. -/// /// For each commit of the zone a diff of the changes made is requested and, -/// if a diff was actually created, it will be returned by -/// [`ZoneUpdater::apply()`]. +/// if a diff was actually created, will be returned by [`apply()`]. /// /// # Usage /// @@ -49,6 +42,14 @@ use super::{WritableZone, WritableZoneNode, Zone, ZoneDiff}; /// [`ZoneUpdate`]s such as /// [`XfrResponseInterpreter`][crate::net::xfr::protocol::XfrResponseInterpreter]. /// +/// To completely replace the content of a zone pass +/// [`ZoneUpdate::DeleteAllRecords`] to [`apply()`] before any other updates. +/// +/// Pass updates to be applied to the zone one at a time to [`apply()`]. +/// +/// Passing [`ZoneUpdate::BeginBatchDelete`] commits any edits in progress and +/// starts editing a new zone version. +/// /// # Replacing the content of a zone /// /// ``` @@ -214,6 +215,8 @@ use super::{WritableZone, WritableZoneNode, Zone, ZoneDiff}; /// # /// # } /// ``` +/// +/// [`apply()`]: ZoneUpdater::apply() pub struct ZoneUpdater { /// The zone to be updated. zone: Zone, From 81852acf517c52f93e2c25534a5a88f9b09bdd26 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 12:30:16 +0200 Subject: [PATCH 222/333] More RustDocs. --- src/zonetree/update.rs | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 69e7d79a6..ddb2712cf 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -42,13 +42,10 @@ use super::{WritableZone, WritableZoneNode, Zone, ZoneDiff}; /// [`ZoneUpdate`]s such as /// [`XfrResponseInterpreter`][crate::net::xfr::protocol::XfrResponseInterpreter]. /// -/// To completely replace the content of a zone pass -/// [`ZoneUpdate::DeleteAllRecords`] to [`apply()`] before any other updates. -/// /// Pass updates to be applied to the zone one at a time to [`apply()`]. /// -/// Passing [`ZoneUpdate::BeginBatchDelete`] commits any edits in progress and -/// starts editing a new zone version. +/// To completely replace the content of a zone pass +/// [`ZoneUpdate::DeleteAllRecords`] to [`apply()`] before any other updates. /// /// # Replacing the content of a zone /// @@ -261,14 +258,12 @@ impl ZoneUpdater { /// committed then any diff made by the `Zone` backing store /// implementation will be returned. /// - ///
- /// - /// This method invokes `WritableZone::commit()` when it receives - /// `ZoneUpdate::Finished`. If `ZoneUpdate::Finished` is not passed to - /// `ZoneUpdater::apply()` there is no guarantee that the applied changes - /// to the zone will take effect. + /// Changes to the zone are committed when [`ZoneUpdate::Finished`] is + /// received, or rolled back if [`ZoneUpdater`] is dropped before + /// receiving [`ZoneUpdate::Finished`]. /// - ///
+ /// Passing [`ZoneUpdate::BeginBatchDelete`] will also commit any edits in + /// progress and re-open the zone for editing again. pub async fn apply( &mut self, update: ZoneUpdate, From cd032dde4a52ee5133f9a7435cc82449e749624e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 12:32:01 +0200 Subject: [PATCH 223/333] Improved a comment. --- src/zonetree/update.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index ddb2712cf..8d09861df 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -271,7 +271,7 @@ impl ZoneUpdater { trace!("Event: {update}"); match update { ZoneUpdate::DeleteAllRecords => { - // To completely replace the content of the zone, i.e. + // To completely replace the content of the zone, i.e. with // something like an AXFR transfer, we can't add records from // a new version of the zone to an existing zone because if // the old version contained a record which the new version @@ -287,9 +287,8 @@ impl ZoneUpdater { ZoneUpdate::AddRecord(rec) => self.add_record(rec).await?, - // Note: Batches first contain deletions then additions, so batch - // deletion signals the start of a batch, and the end of any - // previous batch addition. + // Batch deletion signals the start of a batch, and the end of any + // batch addition that was in progress. ZoneUpdate::BeginBatchDelete(_old_soa) => { let diff = if self.batching { // Commit the previous batch. From cabde2379f71918bf2d3c6f10102aa039a2b9399 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 13:10:40 +0200 Subject: [PATCH 224/333] Keep it simple. --- src/zonetree/update.rs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 8d09861df..50839b227 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -314,10 +314,8 @@ impl ZoneUpdater { } ZoneUpdate::Finished(zone_soa) => { - if !self.batching { - // Update the SOA record. - self.update_soa(zone_soa).await?; - } + // Update the SOA record. + self.update_soa(zone_soa).await?; // Commit the previous batch and return any diff produced. return self.write.commit().await; From d2ae8077546c0060c083162c172c3c4e84de75ae Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 14:01:43 +0200 Subject: [PATCH 225/333] Re-use code instead of duplicating it. --- src/zonetree/in_memory/nodes.rs | 12 +++-------- src/zonetree/mod.rs | 27 +++++++++++++++++++++++++ src/zonetree/update.rs | 35 ++++++++------------------------- 3 files changed, 38 insertions(+), 36 deletions(-) diff --git a/src/zonetree/in_memory/nodes.rs b/src/zonetree/in_memory/nodes.rs index 3510c6ae7..b19986df4 100644 --- a/src/zonetree/in_memory/nodes.rs +++ b/src/zonetree/in_memory/nodes.rs @@ -12,7 +12,7 @@ use parking_lot::{ use tokio::sync::Mutex; use crate::base::iana::{Class, Rtype}; -use crate::base::name::{Label, OwnedLabel, ToLabelIter, ToName}; +use crate::base::name::{Label, OwnedLabel, ToName}; use crate::zonetree::error::{CnameError, OutOfZone, ZoneCutError}; use crate::zonetree::types::{StoredName, ZoneCut}; use crate::zonetree::walk::WalkState; @@ -23,6 +23,7 @@ use crate::zonetree::{ use super::read::ReadZone; use super::versioned::{Version, Versioned}; use super::write::{WriteZone, ZoneVersions}; +use crate::zonetree::util::rel_name_rev_iter; //------------ ZoneApex ------------------------------------------------------ @@ -71,14 +72,7 @@ impl ZoneApex { &self, qname: &'l impl ToName, ) -> Result + Clone, OutOfZone> { - let mut qname = qname.iter_labels().rev(); - for apex_label in self.name().iter_labels().rev() { - let qname_label = qname.next(); - if Some(apex_label) != qname_label { - return Err(OutOfZone); - } - } - Ok(qname) + rel_name_rev_iter(&self.apex_name, qname) } /// Returns the RRsets of this node. diff --git a/src/zonetree/mod.rs b/src/zonetree/mod.rs index 0531ed20c..130b8b503 100644 --- a/src/zonetree/mod.rs +++ b/src/zonetree/mod.rs @@ -123,3 +123,30 @@ pub use self::types::{ }; pub use self::walk::WalkOp; pub use self::zone::{Zone, ZoneKey}; + +/// Zone related utilities. +pub mod util { + use crate::base::name::{Label, ToLabelIter}; + use crate::base::ToName; + + use super::error::OutOfZone; + use super::StoredName; + + /// Gets a reverse iterator to the relative part of a name. + /// + /// Can be used for example to get an iterator over the part of a name + /// that is "under" a zone apex name. + pub fn rel_name_rev_iter<'l>( + base: &StoredName, + qname: &'l impl ToName, + ) -> Result + Clone, OutOfZone> { + let mut qname = qname.iter_labels().rev(); + for apex_label in base.iter_labels().rev() { + let qname_label = qname.next(); + if Some(apex_label) != qname_label { + return Err(OutOfZone); + } + } + Ok(qname) + } +} diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 50839b227..040c6bdb1 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -10,17 +10,17 @@ use std::borrow::ToOwned; use std::boxed::Box; use bytes::Bytes; -use tracing::{error, trace}; +use tracing::trace; -use crate::base::name::{FlattenInto, Label, ToLabelIter}; +use crate::base::name::FlattenInto; use crate::base::scan::ScannerError; -use crate::base::{Name, ParsedName, Record, Rtype, ToName}; +use crate::base::{ParsedName, Record, Rtype}; use crate::net::xfr::protocol::ParsedRecord; use crate::rdata::ZoneRecordData; use crate::zonetree::{Rrset, SharedRrset}; -use super::error::OutOfZone; use super::types::ZoneUpdate; +use super::util::rel_name_rev_iter; use super::{WritableZone, WritableZoneNode, Zone, ZoneDiff}; /// Apply a sequence of [`ZoneUpdate`]s to update the content of a [`Zone`]. @@ -327,23 +327,6 @@ impl ZoneUpdater { } impl ZoneUpdater { - fn mk_relative_name_iterator<'l>( - apex_name: &Name, - qname: &'l impl ToName, - ) -> Result + Clone, OutOfZone> { - let mut qname = qname.iter_labels().rev(); - for apex_label in apex_name.iter_labels().rev() { - let qname_label = qname.next(); - if Some(apex_label) != qname_label { - error!( - "Qname '{qname_label:?}' is not in zone '{apex_name}'" - ); - return Err(OutOfZone); - } - } - Ok(qname) - } - /// Given a zone record, obtain a [`WritableZoneNode`] for the owner. /// /// A [`Zone`] is a tree structure which can be modified by descending the @@ -369,11 +352,9 @@ impl ZoneUpdater { let mut end_node: Option> = None; - let name = - Self::mk_relative_name_iterator(self.zone.apex_name(), &owner) - .map_err(|_| { - std::io::Error::custom("Record owner name out of zone") - })?; + let name = rel_name_rev_iter(self.zone.apex_name(), &owner).map_err( + |_| std::io::Error::custom("Record owner name out of zone"), + )?; let writable = self.write.writable.as_ref().unwrap(); @@ -522,7 +503,7 @@ mod tests { use crate::base::net::Ipv4Addr; use crate::base::rdata::ComposeRecordData; use crate::base::{ - Message, MessageBuilder, ParsedName, Record, Serial, Ttl, + Message, MessageBuilder, Name, ParsedName, Record, Serial, Ttl, }; use crate::net::xfr::protocol::XfrResponseInterpreter; use crate::rdata::{Soa, A}; From 02b8c861e43d2b788807d1be46325e881bd84d5f Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 14:53:33 +0200 Subject: [PATCH 226/333] Cleanup. --- src/zonetree/update.rs | 34 ++++++++++++++++------------------ 1 file changed, 16 insertions(+), 18 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 040c6bdb1..6730ea9f0 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -8,6 +8,7 @@ use core::pin::Pin; use std::borrow::ToOwned; use std::boxed::Box; +use std::io::Error as IoError; use bytes::Bytes; use tracing::trace; @@ -344,32 +345,29 @@ impl ZoneUpdater { /// /// This function may panic if it is unable to create new tree nodes for /// the record owner name. - async fn get_writable_node_for_owner( + async fn get_writable_child_node_for_owner( &mut self, rec: &ParsedRecord, ) -> std::io::Result>> { let owner = rec.owner().to_owned(); - let mut end_node: Option> = None; + let mut it = rel_name_rev_iter(self.zone.apex_name(), &owner) + .map_err(|_| IoError::custom("Record owner name out of zone"))?; - let name = rel_name_rev_iter(self.zone.apex_name(), &owner).map_err( - |_| std::io::Error::custom("Record owner name out of zone"), - )?; + let Some(label) = it.next() else { + return Ok(None); + }; let writable = self.write.writable.as_ref().unwrap(); + let mut node = writable.update_child(label).await?; - for label in name { - trace!("Relativised label: {label}"); - end_node = Some( - match end_node { - Some(new_node) => new_node.update_child(label), - None => writable.update_child(label), - } - .await?, - ); + // Find (create if missing) the tree node for the owner name + // of the given record. + for label in it { + node = node.update_child(label).await?; } - Ok(end_node) + Ok(Some(node)) } async fn update_soa( @@ -380,7 +378,7 @@ impl ZoneUpdater { >, ) -> std::io::Result<()> { if new_soa.rtype() != Rtype::SOA { - return Err(std::io::Error::custom("Invalid SOA rtype")); + return Err(IoError::custom("Invalid SOA rtype")); } let mut rrset = Rrset::new(Rtype::SOA, new_soa.ttl()); @@ -401,7 +399,7 @@ impl ZoneUpdater { ZoneRecordData>, >, ) -> std::io::Result<()> { - let end_node = self.get_writable_node_for_owner(&rec).await?; + let end_node = self.get_writable_child_node_for_owner(&rec).await?; let mut rrset = Rrset::new(rec.rtype(), rec.ttl()); let rtype = rec.rtype(); let data = rec.data(); @@ -430,7 +428,7 @@ impl ZoneUpdater { ZoneRecordData>, >, ) -> std::io::Result<()> { - let end_node = self.get_writable_node_for_owner(&rec).await?; + let end_node = self.get_writable_child_node_for_owner(&rec).await?; let mut rrset = Rrset::new(rec.rtype(), rec.ttl()); let rtype = rec.rtype(); let data = rec.into_data().flatten_into(); From e4239b8f094d203a7f9d7f3804bc56c8c139b89c Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 14:55:19 +0200 Subject: [PATCH 227/333] Remove unnecessary clone. --- src/zonetree/update.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 6730ea9f0..8cd2b52b3 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -349,9 +349,7 @@ impl ZoneUpdater { &mut self, rec: &ParsedRecord, ) -> std::io::Result>> { - let owner = rec.owner().to_owned(); - - let mut it = rel_name_rev_iter(self.zone.apex_name(), &owner) + let mut it = rel_name_rev_iter(self.zone.apex_name(), rec.owner()) .map_err(|_| IoError::custom("Record owner name out of zone"))?; let Some(label) = it.next() else { From c0cea1708af6146d6c888ec52cdc738cd8bc6e39 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 15:34:23 +0200 Subject: [PATCH 228/333] Use an own error type. --- src/zonetree/update.rs | 60 +++++++++++++++++++++++++++++++++--------- 1 file changed, 48 insertions(+), 12 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 8cd2b52b3..7b943cb1c 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -20,6 +20,7 @@ use crate::net::xfr::protocol::ParsedRecord; use crate::rdata::ZoneRecordData; use crate::zonetree::{Rrset, SharedRrset}; +use super::error::OutOfZone; use super::types::ZoneUpdate; use super::util::rel_name_rev_iter; use super::{WritableZone, WritableZoneNode, Zone, ZoneDiff}; @@ -239,7 +240,7 @@ impl ZoneUpdater { /// Use [`apply`][Self::apply] to apply changes to the zone. pub fn new( zone: Zone, - ) -> Pin> + Send>> { + ) -> Pin> + Send>> { Box::pin(async move { let write = WriteState::new(zone.clone()).await?; @@ -268,7 +269,7 @@ impl ZoneUpdater { pub async fn apply( &mut self, update: ZoneUpdate, - ) -> std::io::Result> { + ) -> Result, Error> { trace!("Event: {update}"); match update { ZoneUpdate::DeleteAllRecords => { @@ -348,9 +349,8 @@ impl ZoneUpdater { async fn get_writable_child_node_for_owner( &mut self, rec: &ParsedRecord, - ) -> std::io::Result>> { - let mut it = rel_name_rev_iter(self.zone.apex_name(), rec.owner()) - .map_err(|_| IoError::custom("Record owner name out of zone"))?; + ) -> Result>, Error> { + let mut it = rel_name_rev_iter(self.zone.apex_name(), rec.owner())?; let Some(label) = it.next() else { return Ok(None); @@ -396,7 +396,7 @@ impl ZoneUpdater { ParsedName, ZoneRecordData>, >, - ) -> std::io::Result<()> { + ) -> Result<(), Error> { let end_node = self.get_writable_child_node_for_owner(&rec).await?; let mut rrset = Rrset::new(rec.rtype(), rec.ttl()); let rtype = rec.rtype(); @@ -416,7 +416,9 @@ impl ZoneUpdater { } trace!("Removing single RR of {rtype} so updating RRSET"); - node.update_rrset(SharedRrset::new(rrset)).await + node.update_rrset(SharedRrset::new(rrset)).await?; + + Ok(()) } async fn add_record( @@ -425,7 +427,7 @@ impl ZoneUpdater { ParsedName, ZoneRecordData>, >, - ) -> std::io::Result<()> { + ) -> Result<(), Error> { let end_node = self.get_writable_child_node_for_owner(&rec).await?; let mut rrset = Rrset::new(rec.rtype(), rec.ttl()); let rtype = rec.rtype(); @@ -443,7 +445,9 @@ impl ZoneUpdater { } } - node.update_rrset(SharedRrset::new(rrset)).await + node.update_rrset(SharedRrset::new(rrset)).await?; + + Ok(()) } } @@ -469,19 +473,22 @@ impl WriteState { Ok(()) } - async fn commit(&mut self) -> std::io::Result> { + async fn commit(&mut self) -> Result, Error> { // Commit the deletes and adds that just occurred if let Some(writable) = self.writable.take() { // Ensure that there are no dangling references to the created // diff (otherwise commit() will panic). drop(writable); - self.write.commit(false).await + + let diff = self.write.commit(false).await?; + + Ok(diff) } else { Ok(None) } } - async fn reopen(&mut self) -> std::io::Result<()> { + async fn reopen(&mut self) -> Result<(), Error> { self.writable = Some(self.write.open(true).await?); Ok(()) } @@ -623,3 +630,32 @@ mod tests { .unwrap(); } } + +//------------ Error ---------------------------------------------------------- + +/// Zone update error. +#[derive(Debug)] +pub enum Error { + /// The record owner is outside the zone. + OutOfZone, + + /// The record must be a SOA record. + NotSoaRecord, + + /// An I/O error occurred while updating the zone. + IoError(std::io::Error), +} + +//--- From + +impl From for Error { + fn from(err: std::io::Error) -> Self { + Self::IoError(err) + } +} + +impl From for Error { + fn from(_: OutOfZone) -> Self { + Self::OutOfZone + } +} From 714b293a23e21f0224a453b94dc2df0f693d75a6 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 15:35:58 +0200 Subject: [PATCH 229/333] More use of an own error type. --- src/zonetree/update.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 7b943cb1c..fe526a8cb 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -8,13 +8,11 @@ use core::pin::Pin; use std::borrow::ToOwned; use std::boxed::Box; -use std::io::Error as IoError; use bytes::Bytes; use tracing::trace; use crate::base::name::FlattenInto; -use crate::base::scan::ScannerError; use crate::base::{ParsedName, Record, Rtype}; use crate::net::xfr::protocol::ParsedRecord; use crate::rdata::ZoneRecordData; @@ -374,9 +372,9 @@ impl ZoneUpdater { ParsedName, ZoneRecordData>, >, - ) -> std::io::Result<()> { + ) -> Result<(), Error> { if new_soa.rtype() != Rtype::SOA { - return Err(IoError::custom("Invalid SOA rtype")); + return Err(Error::NotSoaRecord); } let mut rrset = Rrset::new(Rtype::SOA, new_soa.ttl()); @@ -386,7 +384,9 @@ impl ZoneUpdater { .as_ref() .unwrap() .update_rrset(SharedRrset::new(rrset)) - .await + .await?; + + Ok(()) } /// Find and delete a record in the zone by exact match. From 8ed8bf4286977df79c4ffd825327b379c41a4b82 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 15:57:48 +0200 Subject: [PATCH 230/333] Cleanup and more RustDoc. --- src/zonetree/update.rs | 52 +++++++++++++++++++++++++----------------- 1 file changed, 31 insertions(+), 21 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index fe526a8cb..ae9c800da 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -283,9 +283,13 @@ impl ZoneUpdater { self.write.remove_all().await?; } - ZoneUpdate::DeleteRecord(rec) => self.delete_record(rec).await?, + ZoneUpdate::DeleteRecord(rec) => { + self.delete_record_from_rrset(rec).await? + } - ZoneUpdate::AddRecord(rec) => self.add_record(rec).await?, + ZoneUpdate::AddRecord(rec) => { + self.add_record_to_rrset(rec).await? + } // Batch deletion signals the start of a batch, and the end of any // batch addition that was in progress. @@ -366,6 +370,7 @@ impl ZoneUpdater { Ok(Some(node)) } + /// Create or update the SOA RRset using the given SOA record. async fn update_soa( &mut self, new_soa: Record< @@ -389,25 +394,26 @@ impl ZoneUpdater { Ok(()) } - /// Find and delete a record in the zone by exact match. - async fn delete_record( + /// Find and delete a resource record in the zone by exact match. + async fn delete_record_from_rrset( &mut self, rec: Record< ParsedName, ZoneRecordData>, >, ) -> Result<(), Error> { - let end_node = self.get_writable_child_node_for_owner(&rec).await?; + // Find or create the point to edit in the node tree. + let tree_node = self.get_writable_child_node_for_owner(&rec).await?; + let writable = self.write.writable.as_ref().unwrap(); + let tree_node = tree_node.as_ref().unwrap_or(writable); + + // Prepare an RRset that contains all of the records of the existing + // RRset in the tree except the one to delete. let mut rrset = Rrset::new(rec.rtype(), rec.ttl()); let rtype = rec.rtype(); let data = rec.data(); - let writable = self.write.writable.as_ref().unwrap(); - - trace!("Deleting RR for {rtype}"); - let node = end_node.as_ref().unwrap_or(writable); - - if let Some(existing_rrset) = node.get_rrset(rtype).await? { + if let Some(existing_rrset) = tree_node.get_rrset(rtype).await? { for existing_data in existing_rrset.data() { if existing_data != data { rrset.push_data(existing_data.clone()); @@ -415,37 +421,41 @@ impl ZoneUpdater { } } - trace!("Removing single RR of {rtype} so updating RRSET"); - node.update_rrset(SharedRrset::new(rrset)).await?; + // Replace the RRset in the tree with the new smaller one. + tree_node.update_rrset(SharedRrset::new(rrset)).await?; Ok(()) } - async fn add_record( + /// Add a resource record to a new or existing RRset. + async fn add_record_to_rrset( &mut self, rec: Record< ParsedName, ZoneRecordData>, >, ) -> Result<(), Error> { - let end_node = self.get_writable_child_node_for_owner(&rec).await?; + // Find or create the point to edit in the node tree. + let tree_node = self.get_writable_child_node_for_owner(&rec).await?; + let writable = self.write.writable.as_ref().unwrap(); + let tree_node = tree_node.as_ref().unwrap_or(writable); + + // Prepare an RRset that contains all of the records of the existing + // RRset in the tree plus the one to add. let mut rrset = Rrset::new(rec.rtype(), rec.ttl()); let rtype = rec.rtype(); let data = rec.into_data().flatten_into(); - let writable = self.write.writable.as_ref().unwrap(); - trace!("Adding RR: {:?}", rrset); rrset.push_data(data); - let node = end_node.as_ref().unwrap_or(writable); - - if let Some(existing_rrset) = node.get_rrset(rtype).await? { + if let Some(existing_rrset) = tree_node.get_rrset(rtype).await? { for existing_data in existing_rrset.data() { rrset.push_data(existing_data.clone()); } } - node.update_rrset(SharedRrset::new(rrset)).await?; + // Replace the Rrset in the tree with the new bigger one. + tree_node.update_rrset(SharedRrset::new(rrset)).await?; Ok(()) } From 9d0bc805a12b7294bad00a23f86ad7ffaaf44196 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 16:42:18 +0200 Subject: [PATCH 231/333] More RustDocs and cleanup. --- src/zonetree/update.rs | 94 ++++++++++++++++++++++++++++++------------ 1 file changed, 68 insertions(+), 26 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index ae9c800da..03e384523 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -12,7 +12,7 @@ use std::boxed::Box; use bytes::Bytes; use tracing::trace; -use crate::base::name::FlattenInto; +use crate::base::name::{FlattenInto, Label}; use crate::base::{ParsedName, Record, Rtype}; use crate::net::xfr::protocol::ParsedRecord; use crate::rdata::ZoneRecordData; @@ -222,7 +222,7 @@ pub struct ZoneUpdater { /// /// For each new zone version any old write state has to be committed and /// a new write state opened. - write: WriteState, + write: ReopenableZoneWriter, /// Whether or not we entered an IXFR-like batching mode. batching: bool, @@ -240,7 +240,7 @@ impl ZoneUpdater { zone: Zone, ) -> Pin> + Send>> { Box::pin(async move { - let write = WriteState::new(zone.clone()).await?; + let write = ReopenableZoneWriter::new(zone.clone()).await?; Ok(Self { zone, @@ -358,16 +358,15 @@ impl ZoneUpdater { return Ok(None); }; - let writable = self.write.writable.as_ref().unwrap(); - let mut node = writable.update_child(label).await?; + let mut child_node = self.write.update_child(label).await?; // Find (create if missing) the tree node for the owner name // of the given record. for label in it { - node = node.update_child(label).await?; + child_node = child_node.update_child(label).await?; } - Ok(Some(node)) + Ok(Some(child_node)) } /// Create or update the SOA RRset using the given SOA record. @@ -385,10 +384,7 @@ impl ZoneUpdater { let mut rrset = Rrset::new(Rtype::SOA, new_soa.ttl()); rrset.push_data(new_soa.data().to_owned().flatten_into()); self.write - .writable - .as_ref() - .unwrap() - .update_rrset(SharedRrset::new(rrset)) + .update_root_rrset(SharedRrset::new(rrset)) .await?; Ok(()) @@ -404,8 +400,7 @@ impl ZoneUpdater { ) -> Result<(), Error> { // Find or create the point to edit in the node tree. let tree_node = self.get_writable_child_node_for_owner(&rec).await?; - let writable = self.write.writable.as_ref().unwrap(); - let tree_node = tree_node.as_ref().unwrap_or(writable); + let tree_node = tree_node.as_ref().unwrap_or(self.write.root()); // Prepare an RRset that contains all of the records of the existing // RRset in the tree except the one to delete. @@ -437,8 +432,7 @@ impl ZoneUpdater { ) -> Result<(), Error> { // Find or create the point to edit in the node tree. let tree_node = self.get_writable_child_node_for_owner(&rec).await?; - let writable = self.write.writable.as_ref().unwrap(); - let tree_node = tree_node.as_ref().unwrap_or(writable); + let tree_node = tree_node.as_ref().unwrap_or(self.write.root()); // Prepare an RRset that contains all of the records of the existing // RRset in the tree plus the one to add. @@ -461,28 +455,35 @@ impl ZoneUpdater { } } -//------------ WriteState ----------------------------------------------------- +//------------ MultiVersionWriteHandle ---------------------------------------- -struct WriteState { +/// State for writing multiple zone versions in sequence. +/// +/// This type provides write access to the next version of a zone and +/// convenience methods for working with the zone. +/// +/// If needed after commiting one version of the zone being edited the writer +/// can be re-opened to write the next version of the zone. +struct ReopenableZoneWriter { + /// A write interface to a zone. write: Box, + + /// A write interface to the root node of a zone for a particular zone + /// version. writable: Option>, } -impl WriteState { +impl ReopenableZoneWriter { + /// Creates a writer for the given [`Zone`]. async fn new(zone: Zone) -> std::io::Result { let write = zone.write().await; let writable = Some(write.open(true).await?); Ok(Self { write, writable }) } - async fn remove_all(&mut self) -> std::io::Result<()> { - if let Some(writable) = &mut self.writable { - writable.remove_all().await?; - } - - Ok(()) - } - + /// Commits any pending changes to the [`Zone`] being written to. + /// + /// Returns the created diff, if any. async fn commit(&mut self) -> Result, Error> { // Commit the deletes and adds that just occurred if let Some(writable) = self.writable.take() { @@ -498,12 +499,53 @@ impl WriteState { } } + /// Replaces the current root node write interface with a new one. + /// + /// Call [`commit()`] before calling this method. async fn reopen(&mut self) -> Result<(), Error> { self.writable = Some(self.write.open(true).await?); Ok(()) } + + /// Convenience method to mark all nodes in the tree as removed. + /// + /// Current readers will not be affected until [`commit()`] is called. + async fn remove_all(&mut self) -> std::io::Result<()> { + if let Some(writable) = &mut self.writable { + writable.remove_all().await?; + } + + Ok(()) + } + + /// Get a write interface to a child node of the tree. + /// + /// Use this to modify child nodes in the tree. + async fn update_child( + &self, + label: &Label, + ) -> std::io::Result> { + self.root().update_child(label).await + } + + /// Replace the RRset at the root node with the given RRset. + async fn update_root_rrset( + &self, + rrset: SharedRrset, + ) -> std::io::Result<()> { + self.root().update_rrset(rrset).await + } + + /// Helper method to access the current root node zone writer. + #[allow(clippy::borrowed_box)] + fn root(&self) -> &Box { + // SAFETY: Writable is always Some so is safe to unwrap. + self.writable.as_ref().unwrap() + } } +//------------ Tests ---------------------------------------------------------- + #[cfg(test)] mod tests { use core::str::FromStr; From 856575334038c43d9c7b8c4b8b4f950ab3d83982 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 16:42:24 +0200 Subject: [PATCH 232/333] Remove an outdated comment. --- src/zonetree/in_memory/write.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index 4590759dd..87ea0ad62 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -103,8 +103,6 @@ impl WritableZone for WriteZone { let new_apex = WriteNode::new_apex(self.clone(), create_diff); if let Ok(write_node) = &new_apex { - // Note: the start and end serial of the diff will be filled in - // when commit() is invoked. *self.diff.lock().unwrap() = write_node.diff(); } From d615b4070cdc9279efc11f96158e811027ef90c6 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 16:59:09 +0200 Subject: [PATCH 233/333] More RustDocs. --- src/zonetree/types.rs | 20 ++++++++++++++++---- src/zonetree/update.rs | 5 +++-- src/zonetree/zone.rs | 9 +++++++-- 3 files changed, 26 insertions(+), 8 deletions(-) diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index 640290a05..d00f12426 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -254,6 +254,8 @@ pub struct ZoneCut { //------------ ZoneDiffBuilder ----------------------------------------------- /// A [`ZoneDiff`] builder. +/// +/// Removes are assumed to occur before adds. #[derive(Debug, Default)] pub struct ZoneDiffBuilder { /// The records added to the Zone. @@ -264,12 +266,12 @@ pub struct ZoneDiffBuilder { } impl ZoneDiffBuilder { - /// TODO + /// Creates a new instance of the builder. pub fn new() -> Self { Default::default() } - /// TODO + /// Record in the diff that a resource record was added. pub fn add( &mut self, owner: StoredName, @@ -279,7 +281,7 @@ impl ZoneDiffBuilder { self.added.insert((owner, rtype), rrset); } - /// TODO + /// Record in the diff that a resource record was removed. pub fn remove( &mut self, owner: StoredName, @@ -289,7 +291,15 @@ impl ZoneDiffBuilder { self.removed.insert((owner, rtype), rrset); } - /// TODO + /// Exchange this builder instnace for an immutable [`ZoneDiff`]. + /// + /// The start serial should be the zone version to which the diffs should + /// be applied. The end serial denotes the zone version that results from + /// applying this diff. + /// + /// Note: No check is currently done that the start and end serials match + /// the SOA records in the removed and added records contained within the + /// diff. pub fn build(self, start_serial: Serial, end_serial: Serial) -> ZoneDiff { ZoneDiff { start_serial, @@ -303,6 +313,8 @@ impl ZoneDiffBuilder { //------------ ZoneDiff ------------------------------------------------------ /// The differences between one serial and another for a Zone. +/// +/// Removes are assumed to occur before adds. #[derive(Clone, Debug)] pub struct ZoneDiff { /// The serial number of the Zone which was modified. diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 03e384523..bfcea4137 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -501,7 +501,7 @@ impl ReopenableZoneWriter { /// Replaces the current root node write interface with a new one. /// - /// Call [`commit()`] before calling this method. + /// Call [`commit()`][Self::commit] before calling this method. async fn reopen(&mut self) -> Result<(), Error> { self.writable = Some(self.write.open(true).await?); Ok(()) @@ -509,7 +509,8 @@ impl ReopenableZoneWriter { /// Convenience method to mark all nodes in the tree as removed. /// - /// Current readers will not be affected until [`commit()`] is called. + /// Current readers will not be affected until [`commit()`][Self::commit] + /// is called. async fn remove_all(&mut self) -> std::io::Result<()> { if let Some(writable) = &mut self.writable { writable.remove_all().await?; diff --git a/src/zonetree/zone.rs b/src/zonetree/zone.rs index dfa7a6013..6c4db9873 100644 --- a/src/zonetree/zone.rs +++ b/src/zonetree/zone.rs @@ -15,7 +15,11 @@ use super::{parsed, ReadableZone, ZoneStore}; //------------ ZoneKey ------------------------------------------------------- -/// TODO +/// A key that uniquely identifies a zone. +/// +/// A zone is identified by the owner name of the apex and its class. Every +/// record in a zone must be at or under the apex owner name and be of the +/// same class. pub type ZoneKey = (StoredName, Class); //------------ Zone ---------------------------------------------------------- @@ -40,7 +44,8 @@ impl Zone { } } - /// TODO + /// Exchange this [`Zone`] wrapper for the actual underlying backing store + /// implementation. pub fn into_inner(self) -> Arc { self.store } From 60b6a47414089b8f67e6555465f6fdac52f8349f Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 21:03:16 +0200 Subject: [PATCH 234/333] More RustDocs. --- src/zonetree/types.rs | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index d00f12426..f6bc8db00 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -254,7 +254,7 @@ pub struct ZoneCut { //------------ ZoneDiffBuilder ----------------------------------------------- /// A [`ZoneDiff`] builder. -/// +/// /// Removes are assumed to occur before adds. #[derive(Debug, Default)] pub struct ZoneDiffBuilder { @@ -292,11 +292,11 @@ impl ZoneDiffBuilder { } /// Exchange this builder instnace for an immutable [`ZoneDiff`]. - /// + /// /// The start serial should be the zone version to which the diffs should /// be applied. The end serial denotes the zone version that results from /// applying this diff. - /// + /// /// Note: No check is currently done that the start and end serials match /// the SOA records in the removed and added records contained within the /// diff. @@ -312,21 +312,21 @@ impl ZoneDiffBuilder { //------------ ZoneDiff ------------------------------------------------------ -/// The differences between one serial and another for a Zone. -/// +/// The differences between one serial and another for a DNS zone. +/// /// Removes are assumed to occur before adds. #[derive(Clone, Debug)] pub struct ZoneDiff { - /// The serial number of the Zone which was modified. + /// The serial number of the zone which was modified. pub start_serial: Serial, - /// The serial number of the Zone that resulted from the modifications. + /// The serial number of the Zzone that resulted from the modifications. pub end_serial: Serial, - /// The records added to the Zone. + /// The RRsets added to the zone. pub added: Arc>, - /// The records removed from the Zone. + /// The RRsets removed from the zone. pub removed: Arc>, } From d718eaf722221def0c3a1a121d3c45528bc02bfe Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 21:37:33 +0200 Subject: [PATCH 235/333] Organize imports. --- src/zonetree/in_memory/nodes.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/zonetree/in_memory/nodes.rs b/src/zonetree/in_memory/nodes.rs index b19986df4..7e7dacd2d 100644 --- a/src/zonetree/in_memory/nodes.rs +++ b/src/zonetree/in_memory/nodes.rs @@ -15,6 +15,7 @@ use crate::base::iana::{Class, Rtype}; use crate::base::name::{Label, OwnedLabel, ToName}; use crate::zonetree::error::{CnameError, OutOfZone, ZoneCutError}; use crate::zonetree::types::{StoredName, ZoneCut}; +use crate::zonetree::util::rel_name_rev_iter; use crate::zonetree::walk::WalkState; use crate::zonetree::{ ReadableZone, SharedRr, SharedRrset, WritableZone, ZoneStore, @@ -23,7 +24,6 @@ use crate::zonetree::{ use super::read::ReadZone; use super::versioned::{Version, Versioned}; use super::write::{WriteZone, ZoneVersions}; -use crate::zonetree::util::rel_name_rev_iter; //------------ ZoneApex ------------------------------------------------------ From acfea32db437377be72d343899730c63f606f62b Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 21:37:43 +0200 Subject: [PATCH 236/333] More RustDocs. --- src/zonetree/tree.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/zonetree/tree.rs b/src/zonetree/tree.rs index d02423870..12915783c 100644 --- a/src/zonetree/tree.rs +++ b/src/zonetree/tree.rs @@ -185,7 +185,7 @@ impl ZoneSetNode { //------------ ZoneSetIter --------------------------------------------------- -/// TODO +/// An iterator over the [`Zone`]s in a [`ZoneTree`]. pub struct ZoneSetIter<'a> { roots: hash_map::Values<'a, Class, ZoneSetNode>, nodes: NodesIter<'a>, From 303e5262d97af012e587a5c551cee40f2142b23e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 19 Sep 2024 21:54:57 +0200 Subject: [PATCH 237/333] More RustDocs. --- src/zonetree/in_memory/mod.rs | 3 +++ src/zonetree/in_memory/nodes.rs | 2 +- src/zonetree/in_memory/read.rs | 2 +- src/zonetree/in_memory/versioned.rs | 4 ++++ src/zonetree/in_memory/write.rs | 13 ++++++++++++- src/zonetree/zone.rs | 2 +- 6 files changed, 22 insertions(+), 4 deletions(-) diff --git a/src/zonetree/in_memory/mod.rs b/src/zonetree/in_memory/mod.rs index d7b50db5b..59c36b45b 100644 --- a/src/zonetree/in_memory/mod.rs +++ b/src/zonetree/in_memory/mod.rs @@ -1,3 +1,6 @@ +//! An in-memory backing store for [`Zone`]s. +//! +//! [`Zone`]: super::Zone mod builder; mod nodes; mod read; diff --git a/src/zonetree/in_memory/nodes.rs b/src/zonetree/in_memory/nodes.rs index 7e7dacd2d..f88b54868 100644 --- a/src/zonetree/in_memory/nodes.rs +++ b/src/zonetree/in_memory/nodes.rs @@ -1,4 +1,4 @@ -//! The nodes in a zone tree. +//! The resource record tree nodes of an in-memory zone. use std::boxed::Box; use std::collections::{hash_map, HashMap}; diff --git a/src/zonetree/in_memory/read.rs b/src/zonetree/in_memory/read.rs index de71e11d2..977e771af 100644 --- a/src/zonetree/in_memory/read.rs +++ b/src/zonetree/in_memory/read.rs @@ -1,4 +1,4 @@ -//! Quering for zone data. +//! Read access to in-memory zones. use core::iter; use std::sync::Arc; diff --git a/src/zonetree/in_memory/versioned.rs b/src/zonetree/in_memory/versioned.rs index 7d49329bd..bdc272d23 100644 --- a/src/zonetree/in_memory/versioned.rs +++ b/src/zonetree/in_memory/versioned.rs @@ -1,3 +1,4 @@ +//! Data types for storing in-memory zone data by zone version. use crate::base::serial::Serial; use serde::{Deserialize, Serialize}; use std::vec::Vec; @@ -31,6 +32,9 @@ impl Default for Version { //------------ Versioned ----------------------------------------------------- +/// A history preserving ordered map of data keyed by zone version. +/// +/// Updates and inserts preserve previous versions of the stored data. #[derive(Clone, Debug)] pub struct Versioned { data: Vec<(Version, Option)>, diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index 87ea0ad62..6c8692a5b 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -1,4 +1,4 @@ -//! Write access to zones. +//! Write access to in-memory zones. use core::future::ready; use std::boxed::Box; @@ -29,6 +29,7 @@ use crate::rdata::ZoneRecordData; //------------ WriteZone ----------------------------------------------------- +/// Serialized write operations on in-memory zones with auto-diffing support. pub struct WriteZone { apex: Arc, _lock: Option>, @@ -231,12 +232,20 @@ impl WritableZone for WriteZone { } } +/// Returns the inner value, if the Arc has exactly one strong reference. +/// +/// Wrapper around [`Arc::into_inner()`] with an implementation back-ported +/// for Rust <1.70.0 when [`Arc::into_inner()`] did not exist yet. #[rustversion::since(1.70.0)] fn arc_into_inner(this: Arc>) -> Option> { #[allow(clippy::incompatible_msrv)] Arc::into_inner(this) } +/// Returns the inner value, if the Arc has exactly one strong reference. +/// +/// Wrapper around [`Arc::into_inner()`] with an implementation back-ported +/// for Rust <1.70.0 when [`Arc::into_inner()`] did not exist yet. #[rustversion::before(1.70.0)] fn arc_into_inner(this: Arc>) -> Option> { // From: https://doc.rust-lang.org/std/sync/struct.Arc.html#method.into_inner @@ -258,6 +267,7 @@ fn arc_into_inner(this: Arc>) -> Option> { //------------ WriteNode ------------------------------------------------------ +/// Write operations on in-memory zone tree nodes with auto-diffing support. pub struct WriteNode { /// The writer for the zone we are working with. zone: WriteZone, @@ -634,6 +644,7 @@ impl fmt::Display for WriteApexError { //------------ ZoneVersions -------------------------------------------------- +/// An ordered collection of zone versions of which only one is "current". #[derive(Debug)] pub struct ZoneVersions { current: (Version, Arc), diff --git a/src/zonetree/zone.rs b/src/zonetree/zone.rs index 6c4db9873..66f4966f4 100644 --- a/src/zonetree/zone.rs +++ b/src/zonetree/zone.rs @@ -16,7 +16,7 @@ use super::{parsed, ReadableZone, ZoneStore}; //------------ ZoneKey ------------------------------------------------------- /// A key that uniquely identifies a zone. -/// +/// /// A zone is identified by the owner name of the apex and its class. Every /// record in a zone must be at or under the apex owner name and be of the /// same class. From 2de2cdce661663dbb575f6846d827cdccbe25ac9 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 20 Sep 2024 01:29:14 +0200 Subject: [PATCH 238/333] More cleanup and RustDocs. --- src/zonetree/in_memory/write.rs | 240 ++++++++++++++++++++------------ 1 file changed, 150 insertions(+), 90 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index 6c8692a5b..21ef33967 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -17,7 +17,7 @@ use tracing::trace; use crate::base::iana::Rtype; use crate::base::name::Label; -use crate::base::NameBuilder; +use crate::base::{NameBuilder, Serial}; use crate::zonetree::types::{ZoneCut, ZoneDiff, ZoneDiffBuilder}; use crate::zonetree::StoredName; use crate::zonetree::{Rrset, SharedRr}; @@ -59,6 +59,111 @@ impl WriteZone { fn last_published_version(&self) -> Version { self.published_versions.read().current().0 } + + fn bump_soa_serial(&mut self, old_soa_rr: &Option) { + let old_soa_rr = old_soa_rr.as_ref().unwrap(); + let ZoneRecordData::Soa(old_soa) = old_soa_rr.data() else { + unreachable!() + }; + trace!("Commit: old_soa={old_soa:#?}"); + + // Create a SOA record with a higher serial number than the previous + // SOA record. + let mut new_soa_rrset = Rrset::new(Rtype::SOA, old_soa_rr.ttl()); + let new_soa_serial = old_soa.serial().add(1); + let new_soa_data = crate::rdata::Soa::new( + old_soa.mname().clone(), + old_soa.rname().clone(), + new_soa_serial, + old_soa.refresh(), + old_soa.retry(), + old_soa.expire(), + old_soa.minimum(), + ); + new_soa_rrset.push_data(new_soa_data.into()); + + trace!("Commit: new_soa={new_soa_rrset:#?}"); + let new_soa_shared_rrset = SharedRrset::new(new_soa_rrset); + + // Update the SOA record in the new zone version. + self.apex + .rrsets() + .update(new_soa_shared_rrset.clone(), self.new_version); + } + + fn add_soa_remove_diff_entry( + &mut self, + old_soa_rr: Option, + diff: &mut ZoneDiffBuilder, + ) -> Option { + if let Some(old_soa_rr) = old_soa_rr { + let ZoneRecordData::Soa(old_soa) = old_soa_rr.data() else { + unreachable!() + }; + + let mut removed_soa_rrset = + Rrset::new(Rtype::SOA, old_soa_rr.ttl()); + removed_soa_rrset.push_data(old_soa_rr.data().clone()); + let removed_soa_rrset = SharedRrset::new(removed_soa_rrset); + + trace!( + "Diff: recording removal of old SOA: {removed_soa_rrset:#?}" + ); + diff.remove( + self.apex.name().clone(), + Rtype::SOA, + removed_soa_rrset, + ); + + Some(old_soa.serial()) + } else { + None + } + } + + fn add_soa_add_diff_entry( + &mut self, + new_soa_rr: Option, + diff: &mut ZoneDiffBuilder, + ) -> Option { + if let Some(new_soa_rr) = new_soa_rr { + let ZoneRecordData::Soa(new_soa) = new_soa_rr.data() else { + unreachable!() + }; + let mut new_soa_shared_rrset = + Rrset::new(Rtype::SOA, new_soa_rr.ttl()); + new_soa_shared_rrset.push_data(new_soa_rr.data().clone()); + let new_soa_rrset = SharedRrset::new(new_soa_shared_rrset); + + trace!("Diff: recording addition of new SOA: {new_soa_rrset:#?}"); + diff.add(self.apex.name().clone(), Rtype::SOA, new_soa_rrset); + + Some(new_soa.serial()) + } else { + None + } + } + + fn publish_new_zone_version(&mut self) { + trace!( + "Commit: Making zone version '{:#?}' current", + self.new_version + ); + let marker = self + .published_versions + .write() + .update_current(self.new_version); + self.published_versions + .write() + .push_version(self.new_version, marker); + + trace!("Commit: zone versions: {:#?}", self.published_versions); + trace!("Commit: zone dump:\n{:#?}", self.apex); + + // Start the next version. + self.new_version = self.new_version.next(); + self.dirty = false; + } } //--- impl Clone @@ -119,6 +224,15 @@ impl WritableZone for WriteZone { Box::pin(ready(res)) } + /// Publish in-progress zone edits. + /// + /// If `bump_soa_serial` is true AND the zone has an existing SOA record + /// AND the to-be-published zone version does NOT have a new SOA record, + /// then a copy of the old SOA record with its serial number increased + /// will be saved. + /// + /// If a diff has been captured, also ensure that it contains diff entries + /// for removing the old SOA and adding the new SOA. fn commit( &mut self, bump_soa_serial: bool, @@ -131,102 +245,48 @@ impl WritableZone for WriteZone { > { let mut out_diff = None; - // An empty zone that is being filled by AXFR won't have an existing SOA. - if let Some(old_soa_rr) = - self.apex.get_soa(self.last_published_version()) - { - let ZoneRecordData::Soa(old_soa) = old_soa_rr.data() else { - unreachable!() - }; - trace!("Commit: old_soa={old_soa:#?}"); - - if bump_soa_serial { - // Ensure that the SOA record in the zone is updated. - let mut new_soa_rrset = - Rrset::new(Rtype::SOA, old_soa_rr.ttl()); - let new_soa_serial = old_soa.serial().add(1); - let new_soa_data = crate::rdata::Soa::new( - old_soa.mname().clone(), - old_soa.rname().clone(), - new_soa_serial, - old_soa.refresh(), - old_soa.retry(), - old_soa.expire(), - old_soa.minimum(), - ); - new_soa_rrset.push_data(new_soa_data.into()); - trace!("Commit: new_soa={new_soa_rrset:#?}"); - let new_soa_shared_rrset = SharedRrset::new(new_soa_rrset); + // If bump_soa_serial is true AND if the zone already had a SOA record + // AND no SOA record exists in the new version of the zone: add a SOA + // record with a higher serial than the previous SOA record. + // + // For an empty zone being populated by AXFR this won't be possible as + // there won't be an existing SOA to increment, but there should in + // that case be a SOA record in the new version of the zone anyway. + + let old_soa_rr = self.apex.get_soa(self.last_published_version()); + let new_soa_rr = self.apex.get_soa(self.new_version); + + let mut soa_serial_bumped = false; + if bump_soa_serial && old_soa_rr.is_some() && new_soa_rr.is_none() { + self.bump_soa_serial(&old_soa_rr); + soa_serial_bumped = true; + } - self.apex - .rrsets() - .update(new_soa_shared_rrset.clone(), self.new_version); - } + // Extract (and finish) the created diff, if any. + let diff = self.diff.lock().unwrap().take(); - // Extract the created diff, if any. - if let Some(diff) = self.diff.lock().unwrap().take() { - let new_soa_rr = self.apex.get_soa(self.new_version).unwrap(); - let ZoneRecordData::Soa(new_soa) = new_soa_rr.data() else { - unreachable!() - }; - - let diff = arc_into_inner(diff).unwrap(); - let mut diff = Mutex::into_inner(diff).unwrap(); - - if bump_soa_serial { - let mut removed_soa_rrset = - Rrset::new(Rtype::SOA, old_soa_rr.ttl()); - removed_soa_rrset.push_data(old_soa_rr.data().clone()); - let removed_soa_rrset = - SharedRrset::new(removed_soa_rrset); - - let mut new_soa_shared_rrset = - Rrset::new(Rtype::SOA, new_soa_rr.ttl()); - new_soa_shared_rrset.push_data(new_soa_rr.data().clone()); - let new_soa_rrset = - SharedRrset::new(new_soa_shared_rrset); - - trace!("Diff: recording removal of old SOA: {removed_soa_rrset:#?}"); - diff.remove( - self.apex.name().clone(), - Rtype::SOA, - removed_soa_rrset, - ); + if diff.is_some() && new_soa_rr.is_some() { + let diff = diff.unwrap(); + let diff = arc_into_inner(diff).unwrap(); + let mut diff = Mutex::into_inner(diff).unwrap(); - trace!( - "Diff: recording addition of new SOA: {new_soa_rrset:#?}" - ); - diff.add( - self.apex.name().clone(), - Rtype::SOA, - new_soa_rrset, - ); - } + // Generate a diff entry for the update of the SOA record + if soa_serial_bumped { + let old_serial = + self.add_soa_remove_diff_entry(old_soa_rr, &mut diff); + + let new_serial = + self.add_soa_add_diff_entry(new_soa_rr, &mut diff); - out_diff = - Some(diff.build(old_soa.serial(), new_soa.serial())); + if old_serial.is_some() && new_serial.is_some() { + let old_serial = old_serial.unwrap(); + let new_serial = new_serial.unwrap(); + out_diff = Some(diff.build(old_serial, new_serial)); + } } } - // Make the new version visible. - trace!( - "Commit: Making zone version '{:#?}' current", - self.new_version - ); - let marker = self - .published_versions - .write() - .update_current(self.new_version); - self.published_versions - .write() - .push_version(self.new_version, marker); - - trace!("Commit: zone versions: {:#?}", self.published_versions); - trace!("Commit: zone dump:\n{:#?}", self.apex); - - // Start the next version. - self.new_version = self.new_version.next(); - self.dirty = false; + self.publish_new_zone_version(); Box::pin(ready(Ok(out_diff))) } From 1d7ea638d9b701830229ee20ece5d6c993819151 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 20 Sep 2024 10:16:53 +0200 Subject: [PATCH 239/333] Remove added code that is not used by this branch. --- src/zonetree/mod.rs | 2 +- src/zonetree/zone.rs | 17 ----------------- 2 files changed, 1 insertion(+), 18 deletions(-) diff --git a/src/zonetree/mod.rs b/src/zonetree/mod.rs index 130b8b503..035e28742 100644 --- a/src/zonetree/mod.rs +++ b/src/zonetree/mod.rs @@ -122,7 +122,7 @@ pub use self::types::{ Rrset, SharedRr, SharedRrset, StoredName, StoredRecord, ZoneDiff, }; pub use self::walk::WalkOp; -pub use self::zone::{Zone, ZoneKey}; +pub use self::zone::Zone; /// Zone related utilities. pub mod util { diff --git a/src/zonetree/zone.rs b/src/zonetree/zone.rs index 66f4966f4..7663db032 100644 --- a/src/zonetree/zone.rs +++ b/src/zonetree/zone.rs @@ -13,15 +13,6 @@ use super::traits::WritableZone; use super::types::StoredName; use super::{parsed, ReadableZone, ZoneStore}; -//------------ ZoneKey ------------------------------------------------------- - -/// A key that uniquely identifies a zone. -/// -/// A zone is identified by the owner name of the apex and its class. Every -/// record in a zone must be at or under the apex owner name and be of the -/// same class. -pub type ZoneKey = (StoredName, Class); - //------------ Zone ---------------------------------------------------------- /// A single DNS zone. @@ -74,14 +65,6 @@ impl Zone { { self.store.clone().write() } - - /// Gets a key that uniquely identifies this zone. - /// - /// Note: Assumes that there is only ever one instance of a zone with a - /// given apex name and class in a set of zones. - pub fn key(&self) -> ZoneKey { - (self.apex_name().clone(), self.class()) - } } //--- TryFrom From 1ecc80df9eb52bd5a028aae3a36fb47a821ee9b0 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 20 Sep 2024 10:18:10 +0200 Subject: [PATCH 240/333] Remove added code that is not used by this branch. --- src/zonetree/zone.rs | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/src/zonetree/zone.rs b/src/zonetree/zone.rs index 7663db032..ca3058a74 100644 --- a/src/zonetree/zone.rs +++ b/src/zonetree/zone.rs @@ -21,12 +21,6 @@ pub struct Zone { store: Arc, } -impl AsRef for Zone { - fn as_ref(&self) -> &dyn ZoneStore { - self.store.as_ref() - } -} - impl Zone { /// Creates a new [`Zone`] instance with the given data. pub fn new(data: impl ZoneStore + 'static) -> Self { @@ -35,14 +29,6 @@ impl Zone { } } - /// Exchange this [`Zone`] wrapper for the actual underlying backing store - /// implementation. - pub fn into_inner(self) -> Arc { - self.store - } -} - -impl Zone { /// Gets the CLASS of this zone. pub fn class(&self) -> Class { self.store.class() From f55362b59a9fa3c6fa5c49ab762e5440ffe8d301 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 20 Sep 2024 11:28:20 +0200 Subject: [PATCH 241/333] More RustDocs. Remove the unused dirty flag. --- src/zonetree/in_memory/write.rs | 50 +++++++++++++++++++++++++++------ 1 file changed, 41 insertions(+), 9 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index 21ef33967..d427556f1 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -31,11 +31,49 @@ use crate::rdata::ZoneRecordData; /// Serialized write operations on in-memory zones with auto-diffing support. pub struct WriteZone { + /// The zone to edit. apex: Arc, + + /// A write lock on the zone. + /// + /// This lock is granted by [`ZoneApex::write()`] and held by us until we + /// are finished. Further calls to [`ZoneApex::write()`] will block until + /// we are dropped and release the lock. + /// + /// [ZoneApex::write()]: ZoneApex::write() _lock: Option>, + + /// The version number of the new zone version to create. + /// + /// This is set initially in [`new()`] and is incremented by [`commit()`] + /// after the new zone version has been published. + /// + /// Note: There is currently no mechanism for controlling the version + /// number of the next zone version to be published. However, this version + /// number is for internal use and is not (yet?) constrained to match the + /// SOA serial in the zone when the zone is published. Users can therefore + /// use whatever serial incrementing policy they desire as they control + /// the content of the SOA record in the zone. new_version: Version, - dirty: bool, + + /// The set of versions already published in this zone prior to starting + /// the write operation. published_versions: Arc>, + + /// The set of differences accumulated as changes are made to the zone. + /// + /// The outermost Arc>> is needed so that [`open()`] can + /// store a [`ZoneDiffBuilder`] created by [`WriteNode`] and because + /// [`open()`] takes &self it cannot mutate itself and store it that way. + /// It also can't just store a reference to [`ZoneDiffBuilder`] as it + /// needs to call [`ZoneDiffBuilder::build()`] in [`commit()`] which + /// requires that the builder be consumed (and thus owned, ). It is stored + /// as an Option because storing a diff is costly thus optional. + /// + /// The innermost Arc> is needed because each time + /// [`WriteNode::update_child()`] is called it creates a new [`WriteNode`] + /// which also needs to be able to add and remove things from the same + /// diff collection. diff: Arc>>>>, } @@ -50,9 +88,8 @@ impl WriteZone { apex, _lock: Some(_lock), new_version, - dirty: false, published_versions, - diff: Arc::new(Mutex::new(None)), + diff: Default::default(), } } @@ -162,7 +199,6 @@ impl WriteZone { // Start the next version. self.new_version = self.new_version.next(); - self.dirty = false; } } @@ -174,7 +210,6 @@ impl Clone for WriteZone { apex: self.apex.clone(), _lock: None, new_version: self.new_version, - dirty: self.dirty, published_versions: self.published_versions.clone(), diff: self.diff.clone(), } @@ -185,10 +220,7 @@ impl Clone for WriteZone { impl Drop for WriteZone { fn drop(&mut self) { - if self.dirty { - self.apex.rollback(self.new_version); - self.dirty = false; - } + self.apex.rollback(self.new_version); } } From c43fe0c104ea0be3b3866fb1ad78084f5a06ba46 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 20 Sep 2024 11:32:31 +0200 Subject: [PATCH 242/333] Cargo fmt. --- src/zonetree/in_memory/write.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index d427556f1..47cbed3cd 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -35,19 +35,19 @@ pub struct WriteZone { apex: Arc, /// A write lock on the zone. - /// + /// /// This lock is granted by [`ZoneApex::write()`] and held by us until we /// are finished. Further calls to [`ZoneApex::write()`] will block until /// we are dropped and release the lock. - /// + /// /// [ZoneApex::write()]: ZoneApex::write() _lock: Option>, /// The version number of the new zone version to create. - /// + /// /// This is set initially in [`new()`] and is incremented by [`commit()`] /// after the new zone version has been published. - /// + /// /// Note: There is currently no mechanism for controlling the version /// number of the next zone version to be published. However, this version /// number is for internal use and is not (yet?) constrained to match the @@ -69,7 +69,7 @@ pub struct WriteZone { /// needs to call [`ZoneDiffBuilder::build()`] in [`commit()`] which /// requires that the builder be consumed (and thus owned, ). It is stored /// as an Option because storing a diff is costly thus optional. - /// + /// /// The innermost Arc> is needed because each time /// [`WriteNode::update_child()`] is called it creates a new [`WriteNode`] /// which also needs to be able to add and remove things from the same From 0f81e33774a073bca35936edbb8826ce29d55b9c Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 20 Sep 2024 12:14:59 +0200 Subject: [PATCH 243/333] Remove commented out code. --- src/zonetree/in_memory/write.rs | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index 47cbed3cd..a5f8b4db6 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -460,11 +460,7 @@ impl WriteNode { } } - // if rrset.is_empty() { - // rrsets.remove(rrset.rtype(), self.zone.version.prev()); - // } else { rrsets.update(rrset, self.zone.new_version); - // } self.check_nx_domain()?; Ok(()) } From 4aa68f6633850fb982f7bb4f279f5a904f052069 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 20 Sep 2024 17:28:23 +0200 Subject: [PATCH 244/333] FIX: Don't always rollback changes. Instead add the dirty flag back in, but this time make sure it is set and unset correctly. --- src/zonetree/in_memory/write.rs | 22 ++++++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index a5f8b4db6..dab35db5f 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -1,6 +1,9 @@ //! Write access to in-memory zones. use core::future::ready; +use core::sync::atomic::AtomicBool; +use core::sync::atomic::Ordering; + use std::boxed::Box; use std::future::Future; use std::pin::Pin; @@ -18,6 +21,7 @@ use tracing::trace; use crate::base::iana::Rtype; use crate::base::name::Label; use crate::base::{NameBuilder, Serial}; +use crate::rdata::ZoneRecordData; use crate::zonetree::types::{ZoneCut, ZoneDiff, ZoneDiffBuilder}; use crate::zonetree::StoredName; use crate::zonetree::{Rrset, SharedRr}; @@ -25,7 +29,6 @@ use crate::zonetree::{SharedRrset, WritableZone, WritableZoneNode}; use super::nodes::{Special, ZoneApex, ZoneNode}; use super::versioned::{Version, VersionMarker}; -use crate::rdata::ZoneRecordData; //------------ WriteZone ----------------------------------------------------- @@ -75,6 +78,13 @@ pub struct WriteZone { /// which also needs to be able to add and remove things from the same /// diff collection. diff: Arc>>>>, + + /// The zone is dirty if changes have been made but not yet committed. + /// + /// This flag is set when a zone is opened for editing, and cleared when + /// it is committed. If not cleared, on drop any changes made will be + /// rolled back. + dirty: Arc, } impl WriteZone { @@ -90,6 +100,7 @@ impl WriteZone { new_version, published_versions, diff: Default::default(), + dirty: Default::default(), } } @@ -199,6 +210,8 @@ impl WriteZone { // Start the next version. self.new_version = self.new_version.next(); + + self.dirty.store(false, Ordering::SeqCst); } } @@ -212,6 +225,7 @@ impl Clone for WriteZone { new_version: self.new_version, published_versions: self.published_versions.clone(), diff: self.diff.clone(), + dirty: Default::default(), } } } @@ -220,7 +234,9 @@ impl Clone for WriteZone { impl Drop for WriteZone { fn drop(&mut self) { - self.apex.rollback(self.new_version); + if self.dirty.swap(false, Ordering::SeqCst) { + self.apex.rollback(self.new_version); + } } } @@ -253,6 +269,8 @@ impl WritableZone for WriteZone { ) }); + self.dirty.store(true, Ordering::SeqCst); + Box::pin(ready(res)) } From be95515328fb239a2fafc29cc7945b6687e0eba9 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 20 Sep 2024 17:29:16 +0200 Subject: [PATCH 245/333] FIX: Store diffs even if the SOA serial was not bumped on commit. --- src/zonetree/in_memory/write.rs | 20 ++++++++------------ 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index dab35db5f..a09e06f31 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -306,10 +306,8 @@ impl WritableZone for WriteZone { let old_soa_rr = self.apex.get_soa(self.last_published_version()); let new_soa_rr = self.apex.get_soa(self.new_version); - let mut soa_serial_bumped = false; if bump_soa_serial && old_soa_rr.is_some() && new_soa_rr.is_none() { self.bump_soa_serial(&old_soa_rr); - soa_serial_bumped = true; } // Extract (and finish) the created diff, if any. @@ -321,18 +319,16 @@ impl WritableZone for WriteZone { let mut diff = Mutex::into_inner(diff).unwrap(); // Generate a diff entry for the update of the SOA record - if soa_serial_bumped { - let old_serial = - self.add_soa_remove_diff_entry(old_soa_rr, &mut diff); + let old_serial = + self.add_soa_remove_diff_entry(old_soa_rr, &mut diff); - let new_serial = - self.add_soa_add_diff_entry(new_soa_rr, &mut diff); + let new_serial = + self.add_soa_add_diff_entry(new_soa_rr, &mut diff); - if old_serial.is_some() && new_serial.is_some() { - let old_serial = old_serial.unwrap(); - let new_serial = new_serial.unwrap(); - out_diff = Some(diff.build(old_serial, new_serial)); - } + if old_serial.is_some() && new_serial.is_some() { + let old_serial = old_serial.unwrap(); + let new_serial = new_serial.unwrap(); + out_diff = Some(diff.build(old_serial, new_serial)); } } From 25f0e1e411909931f605f1720a0cefcb87115bb1 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 20 Sep 2024 22:31:19 +0200 Subject: [PATCH 246/333] Extend unit tet to verify created zone content is as expected. Catches the "always drop" bug that was just fixed. --- src/zonetree/update.rs | 44 +++++++++++++++++++++++++++++++++++------- 1 file changed, 37 insertions(+), 7 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index bfcea4137..15f131eaf 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -550,6 +550,7 @@ impl ReopenableZoneWriter { #[cfg(test)] mod tests { use core::str::FromStr; + use std::vec::Vec; use bytes::BytesMut; use octseq::Octets; @@ -568,29 +569,58 @@ mod tests { use super::*; #[tokio::test] - async fn simple_test() { + async fn write_soa_read_soa() { init_logging(); let zone = mk_empty_zone("example.com"); let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); + let qname = Name::from_str("example.com").unwrap(); + let s = Serial::now(); let soa = mk_soa(s); - let soa = ZoneRecordData::Soa(soa); - let soa = Record::new( - ParsedName::from(Name::from_str("example.com").unwrap()), + let soa_data = ZoneRecordData::Soa(soa.clone()); + let soa_rec = Record::new( + ParsedName::from(qname.clone()), Class::IN, Ttl::from_secs(0), - soa, + soa_data, ); updater - .apply(ZoneUpdate::AddRecord(soa.clone())) + .apply(ZoneUpdate::AddRecord(soa_rec.clone())) .await .unwrap(); - updater.apply(ZoneUpdate::Finished(soa)).await.unwrap(); + updater + .apply(ZoneUpdate::Finished(soa_rec.clone())) + .await + .unwrap(); + + let query = MessageBuilder::new_vec(); + let mut query = query.question(); + query.push((qname.clone(), Rtype::SOA)).unwrap(); + let message: Message> = query.into(); + + let builder = MessageBuilder::new_bytes(); + let answer: Message = zone + .read() + .query(qname, Rtype::SOA) + .unwrap() + .to_message(&message, builder) + .into(); + + let found_soa_rec = answer + .answer() + .unwrap() + .limit_to::>() + .next() + .unwrap() + .unwrap() + .into_data(); + + assert_eq!(found_soa_rec, soa); } #[tokio::test] From c546c2a3fac5adb1e2e5b94c66cc8194a7abe302 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 12:52:04 +0200 Subject: [PATCH 247/333] Don't leave behind an empty RRset node for a version if the update removes all of its constituent RRs. --- src/zonetree/in_memory/nodes.rs | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/src/zonetree/in_memory/nodes.rs b/src/zonetree/in_memory/nodes.rs index f88b54868..864682fbb 100644 --- a/src/zonetree/in_memory/nodes.rs +++ b/src/zonetree/in_memory/nodes.rs @@ -254,11 +254,15 @@ impl NodeRrsets { /// Updates an RRset. pub fn update(&self, rrset: SharedRrset, version: Version) { - self.rrsets - .write() - .entry(rrset.rtype()) - .or_default() - .update(rrset, version) + if rrset.is_empty() { + self.remove_rtype(rrset.rtype(), version); + } else { + self.rrsets + .write() + .entry(rrset.rtype()) + .or_default() + .update(rrset, version); + } } /// Removes the RRset for the given type. @@ -267,7 +271,7 @@ impl NodeRrsets { .write() .entry(rtype) .or_default() - .remove(version) + .remove(version); } pub fn rollback(&self, version: Version) { From 2e9abacf523e29034b2aa26d4144d99f10644ac2 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 12:53:06 +0200 Subject: [PATCH 248/333] Don't skip deleted node versions when get()'ing the best match for a node by version. --- src/zonetree/in_memory/versioned.rs | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/zonetree/in_memory/versioned.rs b/src/zonetree/in_memory/versioned.rs index bdc272d23..6a579d9a1 100644 --- a/src/zonetree/in_memory/versioned.rs +++ b/src/zonetree/in_memory/versioned.rs @@ -46,13 +46,17 @@ impl Versioned { } pub fn get(&self, version: Version) -> Option<&T> { - self.data.iter().rev().find_map(|item| { + let res = self.data.iter().rev().find_map(|item| { if item.0 <= version { - item.1.as_ref() + // Allow returning of empty values. + Some(item.1.as_ref()) } else { None } - }) + }); + + // Flatten Some(None) to None for empty values. + res.flatten() } pub fn update(&mut self, version: Version, value: T) { From a94f0985ddac26de4d4556144b54fc25cc31c3ca Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 12:53:44 +0200 Subject: [PATCH 249/333] Only mark the zone as dirty if WriteNode creation actually succeeded. --- src/zonetree/in_memory/write.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index a09e06f31..de4432690 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -258,6 +258,7 @@ impl WritableZone for WriteZone { if let Ok(write_node) = &new_apex { *self.diff.lock().unwrap() = write_node.diff(); + self.dirty.store(true, Ordering::SeqCst); } let res = new_apex @@ -269,8 +270,6 @@ impl WritableZone for WriteZone { ) }); - self.dirty.store(true, Ordering::SeqCst); - Box::pin(ready(res)) } From 81afa23f7624afa604ab47d9f2488eaff4d41c32 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 12:55:49 +0200 Subject: [PATCH 250/333] Determine ZoneDiff start and end serial from the given RRsets, so they can't be a mismatch with each other. --- src/zonetree/in_memory/write.rs | 10 +++-- src/zonetree/types.rs | 78 ++++++++++++++++++++++++++++++--- 2 files changed, 78 insertions(+), 10 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index de4432690..096258adc 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -325,9 +325,13 @@ impl WritableZone for WriteZone { self.add_soa_add_diff_entry(new_soa_rr, &mut diff); if old_serial.is_some() && new_serial.is_some() { - let old_serial = old_serial.unwrap(); - let new_serial = new_serial.unwrap(); - out_diff = Some(diff.build(old_serial, new_serial)); + let Ok(zone_diff) = diff.build() else { + return Box::pin(ready(Err(std::io::Error::new( + ErrorKind::Other, + "Diff lacks SOA records", + )))); + }; + out_diff = Some(zone_diff); } } diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index f6bc8db00..46b7e0b13 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -300,13 +300,8 @@ impl ZoneDiffBuilder { /// Note: No check is currently done that the start and end serials match /// the SOA records in the removed and added records contained within the /// diff. - pub fn build(self, start_serial: Serial, end_serial: Serial) -> ZoneDiff { - ZoneDiff { - start_serial, - end_serial, - added: Arc::new(self.added), - removed: Arc::new(self.removed), - } + pub fn build(self) -> Result { + ZoneDiff::new(self.added, self.removed) } } @@ -330,6 +325,75 @@ pub struct ZoneDiff { pub removed: Arc>, } +impl ZoneDiff { + /// Creates a new immutable zone diff. + /// + /// Returns `Err(ZoneDiffError::MissingStartSoa)` If the removed records + /// do not include a zone SOA. + /// + /// Returns `Err(ZoneDiffError::MissingEndSoa)` If the added records do + /// not include a zone SOA. + /// + /// Returns Ok otherwise. + fn new( + added: HashMap<(Name, Rtype), SharedRrset>, + removed: HashMap<(Name, Rtype), SharedRrset>, + ) -> Result { + // Determine the old and new SOA serials by looking at the added and + // removed records. + let start_serial = removed + .iter() + .find_map(|((_, rtype), rrset)| { + if *rtype == Rtype::SOA { + if let Some(ZoneRecordData::Soa(soa)) = + rrset.data().first() + { + return Some(soa.serial()); + } + } + None + }) + .ok_or(ZoneDiffError::MissingStartSoa)?; + + let end_serial = added + .iter() + .find_map(|((_, rtype), rrset)| { + if *rtype == Rtype::SOA { + if let Some(ZoneRecordData::Soa(soa)) = + rrset.data().first() + { + return Some(soa.serial()); + } + } + None + }) + .ok_or(ZoneDiffError::MissingEndSoa)?; + + Ok(Self { + start_serial, + end_serial, + added: added.into(), + removed: removed.into(), + }) + } +} + +//------------ ZoneDiffError -------------------------------------------------- + +/// Creating a [`ZoneDiff`] failed for some reason. +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +pub enum ZoneDiffError { + /// Missing start SOA. + /// + /// A zone diff requires a starting SOA. + MissingStartSoa, + + /// Missing end SOA. + /// + /// A zone diff requires a starting SOA. + MissingEndSoa, +} + //------------ ZoneUpdate ----------------------------------------------------- /// An update to be applied to a [`Zone`]. From 6335e9568b99205ba9d0ee552d07ed1347d26872 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 12:56:13 +0200 Subject: [PATCH 251/333] Determine ZoneDiff start and end serial from the given RRsets, so they can't be a mismatch with each other. --- src/zonetree/in_memory/write.rs | 11 +++-- src/zonetree/types.rs | 78 ++++++++++++++++++++++++++++++--- 2 files changed, 79 insertions(+), 10 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index de4432690..48611ab4f 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -6,6 +6,7 @@ use core::sync::atomic::Ordering; use std::boxed::Box; use std::future::Future; +use std::io::ErrorKind; use std::pin::Pin; use std::sync::Arc; use std::sync::Mutex; @@ -325,9 +326,13 @@ impl WritableZone for WriteZone { self.add_soa_add_diff_entry(new_soa_rr, &mut diff); if old_serial.is_some() && new_serial.is_some() { - let old_serial = old_serial.unwrap(); - let new_serial = new_serial.unwrap(); - out_diff = Some(diff.build(old_serial, new_serial)); + let Ok(zone_diff) = diff.build() else { + return Box::pin(ready(Err(std::io::Error::new( + ErrorKind::Other, + "Diff lacks SOA records", + )))); + }; + out_diff = Some(zone_diff); } } diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index f6bc8db00..46b7e0b13 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -300,13 +300,8 @@ impl ZoneDiffBuilder { /// Note: No check is currently done that the start and end serials match /// the SOA records in the removed and added records contained within the /// diff. - pub fn build(self, start_serial: Serial, end_serial: Serial) -> ZoneDiff { - ZoneDiff { - start_serial, - end_serial, - added: Arc::new(self.added), - removed: Arc::new(self.removed), - } + pub fn build(self) -> Result { + ZoneDiff::new(self.added, self.removed) } } @@ -330,6 +325,75 @@ pub struct ZoneDiff { pub removed: Arc>, } +impl ZoneDiff { + /// Creates a new immutable zone diff. + /// + /// Returns `Err(ZoneDiffError::MissingStartSoa)` If the removed records + /// do not include a zone SOA. + /// + /// Returns `Err(ZoneDiffError::MissingEndSoa)` If the added records do + /// not include a zone SOA. + /// + /// Returns Ok otherwise. + fn new( + added: HashMap<(Name, Rtype), SharedRrset>, + removed: HashMap<(Name, Rtype), SharedRrset>, + ) -> Result { + // Determine the old and new SOA serials by looking at the added and + // removed records. + let start_serial = removed + .iter() + .find_map(|((_, rtype), rrset)| { + if *rtype == Rtype::SOA { + if let Some(ZoneRecordData::Soa(soa)) = + rrset.data().first() + { + return Some(soa.serial()); + } + } + None + }) + .ok_or(ZoneDiffError::MissingStartSoa)?; + + let end_serial = added + .iter() + .find_map(|((_, rtype), rrset)| { + if *rtype == Rtype::SOA { + if let Some(ZoneRecordData::Soa(soa)) = + rrset.data().first() + { + return Some(soa.serial()); + } + } + None + }) + .ok_or(ZoneDiffError::MissingEndSoa)?; + + Ok(Self { + start_serial, + end_serial, + added: added.into(), + removed: removed.into(), + }) + } +} + +//------------ ZoneDiffError -------------------------------------------------- + +/// Creating a [`ZoneDiff`] failed for some reason. +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +pub enum ZoneDiffError { + /// Missing start SOA. + /// + /// A zone diff requires a starting SOA. + MissingStartSoa, + + /// Missing end SOA. + /// + /// A zone diff requires a starting SOA. + MissingEndSoa, +} + //------------ ZoneUpdate ----------------------------------------------------- /// An update to be applied to a [`Zone`]. From 8f74fc9f5c9a9fe3f130b5803dc1fe3a26cd8b74 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 12:56:52 +0200 Subject: [PATCH 252/333] Only add the removed and added RRs to a diff, not the entire corresponding RRsets. --- src/zonetree/in_memory/write.rs | 96 ++++++++++++++++++++++++++------- 1 file changed, 77 insertions(+), 19 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index 48611ab4f..1171e3ca1 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -441,7 +441,7 @@ impl WriteNode { Ok(node) } - fn update_rrset(&self, rrset: SharedRrset) -> Result<(), io::Error> { + fn update_rrset(&self, new_rrset: SharedRrset) -> Result<(), io::Error> { let rrsets = match self.node { Either::Right(ref apex) => apex.rrsets(), Either::Left(ref node) => node.rrsets(), @@ -449,36 +449,94 @@ impl WriteNode { trace!("Updating RRset"); if let Some((owner, diff)) = &self.diff { - let changed = if let Some(removed_rrset) = - rrsets.get(rrset.rtype(), self.zone.last_published_version()) + let current_rrset = if let Some(current_rrset) = rrsets + .get(new_rrset.rtype(), self.zone.last_published_version()) { - let changed = rrset != removed_rrset; + let changed = new_rrset != current_rrset; - if changed && !removed_rrset.is_empty() { - trace!("Diff detected: update of existing RRSET - recording removal of the current RRSET: {removed_rrset:#?}"); - diff.lock().unwrap().remove( - owner.clone(), - rrset.rtype(), - removed_rrset.clone(), - ); + if changed && !current_rrset.is_empty() { + Some(current_rrset) + } else { + None } - - changed } else { - true + None }; - if changed && !rrset.is_empty() { - trace!("Diff detected: update of existing RRSET - recording addition of the new RRSET: {rrset:#?}"); + match (current_rrset.is_some(), !new_rrset.is_empty()) { + (true, true) => { + trace!("Diff detected: update of existing RRSET - recording change of RRSET from {current_rrset:?} to {new_rrset:#?}"); + + // Check each resource record in the RRset being updated + // to see if it is missing from the new RRSet. + let new_rrs = new_rrset.as_rrset().data(); + let mut removed_rrs = + Rrset::new(new_rrset.rtype(), new_rrset.ttl()); + for removed_rr in current_rrset + .as_ref() + .unwrap() + .as_rrset() + .data() + .iter() + .filter(|rr| !new_rrs.contains(rr)) + { + removed_rrs.push_data(removed_rr.clone()); + } + + if !removed_rrs.is_empty() { + diff.lock().unwrap().remove( + owner.clone(), + new_rrset.rtype(), + SharedRrset::new(removed_rrs), + ); + } + + // Check each resource record in the new RRset to see if + // it is missing from the RRset being updated. + let old_rrs = + current_rrset.as_ref().unwrap().as_rrset().data(); + let mut added_rrs = + Rrset::new(new_rrset.rtype(), new_rrset.ttl()); + for added_rr in new_rrset + .as_rrset() + .data() + .iter() + .filter(|rr| !old_rrs.contains(rr)) + { + added_rrs.push_data(added_rr.clone()); + } + + if !added_rrs.is_empty() { + diff.lock().unwrap().add( + owner.clone(), + new_rrset.rtype(), + SharedRrset::new(added_rrs), + ); + } + } + (true, false) => { + trace!("Diff detected: update of existing RRSET - recording removal of the current RRSET {current_rrset:#?}"); + diff.lock().unwrap().remove( + owner.clone(), + new_rrset.rtype(), + current_rrset.unwrap().clone(), + ); + } + (false, true) => { + trace!("Diff detected: update of existing RRSET - recording addition of new RRSET {new_rrset:#?}"); diff.lock().unwrap().add( owner.clone(), - rrset.rtype(), - rrset.clone(), + new_rrset.rtype(), + new_rrset.clone(), ); + } + (false, false) => { + // NOOP + } } } - rrsets.update(rrset, self.zone.new_version); + rrsets.update(new_rrset, self.zone.new_version); self.check_nx_domain()?; Ok(()) } From ee5ea62ef5684673ebcd40a2b4f3b0e4d4c73d22 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 12:57:12 +0200 Subject: [PATCH 253/333] Cargo fmt. --- src/zonetree/in_memory/write.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index 1171e3ca1..fbb8c58e5 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -524,11 +524,11 @@ impl WriteNode { } (false, true) => { trace!("Diff detected: update of existing RRSET - recording addition of new RRSET {new_rrset:#?}"); - diff.lock().unwrap().add( - owner.clone(), + diff.lock().unwrap().add( + owner.clone(), new_rrset.rtype(), new_rrset.clone(), - ); + ); } (false, false) => { // NOOP From 5428bd582ef9586420c388facdb4ed6618c7688d Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 13:00:52 +0200 Subject: [PATCH 254/333] Various fixes and improvements: - Release the write lock on ZoneUpdate::Finished and don't allow the ZoneUpdater to be used after that. - Remove the entire RRset if its' last record is removed. - Check more in tests and add more tests. --- src/zonetree/update.rs | 632 +++++++++++++++++++++++++++++++++++++++-- 1 file changed, 605 insertions(+), 27 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 15f131eaf..7ec28f93b 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -224,8 +224,8 @@ pub struct ZoneUpdater { /// a new write state opened. write: ReopenableZoneWriter, - /// Whether or not we entered an IXFR-like batching mode. - batching: bool, + /// The current state of the updater. + state: ZoneUpdaterState, } impl ZoneUpdater { @@ -245,7 +245,7 @@ impl ZoneUpdater { Ok(Self { zone, write, - batching: false, + state: Default::default(), }) }) } @@ -268,7 +268,12 @@ impl ZoneUpdater { &mut self, update: ZoneUpdate, ) -> Result, Error> { - trace!("Event: {update}"); + trace!("Update: {update}"); + + if self.state == ZoneUpdaterState::Finished { + return Err(Error::Finished); + } + match update { ZoneUpdate::DeleteAllRecords => { // To completely replace the content of the zone, i.e. with @@ -294,19 +299,13 @@ impl ZoneUpdater { // Batch deletion signals the start of a batch, and the end of any // batch addition that was in progress. ZoneUpdate::BeginBatchDelete(_old_soa) => { - let diff = if self.batching { - // Commit the previous batch. - let diff = self.write.commit().await?; + // Commit the previous batch. + let diff = self.write.commit().await?; - // Open a writer for the new batch. - self.write.reopen().await?; + // Open a writer for the new batch. + self.write.reopen().await?; - diff - } else { - None - }; - - self.batching = true; + self.state = ZoneUpdaterState::Batching; return Ok(diff); } @@ -314,7 +313,7 @@ impl ZoneUpdater { ZoneUpdate::BeginBatchAdd(new_soa) => { // Update the SOA record. self.update_soa(new_soa).await?; - self.batching = true; + self.state = ZoneUpdaterState::Batching; } ZoneUpdate::Finished(zone_soa) => { @@ -322,12 +321,25 @@ impl ZoneUpdater { self.update_soa(zone_soa).await?; // Commit the previous batch and return any diff produced. - return self.write.commit().await; + let diff = self.write.commit().await?; + + // Close this updater + self.write.close()?; + self.state = ZoneUpdaterState::Finished; + + return Ok(diff); } } Ok(None) } + + /// Has zone updating finished? + /// + /// If true, further calls to [`apply()`] will fail. + pub fn is_finished(&self) -> bool { + self.state == ZoneUpdaterState::Finished + } } impl ZoneUpdater { @@ -417,7 +429,11 @@ impl ZoneUpdater { } // Replace the RRset in the tree with the new smaller one. - tree_node.update_rrset(SharedRrset::new(rrset)).await?; + if rrset.is_empty() { + tree_node.remove_rrset(rrset.rtype()).await?; + } else { + tree_node.update_rrset(SharedRrset::new(rrset)).await?; + } Ok(()) } @@ -455,7 +471,27 @@ impl ZoneUpdater { } } -//------------ MultiVersionWriteHandle ---------------------------------------- +//------------ ZoneUpdaterState ----------------------------------------------- + +/// The current state of a [`ZoneUpdater`]. +#[derive(Copy, Clone, Debug, Default, PartialEq, Eq)] +enum ZoneUpdaterState { + #[default] + Normal, + + /// IXFR-like batching mode. + Batching, + + /// Finished. + /// + /// [`ZoneUpdate::Finished`] was encountered. + /// + /// The [`ZoneUpdater`] has closed the [`WritableZone`] and can no longer + /// be used. + Finished, +} + +//------------ ReopenableZoneWriter ------------------------------------------- /// State for writing multiple zone versions in sequence. /// @@ -466,7 +502,7 @@ impl ZoneUpdater { /// can be re-opened to write the next version of the zone. struct ReopenableZoneWriter { /// A write interface to a zone. - write: Box, + write: Option>, /// A write interface to the root node of a zone for a particular zone /// version. @@ -478,6 +514,7 @@ impl ReopenableZoneWriter { async fn new(zone: Zone) -> std::io::Result { let write = zone.write().await; let writable = Some(write.open(true).await?); + let write = Some(write); Ok(Self { write, writable }) } @@ -491,7 +528,12 @@ impl ReopenableZoneWriter { // diff (otherwise commit() will panic). drop(writable); - let diff = self.write.commit(false).await?; + let diff = self + .write + .as_mut() + .ok_or(Error::Finished)? + .commit(false) + .await?; Ok(diff) } else { @@ -503,7 +545,20 @@ impl ReopenableZoneWriter { /// /// Call [`commit()`][Self::commit] before calling this method. async fn reopen(&mut self) -> Result<(), Error> { - self.writable = Some(self.write.open(true).await?); + self.writable = Some( + self.write + .as_mut() + .ok_or(Error::Finished)? + .open(true) + .await?, + ); + Ok(()) + } + + /// Close all write state, if not closed already. + fn close(&mut self) -> Result<(), Error> { + self.writable.take(); + self.write.take().ok_or(Error::Finished)?; Ok(()) } @@ -550,6 +605,9 @@ impl ReopenableZoneWriter { #[cfg(test)] mod tests { use core::str::FromStr; + use core::sync::atomic::{AtomicUsize, Ordering}; + + use std::sync::Arc; use std::vec::Vec; use bytes::BytesMut; @@ -563,7 +621,7 @@ mod tests { Message, MessageBuilder, Name, ParsedName, Record, Serial, Ttl, }; use crate::net::xfr::protocol::XfrResponseInterpreter; - use crate::rdata::{Soa, A}; + use crate::rdata::{Ns, Soa, A}; use crate::zonetree::ZoneBuilder; use super::*; @@ -593,7 +651,94 @@ mod tests { .await .unwrap(); + let diff = updater + .apply(ZoneUpdate::Finished(soa_rec.clone())) + .await + .unwrap(); + + let query = MessageBuilder::new_vec(); + let mut query = query.question(); + query.push((qname.clone(), Rtype::SOA)).unwrap(); + let message: Message> = query.into(); + + let builder = MessageBuilder::new_bytes(); + let answer: Message = zone + .read() + .query(qname, Rtype::SOA) + .unwrap() + .to_message(&message, builder) + .into(); + + let found_soa_rec = answer + .answer() + .unwrap() + .limit_to::>() + .next() + .unwrap() + .unwrap() + .into_data(); + + assert_eq!(found_soa_rec, soa); + + // No diff because there is no prior SOA serial + assert!(diff.is_none()); + } + + #[tokio::test] + async fn diff_check() { + init_logging(); + + let zone = mk_empty_zone("example.com"); + + let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); + + let qname = Name::from_str("example.com").unwrap(); + + let s = Serial(20240922); + let soa = mk_soa(s); + let soa_data = ZoneRecordData::Soa(soa.clone()); + let soa_rec = Record::new( + ParsedName::from(qname.clone()), + Class::IN, + Ttl::from_secs(0), + soa_data, + ); + + updater + .apply(ZoneUpdate::AddRecord(soa_rec.clone())) + .await + .unwrap(); + + let diff = updater + .apply(ZoneUpdate::Finished(soa_rec.clone())) + .await + .unwrap(); + + // No diff because there is no prior SOA serial + assert!(diff.is_none()); + + let soa = mk_soa(s.add(1)); + let soa_data = ZoneRecordData::Soa(soa.clone()); + let soa_rec = Record::new( + ParsedName::from(qname.clone()), + Class::IN, + Ttl::from_secs(0), + soa_data, + ); + + assert!(updater.is_finished()); + + let res = updater.apply(ZoneUpdate::AddRecord(soa_rec.clone())).await; + assert!(matches!(res, Err(crate::zonetree::update::Error::Finished))); + + let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); + updater + .apply(ZoneUpdate::AddRecord(soa_rec.clone())) + .await + .unwrap(); + + let diff = updater .apply(ZoneUpdate::Finished(soa_rec.clone())) .await .unwrap(); @@ -621,6 +766,12 @@ mod tests { .into_data(); assert_eq!(found_soa_rec, soa); + + assert!(diff.is_some()); + let diff = diff.unwrap(); + + assert_eq!(diff.start_serial, Serial(20240922)); + assert_eq!(diff.end_serial, Serial(20240923)); } #[tokio::test] @@ -642,9 +793,11 @@ mod tests { let serial = Serial::now(); let soa = mk_soa(serial); add_answer_record(&req, &mut answer, soa.clone()); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::LOCALHOST)); - add_answer_record(&req, &mut answer, A::new(Ipv4Addr::BROADCAST)); - add_answer_record(&req, &mut answer, soa); + let a_1 = A::new(Ipv4Addr::LOCALHOST); + add_answer_record(&req, &mut answer, a_1.clone()); + let a_2 = A::new(Ipv4Addr::BROADCAST); + add_answer_record(&req, &mut answer, a_2.clone()); + add_answer_record(&req, &mut answer, soa.clone()); let resp = answer.into_message(); // Process the response. @@ -655,7 +808,429 @@ mod tests { updater.apply(update).await.unwrap(); } - dbg!(zone); + // -------------------------------------------------------------------- + // Check the contents of the constructed zone. + // -------------------------------------------------------------------- + + // example.com SOA + let query = MessageBuilder::new_vec(); + let mut query = query.question(); + let qname = Name::from_str("example.com").unwrap(); + query.push((qname.clone(), Rtype::SOA)).unwrap(); + let message: Message> = query.into(); + + let builder = MessageBuilder::new_bytes(); + let answer: Message = zone + .read() + .query(qname, Rtype::SOA) + .unwrap() + .to_message(&message, builder) + .into(); + + let mut answers = answer.answer().unwrap().limit_to::>(); + assert_eq!(answers.next().unwrap().unwrap().into_data(), soa); + assert_eq!(answers.next(), None); + + // example. A + let query = MessageBuilder::new_vec(); + let mut query = query.question(); + let qname = Name::from_str("example.com").unwrap(); + query.push((qname.clone(), Rtype::A)).unwrap(); + let message: Message> = query.into(); + + let builder = MessageBuilder::new_bytes(); + let answer: Message = zone + .read() + .query(qname, Rtype::A) + .unwrap() + .to_message(&message, builder) + .into(); + + let mut answers = answer.answer().unwrap().limit_to::(); + assert_eq!(answers.next().unwrap().unwrap().into_data(), a_2); + assert_eq!(answers.next().unwrap().unwrap().into_data(), a_1); + assert_eq!(answers.next(), None); + } + + #[tokio::test] + async fn rfc_1995_ixfr_example() { + fn mk_rfc_1995_ixfr_example_soa( + serial: u32, + ) -> Record, ZoneRecordData>> + { + Record::new( + ParsedName::from(Name::from_str("JAIN.AD.JP.").unwrap()), + Class::IN, + Ttl::from_secs(0), + Soa::new( + ParsedName::from( + Name::from_str("NS.JAIN.AD.JP.").unwrap(), + ), + ParsedName::from( + Name::from_str("mohta.jain.ad.jp.").unwrap(), + ), + Serial(serial), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ) + .into(), + ) + } + + init_logging(); + + // -------------------------------------------------------------------- + // Construct a zone according to the example in RFC 1995 section 7. + // -------------------------------------------------------------------- + + // https://datatracker.ietf.org/doc/html/rfc1995#section-7 + // 7. Example + // "Given the following three generations of data with the current + // serial number of 3," + let zone = mk_empty_zone("JAIN.AD.JP."); + + let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); + // JAIN.AD.JP. IN SOA NS.JAIN.AD.JP. mohta.jain.ad.jp. ( + // 1 600 600 3600000 604800) + let soa_1 = mk_rfc_1995_ixfr_example_soa(1); + updater + .apply(ZoneUpdate::AddRecord(soa_1.clone())) + .await + .unwrap(); + + // IN NS NS.JAIN.AD.JP. + let ns_1 = Record::new( + ParsedName::from(Name::from_str("JAIN.AD.JP.").unwrap()), + Class::IN, + Ttl::from_secs(0), + Ns::new(ParsedName::from( + Name::from_str("NS.JAIN.AD.JP.").unwrap(), + )) + .into(), + ); + updater + .apply(ZoneUpdate::AddRecord(ns_1.clone())) + .await + .unwrap(); + + // NS.JAIN.AD.JP. IN A 133.69.136.1 + let a_1 = Record::new( + ParsedName::from(Name::from_str("NS.JAIN.AD.JP.").unwrap()), + Class::IN, + Ttl::from_secs(0), + A::new(Ipv4Addr::new(133, 69, 136, 1)).into(), + ); + updater + .apply(ZoneUpdate::AddRecord(a_1.clone())) + .await + .unwrap(); + + // NEZU.JAIN.AD.JP. IN A 133.69.136.5 + let nezu = Record::new( + ParsedName::from(Name::from_str("NEZU.JAIN.AD.JP.").unwrap()), + Class::IN, + Ttl::from_secs(0), + A::new(Ipv4Addr::new(133, 69, 136, 5)).into(), + ); + updater + .apply(ZoneUpdate::AddRecord(nezu.clone())) + .await + .unwrap(); + + // "NEZU.JAIN.AD.JP. is removed and JAIN-BB.JAIN.AD.JP. is added." + let diff_1 = updater + .apply(ZoneUpdate::BeginBatchDelete(soa_1.clone())) + .await + .unwrap(); + updater + .apply(ZoneUpdate::DeleteRecord(nezu.clone())) + .await + .unwrap(); + let soa_2 = mk_rfc_1995_ixfr_example_soa(2); + updater + .apply(ZoneUpdate::BeginBatchAdd(soa_2.clone())) + .await + .unwrap(); + let a_2 = Record::new( + ParsedName::from(Name::from_str("JAIN-BB.JAIN.AD.JP.").unwrap()), + Class::IN, + Ttl::from_secs(0), + A::new(Ipv4Addr::new(133, 69, 136, 4)).into(), + ); + updater + .apply(ZoneUpdate::AddRecord(a_2.clone())) + .await + .unwrap(); + let a_3 = Record::new( + ParsedName::from(Name::from_str("JAIN-BB.JAIN.AD.JP.").unwrap()), + Class::IN, + Ttl::from_secs(0), + A::new(Ipv4Addr::new(192, 41, 197, 2)).into(), + ); + updater + .apply(ZoneUpdate::AddRecord(a_3.clone())) + .await + .unwrap(); + + // // "One of the IP addresses of JAIN-BB.JAIN.AD.JP. is changed." + let diff_2 = updater + .apply(ZoneUpdate::BeginBatchDelete(soa_2.clone())) + .await + .unwrap(); + updater + .apply(ZoneUpdate::DeleteRecord(a_2.clone())) + .await + .unwrap(); + let soa_3 = mk_rfc_1995_ixfr_example_soa(3); + updater + .apply(ZoneUpdate::BeginBatchAdd(soa_3.clone())) + .await + .unwrap(); + let a_4 = Record::new( + ParsedName::from(Name::from_str("JAIN-BB.JAIN.AD.JP.").unwrap()), + Class::IN, + Ttl::from_secs(0), + A::new(Ipv4Addr::new(133, 69, 136, 3)).into(), + ); + updater + .apply(ZoneUpdate::AddRecord(a_4.clone())) + .await + .unwrap(); + + let diff_3 = updater + .apply(ZoneUpdate::Finished(soa_3.clone())) + .await + .unwrap(); + + // -------------------------------------------------------------------- + // Check the contents of the constructed zone. + // -------------------------------------------------------------------- + + let count = Arc::new(AtomicUsize::new(0)); + let cloned_count = count.clone(); + zone.read().walk(Box::new(move |_name, _rrset| { + cloned_count.fetch_add(1, Ordering::SeqCst); + })); + + assert_eq!(count.load(Ordering::SeqCst), 4); + + // JAIN.AD.JP. SOA + let query = MessageBuilder::new_vec(); + let mut query = query.question(); + let qname = Name::from_str("JAIN.AD.JP.").unwrap(); + query.push((qname.clone(), Rtype::SOA)).unwrap(); + let message: Message> = query.into(); + + let builder = MessageBuilder::new_bytes(); + let answer: Message = zone + .read() + .query(qname, Rtype::SOA) + .unwrap() + .to_message(&message, builder) + .into(); + + let mut answers = + answer.answer().unwrap().limit_to::>(); + assert_eq!(answers.next().unwrap().unwrap(), soa_3); + assert_eq!(answers.next(), None); + + // JAIN.AD.JP. NS + let query = MessageBuilder::new_vec(); + let mut query = query.question(); + let qname = Name::from_str("JAIN.AD.JP.").unwrap(); + query.push((qname.clone(), Rtype::NS)).unwrap(); + let message: Message> = query.into(); + + let builder = MessageBuilder::new_bytes(); + let answer: Message = zone + .read() + .query(qname, Rtype::NS) + .unwrap() + .to_message(&message, builder) + .into(); + + let mut answers = + answer.answer().unwrap().limit_to::>(); + assert_eq!(answers.next().unwrap().unwrap(), ns_1); + assert_eq!(answers.next(), None); + + // NS.JAIN.AD.JP. A + let query = MessageBuilder::new_vec(); + let mut query = query.question(); + let qname = Name::from_str("NS.JAIN.AD.JP.").unwrap(); + query.push((qname.clone(), Rtype::A)).unwrap(); + let message: Message> = query.into(); + + let builder = MessageBuilder::new_bytes(); + let answer: Message = zone + .read() + .query(qname, Rtype::A) + .unwrap() + .to_message(&message, builder) + .into(); + + let mut answers = + answer.answer().unwrap().limit_to::>(); + assert_eq!(answers.next().unwrap().unwrap(), a_1); + assert_eq!(answers.next(), None); + + // JAIN-BB.JAIN.AD.JP. A + let query = MessageBuilder::new_vec(); + let mut query = query.question(); + let qname = Name::from_str("JAIN-BB.JAIN.AD.JP.").unwrap(); + query.push((qname.clone(), Rtype::A)).unwrap(); + let message: Message> = query.into(); + + let builder = MessageBuilder::new_bytes(); + let answer: Message = zone + .read() + .query(qname, Rtype::A) + .unwrap() + .to_message(&message, builder) + .into(); + + let mut answers = + answer.answer().unwrap().limit_to::>(); + assert_eq!(answers.next().unwrap().unwrap(), a_4); + assert_eq!(answers.next().unwrap().unwrap(), a_3); + assert_eq!(answers.next(), None); + + // "or with the following incremental message:" + + // -------------------------------------------------------------------- + // Check the contents of diff 1: + // -------------------------------------------------------------------- + + // No prior SOA so no diff. + assert!(diff_1.is_none()); + + // -------------------------------------------------------------------- + // Check the contents of diff 2: + // -------------------------------------------------------------------- + + // Diff from SOA serial 1 to SOA serial 2 + assert!(diff_2.is_some()); + let diff_2 = diff_2.unwrap(); + assert_eq!(diff_2.start_serial, Serial(1)); + assert_eq!(diff_2.end_serial, Serial(2)); + + // Removed: SOA and one A record at NEZU.JAIN.AD.JP. + assert_eq!(diff_2.removed.len(), 2); + let mut expected = vec![nezu.into_data()]; + let mut actual = diff_2 + .removed + .get(&(Name::from_str("NEZU.JAIN.AD.JP.").unwrap(), Rtype::A)) + .unwrap() + .data() + .to_vec(); + expected.sort(); + actual.sort(); + assert_eq!(expected, actual); + + // Added: SOA and two A records at JAIN-BB.JAIN.AD.JP. + assert_eq!(diff_2.added.len(), 2); + let mut expected = vec![a_2.clone().into_data(), a_3.into_data()]; + let mut actual = diff_2 + .added + .get(&(Name::from_str("JAIN-BB.JAIN.AD.JP.").unwrap(), Rtype::A)) + .unwrap() + .data() + .to_vec(); + expected.sort(); + actual.sort(); + assert_eq!(expected, actual); + + // -------------------------------------------------------------------- + // Check the contents of diff 3: + // -------------------------------------------------------------------- + + // Diff from SOA serial 2 to SOA serial 3 + assert!(diff_3.is_some()); + let diff_3 = diff_3.unwrap(); + assert_eq!(diff_3.start_serial, Serial(2)); + assert_eq!(diff_3.end_serial, Serial(3)); + + // Removed: SOA and one A record at JAIN-BB.JAIN.AD.JP. + assert_eq!(diff_3.removed.len(), 2); + let mut expected = vec![a_2.into_data()]; + let mut actual = diff_3 + .removed + .get(&(Name::from_str("JAIN-BB.JAIN.AD.JP.").unwrap(), Rtype::A)) + .unwrap() + .data() + .to_vec(); + expected.sort(); + actual.sort(); + assert_eq!(expected, actual); + + // Added: SOA and one A record at JAIN-BB.JAIN.AD.JP. + assert_eq!(diff_3.added.len(), 2); + let mut expected = vec![a_4.into_data()]; + let mut actual = diff_3 + .added + .get(&(Name::from_str("JAIN-BB.JAIN.AD.JP.").unwrap(), Rtype::A)) + .unwrap() + .data() + .to_vec(); + expected.sort(); + actual.sort(); + assert_eq!(expected, actual); + } + + #[tokio::test] + async fn check_rollback() { + init_logging(); + + let zone = mk_empty_zone("example.com"); + + let mut updater = ZoneUpdater::new(zone.clone()).await.unwrap(); + + // Create an AXFR request to reply to. + let req = mk_request("example.com", Rtype::AXFR).into_message(); + + // Create an XFR response interpreter. + let mut interpreter = XfrResponseInterpreter::new(); + + // Create an AXFR response. + let mut answer = mk_empty_answer(&req, Rcode::NOERROR); + let serial = Serial::now(); + let soa = mk_soa(serial); + add_answer_record(&req, &mut answer, soa.clone()); + let a_1 = A::new(Ipv4Addr::LOCALHOST); + add_answer_record(&req, &mut answer, a_1.clone()); + let a_2 = A::new(Ipv4Addr::BROADCAST); + add_answer_record(&req, &mut answer, a_2.clone()); + add_answer_record(&req, &mut answer, soa.clone()); + let resp = answer.into_message(); + + // Process the response. + let it = interpreter.interpret_response(resp).unwrap(); + + for update in it { + let update = update.unwrap(); + // Don't pass ZoneUpdate::Finished to ZoneUpdater thereby preventing + // it from commiting the changes to the zone. + if !matches!(update, ZoneUpdate::Finished(_)) { + updater.apply(update).await.unwrap(); + } + } + + // Drop the ZoneUpdater to show that it definitely doesn't commit. + drop(updater); + + // -------------------------------------------------------------------- + // Check the contents of the constructed zone. + // -------------------------------------------------------------------- + + let count = Arc::new(AtomicUsize::new(0)); + let cloned_count = count.clone(); + zone.read().walk(Box::new(move |_name, _rrset| { + cloned_count.fetch_add(1, Ordering::SeqCst); + })); + + assert_eq!(count.load(Ordering::SeqCst), 0); } //------------ Helper functions ------------------------------------------- @@ -727,6 +1302,9 @@ pub enum Error { /// An I/O error occurred while updating the zone. IoError(std::io::Error), + + /// The updater has finished and cannot be used anymore. + Finished, } //--- From From 899dbbb1bab5e188933e6d99c3a164a07b930468 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 15:18:29 +0200 Subject: [PATCH 255/333] More RustDocs. --- src/net/server/middleware/notify.rs | 51 +++++++++++++++++++++++++---- 1 file changed, 44 insertions(+), 7 deletions(-) diff --git a/src/net/server/middleware/notify.rs b/src/net/server/middleware/notify.rs index 36be5202f..71610fbd6 100644 --- a/src/net/server/middleware/notify.rs +++ b/src/net/server/middleware/notify.rs @@ -1,4 +1,44 @@ -//! DNS NOTIFY related message processing. +//! RFC 1996 DNS NOTIFY related message processing. +//! +//! Quoting [RFC 1996], DNS NOTIFY is the mechanism _"by which a master server +//! advises a set of slave servers that the master's data has been changed and +//! that a query should be initiated to discover the new data."_ +//! +//! The middleware service requires an implementation of the [`Notifiable`] +//! trait to which it forwards received notifications, referred to as the +//! notify target from here on. +//! +//! The middleware service is intended to be used by "slave" implementations +//! and provides a thin layer around receiving and responding to DNS NOTIFY +//! messages, extracting the key data and making it available to the notify +//! target. +//! +//! No actual handling of the received data is done by this module. In +//! particular the following parts of RFC 1996 are NOT implemented and instead +//! are left to the notify target to handle: +//! +//! - Messages with non-zero values in fields not described by RFC 1996 are +//! NOT ignored by this middleware. (RFC 1996 section 3.2) +//! +//! - This module does NOT _"query its masters"_ or initiate XFR transfers. +//! (RFC 1996 section 3.11) +//! +//! - Any "unsecure hint" contained in the answer section is ignored by this +//! middleware and is NOT passed to the notify target. (RFC 1996 section +//! 3.7) +//! +//! - NOTIFY requests received from unknown masters are NOT ignored or logged +//! as this middleware has no knowledge of the known masters. (RFC 1996 +//! section 3.10) +//! +//! - No defering of _"action on any subsequent NOTIFY with the same until it has completed the transcation begun by the first +//! NOTIFY"_ is done by this middleware, as it has no knowledge of whether +//! the notify target begins or completes a transaction. (RFC 1996 section +//! 4.4) +//! +//! [RFC 1996]: https://www.rfc-editor.org/info/rfc1996 + use core::future::{ready, Future, Ready}; use core::marker::PhantomData; use core::ops::ControlFlow; @@ -29,13 +69,10 @@ use crate::zonetree::StoredName; /// A DNS NOTIFY middleware service. /// -/// Standards covered by ths implementation: -/// -/// | RFC | Status | -/// |--------|---------| -/// | [1996] | TBD | +/// [NotifyMiddlewareSvc] implements an [RFC 1996] compliant recipient of DNS +/// NOTIFY messages. /// -/// [1996]: https://datatracker.ietf.org/doc/html/rfc1996 +/// See the [module documentation][super] for more information. #[derive(Clone, Debug)] pub struct NotifyMiddlewareSvc { /// The upstream [`Service`] to pass requests to and receive responses From 4bf2a58e017b05e2168baae6866d79fc0fe5f155 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 15:19:13 +0200 Subject: [PATCH 256/333] Fix broken RustDoc link. --- src/net/server/middleware/notify.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/net/server/middleware/notify.rs b/src/net/server/middleware/notify.rs index 71610fbd6..29ed18bcd 100644 --- a/src/net/server/middleware/notify.rs +++ b/src/net/server/middleware/notify.rs @@ -73,6 +73,8 @@ use crate::zonetree::StoredName; /// NOTIFY messages. /// /// See the [module documentation][super] for more information. +/// +/// [RFC 1996]: https://www.rfc-editor.org/info/rfc1996 #[derive(Clone, Debug)] pub struct NotifyMiddlewareSvc { /// The upstream [`Service`] to pass requests to and receive responses From 1d8c40aa009bd9c3612c0df0212f6d9c98bfdcf4 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 15:26:25 +0200 Subject: [PATCH 257/333] More RustDoc and remove an unnecessary helper function. --- src/net/server/middleware/notify.rs | 30 +++++++++++++++++++---------- 1 file changed, 20 insertions(+), 10 deletions(-) diff --git a/src/net/server/middleware/notify.rs b/src/net/server/middleware/notify.rs index 29ed18bcd..150cfb3c0 100644 --- a/src/net/server/middleware/notify.rs +++ b/src/net/server/middleware/notify.rs @@ -91,6 +91,9 @@ impl NotifyMiddlewareSvc { /// Creates an instance of this middleware service. + /// + /// The given notify target must implement the [`Notifiable`] trait in + /// order to actually use this middleware with the target. #[must_use] pub fn new(next_svc: NextSvc, notify_target: N) -> Self { Self { @@ -110,6 +113,10 @@ where NextSvc::Target: Composer + Default, N: Clone + Notifiable + Sync + Send, { + /// Pre-process received DNS NOTIFY queries. + /// + /// Other types of query will be propagated unmodified to the next + /// middleware or application service in the layered stack of services. async fn preprocess( req: &Request, notify_target: N, @@ -176,9 +183,9 @@ where req.client_addr(), q.qname() ); - ControlFlow::Break(Self::to_stream_compatible( + ControlFlow::Break(once(ready(Ok(CallResult::new( mk_error_response(msg, OptRcode::NOTAUTH), - )) + ))))) } Err(NotifyError::Other) => { @@ -187,9 +194,9 @@ where req.client_addr(), q.qname() ); - ControlFlow::Break(Self::to_stream_compatible( + ControlFlow::Break(once(ready(Ok(CallResult::new( mk_error_response(msg, OptRcode::SERVFAIL), - )) + ))))) } Ok(()) => { @@ -231,12 +238,10 @@ where } } - fn to_stream_compatible( - response: AdditionalBuilder>, - ) -> Once::Item>> { - once(ready(Ok(CallResult::new(response)))) - } - + /// Is this message for us? + /// + /// Returns `Some(Question)` if the given query uses OPCODE NOTIFY and has + /// a first question with a QTYPE of `SOA`, `None` otherwise. fn get_relevant_question( msg: &Message, ) -> Option>>> { @@ -251,6 +256,11 @@ where None } + /// Create a copy of the given message. + /// + /// The copy will be returned as an [`AdditionalBuilder`] so that the + /// caller can further modify it before using it. + /// ` // Based on RequestMessage::append_message_impl(). fn copy_message( source: &Message, From d26ede950304696ccbab7fed80df80d4043500bf Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 15:29:27 +0200 Subject: [PATCH 258/333] Cargo fmt. --- src/net/server/middleware/notify.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/net/server/middleware/notify.rs b/src/net/server/middleware/notify.rs index 150cfb3c0..b39767bdc 100644 --- a/src/net/server/middleware/notify.rs +++ b/src/net/server/middleware/notify.rs @@ -239,7 +239,7 @@ where } /// Is this message for us? - /// + /// /// Returns `Some(Question)` if the given query uses OPCODE NOTIFY and has /// a first question with a QTYPE of `SOA`, `None` otherwise. fn get_relevant_question( @@ -257,7 +257,7 @@ where } /// Create a copy of the given message. - /// + /// /// The copy will be returned as an [`AdditionalBuilder`] so that the /// caller can further modify it before using it. /// ` From c414360f71cf331985c0bfe4f605d09089585772 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 16:09:52 +0200 Subject: [PATCH 259/333] Added RustDocs. --- src/net/server/batcher.rs | 77 ++++++++++++++++++++++++++++++++++++++- 1 file changed, 75 insertions(+), 2 deletions(-) diff --git a/src/net/server/batcher.rs b/src/net/server/batcher.rs index 1e1e416fe..ed280ea46 100644 --- a/src/net/server/batcher.rs +++ b/src/net/server/batcher.rs @@ -1,5 +1,29 @@ -//! Resource record batching. - +//! Resource record batching for response stream splitting. +//! +//! Some DNS requests can result in very large responses that have to be split +//! across multiple messages to be sent back to the client. This applies +//! primarily, if not solely, to responses to AXFR ([RFC 5936]) and IXFR ([RFC +//! 1995]) requests. +//! +//! For example RFC 5936 section 2.2 AXFR Response includes the following +//! paragraph: +//! +//! > _"Each AXFR response message SHOULD contain a sufficient number of +//! > RRs to reasonably amortize the per-message overhead, up to the +//! > largest number that will fit within a DNS message (taking the +//! > required content of the other sections into account, as described +//! > below)."_ +//! +//! This module defines a [ResourceRecordbatcher] trait and a +//! [CallbackBatcher] implementation of the trait, which can split be used a +//! stream of resource records into "batches" filling each DNS response +//! message as much as possible one message at a time from a received stream +//! of resource records. +//! +//! [RFC 1995]: https://www.rfc-editor.org/info/rfc1995 +//! [RFC 5936]: https://www.rfc-editor.org/info/rfc5936 +//! [ResourceRecordBatcher]: ResourceRecordBatcher +//! [CallbackBatcher]: CallbackBatcher use core::marker::PhantomData; use std::fmt::Debug; @@ -18,31 +42,49 @@ use super::util::mk_builder_for_target; //----------- PushResult ------------------------------------------------------ +/// The result of attempting to push a resource record into a +/// [ResourceRecordBatcher]. pub enum PushResult { + /// The resource record was successfully pushed and more will fit. PushedAndReadyForMore, + + /// The resource record was successfully pushed and no more will fit. PushedAndLimitReached(AnswerBuilder>), + + /// The resource record was not pushed because the message is full. NotPushedMessageFull(AnswerBuilder>), + + /// The resource record was not pushed but trying again might work. Retry, } //------------ ResourceRecordBatcher ------------------------------------------ +/// An accumulator of resource records into one or more DNS responses. pub trait ResourceRecordBatcher where RequestOctets: Octets, Target: Composer + Default, { + /// The type returned on error. type Error: From + Debug; + /// Attempt to push a single resource record into a response message. #[allow(clippy::result_unit_err)] fn push( &mut self, record: impl ComposeRecord, ) -> Result, Self::Error>; + /// Signal that the last resource record has been pushed for this response + /// sequence. #[allow(clippy::result_unit_err)] fn finish(&mut self) -> Result<(), Self::Error>; + /// Creates a new `AnswerBuilder` for the given request message. + /// + /// The default implementation sets the RCODE of the new builder to + /// `NOERROR`. fn mk_answer_builder( &self, msg: &Message, @@ -54,11 +96,13 @@ where //------------ Callbacks ------------------------------------------------------ +/// A set of callback functions for use with [`CallbackBatcher`]. pub trait Callbacks where RequestOctets: Octets, Target: Composer + Default, { + /// The type returned on error. type Error: From + Debug; /// Prepare a message builder to push records into. @@ -92,15 +136,24 @@ where //------------ CallbackBatcher ------------------------------------------------ +/// A [`ResourceRecordBatcher`] impl that delegates work to callacks. pub struct CallbackBatcher where RequestOctets: Octets, Target: Composer + Default, C: Callbacks, { + /// The request message being responded to. + /// + /// Needed for constructing response messages. req_msg: Arc>, + + /// The current answer builder in use. answer: Option>, PushError>>, + + /// User defined callback specific state. callback_state: T, + _phantom: PhantomData, } @@ -110,6 +163,13 @@ where Target: Composer + Default, C: Callbacks, { + /// Creates a new instance for building responses to the given request. + /// + /// Delegates certain parts of the task to the callbacks supplied via + /// generic type `C` which must implement the [`Callbacks`] trait. + /// + /// Any state that should be forwarded to the callbacks can be provided + /// using the `callback_state` parameter. pub fn new( req_msg: Arc>, callback_state: T, @@ -122,6 +182,11 @@ where } } + /// Returns the callback specific state. + /// + /// This convenience method returns a reference to the callback specific + /// state that was supplied when the instance was created by calling + /// [`new()`][Self::new]. pub fn callback_state(&self) -> &T { &self.callback_state } @@ -133,6 +198,9 @@ where Target: Composer + Default, C: Callbacks, { + /// Try and push a resource record into the current response. + /// + /// Invokes [`Callbacks::batch_ready()`] on success. fn try_push( &mut self, record: &impl ComposeRecord, @@ -150,6 +218,10 @@ where } } + /// Push a resource record into the current response. + /// + /// Invokes [`Callbacks::batch_started()`] and + /// [`Callbacks::record_pushed()`] if appropriate. fn push_ref( &mut self, record: &impl ComposeRecord, @@ -239,6 +311,7 @@ where Target: Composer + Default, C: Callbacks, { + /// Notes via trace logging if an in-progress answer is dropped. fn drop(&mut self) { if self.answer.is_some() { trace!("Dropping unfinished batcher, was that intentional or did you forget to call finish()?"); From dce66a12bc532f13f9860c97ef7c4285fca0472d Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 16:15:19 +0200 Subject: [PATCH 260/333] More RustDocs. --- src/net/server/batcher.rs | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/net/server/batcher.rs b/src/net/server/batcher.rs index ed280ea46..601151b50 100644 --- a/src/net/server/batcher.rs +++ b/src/net/server/batcher.rs @@ -20,6 +20,20 @@ //! message as much as possible one message at a time from a received stream //! of resource records. //! +//! # Usage +//! +//! - `impl Callbacks` for a struct providing your own `batch_ready()` and +//! `record_pushed()` fn implementations. +//! +//! - Create an instance of `CallbackBatcher` generic over your `Callbacks` +//! impl type. +//! +//! - Call `CallbackBatcher::push()` repeatedly until no more resource records +//! are left to push for the current response. +//! +//! - Call `CallbackBatcher::finish()` to ensure that a last partial response +//! is also completely handled. +//! //! [RFC 1995]: https://www.rfc-editor.org/info/rfc1995 //! [RFC 5936]: https://www.rfc-editor.org/info/rfc5936 //! [ResourceRecordBatcher]: ResourceRecordBatcher From 807d9f417e80a333e591b3aa11cf9be21e7b62f2 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 16:16:40 +0200 Subject: [PATCH 261/333] More RustDocs. --- src/net/server/batcher.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/net/server/batcher.rs b/src/net/server/batcher.rs index 601151b50..fff84f02c 100644 --- a/src/net/server/batcher.rs +++ b/src/net/server/batcher.rs @@ -22,17 +22,17 @@ //! //! # Usage //! -//! - `impl Callbacks` for a struct providing your own `batch_ready()` and -//! `record_pushed()` fn implementations. +//! 1. `impl Callbacks` for a struct providing your own `batch_ready()` and +//! `record_pushed()` fn implementations. //! -//! - Create an instance of `CallbackBatcher` generic over your `Callbacks` -//! impl type. +//! 2. Create an instance of `CallbackBatcher` generic over your `Callbacks` +//! impl type. //! -//! - Call `CallbackBatcher::push()` repeatedly until no more resource records -//! are left to push for the current response. +//! 3. Call `CallbackBatcher::push()` repeatedly until no more resource +//! records are left to push for the current response. //! -//! - Call `CallbackBatcher::finish()` to ensure that a last partial response -//! is also completely handled. +//! 4. Call `CallbackBatcher::finish()` to ensure that a last partial response +//! is also completely handled. //! //! [RFC 1995]: https://www.rfc-editor.org/info/rfc1995 //! [RFC 5936]: https://www.rfc-editor.org/info/rfc5936 From 447632bbe6c227f96c969473bc6cee44b32b8a82 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 16:26:34 +0200 Subject: [PATCH 262/333] Compilation fixes due to upstream changes. --- src/net/server/middleware/xfr.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index ae9b54574..b6ee7b1ae 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -1644,7 +1644,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 rrset.push_data(A::new(p("192.41.197.2")).into()); diff.add(n("JAIN-BB.JAIN.AD.JP"), Rtype::A, SharedRrset::new(rrset)); - diffs.push(diff.build(Serial(1), Serial(2))); + diffs.push(diff.build().unwrap()); // Diff 2: One of the IP addresses of JAIN-BB.JAIN.AD.JP. is changed. let mut diff = ZoneDiffBuilder::new(); @@ -1691,7 +1691,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 rrset.push_data(A::new(p("133.69.136.3")).into()); diff.add(n("JAIN-BB.JAIN.AD.JP"), Rtype::A, SharedRrset::new(rrset)); - diffs.push(diff.build(Serial(2), Serial(3))); + diffs.push(diff.build().unwrap()); // Create an object that knows how to provide zone and diff data for // our zone and diffs. From 81dbbff7256ad04fedf7ceb18b1bb44b528b86ad Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 22:32:41 +0200 Subject: [PATCH 263/333] Backport improvements from the `xfr` branch. --- src/zonetree/in_memory/write.rs | 15 +++++++-------- src/zonetree/types.rs | 27 +++++++++++++++++++++++++++ 2 files changed, 34 insertions(+), 8 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index fbb8c58e5..22dc9ca12 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -6,7 +6,6 @@ use core::sync::atomic::Ordering; use std::boxed::Box; use std::future::Future; -use std::io::ErrorKind; use std::pin::Pin; use std::sync::Arc; use std::sync::Mutex; @@ -17,7 +16,7 @@ use std::{fmt, io}; use futures_util::future::Either; use parking_lot::RwLock; use tokio::sync::OwnedMutexGuard; -use tracing::trace; +use tracing::{trace, warn}; use crate::base::iana::Rtype; use crate::base::name::Label; @@ -326,13 +325,13 @@ impl WritableZone for WriteZone { self.add_soa_add_diff_entry(new_soa_rr, &mut diff); if old_serial.is_some() && new_serial.is_some() { - let Ok(zone_diff) = diff.build() else { - return Box::pin(ready(Err(std::io::Error::new( - ErrorKind::Other, - "Diff lacks SOA records", - )))); + out_diff = match diff.build() { + Ok(zone_diff) => Some(zone_diff), + Err(err) => { + warn!("Error constructing diff: {err}"); + None + } }; - out_diff = Some(zone_diff); } } diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index 46b7e0b13..88e413e8d 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -7,6 +7,7 @@ use std::vec::Vec; use bytes::Bytes; use serde::{Deserialize, Serialize}; +use tracing::trace; use crate::base::name::Name; use crate::base::rdata::RecordData; @@ -369,6 +370,15 @@ impl ZoneDiff { }) .ok_or(ZoneDiffError::MissingEndSoa)?; + if start_serial == end_serial || end_serial < start_serial { + trace!("Diff construction error: serial {start_serial} -> serial {end_serial}:\nremoved: {removed:#?}\nadded: {added:#?}\n"); + return Err(ZoneDiffError::InvalidSerialRange); + } + + trace!( + "Built diff from serial {start_serial} to serial {end_serial}" + ); + Ok(Self { start_serial, end_serial, @@ -392,6 +402,23 @@ pub enum ZoneDiffError { /// /// A zone diff requires a starting SOA. MissingEndSoa, + + /// End SOA serial is equal to or less than the start SOA serial. + InvalidSerialRange, +} + +//--- Display + +impl std::fmt::Display for ZoneDiffError { + fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { + match self { + ZoneDiffError::MissingStartSoa => f.write_str("MissingStartSoa"), + ZoneDiffError::MissingEndSoa => f.write_str("MissingEndSoa"), + ZoneDiffError::InvalidSerialRange => { + f.write_str("InvalidSerialRange") + } + } + } } //------------ ZoneUpdate ----------------------------------------------------- From 296ed37319ce5d40f044c6ef1a8bccc04c112300 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 23:23:00 +0200 Subject: [PATCH 264/333] Note SOA QTYPE support only. --- src/net/server/middleware/notify.rs | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/net/server/middleware/notify.rs b/src/net/server/middleware/notify.rs index b39767bdc..85b552c1a 100644 --- a/src/net/server/middleware/notify.rs +++ b/src/net/server/middleware/notify.rs @@ -14,8 +14,7 @@ //! target. //! //! No actual handling of the received data is done by this module. In -//! particular the following parts of RFC 1996 are NOT implemented and instead -//! are left to the notify target to handle: +//! particular the following parts of RFC 1996 are NOT implemented: //! //! - Messages with non-zero values in fields not described by RFC 1996 are //! NOT ignored by this middleware. (RFC 1996 section 3.2) @@ -36,6 +35,10 @@ //! NOTIFY"_ is done by this middleware, as it has no knowledge of whether //! the notify target begins or completes a transaction. (RFC 1996 section //! 4.4) +//! +//! - Only QTYPE SOA is supported. NOTIFY messages with other QTYPEs will be +//! propagated unmodified to the next middleware or application service in +//! the layered stack of services. //! //! [RFC 1996]: https://www.rfc-editor.org/info/rfc1996 From d533b8911208bf0829f899e76ee5f0eeac5eb139 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 24 Sep 2024 10:17:11 +0200 Subject: [PATCH 265/333] Cargo fmt. --- src/net/server/middleware/notify.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/server/middleware/notify.rs b/src/net/server/middleware/notify.rs index 85b552c1a..77d0f4fb5 100644 --- a/src/net/server/middleware/notify.rs +++ b/src/net/server/middleware/notify.rs @@ -35,7 +35,7 @@ //! NOTIFY"_ is done by this middleware, as it has no knowledge of whether //! the notify target begins or completes a transaction. (RFC 1996 section //! 4.4) -//! +//! //! - Only QTYPE SOA is supported. NOTIFY messages with other QTYPEs will be //! propagated unmodified to the next middleware or application service in //! the layered stack of services. From feb646d1aea5381c34b02e87fb0cc9a67a13b75d Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 24 Sep 2024 12:26:49 +0200 Subject: [PATCH 266/333] RustDocs and get rid of unnecessary function. --- src/net/server/middleware/xfr.rs | 154 +++++++++++++------------------ 1 file changed, 64 insertions(+), 90 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index b6ee7b1ae..48e8869d8 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -1,4 +1,25 @@ -//! XFR request handling middleware. +//! RFC 5936 AXFR and RFC 1995 IXFR request handling middleware. +//! +//! This module provides the [`XfrMiddlewareSvc`] service which responds to +//! [RFC 5936] AXFR and [RFC 1995] IXFR requests to perform entire or +//! incremental difference based zone transfers. +//! +//! Determining which requests to honour and with what data is delegated to a +//! caller supplied implementation of the [`XfrDataProvider`] trait. +//! +//! [`XfrRrBatcher`], primarily intended for internal use by +//! [`XfrMiddlewareSvc`], handles splitting of large zone transfer replies +//! into batches with as many resource records per response as will fit. +//! +//! [RFC 5936]: https://www.rfc-editor.org/info/rfc5936 +//! [RFC 1995]: https://www.rfc-editor.org/info/rfc1995 + + // https://datatracker.ietf.org/doc/html/rfc1995#section-2 + // 2. Brief Description of the Protocol + // "To ensure integrity, servers should use UDP checksums for all + // UDP responses. A cautious client which receives a UDP packet + // with a checksum value of zero should ignore the result and try a + // TCP IXFR instead." use core::future::{ready, Future, Ready}; use core::marker::PhantomData; @@ -43,33 +64,27 @@ use crate::zonetree::{ //------------ XfrMiddlewareSvc ---------------------------------------------- -/// A [`MiddlewareProcessor`] for responding to XFR requests. -/// -/// Standards covered by ths implementation: -/// -/// | RFC | Status | -/// |--------|---------| -/// | [1034] | TBD | -/// | [1035] | TBD | -/// | [1995] | TBD | -/// | [5936] | TBD | -/// -/// [`MiddlewareProcessor`]: -/// crate::net::server::middleware::processor::MiddlewareProcessor -/// [1034]: https://datatracker.ietf.org/doc/html/rfc1034 -/// [1035]: https://datatracker.ietf.org/doc/html/rfc1035 -/// [1995]: https://datatracker.ietf.org/doc/html/rfc1995 -/// [5936]: https://datatracker.ietf.org/doc/html/rfc5936 +/// RFC 5936 AXFR and RFC 1995 IXFR request handling middleware. +/// +/// See the [module documentation] for more information. +/// +/// [module documentation]: super #[derive(Clone, Debug)] pub struct XfrMiddlewareSvc { /// The upstream [`Service`] to pass requests to and receive responses /// from. next_svc: NextSvc, + /// A caller supplied implementation of [`XfrDataProvider`] for + /// determining which requests to answer and with which data. xfr_data_provider: XDP, + /// A limit on the number of XFR related zone walking operations + /// that may run concurrently. zone_walking_semaphore: Arc, + /// A limit on the number of XFR related response batching operations that + /// may run concurrently. batcher_semaphore: Arc, _phantom: PhantomData<(RequestOctets, Metadata)>, @@ -80,7 +95,13 @@ impl where XDP: XfrDataProvider, { - /// Creates a new processor instance. + /// Creates a new instance of this middleware. + /// + /// Takes an implementation of [`XfrDataProvider`] as a parameter to + /// determine which requests to honour and with which data. + /// + /// The `max_concurrency` parameter limits the number of simultaneous zone + /// transfer operations that may occur concurrently without blocking. #[must_use] pub fn new( next_svc: NextSvc, @@ -112,6 +133,10 @@ where NextSvc::Stream: Send + Sync, XDP: XfrDataProvider, { + /// Pre-process received DNS XFR queries. + /// + /// Other types of query will be propagated unmodified to the next + /// middleware or application service in the layered stack of services. pub async fn preprocess( zone_walking_semaphore: Arc, batcher_semaphore: Arc, @@ -299,6 +324,7 @@ where } } + /// Log a message and break with an DNS error response stream. #[allow(clippy::type_complexity)] fn log_and_break( q: &Question::Range<'_>>>, @@ -322,6 +348,7 @@ where ControlFlow::Break(Self::to_stream(mk_error_response(msg, rcode))) } + /// Generate and send an AXFR response for a given request and zone. #[allow(clippy::too_many_arguments)] async fn send_axfr_response( zone_walk_semaphore: Arc, @@ -585,7 +612,8 @@ where Ok(MiddlewareStream::Result(stream)) } - // Returns None if fallback to AXFR should be done. + // Generate and send an IXFR response for the given request and zone + // diffs. #[allow(clippy::too_many_arguments)] async fn send_ixfr_response( batcher_semaphore: Arc, @@ -600,84 +628,26 @@ where ::Item, >, OptRcode, - > { - // https://datatracker.ietf.org/doc/html/rfc1995#section-2 - // 2. Brief Description of the Protocol - // "Transport of a query may be by either UDP or TCP. If an IXFR - // query is via UDP, the IXFR server may attempt to reply using UDP - // if the entire response can be contained in a single DNS packet. - // If the UDP reply does not fit, the query is responded to with a - // single SOA record of the server's current version to inform the - // client that a TCP query should be initiated." - // - // https://datatracker.ietf.org/doc/html/rfc1995#section-3 - // 3. Query Format - // "The IXFR query packet format is the same as that of a normal DNS - // query, but with the query type being IXFR and the authority - // section containing the SOA record of client's version of the - // zone." - // - // https://datatracker.ietf.org/doc/html/rfc1995#section-4 - // 4. Response Format - // "If incremental zone transfer is not available, the entire zone - // is returned. The first and the last RR of the response is the - // SOA record of the zone. I.e. the behavior is the same as an - // AXFR response except the query type is IXFR." - // - // https://datatracker.ietf.org/doc/html/rfc1995#section-2 - // 2. Brief Description of the Protocol - // "To ensure integrity, servers should use UDP checksums for all - // UDP responses. A cautious client which receives a UDP packet - // with a checksum value of zero should ignore the result and try a - // TCP IXFR instead." - if let AnswerContent::Data(rrset) = zone_soa_answer.content() { - if rrset.data().len() == 1 { - if let ZoneRecordData::Soa(soa) = - rrset.first().unwrap().data() - { - let zone_serial = soa.serial(); - - // TODO: if cached then return cached IXFR response - return Self::compute_ixfr( - batcher_semaphore, - req, - query_serial, - zone_serial, - zone_soa_answer, - diffs, - ) - .await; - } - } - } - - Err(OptRcode::SERVFAIL) - } - - #[allow(clippy::too_many_arguments)] - async fn compute_ixfr( - batcher_semaphore: Arc, - req: &Request, - query_serial: Serial, - zone_serial: Serial, - zone_soa_answer: &Answer, - diffs: Vec>, - ) -> Result< - XfrMiddlewareStream< - NextSvc::Future, - NextSvc::Stream, - ::Item, - >, - OptRcode, > { let msg = req.message(); let AnswerContent::Data(zone_soa_rrset) = zone_soa_answer.content().clone() else { - unreachable!() + return Err(OptRcode::SERVFAIL); + }; + + if zone_soa_rrset.data().len() != 1 { + return Err(OptRcode::SERVFAIL); + } + + let first_rr = zone_soa_rrset.first().unwrap(); + let ZoneRecordData::Soa(soa) = first_rr.data() else { + return Err(OptRcode::SERVFAIL); }; + let zone_serial = soa.serial(); + // Note: Unlike RFC 5936 for AXFR, neither RFC 1995 nor RFC 9103 say // anything about whether an IXFR response can consist of more than // one response message, but given the 2^16 byte maximum response size @@ -911,6 +881,10 @@ where } } + /// Is this message for us? + /// + /// Returns `Some(Question)` if the given query uses OPCODE QUERYY and has + /// a first question with a QTYPE of `AXFR` or `IXFR`, `None` otherwise. fn get_relevant_question( msg: &Message, ) -> Option>>> { From f84c0334ae8072735e40026f14543d87661ef5f3 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 24 Sep 2024 12:32:32 +0200 Subject: [PATCH 267/333] Cargo fmt and RustDoc tweaks. --- src/net/server/middleware/xfr.rs | 31 +++++++++++++++---------------- 1 file changed, 15 insertions(+), 16 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 48e8869d8..3d6c08ac9 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -3,24 +3,23 @@ //! This module provides the [`XfrMiddlewareSvc`] service which responds to //! [RFC 5936] AXFR and [RFC 1995] IXFR requests to perform entire or //! incremental difference based zone transfers. -//! +//! //! Determining which requests to honour and with what data is delegated to a //! caller supplied implementation of the [`XfrDataProvider`] trait. -//! +//! //! [`XfrRrBatcher`], primarily intended for internal use by //! [`XfrMiddlewareSvc`], handles splitting of large zone transfer replies //! into batches with as many resource records per response as will fit. -//! +//! +//! # Limitations +//! +//! * RFC 1995 2 Brief Description of the Protocol states: _"To ensure +//! integrity, servers should use UDP checksums for all UDP responses."_. +//! This is not implemented. +//! //! [RFC 5936]: https://www.rfc-editor.org/info/rfc5936 //! [RFC 1995]: https://www.rfc-editor.org/info/rfc1995 - // https://datatracker.ietf.org/doc/html/rfc1995#section-2 - // 2. Brief Description of the Protocol - // "To ensure integrity, servers should use UDP checksums for all - // UDP responses. A cautious client which receives a UDP packet - // with a checksum value of zero should ignore the result and try a - // TCP IXFR instead." - use core::future::{ready, Future, Ready}; use core::marker::PhantomData; use core::ops::{ControlFlow, Deref}; @@ -65,10 +64,10 @@ use crate::zonetree::{ //------------ XfrMiddlewareSvc ---------------------------------------------- /// RFC 5936 AXFR and RFC 1995 IXFR request handling middleware. -/// -/// See the [module documentation] for more information. -/// -/// [module documentation]: super +/// +/// See the [module documentation] for a high level introduction. +/// +/// [module documentation]: crate::net::server::middleware::xfr #[derive(Clone, Debug)] pub struct XfrMiddlewareSvc { /// The upstream [`Service`] to pass requests to and receive responses @@ -96,10 +95,10 @@ where XDP: XfrDataProvider, { /// Creates a new instance of this middleware. - /// + /// /// Takes an implementation of [`XfrDataProvider`] as a parameter to /// determine which requests to honour and with which data. - /// + /// /// The `max_concurrency` parameter limits the number of simultaneous zone /// transfer operations that may occur concurrently without blocking. #[must_use] From 0594753578497297e768b703e9f55d6bab7e183f Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 24 Sep 2024 12:38:19 +0200 Subject: [PATCH 268/333] Don't use magic values. --- src/net/server/middleware/xfr.rs | 20 +++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 3d6c08ac9..402a19512 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -61,6 +61,19 @@ use crate::zonetree::{ ZoneDiff, ZoneTree, }; +//------------ Constants ----------------------------------------------------- + +/// https://datatracker.ietf.org/doc/html/rfc1035#section-2.3.4 +/// 2.3.4. Size limits +/// "UDP messages 512 octets or less" +const MAX_UDP_MSG_BYTE_LEN: u16 = 512; + +/// https://datatracker.ietf.org/doc/html/rfc1035#section-4.2.2 +/// 4.2.2. TCP usage +/// "The message is prefixed with a two byte length field which gives the +/// message length, excluding the two byte length field" +const MAX_TCP_MSG_BYTE_LEN: u16 = u16::MAX; + //------------ XfrMiddlewareSvc ---------------------------------------------- /// RFC 5936 AXFR and RFC 1995 IXFR request handling middleware. @@ -901,12 +914,13 @@ where fn calc_msg_bytes_available(req: &Request) -> usize { let bytes_available = match req.transport_ctx() { TransportSpecificContext::Udp(ctx) => { - let max_msg_size = - ctx.max_response_size_hint().unwrap_or(512); + let max_msg_size = ctx + .max_response_size_hint() + .unwrap_or(MAX_UDP_MSG_BYTE_LEN); max_msg_size - req.num_reserved_bytes() } TransportSpecificContext::NonUdp(_) => { - 65535 - req.num_reserved_bytes() + MAX_TCP_MSG_BYTE_LEN - req.num_reserved_bytes() } }; From 0f2b68cce6dde38112e91e0fdac18426e68d1878 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 24 Sep 2024 12:44:34 +0200 Subject: [PATCH 269/333] More RustDoc and comment tweaks. --- src/net/server/middleware/xfr.rs | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 402a19512..c7b18a473 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -6,6 +6,9 @@ //! //! Determining which requests to honour and with what data is delegated to a //! caller supplied implementation of the [`XfrDataProvider`] trait. +//! [`XfrDataProvider`] implementations for [`Zone`] and [`ZoneTree`] are +//! provided allowing those types to be used as-is as XFR data providers with +//! this middleware service. //! //! [`XfrRrBatcher`], primarily intended for internal use by //! [`XfrMiddlewareSvc`], handles splitting of large zone transfer replies @@ -1022,6 +1025,10 @@ pub trait XfrDataProvider { /// available. /// /// Returns Err otherwise. + /// + /// Pass `Some` zone SOA serial number in the `diff_from` parameter to + /// request `ZoneDiff`s from the specified serial to the current SOA + /// serial number of the zone, inclusive, if available. #[allow(clippy::type_complexity)] fn request( &self, @@ -1043,7 +1050,7 @@ pub trait XfrDataProvider { Octs: Octets + Send + Sync; } -//--- impl for AsRef +//--- impl XfrDataProvider for Deref impl XfrDataProvider for U where @@ -1073,7 +1080,7 @@ where } } -//--- impl for Zone +//--- impl XfrDataProvider for Zone impl XfrDataProvider for Zone { /// Request data needed to respond to an XFR request. @@ -1114,7 +1121,7 @@ impl XfrDataProvider for Zone { } } -//--- impl for ZoneTree +//--- impl XfrDataProvider for ZoneTree impl XfrDataProvider for ZoneTree { /// Request data needed to respond to an XFR request. From cf9c754831dc1eaddd61a64dbc1c565b225c2001 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 24 Sep 2024 12:54:17 +0200 Subject: [PATCH 270/333] Document how to require TSIG signed XFX requests. --- src/net/server/middleware/xfr.rs | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index c7b18a473..248059f1f 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -9,11 +9,23 @@ //! [`XfrDataProvider`] implementations for [`Zone`] and [`ZoneTree`] are //! provided allowing those types to be used as-is as XFR data providers with //! this middleware service. -//! +//! //! [`XfrRrBatcher`], primarily intended for internal use by //! [`XfrMiddlewareSvc`], handles splitting of large zone transfer replies //! into batches with as many resource records per response as will fit. //! +//! # Requiring TSIG signed XFR requests +//! +//! To require XFR requests to be TSIG authenticated, implement +//! [`XfrDataProvider>`], extract the key data using +//! [`Request::metadata()`] and verify that a TSIG key was used to sign the +//! request, and that the name and algorithm of the used key are acceptable to +//! you. +//! +//! You can then use [`TsigMiddlewareSvc`] below [`XfrMiddlewareSvc`] in the +//! middleware layer stack so that the used `Key` is made available from the +//! TSIG middleware to the XFR middleware. +//! //! # Limitations //! //! * RFC 1995 2 Brief Description of the Protocol states: _"To ensure @@ -22,6 +34,8 @@ //! //! [RFC 5936]: https://www.rfc-editor.org/info/rfc5936 //! [RFC 1995]: https://www.rfc-editor.org/info/rfc1995 +//! [`TsigMiddlewareSvc`]: +//! crate::net::server::middleware::tsig::TsigMiddlewareSvc use core::future::{ready, Future, Ready}; use core::marker::PhantomData; From 73ecabde55a65efaed65df8dcd6f14e4ee8975d6 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 24 Sep 2024 12:54:40 +0200 Subject: [PATCH 271/333] RustDoc correction. --- src/net/server/middleware/xfr.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 248059f1f..79231db16 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -14,7 +14,7 @@ //! [`XfrMiddlewareSvc`], handles splitting of large zone transfer replies //! into batches with as many resource records per response as will fit. //! -//! # Requiring TSIG signed XFR requests +//! # Requiring TSIG authenticated XFR requests //! //! To require XFR requests to be TSIG authenticated, implement //! [`XfrDataProvider>`], extract the key data using From 5ee9e07e677825460896dce54cc679fa040efea4 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 24 Sep 2024 12:55:28 +0200 Subject: [PATCH 272/333] Improved RustDoc. --- src/net/server/middleware/xfr.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 79231db16..f4010a11e 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -22,7 +22,8 @@ //! request, and that the name and algorithm of the used key are acceptable to //! you. //! -//! You can then use [`TsigMiddlewareSvc`] below [`XfrMiddlewareSvc`] in the +//! You can then use your [`XfrDataProvider`] impl with [`XfrMiddlewareSvc`], +//! and add [`TsigMiddlewareSvc`] directly before [`XfrMiddlewareSvc`] in the //! middleware layer stack so that the used `Key` is made available from the //! TSIG middleware to the XFR middleware. //! From 2e5ea83376ca0eabfc8b05e5cb3f5b07ca85b5da Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 24 Sep 2024 12:58:08 +0200 Subject: [PATCH 273/333] Cargo fmt. --- src/net/server/middleware/xfr.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index f4010a11e..888bea119 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -9,24 +9,24 @@ //! [`XfrDataProvider`] implementations for [`Zone`] and [`ZoneTree`] are //! provided allowing those types to be used as-is as XFR data providers with //! this middleware service. -//! +//! //! [`XfrRrBatcher`], primarily intended for internal use by //! [`XfrMiddlewareSvc`], handles splitting of large zone transfer replies //! into batches with as many resource records per response as will fit. //! //! # Requiring TSIG authenticated XFR requests -//! +//! //! To require XFR requests to be TSIG authenticated, implement //! [`XfrDataProvider>`], extract the key data using //! [`Request::metadata()`] and verify that a TSIG key was used to sign the //! request, and that the name and algorithm of the used key are acceptable to //! you. -//! +//! //! You can then use your [`XfrDataProvider`] impl with [`XfrMiddlewareSvc`], //! and add [`TsigMiddlewareSvc`] directly before [`XfrMiddlewareSvc`] in the //! middleware layer stack so that the used `Key` is made available from the //! TSIG middleware to the XFR middleware. -//! +//! //! # Limitations //! //! * RFC 1995 2 Brief Description of the Protocol states: _"To ensure From dd4da4f17f94fcf1167922f7b0e68b3a65df84ac Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 24 Sep 2024 15:43:03 +0200 Subject: [PATCH 274/333] Don't require the diffs supplied to XfrMiddlewareSvc to be in-memory HashMaps, instead abstract over the underlying implementation using a trait. Still needs proper RustDocs, and maybe async support. --- src/net/server/middleware/xfr.rs | 72 ++++++++++----- src/zonetree/in_memory/write.rs | 18 ++-- src/zonetree/mod.rs | 7 +- src/zonetree/traits.rs | 103 ++++++++++++++++++++- src/zonetree/types.rs | 148 ++++++++++++++++++++++++++++--- src/zonetree/update.rs | 6 +- 6 files changed, 303 insertions(+), 51 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 888bea119..deecc6425 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -74,10 +74,12 @@ use crate::net::server::service::{ use crate::net::server::util::{mk_builder_for_target, mk_error_response}; use crate::rdata::{Soa, ZoneRecordData}; use crate::zonetree::error::OutOfZone; +use crate::zonetree::types::EmptyZoneDiff; use crate::zonetree::{ Answer, AnswerContent, ReadableZone, SharedRrset, StoredName, Zone, - ZoneDiff, ZoneTree, + ZoneDiff, ZoneDiffItem, ZoneTree, }; +use std::fmt::Debug; //------------ Constants ----------------------------------------------------- @@ -162,6 +164,7 @@ where NextSvc::Target: Composer + Default + Send + Sync, NextSvc::Stream: Send + Sync, XDP: XfrDataProvider, + XDP::Diff: Debug + 'static, { /// Pre-process received DNS XFR queries. /// @@ -650,7 +653,7 @@ where req: &Request, query_serial: Serial, zone_soa_answer: &Answer, - diffs: Vec>, + diffs: Vec, ) -> Result< XfrMiddlewareStream< NextSvc::Future, @@ -658,7 +661,10 @@ where ::Item, >, OptRcode, - > { + > + where + XDP::Diff: Send + 'static, + { let msg = req.message(); let AnswerContent::Data(zone_soa_rrset) = @@ -805,8 +811,8 @@ where // and added RRs. The first RR of the deleted RRs is the // older SOA RR and the first RR of the added RRs is the // newer SOA RR. - let soa_k = &(owner.clone(), Rtype::SOA); - let removed_soa = diff.removed.get(soa_k).unwrap(); // The zone MUST have a SOA record + let removed_soa = + diff.get_removed(owner.clone(), Rtype::SOA).unwrap(); // The zone MUST have a SOA record batcher .push(( owner.clone(), @@ -816,8 +822,10 @@ where )) .unwrap(); // TODO - diff.removed.iter().for_each(|((owner, rtype), rrset)| { + diff.iter_removed().for_each(|item| { + let (owner, rtype) = item.key(); if *rtype != Rtype::SOA { + let rrset = item.value(); for rr in rrset.data() { batcher .push(( @@ -831,7 +839,8 @@ where } }); - let added_soa = diff.added.get(soa_k).unwrap(); // The zone MUST have a SOA record + let added_soa = + diff.get_added(owner.clone(), Rtype::SOA).unwrap(); // The zone MUST have a SOA record batcher .push(( owner.clone(), @@ -841,8 +850,10 @@ where )) .unwrap(); // TODO - diff.added.iter().for_each(|((owner, rtype), rrset)| { + diff.iter_added().for_each(|item| { + let (owner, rtype) = item.key(); if *rtype != Rtype::SOA { + let rrset = item.value(); for rr in rrset.data() { batcher .push(( @@ -958,6 +969,7 @@ where NextSvc::Target: Composer + Default + Send + Sync, NextSvc::Stream: Send + Sync, XDP: XfrDataProvider + Clone + Sync + Send + 'static, + XDP::Diff: Debug + Sync, Metadata: Clone + Default + Sync + Send + 'static, { type Target = NextSvc::Target; @@ -1034,6 +1046,8 @@ pub enum XfrDataProviderError { /// A provider of data needed for responding to XFR requests. pub trait XfrDataProvider { + type Diff: ZoneDiff + Send; + /// Request data needed to respond to an XFR request. /// /// Returns Ok if the request is allowed and the requested data is @@ -1053,7 +1067,7 @@ pub trait XfrDataProvider { Box< dyn Future< Output = Result< - (Zone, Vec>), + (Zone, Vec), XfrDataProviderError, >, > + Sync @@ -1072,6 +1086,8 @@ where T: XfrDataProvider + 'static, U: Deref, { + type Diff = T::Diff; + fn request( &self, req: &Request, @@ -1080,7 +1096,7 @@ where Box< dyn Future< Output = Result< - (Zone, Vec>), + (Zone, Vec), XfrDataProviderError, >, > + Sync @@ -1098,6 +1114,8 @@ where //--- impl XfrDataProvider for Zone impl XfrDataProvider for Zone { + type Diff = EmptyZoneDiff; + /// Request data needed to respond to an XFR request. /// /// Returns Ok(Self, vec![]) if the given apex name and class match this @@ -1112,7 +1130,7 @@ impl XfrDataProvider for Zone { Box< dyn Future< Output = Result< - (Zone, Vec>), + (Zone, Vec), XfrDataProviderError, >, > + Sync @@ -1139,6 +1157,8 @@ impl XfrDataProvider for Zone { //--- impl XfrDataProvider for ZoneTree impl XfrDataProvider for ZoneTree { + type Diff = EmptyZoneDiff; + /// Request data needed to respond to an XFR request. /// /// Returns Ok(zone, vec![]) if the given apex name and class match a zone @@ -1153,7 +1173,7 @@ impl XfrDataProvider for ZoneTree { Box< dyn Future< Output = Result< - (Zone, Vec>), + (Zone, Vec), XfrDataProviderError, >, > + Sync @@ -1382,7 +1402,7 @@ mod tests { use crate::tsig::{Algorithm, Key, KeyName}; use crate::zonefile::inplace::Zonefile; use crate::zonetree::types::Rrset; - use crate::zonetree::ZoneDiffBuilder; + use crate::zonetree::{InMemoryZoneDiff, InMemoryZoneDiffBuilder}; use super::*; @@ -1612,7 +1632,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 let zone = load_zone(rfc_1995_zone.as_bytes()); // Diff 1: NEZU.JAIN.AD.JP. is removed and JAIN-BB.JAIN.AD.JP. is added. - let mut diff = ZoneDiffBuilder::new(); + let mut diff = InMemoryZoneDiffBuilder::new(); // -- Remove the old SOA. let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); @@ -1656,7 +1676,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 diffs.push(diff.build().unwrap()); // Diff 2: One of the IP addresses of JAIN-BB.JAIN.AD.JP. is changed. - let mut diff = ZoneDiffBuilder::new(); + let mut diff = InMemoryZoneDiffBuilder::new(); // -- Remove the old SOA. let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); @@ -1717,7 +1737,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 let zone_soa = get_zone_soa(&zone).await; - // could be replied to with the following full zone transfer message: + // could be replied to with the following incremental message: let mut expected_records: ExpectedRecords = vec![ (n("JAIN.AD.JP."), zone_soa.clone().into()), ( @@ -1840,6 +1860,8 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 } impl XfrDataProvider>> for KeyReceivingXfrDataProvider { + type Diff = EmptyZoneDiff; + #[allow(clippy::type_complexity)] fn request( &self, @@ -1849,7 +1871,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 Box< dyn Future< Output = Result< - (Zone, Vec>), + (Zone, Vec), XfrDataProviderError, >, > + Sync @@ -2042,7 +2064,10 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 ::Stream, <::Stream as Stream>::Item, >, - > { + > + where + XDP::Diff: Debug + 'static, + { XfrMiddlewareSvc::, TestNextSvc, XDP, Metadata>::preprocess( Arc::new(Semaphore::new(1)), Arc::new(Semaphore::new(1)), @@ -2068,7 +2093,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 assert!(resp.is_answer(req)); let mut records = resp.answer().unwrap().peekable(); - for rec in records.by_ref() { + for (idx, rec) in records.by_ref().enumerate() { let rec = rec.unwrap(); let rec = rec @@ -2083,7 +2108,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 }) .unwrap_or_else(|| { panic!( - "XFR record {} {} {} {} was not expected", + "XFR record {idx} {} {} {} {} was not expected", rec.owner(), rec.class(), rec.rtype(), @@ -2122,11 +2147,11 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 struct ZoneWithDiffs { zone: Zone, - diffs: Vec>, + diffs: Vec>, } impl ZoneWithDiffs { - fn new(zone: Zone, diffs: Vec) -> Self { + fn new(zone: Zone, diffs: Vec) -> Self { Self { zone, diffs: diffs.into_iter().map(Arc::new).collect(), @@ -2135,6 +2160,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 } impl XfrDataProvider for ZoneWithDiffs { + type Diff = Arc; fn request( &self, req: &Request, @@ -2143,7 +2169,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 Box< dyn Future< Output = Result< - (Zone, Vec>), + (Zone, Vec>), XfrDataProviderError, >, > + Sync diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index 22dc9ca12..c4241f438 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -22,7 +22,9 @@ use crate::base::iana::Rtype; use crate::base::name::Label; use crate::base::{NameBuilder, Serial}; use crate::rdata::ZoneRecordData; -use crate::zonetree::types::{ZoneCut, ZoneDiff, ZoneDiffBuilder}; +use crate::zonetree::types::{ + InMemoryZoneDiff, InMemoryZoneDiffBuilder, ZoneCut, +}; use crate::zonetree::StoredName; use crate::zonetree::{Rrset, SharedRr}; use crate::zonetree::{SharedRrset, WritableZone, WritableZoneNode}; @@ -77,7 +79,7 @@ pub struct WriteZone { /// [`WriteNode::update_child()`] is called it creates a new [`WriteNode`] /// which also needs to be able to add and remove things from the same /// diff collection. - diff: Arc>>>>, + diff: Arc>>>>, /// The zone is dirty if changes have been made but not yet committed. /// @@ -142,7 +144,7 @@ impl WriteZone { fn add_soa_remove_diff_entry( &mut self, old_soa_rr: Option, - diff: &mut ZoneDiffBuilder, + diff: &mut InMemoryZoneDiffBuilder, ) -> Option { if let Some(old_soa_rr) = old_soa_rr { let ZoneRecordData::Soa(old_soa) = old_soa_rr.data() else { @@ -172,7 +174,7 @@ impl WriteZone { fn add_soa_add_diff_entry( &mut self, new_soa_rr: Option, - diff: &mut ZoneDiffBuilder, + diff: &mut InMemoryZoneDiffBuilder, ) -> Option { if let Some(new_soa_rr) = new_soa_rr { let ZoneRecordData::Soa(new_soa) = new_soa_rr.data() else { @@ -287,7 +289,7 @@ impl WritableZone for WriteZone { bump_soa_serial: bool, ) -> Pin< Box< - dyn Future, io::Error>> + dyn Future, io::Error>> + Send + Sync, >, @@ -385,7 +387,7 @@ pub struct WriteNode { node: Either, Arc>, /// The diff we are building, if enabled. - diff: Option<(StoredName, Arc>)>, + diff: Option<(StoredName, Arc>)>, } impl WriteNode { @@ -398,7 +400,7 @@ impl WriteNode { let diff = if create_diff { Some(( zone.apex.name().clone(), - Arc::new(Mutex::new(ZoneDiffBuilder::new())), + Arc::new(Mutex::new(InMemoryZoneDiffBuilder::new())), )) } else { None @@ -677,7 +679,7 @@ impl WriteNode { Ok(()) } - fn diff(&self) -> Option>> { + fn diff(&self) -> Option>> { self.diff .as_ref() .map(|(_, diff_builder)| diff_builder.clone()) diff --git a/src/zonetree/mod.rs b/src/zonetree/mod.rs index 574d49ab9..5556f234e 100644 --- a/src/zonetree/mod.rs +++ b/src/zonetree/mod.rs @@ -115,12 +115,13 @@ mod zone; pub use self::answer::{Answer, AnswerAuthority, AnswerContent}; pub use self::in_memory::ZoneBuilder; pub use self::traits::{ - ReadableZone, WritableZone, WritableZoneNode, ZoneStore, + ReadableZone, WritableZone, WritableZoneNode, ZoneDiff, ZoneDiffItem, + ZoneStore, }; pub use self::tree::{ZoneSetIter, ZoneTree}; pub use self::types::{ - Rrset, SharedRr, SharedRrset, StoredName, StoredRecord, ZoneDiff, - ZoneDiffBuilder, + InMemoryZoneDiff, InMemoryZoneDiffBuilder, Rrset, SharedRr, SharedRrset, + StoredName, StoredRecord, }; pub use self::walk::WalkOp; pub use self::zone::Zone; diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index 37904047f..b87982f96 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -21,12 +21,13 @@ use bytes::Bytes; use crate::base::iana::Class; use crate::base::name::Label; -use crate::base::{Name, Rtype}; +use crate::base::{Name, Rtype, Serial, ToName}; use super::answer::Answer; use super::error::OutOfZone; -use super::types::{ZoneCut, ZoneDiff}; +use super::types::{InMemoryZoneDiff, ZoneCut}; use super::{SharedRr, SharedRrset, StoredName, WalkOp}; +use core::ops::Deref; //------------ ZoneStore ----------------------------------------------------- @@ -155,7 +156,7 @@ pub trait WritableZone: Send + Sync { bump_soa_serial: bool, ) -> Pin< Box< - dyn Future, io::Error>> + dyn Future, io::Error>> + Send + Sync, >, @@ -234,3 +235,99 @@ pub trait WritableZoneNode: Send + Sync { &self, ) -> Pin> + Send + Sync>>; } + +//------------ ZoneDiffItem --------------------------------------------------- + +/// TODO +pub trait ZoneDiffItem { + /// TODO + fn key(&self) -> &(StoredName, Rtype); + + /// TODO + fn value(&self) -> &SharedRrset; +} + +//------------ ZoneDiff ------------------------------------------------------- + +/// TODO +pub trait ZoneDiff { + /// TODO + type Item<'a>: ZoneDiffItem + where + Self: 'a; + + /// TODO + type Iterator<'a>: Iterator> + where + Self: 'a; + + /// The serial number of the zone which was modified. + fn start_serial(&self) -> Serial; + + /// The serial number of the zone that resulted from the modifications. + fn end_serial(&self) -> Serial; + + /// An iterator over the RRsets that were added to the zone. + fn iter_added(&self) -> Self::Iterator<'_>; + + /// An iterator over the RRsets that were removed from the zone. + fn iter_removed(&self) -> Self::Iterator<'_>; + + /// Get an RRset that was added to the zone, if present in the diff. + fn get_added( + &self, + name: impl ToName, + rtype: Rtype, + ) -> Option<&SharedRrset>; + + /// Get an RRset that was removed from the zone, if present in the diff. + fn get_removed( + &self, + name: impl ToName, + rtype: Rtype, + ) -> Option<&SharedRrset>; +} + +//--- impl ZoneDiff for Arc + +impl ZoneDiff for Arc { + type Item<'a> = T::Item<'a> + where + Self: 'a; + + type Iterator<'a> = T::Iterator<'a> + where + Self: 'a; + + fn start_serial(&self) -> Serial { + Arc::deref(self).start_serial() + } + + fn end_serial(&self) -> Serial { + Arc::deref(self).end_serial() + } + + fn iter_added(&self) -> Self::Iterator<'_> { + Arc::deref(self).iter_added() + } + + fn iter_removed(&self) -> Self::Iterator<'_> { + Arc::deref(self).iter_removed() + } + + fn get_added( + &self, + name: impl ToName, + rtype: Rtype, + ) -> Option<&SharedRrset> { + Arc::deref(self).get_added(name, rtype) + } + + fn get_removed( + &self, + name: impl ToName, + rtype: Rtype, + ) -> Option<&SharedRrset> { + Arc::deref(self).get_removed(name, rtype) + } +} diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index 88e413e8d..1c2d3f601 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -9,11 +9,12 @@ use bytes::Bytes; use serde::{Deserialize, Serialize}; use tracing::trace; +use super::traits::{ZoneDiff, ZoneDiffItem}; use crate::base::name::Name; use crate::base::rdata::RecordData; use crate::base::record::Record; -use crate::base::Serial; use crate::base::{iana::Rtype, Ttl}; +use crate::base::{Serial, ToName}; use crate::rdata::ZoneRecordData; //------------ Type Aliases -------------------------------------------------- @@ -252,13 +253,13 @@ pub struct ZoneCut { pub glue: Vec, } -//------------ ZoneDiffBuilder ----------------------------------------------- +//------------ InMemoryZoneDiffBuilder ---------------------------------------- -/// A [`ZoneDiff`] builder. +/// An [`InMemoryZoneDiff`] builder. /// /// Removes are assumed to occur before adds. #[derive(Debug, Default)] -pub struct ZoneDiffBuilder { +pub struct InMemoryZoneDiffBuilder { /// The records added to the Zone. added: HashMap<(StoredName, Rtype), SharedRrset>, @@ -266,7 +267,7 @@ pub struct ZoneDiffBuilder { removed: HashMap<(StoredName, Rtype), SharedRrset>, } -impl ZoneDiffBuilder { +impl InMemoryZoneDiffBuilder { /// Creates a new instance of the builder. pub fn new() -> Self { Default::default() @@ -301,22 +302,22 @@ impl ZoneDiffBuilder { /// Note: No check is currently done that the start and end serials match /// the SOA records in the removed and added records contained within the /// diff. - pub fn build(self) -> Result { - ZoneDiff::new(self.added, self.removed) + pub fn build(self) -> Result { + InMemoryZoneDiff::new(self.added, self.removed) } } -//------------ ZoneDiff ------------------------------------------------------ +//------------ InMemoryZoneDiff ----------------------------------------------- /// The differences between one serial and another for a DNS zone. /// /// Removes are assumed to occur before adds. #[derive(Clone, Debug)] -pub struct ZoneDiff { +pub struct InMemoryZoneDiff { /// The serial number of the zone which was modified. pub start_serial: Serial, - /// The serial number of the Zzone that resulted from the modifications. + /// The serial number of the zone that resulted from the modifications. pub end_serial: Serial, /// The RRsets added to the zone. @@ -326,7 +327,7 @@ pub struct ZoneDiff { pub removed: Arc>, } -impl ZoneDiff { +impl InMemoryZoneDiff { /// Creates a new immutable zone diff. /// /// Returns `Err(ZoneDiffError::MissingStartSoa)` If the removed records @@ -388,6 +389,131 @@ impl ZoneDiff { } } +//--- impl ZoneDiff + +impl<'a> ZoneDiffItem for (&'a (StoredName, Rtype), &'a SharedRrset) { + fn key(&self) -> &(StoredName, Rtype) { + self.0 + } + + fn value(&self) -> &SharedRrset { + self.1 + } +} + +impl ZoneDiff for InMemoryZoneDiff { + type Item<'a> = (&'a (StoredName, Rtype), &'a SharedRrset) + where + Self: 'a; + + type Iterator<'a> = std::collections::hash_map::Iter<'a, (StoredName, Rtype), SharedRrset> + where + Self: 'a; + + fn start_serial(&self) -> Serial { + self.start_serial + } + + fn end_serial(&self) -> Serial { + self.end_serial + } + + fn iter_added(&self) -> Self::Iterator<'_> { + self.added.iter() + } + + fn iter_removed(&self) -> Self::Iterator<'_> { + self.removed.iter() + } + + fn get_added( + &self, + name: impl ToName, + rtype: Rtype, + ) -> Option<&SharedRrset> { + self.added.get(&(name.to_name(), rtype)) + } + + fn get_removed( + &self, + name: impl ToName, + rtype: Rtype, + ) -> Option<&SharedRrset> { + self.removed.get(&(name.to_name(), rtype)) + } +} + +/// TODO +pub struct EmptyZoneDiffItem; + +impl ZoneDiffItem for EmptyZoneDiffItem { + fn key(&self) -> &(StoredName, Rtype) { + unreachable!() + } + + fn value(&self) -> &SharedRrset { + unreachable!() + } +} + +/// TODO +#[derive(Debug)] +pub struct EmptyZoneDiffIterator; + +impl Iterator for EmptyZoneDiffIterator { + type Item = EmptyZoneDiffItem; + + fn next(&mut self) -> Option { + None + } +} + +/// TODO +#[derive(Debug)] +pub struct EmptyZoneDiff; + +impl ZoneDiff for EmptyZoneDiff { + type Item<'a> = EmptyZoneDiffItem + where + Self: 'a; + + type Iterator<'a> = EmptyZoneDiffIterator + where + Self: 'a; + + fn start_serial(&self) -> Serial { + Serial(0) + } + + fn end_serial(&self) -> Serial { + Serial(0) + } + + fn iter_added(&self) -> Self::Iterator<'_> { + EmptyZoneDiffIterator + } + + fn iter_removed(&self) -> Self::Iterator<'_> { + EmptyZoneDiffIterator + } + + fn get_added( + &self, + _name: impl ToName, + _rtype: Rtype, + ) -> Option<&SharedRrset> { + None + } + + fn get_removed( + &self, + _name: impl ToName, + _rtype: Rtype, + ) -> Option<&SharedRrset> { + None + } +} + //------------ ZoneDiffError -------------------------------------------------- /// Creating a [`ZoneDiff`] failed for some reason. diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 7ec28f93b..1e5b62d24 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -21,7 +21,7 @@ use crate::zonetree::{Rrset, SharedRrset}; use super::error::OutOfZone; use super::types::ZoneUpdate; use super::util::rel_name_rev_iter; -use super::{WritableZone, WritableZoneNode, Zone, ZoneDiff}; +use super::{InMemoryZoneDiff, WritableZone, WritableZoneNode, Zone}; /// Apply a sequence of [`ZoneUpdate`]s to update the content of a [`Zone`]. /// @@ -267,7 +267,7 @@ impl ZoneUpdater { pub async fn apply( &mut self, update: ZoneUpdate, - ) -> Result, Error> { + ) -> Result, Error> { trace!("Update: {update}"); if self.state == ZoneUpdaterState::Finished { @@ -521,7 +521,7 @@ impl ReopenableZoneWriter { /// Commits any pending changes to the [`Zone`] being written to. /// /// Returns the created diff, if any. - async fn commit(&mut self) -> Result, Error> { + async fn commit(&mut self) -> Result, Error> { // Commit the deletes and adds that just occurred if let Some(writable) = self.writable.take() { // Ensure that there are no dangling references to the created From d3a08a5f4997240dc741273cd37d664f6dacf3d9 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 24 Sep 2024 16:49:25 +0200 Subject: [PATCH 275/333] Make ZoneDiff responses async, in case e.g. diffs are fetched from disk or a database. --- src/net/server/middleware/xfr.rs | 34 ++++++++++---- src/zonetree/traits.rs | 48 +++++++++++-------- src/zonetree/types.rs | 79 +++++++++++++++++++------------- 3 files changed, 103 insertions(+), 58 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index deecc6425..e36852cf1 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -43,12 +43,14 @@ use core::marker::PhantomData; use core::ops::{ControlFlow, Deref}; use std::boxed::Box; +use std::fmt::Debug; use std::pin::Pin; use std::sync::Arc; use std::vec::Vec; use bytes::Bytes; use futures_util::stream::{once, Once, Stream}; +use futures_util::{pin_mut, StreamExt}; use octseq::Octets; use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; use tokio::sync::Semaphore; @@ -79,7 +81,6 @@ use crate::zonetree::{ Answer, AnswerContent, ReadableZone, SharedRrset, StoredName, Zone, ZoneDiff, ZoneDiffItem, ZoneTree, }; -use std::fmt::Debug; //------------ Constants ----------------------------------------------------- @@ -165,6 +166,7 @@ where NextSvc::Stream: Send + Sync, XDP: XfrDataProvider, XDP::Diff: Debug + 'static, + for<'a> ::Stream<'a>: Send, { /// Pre-process received DNS XFR queries. /// @@ -811,8 +813,10 @@ where // and added RRs. The first RR of the deleted RRs is the // older SOA RR and the first RR of the added RRs is the // newer SOA RR. - let removed_soa = - diff.get_removed(owner.clone(), Rtype::SOA).unwrap(); // The zone MUST have a SOA record + let removed_soa = diff + .get_removed(owner.clone(), Rtype::SOA) + .await + .unwrap(); // The zone MUST have a SOA record batcher .push(( owner.clone(), @@ -822,7 +826,9 @@ where )) .unwrap(); // TODO - diff.iter_removed().for_each(|item| { + let removed_stream = diff.removed(); + pin_mut!(removed_stream); + while let Some(item) = removed_stream.next().await { let (owner, rtype) = item.key(); if *rtype != Rtype::SOA { let rrset = item.value(); @@ -837,10 +843,10 @@ where .unwrap(); // TODO } } - }); + } let added_soa = - diff.get_added(owner.clone(), Rtype::SOA).unwrap(); // The zone MUST have a SOA record + diff.get_added(owner.clone(), Rtype::SOA).await.unwrap(); // The zone MUST have a SOA record batcher .push(( owner.clone(), @@ -850,7 +856,9 @@ where )) .unwrap(); // TODO - diff.iter_added().for_each(|item| { + let added_stream = diff.added(); + pin_mut!(added_stream); + while let Some(item) = added_stream.next().await { let (owner, rtype) = item.key(); if *rtype != Rtype::SOA { let rrset = item.value(); @@ -865,7 +873,7 @@ where .unwrap(); // TODO } } - }); + } } batcher @@ -970,6 +978,7 @@ where NextSvc::Stream: Send + Sync, XDP: XfrDataProvider + Clone + Sync + Send + 'static, XDP::Diff: Debug + Sync, + for<'a> ::Stream<'a>: Send, Metadata: Clone + Default + Sync + Send + 'static, { type Target = NextSvc::Target; @@ -2067,6 +2076,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 > where XDP::Diff: Debug + 'static, + for<'a> ::Stream<'a>: Send, { XfrMiddlewareSvc::, TestNextSvc, XDP, Metadata>::preprocess( Arc::new(Semaphore::new(1)), @@ -2101,6 +2111,14 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 .unwrap() .unwrap(); + eprintln!( + "XFR record {idx} {} {} {} {}", + rec.owner(), + rec.class(), + rec.rtype(), + rec.data(), + ); + let pos = expected_records .iter() .position(|(name, data)| { diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index b87982f96..c97dfbdc9 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -9,6 +9,7 @@ //! [`ZoneTree`]: super::ZoneTree use core::any::Any; use core::future::ready; +use core::ops::Deref; use core::pin::Pin; use std::boxed::Box; @@ -18,6 +19,7 @@ use std::io; use std::sync::Arc; use bytes::Bytes; +use futures_util::Stream; use crate::base::iana::Class; use crate::base::name::Label; @@ -27,7 +29,6 @@ use super::answer::Answer; use super::error::OutOfZone; use super::types::{InMemoryZoneDiff, ZoneCut}; use super::{SharedRr, SharedRrset, StoredName, WalkOp}; -use core::ops::Deref; //------------ ZoneStore ----------------------------------------------------- @@ -257,35 +258,40 @@ pub trait ZoneDiff { Self: 'a; /// TODO - type Iterator<'a>: Iterator> + type Stream<'a>: Stream> where Self: 'a; /// The serial number of the zone which was modified. - fn start_serial(&self) -> Serial; + fn start_serial( + &self, + ) -> Pin + Send + '_>>; /// The serial number of the zone that resulted from the modifications. - fn end_serial(&self) -> Serial; + fn end_serial(&self) + -> Pin + Send + '_>>; - /// An iterator over the RRsets that were added to the zone. - fn iter_added(&self) -> Self::Iterator<'_>; + /// An stream of RRsets that were added to the zone. + // TODO: Does this need to be Box>>? + fn added(&self) -> Self::Stream<'_>; - /// An iterator over the RRsets that were removed from the zone. - fn iter_removed(&self) -> Self::Iterator<'_>; + /// An stream of RRsets that were removed from the zone. + // TODO: Does this need to be Box>>? + fn removed(&self) -> Self::Stream<'_>; /// Get an RRset that was added to the zone, if present in the diff. fn get_added( &self, name: impl ToName, rtype: Rtype, - ) -> Option<&SharedRrset>; + ) -> Pin> + Send + '_>>; /// Get an RRset that was removed from the zone, if present in the diff. fn get_removed( &self, name: impl ToName, rtype: Rtype, - ) -> Option<&SharedRrset>; + ) -> Pin> + Send + '_>>; } //--- impl ZoneDiff for Arc @@ -295,31 +301,35 @@ impl ZoneDiff for Arc { where Self: 'a; - type Iterator<'a> = T::Iterator<'a> + type Stream<'a> = T::Stream<'a> where Self: 'a; - fn start_serial(&self) -> Serial { + fn start_serial( + &self, + ) -> Pin + Send + '_>> { Arc::deref(self).start_serial() } - fn end_serial(&self) -> Serial { + fn end_serial( + &self, + ) -> Pin + Send + '_>> { Arc::deref(self).end_serial() } - fn iter_added(&self) -> Self::Iterator<'_> { - Arc::deref(self).iter_added() + fn added(&self) -> Self::Stream<'_> { + Arc::deref(self).added() } - fn iter_removed(&self) -> Self::Iterator<'_> { - Arc::deref(self).iter_removed() + fn removed(&self) -> Self::Stream<'_> { + Arc::deref(self).removed() } fn get_added( &self, name: impl ToName, rtype: Rtype, - ) -> Option<&SharedRrset> { + ) -> Pin> + Send + '_>> { Arc::deref(self).get_added(name, rtype) } @@ -327,7 +337,7 @@ impl ZoneDiff for Arc { &self, name: impl ToName, rtype: Rtype, - ) -> Option<&SharedRrset> { + ) -> Pin> + Send + '_>> { Arc::deref(self).get_removed(name, rtype) } } diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index 1c2d3f601..852ce11fc 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -1,11 +1,17 @@ //! Zone tree related types. -use std::collections::HashMap; +use core::future::{ready, Future}; +use core::pin::Pin; +use core::task::{Context, Poll}; + +use std::boxed::Box; +use std::collections::{hash_map, HashMap}; use std::ops; use std::sync::Arc; use std::vec::Vec; use bytes::Bytes; +use futures_util::stream; use serde::{Deserialize, Serialize}; use tracing::trace; @@ -406,40 +412,44 @@ impl ZoneDiff for InMemoryZoneDiff { where Self: 'a; - type Iterator<'a> = std::collections::hash_map::Iter<'a, (StoredName, Rtype), SharedRrset> + type Stream<'a> = futures_util::stream::Iter> where Self: 'a; - fn start_serial(&self) -> Serial { - self.start_serial + fn start_serial( + &self, + ) -> Pin + Send + '_>> { + Box::pin(ready(self.start_serial)) } - fn end_serial(&self) -> Serial { - self.end_serial + fn end_serial( + &self, + ) -> Pin + Send + '_>> { + Box::pin(ready(self.end_serial)) } - fn iter_added(&self) -> Self::Iterator<'_> { - self.added.iter() + fn added(&self) -> Self::Stream<'_> { + stream::iter(self.added.iter()) } - fn iter_removed(&self) -> Self::Iterator<'_> { - self.removed.iter() + fn removed(&self) -> Self::Stream<'_> { + stream::iter(self.removed.iter()) } fn get_added( &self, name: impl ToName, rtype: Rtype, - ) -> Option<&SharedRrset> { - self.added.get(&(name.to_name(), rtype)) + ) -> Pin> + Send + '_>> { + Box::pin(ready(self.added.get(&(name.to_name(), rtype)))) } fn get_removed( &self, name: impl ToName, rtype: Rtype, - ) -> Option<&SharedRrset> { - self.removed.get(&(name.to_name(), rtype)) + ) -> Pin> + Send + '_>> { + Box::pin(ready(self.removed.get(&(name.to_name(), rtype)))) } } @@ -458,13 +468,16 @@ impl ZoneDiffItem for EmptyZoneDiffItem { /// TODO #[derive(Debug)] -pub struct EmptyZoneDiffIterator; +pub struct EmptyZoneDiffStream; -impl Iterator for EmptyZoneDiffIterator { +impl futures_util::stream::Stream for EmptyZoneDiffStream { type Item = EmptyZoneDiffItem; - fn next(&mut self) -> Option { - None + fn poll_next( + self: Pin<&mut Self>, + _cx: &mut Context<'_>, + ) -> Poll> { + Poll::Ready(None) } } @@ -477,40 +490,44 @@ impl ZoneDiff for EmptyZoneDiff { where Self: 'a; - type Iterator<'a> = EmptyZoneDiffIterator + type Stream<'a> = EmptyZoneDiffStream where Self: 'a; - fn start_serial(&self) -> Serial { - Serial(0) + fn start_serial( + &self, + ) -> Pin + Send + '_>> { + Box::pin(ready(Serial(0))) } - fn end_serial(&self) -> Serial { - Serial(0) + fn end_serial( + &self, + ) -> Pin + Send + '_>> { + Box::pin(ready(Serial(0))) } - fn iter_added(&self) -> Self::Iterator<'_> { - EmptyZoneDiffIterator + fn added(&self) -> Self::Stream<'_> { + EmptyZoneDiffStream } - fn iter_removed(&self) -> Self::Iterator<'_> { - EmptyZoneDiffIterator + fn removed(&self) -> Self::Stream<'_> { + EmptyZoneDiffStream } fn get_added( &self, _name: impl ToName, _rtype: Rtype, - ) -> Option<&SharedRrset> { - None + ) -> Pin> + Send + '_>> { + Box::pin(ready(None)) } fn get_removed( &self, _name: impl ToName, _rtype: Rtype, - ) -> Option<&SharedRrset> { - None + ) -> Pin> + Send + '_>> { + Box::pin(ready(None)) } } From d43aa266a9eb738866278a0329dbb0a169fb030a Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 24 Sep 2024 17:41:45 +0200 Subject: [PATCH 276/333] More RustDocs. --- src/zonetree/traits.rs | 29 +++++++++++++++++++++++------ src/zonetree/types.rs | 9 ++++++--- src/zonetree/update.rs | 2 ++ 3 files changed, 31 insertions(+), 9 deletions(-) diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index c97dfbdc9..376cc236e 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -239,25 +239,42 @@ pub trait WritableZoneNode: Send + Sync { //------------ ZoneDiffItem --------------------------------------------------- -/// TODO +/// One difference item in a set of changes made to a zone. +/// +/// Conceptually a diff is like a set of keys and values, representing a change +/// to a resource record set with key (owner name, resource type) and the value +/// being the changed resource records at that owner and with that type. pub trait ZoneDiffItem { - /// TODO + /// The owner name and resource record type. fn key(&self) -> &(StoredName, Rtype); - /// TODO + /// The changed records. + /// + /// Each record has the same key (owner name and resource record type). fn value(&self) -> &SharedRrset; } //------------ ZoneDiff ------------------------------------------------------- -/// TODO +/// A set of differences between two versions (SOA serial numbers) of a zone. +/// +/// Often referred to simply as a "diff". +/// +/// The default implementation of this trait supplied by the domain crate is +/// the [`InMemoryZoneDiff`]. As the name implies it stores its data in +/// memory. +/// +/// In order however to support less local backing stores for diff data, such +/// as on-disk storage or in a database possibly reached via a network, +/// asynchronous access to the diff is supported via use of [`Future`]s and +/// [`Stream`]s. pub trait ZoneDiff { - /// TODO + /// A single item in the diff. type Item<'a>: ZoneDiffItem where Self: 'a; - /// TODO + /// The type of [`Stream`] used to access the diff records. type Stream<'a>: Stream> where Self: 'a; diff --git a/src/zonetree/types.rs b/src/zonetree/types.rs index 852ce11fc..777ab5eb0 100644 --- a/src/zonetree/types.rs +++ b/src/zonetree/types.rs @@ -453,7 +453,7 @@ impl ZoneDiff for InMemoryZoneDiff { } } -/// TODO +/// The item type used by [`EmptyZoneDiff`]. pub struct EmptyZoneDiffItem; impl ZoneDiffItem for EmptyZoneDiffItem { @@ -466,7 +466,7 @@ impl ZoneDiffItem for EmptyZoneDiffItem { } } -/// TODO +/// The stream type used by [`EmptyZoneDiff`]. #[derive(Debug)] pub struct EmptyZoneDiffStream; @@ -481,7 +481,10 @@ impl futures_util::stream::Stream for EmptyZoneDiffStream { } } -/// TODO +/// A [`ZoneDiff`] implementation that is always empty. +/// +/// Useful when a [`ZoneDiff`] type is needed in a type declaration but for use +/// by a type that does not support zone difference data. #[derive(Debug)] pub struct EmptyZoneDiff; diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 1e5b62d24..70a006b2b 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -337,6 +337,8 @@ impl ZoneUpdater { /// Has zone updating finished? /// /// If true, further calls to [`apply()`] will fail. + /// + /// [`apply()`]: Self::apply pub fn is_finished(&self) -> bool { self.state == ZoneUpdaterState::Finished } From b3254796e8b32be136332cbf9e763432be443b96 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 24 Sep 2024 18:23:27 +0200 Subject: [PATCH 277/333] Return an error and use ?. Remove unnecessary helper functions. --- src/net/server/middleware/xfr.rs | 197 ++++++++++++++----------------- src/zonetree/update.rs | 2 +- 2 files changed, 90 insertions(+), 109 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index e36852cf1..35ac744b4 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -177,18 +177,21 @@ where batcher_semaphore: Arc, req: &Request, xfr_data_provider: XDP, - ) -> ControlFlow< - XfrMiddlewareStream< - NextSvc::Future, - NextSvc::Stream, - ::Item, + ) -> Result< + ControlFlow< + XfrMiddlewareStream< + NextSvc::Future, + NextSvc::Stream, + ::Item, + >, >, + OptRcode, > { let msg = req.message(); // Do we support this type of request? let Some(q) = Self::get_relevant_question(msg) else { - return ControlFlow::Continue(()); + return Ok(ControlFlow::Continue(())); }; // https://datatracker.ietf.org/doc/html/rfc1995#section-3 @@ -207,17 +210,17 @@ where }; if q.qtype() == Rtype::IXFR && ixfr_query_serial.is_none() { - return Self::log_and_break( - &q, - req, - msg, - OptRcode::FORMERR, - "IXFR request lacks authority section SOA", + warn!( + "{} for {} from {} refused: IXFR request lacks authority section SOA", + q.qtype(), + q.qname(), + req.client_addr() ); + return Err(OptRcode::FORMERR); } // Is transfer allowed for the requested zone for this requestor? - let res = xfr_data_provider + let (zone, diffs) = xfr_data_provider .request(req, ixfr_query_serial) .await .map_err(|err| match err { @@ -226,7 +229,13 @@ where // 2.2.1 Header Values // "If a server is not authoritative for the queried // zone, the server SHOULD set the value to NotAuth(9)" - ("unknown zone", OptRcode::NOTAUTH) + debug!( + "{} for {} from {} refused: unknown zone", + q.qtype(), + q.qname(), + req.client_addr() + ); + OptRcode::NOTAUTH } XfrDataProviderError::TemporarilyUnavailable => { @@ -235,33 +244,38 @@ where // SERVFAIL is the appropriate response, not NOTAUTH, as // we know we are supposed to be authoritative for the // zone but we just don't have the data right now. - ("zone not currently available", OptRcode::SERVFAIL) + warn!( + "{} for {} from {} refused: zone not currently available", + q.qtype(), + q.qname(), + req.client_addr() + ); + OptRcode::SERVFAIL } XfrDataProviderError::Refused => { - ("access denied", OptRcode::REFUSED) + warn!( + "{} for {} from {} refused: access denied", + q.qtype(), + q.qname(), + req.client_addr() + ); + OptRcode::REFUSED } - }) - .map_err(|(reason, rcode)| { - Self::log_and_break(&q, req, msg, rcode, reason) - }); - - let Ok((zone, diffs)) = res else { - return res.unwrap_err(); - }; + })?; // Read the zone SOA RR let read = zone.read(); let Ok(zone_soa_answer) = Self::read_soa(&read, q.qname().to_name()).await else { - return Self::log_and_break( - &q, - req, - msg, - OptRcode::SERVFAIL, - "name is outside the zone", + debug!( + "{} for {} from {} refused: name is outside the zone", + q.qtype(), + q.qname(), + req.client_addr() ); + return Err(OptRcode::SERVFAIL); }; match q.qtype() { @@ -277,13 +291,17 @@ where // Therefore, this document does not update RFC 1035 in // this respect: AXFR sessions over UDP transport are not // defined." - Self::log_and_break( - &q, - req, - msg, - OptRcode::NOTIMP, - "AXFR not suppored over UDP", - ) + warn!( + "{} for {} from {} refused: AXFR not supported over UDP", + q.qtype(), + q.qname(), + req.client_addr() + ); + let response = mk_error_response(msg, OptRcode::NOTIMP); + let res = Ok(CallResult::new(response)); + Ok(ControlFlow::Break(MiddlewareStream::Map(once(ready( + res, + ))))) } Rtype::AXFR | Rtype::IXFR if diffs.is_empty() => { @@ -296,8 +314,9 @@ where // behavior is the same as an AXFR response except the // query type is IXFR." info!( - "IXFR for {} from {}: diffs not available, falling back to AXFR", + "IXFR for {} (serial {} from {}: diffs not available, falling back to AXFR", q.qname(), + ixfr_query_serial.unwrap(), req.client_addr() ); } else { @@ -315,12 +334,9 @@ where &zone_soa_answer, read, ) - .await - .unwrap_or_else(|rcode| { - Self::to_stream(mk_error_response(msg, rcode)) - }); + .await?; - ControlFlow::Break(stream) + Ok(ControlFlow::Break(stream)) } Rtype::IXFR => { @@ -347,42 +363,15 @@ where &zone_soa_answer, diffs, ) - .await - .unwrap_or_else(|rcode| { - Self::to_stream(mk_error_response(msg, rcode)) - }); + .await?; - ControlFlow::Break(stream) + Ok(ControlFlow::Break(stream)) } - _ => ControlFlow::Continue(()), + _ => Ok(ControlFlow::Continue(())), } } - /// Log a message and break with an DNS error response stream. - #[allow(clippy::type_complexity)] - fn log_and_break( - q: &Question::Range<'_>>>, - req: &Request, - msg: &Message, - rcode: OptRcode, - reason: &'static str, - ) -> ControlFlow< - XfrMiddlewareStream< - NextSvc::Future, - NextSvc::Stream, - ::Item, - >, - > { - warn!( - "{} for {} from {} refused: {reason}", - q.qtype(), - q.qname(), - req.client_addr() - ); - ControlFlow::Break(Self::to_stream(mk_error_response(msg, rcode))) - } - /// Generate and send an AXFR response for a given request and zone. #[allow(clippy::too_many_arguments)] async fn send_axfr_response( @@ -703,21 +692,13 @@ where // Errata https://www.rfc-editor.org/errata/eid3196 points out that // this is NOT "just as in AXFR" as AXFR does not do that. if query_serial >= zone_serial { + trace!("IXFR finished because query_serial >= zone_serial"); let builder = mk_builder_for_target(); let response = zone_soa_answer.to_message(msg, builder); - trace!("IXFR finished because query_serial >= zone_serial"); - return Ok(Self::to_stream(response)); + let res = Ok(CallResult::new(response)); + return Ok(MiddlewareStream::Map(once(ready(res)))); } - // Get the necessary diffs, if available - // let start_serial = query_serial; - // let end_serial = zone_serial; - // let diffs = zone_info.diffs_for_range(start_serial, end_serial).await; - // if diffs.is_empty() { - // trace!("No diff available for IXFR"); - // return IxfrResult::FallbackToAxfr; - // }; - // TODO: Add something like the Bind `max-ixfr-ratio` option that // "sets the size threshold (expressed as a percentage of the size of // the full zone) beyond which named chooses to use an AXFR response @@ -908,17 +889,6 @@ where sender.send(Ok(call_result)).unwrap(); // TODO: Handle this Result } - fn to_stream( - response: AdditionalBuilder>, - ) -> XfrMiddlewareStream< - NextSvc::Future, - NextSvc::Stream, - ::Item, - > { - let res = Ok(CallResult::new(response)); - MiddlewareStream::Map(once(ready(res))) - } - #[allow(clippy::borrowed_box)] async fn read_soa( read: &Box, @@ -1007,12 +977,20 @@ where ) .await { - ControlFlow::Continue(()) => { + Ok(ControlFlow::Continue(())) => { let request = request.with_new_metadata(()); let stream = next_svc.call(request).await; MiddlewareStream::IdentityStream(stream) } - ControlFlow::Break(stream) => stream, + + Ok(ControlFlow::Break(stream)) => stream, + + Err(rcode) => { + let response = + mk_error_response(request.message(), rcode); + let res = Ok(CallResult::new(response)); + MiddlewareStream::Map(once(ready(res))) + } } }) } @@ -1426,7 +1404,7 @@ mod tests { let req = mk_axfr_request(zone.apex_name(), ()); - let res = do_preprocess(zone.clone(), &req).await; + let res = do_preprocess(zone.clone(), &req).await.unwrap(); let ControlFlow::Break(mut stream) = res else { panic!("AXFR failed"); @@ -1472,7 +1450,7 @@ mod tests { let req = mk_axfr_request(zone.apex_name(), ()); - let res = do_preprocess(zone.clone(), &req).await; + let res = do_preprocess(zone.clone(), &req).await.unwrap(); let ControlFlow::Break(mut stream) = res else { panic!("AXFR failed"); @@ -1553,7 +1531,7 @@ mod tests { let req = mk_udp_axfr_request(zone.apex_name(), ()); - let res = do_preprocess(zone, &req).await; + let res = do_preprocess(zone, &req).await.unwrap(); let ControlFlow::Break(mut stream) = res else { panic!("AXFR failed"); @@ -1595,7 +1573,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 // The following IXFR query let req = mk_udp_ixfr_request(zone.apex_name(), Serial(1), ()); - let res = do_preprocess(zone_with_diffs, &req).await; + let res = do_preprocess(zone_with_diffs, &req).await.unwrap(); let ControlFlow::Break(mut stream) = res else { panic!("IXFR failed"); @@ -1738,7 +1716,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 // The following IXFR query let req = mk_ixfr_request(zone.apex_name(), Serial(1), ()); - let res = do_preprocess(zone_with_diffs, &req).await; + let res = do_preprocess(zone_with_diffs, &req).await.unwrap(); let ControlFlow::Break(mut stream) = res else { panic!("IXFR failed"); @@ -1838,7 +1816,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 let req = mk_udp_ixfr_request(zone.apex_name(), Serial(0), ()); - let res = do_preprocess(zone.clone(), &req).await; + let res = do_preprocess(zone.clone(), &req).await.unwrap(); let ControlFlow::Break(mut stream) = res else { panic!("IXFR failed"); @@ -2067,12 +2045,15 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 async fn do_preprocess>( zone: XDP, req: &Request, Metadata>, - ) -> ControlFlow< - XfrMiddlewareStream< - ::Future, - ::Stream, - <::Stream as Stream>::Item, + ) -> Result< + ControlFlow< + XfrMiddlewareStream< + ::Future, + ::Stream, + <::Stream as Stream>::Item, + >, >, + OptRcode, > where XDP::Diff: Debug + 'static, diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 70a006b2b..62d90e7fd 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -337,7 +337,7 @@ impl ZoneUpdater { /// Has zone updating finished? /// /// If true, further calls to [`apply()`] will fail. - /// + /// /// [`apply()`]: Self::apply pub fn is_finished(&self) -> bool { self.state == ZoneUpdaterState::Finished From 08e969edbeee5ea56c437b3002a5730016b3ca7c Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 25 Sep 2024 09:16:16 +0200 Subject: [PATCH 278/333] Review feedback: remove unnecessary leading underscores in trait fn variable names. --- src/net/server/batcher.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/net/server/batcher.rs b/src/net/server/batcher.rs index fff84f02c..78cfa8928 100644 --- a/src/net/server/batcher.rs +++ b/src/net/server/batcher.rs @@ -142,9 +142,9 @@ where /// Do something with the completed message. #[allow(clippy::result_unit_err)] fn batch_ready( - _state: &T, - _answer: AnswerBuilder>, - _finished: bool, + state: &T, + answer: AnswerBuilder>, + finished: bool, ) -> Result<(), Self::Error>; } From 14c960dc8c7d2bc338ee6f974af64880762a15a7 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 25 Sep 2024 13:59:59 +0200 Subject: [PATCH 279/333] - More unwrap/error handling. - Actually cause SERVFAIL back to the client on fatal error. - Extract AXFR inner async tasks to own types. - Improved variable names in some cases. - Added `AnswerContent::first()` for less noisy uwrapping of data in XfrMiddlewareSvc. --- src/net/server/middleware/xfr.rs | 570 +++++++++++++++++++------------ src/zonetree/answer.rs | 32 +- 2 files changed, 382 insertions(+), 220 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 35ac744b4..c72fb5fd8 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -52,7 +52,9 @@ use bytes::Bytes; use futures_util::stream::{once, Once, Stream}; use futures_util::{pin_mut, StreamExt}; use octseq::Octets; -use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; +use tokio::sync::mpsc::{ + unbounded_channel, Receiver, Sender, UnboundedSender, +}; use tokio::sync::Semaphore; use tokio_stream::wrappers::UnboundedReceiverStream; use tracing::{debug, error, info, trace, warn}; @@ -63,7 +65,8 @@ use crate::base::message_builder::{ }; use crate::base::wire::Composer; use crate::base::{ - Message, Name, ParsedName, Question, Rtype, Serial, StreamTarget, ToName, + Message, Name, ParsedName, Question, RecordData, Rtype, Serial, + StreamTarget, ToName, }; use crate::net::server::batcher::{ CallbackBatcher, Callbacks, ResourceRecordBatcher, @@ -316,7 +319,7 @@ where info!( "IXFR for {} (serial {} from {}: diffs not available, falling back to AXFR", q.qname(), - ixfr_query_serial.unwrap(), + ixfr_query_serial.unwrap(), // SAFETY: Always Some() if IXFR req.client_addr() ); } else { @@ -326,7 +329,7 @@ where req.client_addr() ); } - let stream = Self::send_axfr_response( + let stream = Self::respond_to_axfr_query( zone_walking_semaphore, batcher_semaphore, req, @@ -340,6 +343,7 @@ where } Rtype::IXFR => { + // SAFETY: Always Some() if IXFR let ixfr_query_serial = ixfr_query_serial.unwrap(); info!( "IXFR for {} (serial {ixfr_query_serial}) from {}", @@ -356,7 +360,7 @@ where // the query is responded to with a single SOA record of // the server's current version to inform the client that a // TCP query should be initiated." - let stream = Self::send_ixfr_response( + let stream = Self::respond_to_ixfr_query( batcher_semaphore.clone(), req, ixfr_query_serial, @@ -374,7 +378,7 @@ where /// Generate and send an AXFR response for a given request and zone. #[allow(clippy::too_many_arguments)] - async fn send_axfr_response( + async fn respond_to_axfr_query( zone_walk_semaphore: Arc, batcher_semaphore: Arc, req: &Request, @@ -389,8 +393,6 @@ where >, OptRcode, > { - let msg = req.message(); - let AnswerContent::Data(zone_soa_rrset) = zone_soa_answer.content().clone() else { @@ -401,6 +403,8 @@ where return Err(OptRcode::SERVFAIL); }; + let soft_byte_limit = Self::calc_msg_bytes_available(req); + // TODO // let compatibility_mode = xfr_config.compatibility_mode // == CompatibilityMode::BackwardCompatible; @@ -435,13 +439,13 @@ where // Create a stream that will be immediately returned to the caller. // Async tasks will then push DNS response messages into the stream as // they become available. - let (sender, receiver) = unbounded_channel(); + let (response_tx, receiver) = unbounded_channel(); let stream = UnboundedReceiverStream::new(receiver); // Create a bounded queue for passing RRsets found during zone walking // to a task which will batch the RRs together before pushing them // into the result stream. - let (batcher_tx, mut batcher_rx) = + let (batcher_tx, batcher_rx) = tokio::sync::mpsc::channel::<(StoredName, SharedRrset)>(100); let must_fit_in_single_message = @@ -452,9 +456,9 @@ where // responses. The transport should modify its behaviour to account // for the potentially slow and long running nature of a // transaction. - Self::add_to_stream( + add_to_stream( CallResult::feedback_only(ServiceFeedback::BeginTransaction), - &sender, + &response_tx, ); } @@ -467,179 +471,73 @@ where return Err(OptRcode::SERVFAIL); } + let msg = req.message().clone(); + // Stream the remaining non-SOA zone RRsets in the background to the // batcher. - tokio::spawn(async move { - // Limit the number of concurrently running XFR related zone - // walking operations. - if zone_walk_semaphore.acquire().await.is_err() { - error!("Internal error: Failed to acquire XFR zone walking semaphore"); - return; - } - - let cloned_batcher_tx = batcher_tx.clone(); - let op = - Box::new(move |owner: StoredName, rrset: &SharedRrset| { - if rrset.rtype() != Rtype::SOA { - let _ = cloned_batcher_tx - .blocking_send((owner.clone(), rrset.clone())); - // If the blocking send fails it means that the - // batcher is no longer available. This can happen if - // it was no longer able to pass messages back to the - // underlying transport, which can happen if the - // client closed the connection. We don't log this - // because we can't stop the tree walk and so will - // keep hitting this error until the tree walk is - // complete, causing a lot of noise if we were to log - // this. - } - }); + let zone_funneler = ZoneFunneler::new( + read, + qname, + zone_soa_rrset, + batcher_tx, + zone_walk_semaphore, + ); - // Walk the zone tree, invoking our operation for each leaf. - match read.is_async() { - true => { - read.walk_async(op).await; - if let Err(err) = - batcher_tx.send((qname, zone_soa_rrset)).await - { - error!("Internal error: Failed to send final AXFR SOA to batcher: {err}"); - } - } - false => { - tokio::task::spawn_blocking(move || { - read.walk(op); - if let Err(err) = - batcher_tx.blocking_send((qname, zone_soa_rrset)) - { - error!("Internal error: Failed to send final AXFR SOA to batcher: {err}"); - } - }); - } - } - }); + let batching_responder = BatchingRrResponder::new( + req.message().clone(), + zone_soa_answer.clone(), + batcher_rx, + response_tx.clone(), + compatibility_mode, + soft_byte_limit, + must_fit_in_single_message, + batcher_semaphore, + ); - // Combine RRsets enumerated by zone walking as many as possible per - // DNS response message and pass the created messages downstream to - // the caller. - let msg = msg.clone(); - let soft_byte_limit = Self::calc_msg_bytes_available(req); - let zone_soa_answer = zone_soa_answer.clone(); + let cloned_msg = msg.clone(); + let cloned_response_tx = response_tx.clone(); + // Start the funneler. It will walk the zone and send all of the RRs + // one at a time to the batching responder. tokio::spawn(async move { - // Limit the number of concurrently running XFR batching - // operations. - if batcher_semaphore.acquire().await.is_err() { - error!( - "Internal error: Failed to acquire XFR batcher semaphore" - ); - return; + if let Err(rcode) = zone_funneler.run().await { + let resp = mk_error_response(&cloned_msg, rcode); + add_to_stream(CallResult::new(resp), &cloned_response_tx); } + }); - let Ok(qclass) = msg.sole_question().map(|q| q.qclass()) else { - unreachable!(); - }; - - // Note: NSD apparently uses name compresson on AXFR responses - // because AXFR responses they typically contain lots of - // alphabetically ordered duplicate names which compress well. NSD - // limits AXFR responses to 16,383 RRs because DNS name - // compression uses a 14-bit offset (2^14-1=16383) from the start - // of the message to the first occurence of a name instead of - // repeating the name, and name compression is less effective - // over 16383 bytes. (Credit: Wouter Wijngaards) - // - // TODO: Once we start supporting name compression in responses decide - // if we want to behave the same way. - - let hard_rr_limit = match compatibility_mode { - true => Some(1), - false => None, - }; - - let mut batcher = XfrRrBatcher::build( - msg.clone(), - sender.clone(), - Some(soft_byte_limit), - hard_rr_limit, - must_fit_in_single_message, - ); - - while let Some((owner, rrset)) = batcher_rx.recv().await { - for rr in rrset.data() { - if let Err(err) = - batcher.push((owner.clone(), qclass, rrset.ttl(), rr)) - { - match err { - BatchReadyError::MustFitInSingleMessage => { - // https://datatracker.ietf.org/doc/html/rfc1995#section-2 - // 2. Brief Description of the Protocol - // .. - // "If the UDP reply does not fit, the - // query is responded to with a single SOA - // record of the server's current version - // to inform the client that a TCP query - // should be initiated." - debug_assert!(must_fit_in_single_message); - let builder = mk_builder_for_target(); - let resp = - zone_soa_answer.to_message(&msg, builder); - debug!("IXFR aborted because response does not fit in a single UDP reply"); - Self::add_to_stream( - CallResult::new(resp), - &sender, - ); - batcher_rx.close(); - return; - } - - BatchReadyError::PushError(err) => { - error!( - "Internal error: Failed to send RR to batcher: {err}" - ); - let resp = mk_error_response( - &msg, - OptRcode::SERVFAIL, - ); - Self::add_to_stream( - CallResult::new(resp), - &sender, - ); - batcher_rx.close(); - return; - } - - BatchReadyError::SendError => { - debug!("Batcher was unable to send completed batch, presumably because the receiver was dropped"); - batcher_rx.close(); - return; - } - } - } + // Start the batching responder. It will receive RRs from the funneler + // and push them in batches into the response stream. + tokio::spawn(async move { + match batching_responder.run().await { + Ok(()) => { + trace!("Ending transaction"); + add_to_stream( + CallResult::feedback_only( + ServiceFeedback::EndTransaction, + ), + &response_tx, + ); } - } - batcher.finish().unwrap(); // TODO - - if !must_fit_in_single_message { - trace!("Finishing transaction"); - Self::add_to_stream( - CallResult::feedback_only( - ServiceFeedback::EndTransaction, - ), - &sender, - ); + Err(rcode) => { + let resp = mk_error_response(&msg, rcode); + add_to_stream(CallResult::new(resp), &response_tx); + } } - - batcher_rx.close(); }); + // If either the funneler or batcher responder terminate then so will + // the other as they each own half of a send <-> receive channel and + // abort if the other side of the channel is gone. + Ok(MiddlewareStream::Result(stream)) } // Generate and send an IXFR response for the given request and zone // diffs. #[allow(clippy::too_many_arguments)] - async fn send_ixfr_response( + async fn respond_to_ixfr_query( batcher_semaphore: Arc, req: &Request, query_serial: Serial, @@ -658,23 +556,12 @@ where { let msg = req.message(); - let AnswerContent::Data(zone_soa_rrset) = - zone_soa_answer.content().clone() + let Some((soa_ttl, ZoneRecordData::Soa(soa))) = + zone_soa_answer.content().first() else { return Err(OptRcode::SERVFAIL); }; - if zone_soa_rrset.data().len() != 1 { - return Err(OptRcode::SERVFAIL); - } - - let first_rr = zone_soa_rrset.first().unwrap(); - let ZoneRecordData::Soa(soa) = first_rr.data() else { - return Err(OptRcode::SERVFAIL); - }; - - let zone_serial = soa.serial(); - // Note: Unlike RFC 5936 for AXFR, neither RFC 1995 nor RFC 9103 say // anything about whether an IXFR response can consist of more than // one response message, but given the 2^16 byte maximum response size @@ -691,7 +578,7 @@ where // ^^^^^^^^^^^^^^^ // Errata https://www.rfc-editor.org/errata/eid3196 points out that // this is NOT "just as in AXFR" as AXFR does not do that. - if query_serial >= zone_serial { + if query_serial >= soa.serial() { trace!("IXFR finished because query_serial >= zone_serial"); let builder = mk_builder_for_target(); let response = zone_soa_answer.to_message(msg, builder); @@ -718,7 +605,7 @@ where // responses. The transport should modify its behaviour to account // for the potentially slow and long running nature of a // transaction. - Self::add_to_stream( + add_to_stream( CallResult::feedback_only(ServiceFeedback::BeginTransaction), &sender, ); @@ -735,6 +622,8 @@ where error!( "Internal error: Failed to acquire XFR batcher semaphore" ); + let resp = mk_error_response(&msg, OptRcode::SERVFAIL); + add_to_stream(CallResult::new(resp), &sender); return; } @@ -779,12 +668,7 @@ where ); batcher - .push(( - owner.clone(), - qclass, - zone_soa_rrset.ttl(), - &zone_soa_rrset.data()[0], - )) + .push((owner.clone(), qclass, soa_ttl, &soa)) .unwrap(); // TODO for diff in diffs { @@ -857,20 +741,13 @@ where } } - batcher - .push(( - owner, - qclass, - zone_soa_rrset.ttl(), - &zone_soa_rrset.data()[0], - )) - .unwrap(); // TODO + batcher.push((owner, qclass, soa_ttl, soa)).unwrap(); // TODO batcher.finish().unwrap(); // TODO if !must_fit_in_single_message { trace!("Ending transaction"); - Self::add_to_stream( + add_to_stream( CallResult::feedback_only( ServiceFeedback::EndTransaction, ), @@ -882,13 +759,6 @@ where Ok(MiddlewareStream::Result(stream)) } - fn add_to_stream( - call_result: CallResult, - sender: &UnboundedSender>, - ) { - sender.send(Ok(call_result)).unwrap(); // TODO: Handle this Result - } - #[allow(clippy::borrowed_box)] async fn read_soa( read: &Box, @@ -1184,6 +1054,41 @@ impl XfrDataProvider for ZoneTree { } } +//------------ BatchReadyError ------------------------------------------------ + +#[derive(Clone, Copy, Debug)] +pub enum BatchReadyError { + PushError(PushError), + + SendError, + + MustFitInSingleMessage, +} + +//--- Display + +impl std::fmt::Display for BatchReadyError { + fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { + match self { + BatchReadyError::MustFitInSingleMessage => { + f.write_str("MustFitInSingleMessage") + } + BatchReadyError::PushError(err) => { + f.write_fmt(format_args!("PushError: {err}")) + } + BatchReadyError::SendError => f.write_str("SendError"), + } + } +} + +//--- From + +impl From for BatchReadyError { + fn from(err: PushError) -> Self { + Self::PushError(err) + } +} + //------------ XfrRrBatcher --------------------------------------------------- pub struct XfrRrBatcher { @@ -1270,21 +1175,6 @@ where //--- Callbacks -#[derive(Clone, Copy, Debug)] -pub enum BatchReadyError { - PushError(PushError), - - SendError, - - MustFitInSingleMessage, -} - -impl From for BatchReadyError { - fn from(err: PushError) -> Self { - Self::PushError(err) - } -} - impl Callbacks> for XfrRrBatcher @@ -1369,6 +1259,250 @@ impl CallbackState { } } +//------------ ZoneFunneler --------------------------------------------------- + +struct ZoneFunneler { + read: Box, + qname: StoredName, + zone_soa_rrset: SharedRrset, + batcher_tx: Sender<(Name, SharedRrset)>, + zone_walk_semaphore: Arc, +} + +impl ZoneFunneler { + fn new( + read: Box, + qname: StoredName, + zone_soa_rrset: SharedRrset, + batcher_tx: Sender<(Name, SharedRrset)>, + zone_walk_semaphore: Arc, + ) -> Self { + Self { + read, + qname, + zone_soa_rrset, + batcher_tx, + zone_walk_semaphore, + } + } + + async fn run(self) -> Result<(), OptRcode> { + // Limit the number of concurrently running XFR related zone walking + // operations. + if self.zone_walk_semaphore.acquire().await.is_err() { + error!("Internal error: Failed to acquire XFR zone walking semaphore"); + return Err(OptRcode::SERVFAIL); + } + + let cloned_batcher_tx = self.batcher_tx.clone(); + let op = Box::new(move |owner: StoredName, rrset: &SharedRrset| { + if rrset.rtype() != Rtype::SOA { + let _ = cloned_batcher_tx + .blocking_send((owner.clone(), rrset.clone())); + // If the blocking send fails it means that the + // batcher is no longer available. This can happen if + // it was no longer able to pass messages back to the + // underlying transport, which can happen if the + // client closed the connection. We don't log this + // because we can't stop the tree walk and so will + // keep hitting this error until the tree walk is + // complete, causing a lot of noise if we were to log + // this. + } + }); + + // Walk the zone tree, invoking our operation for each leaf. + match self.read.is_async() { + true => { + self.read.walk_async(op).await; + if let Err(err) = self + .batcher_tx + .send((self.qname, self.zone_soa_rrset)) + .await + { + error!("Internal error: Failed to send final AXFR SOA to batcher: {err}"); + return Err(OptRcode::SERVFAIL); + } + } + false => { + tokio::task::spawn_blocking(move || { + self.read.walk(op); + if let Err(err) = self + .batcher_tx + .blocking_send((self.qname, self.zone_soa_rrset)) + { + error!("Internal error: Failed to send final AXFR SOA to batcher: {err}"); + // Note: The lack of the final SOA will be detected by the batcher. + } + }); + } + } + + Ok(()) + } +} + +//------------ BatchingRrResponder --------------------------------------------- + +struct BatchingRrResponder { + msg: Arc>, + zone_soa_answer: Answer, + batcher_rx: Receiver<(Name, SharedRrset)>, + response_tx: UnboundedSender>, + compatibility_mode: bool, + soft_byte_limit: usize, + must_fit_in_single_message: bool, + batcher_semaphore: Arc, +} + +impl BatchingRrResponder +where + RequestOctets: Octets + Send + Sync + 'static + Unpin, + Target: Composer + Default + Send + Sync + 'static, +{ + #[allow(clippy::too_many_arguments)] + fn new( + msg: Arc>, + zone_soa_answer: Answer, + batcher_rx: Receiver<(Name, SharedRrset)>, + response_tx: UnboundedSender>, + compatibility_mode: bool, + soft_byte_limit: usize, + must_fit_in_single_message: bool, + batcher_semaphore: Arc, + ) -> Self { + Self { + msg, + zone_soa_answer, + batcher_rx, + response_tx, + compatibility_mode, + soft_byte_limit, + must_fit_in_single_message, + batcher_semaphore, + } + } + + async fn run(mut self) -> Result<(), OptRcode> { + // Limit the number of concurrently running XFR batching + // operations. + if self.batcher_semaphore.acquire().await.is_err() { + error!("Internal error: Failed to acquire XFR batcher semaphore"); + return Err(OptRcode::SERVFAIL); + } + + // SAFETY: msg.sole_question() was already checked in + // get_relevant_question(). + let qclass = self.msg.sole_question().unwrap().qclass(); + + // Note: NSD apparently uses name compresson on AXFR responses + // because AXFR responses they typically contain lots of + // alphabetically ordered duplicate names which compress well. NSD + // limits AXFR responses to 16,383 RRs because DNS name + // compression uses a 14-bit offset (2^14-1=16383) from the start + // of the message to the first occurence of a name instead of + // repeating the name, and name compression is less effective + // over 16383 bytes. (Credit: Wouter Wijngaards) + // + // TODO: Once we start supporting name compression in responses decide + // if we want to behave the same way. + + let hard_rr_limit = match self.compatibility_mode { + true => Some(1), + false => None, + }; + + let mut batcher = XfrRrBatcher::build( + self.msg.clone(), + self.response_tx.clone(), + Some(self.soft_byte_limit), + hard_rr_limit, + self.must_fit_in_single_message, + ); + + let mut last_rr_rtype = None; + + while let Some((owner, rrset)) = self.batcher_rx.recv().await { + for rr in rrset.data() { + last_rr_rtype = Some(rr.rtype()); + + if let Err(err) = + batcher.push((owner.clone(), qclass, rrset.ttl(), rr)) + { + match err { + BatchReadyError::MustFitInSingleMessage => { + // https://datatracker.ietf.org/doc/html/rfc1995#section-2 + // 2. Brief Description of the Protocol + // .. + // "If the UDP reply does not fit, the + // query is responded to with a single SOA + // record of the server's current version + // to inform the client that a TCP query + // should be initiated." + debug_assert!(self.must_fit_in_single_message); + debug!("Responding to IXFR with single SOA because response does not fit in a single UDP reply"); + + let resp = self.zone_soa_answer.to_message( + &self.msg, + mk_builder_for_target(), + ); + + add_to_stream( + CallResult::new(resp), + &self.response_tx, + ); + + return Ok(()); + } + + BatchReadyError::PushError(err) => { + error!("Internal error: Failed to send RR to batcher: {err}"); + return Err(OptRcode::SERVFAIL); + } + + BatchReadyError::SendError => { + debug!("Batcher was unable to send completed batch. Was the receiver dropped?"); + return Err(OptRcode::SERVFAIL); + } + } + } + } + } + + if let Err(err) = batcher.finish() { + debug!("Batcher was unable to finish: {err}"); + return Err(OptRcode::SERVFAIL); + } + + if last_rr_rtype != Some(Rtype::SOA) { + error!( + "Internal error: Last RR was {}, expected SOA", + last_rr_rtype.unwrap() + ); + return Err(OptRcode::SERVFAIL); + } + + Ok(()) + } +} + +//------------ add_to_stream() ------------------------------------------------ +fn add_to_stream( + call_result: CallResult, + response_tx: &UnboundedSender>, +) { + if response_tx.send(Ok(call_result)).is_err() { + // We failed to write the message into the response stream. This + // shouldn't happen. We can't now return an error to the client + // because that would require writing to the response stream as + // well. We don't want to panic and take down the entire + // application, so instead just log. + error!("Failed to send DNS message to the internal response stream"); + } +} + +//------------ Tests ---------------------------------------------------------- + #[cfg(test)] mod tests { use core::str::FromStr; diff --git a/src/zonetree/answer.rs b/src/zonetree/answer.rs index 46e0dc659..f02e1df5e 100644 --- a/src/zonetree/answer.rs +++ b/src/zonetree/answer.rs @@ -5,10 +5,10 @@ use octseq::Octets; use crate::base::iana::Rcode; use crate::base::message_builder::AdditionalBuilder; use crate::base::wire::Composer; -use crate::base::Message; use crate::base::MessageBuilder; +use crate::base::{Message, Ttl}; -use super::types::StoredName; +use super::types::{StoredName, StoredRecordData}; use super::{SharedRr, SharedRrset}; //------------ Answer -------------------------------------------------------- @@ -202,6 +202,34 @@ pub enum AnswerContent { NoData, } +impl AnswerContent { + /// Gets the first record TTL and data, if any. + /// + /// This can be used to get both the data as a specific variant, and the + /// associated TTL, in a single step: + /// + /// ``` + /// # let answer = Answer::new(Rcode::NOERROR); + /// let Some((soa_ttl, ZoneRecordData::Soa(soa))) = + /// some_answer.content().first() + /// else { + /// // ... + /// }; + /// ``` + pub fn first(&self) -> Option<(Ttl, StoredRecordData)> { + match self { + AnswerContent::Data(shared_rrset) => shared_rrset + .data() + .first() + .map(|data| (shared_rrset.ttl(), data.clone())), + AnswerContent::Cname(shared_rr) => { + Some((shared_rr.ttl(), shared_rr.data().clone())) + } + AnswerContent::NoData => None, + } + } +} + //------------ AnswerAuthority ----------------------------------------------- /// The authority section of a query answer. From 1a90d6fb7c5446fe9f4755e31a47fdd8d4360220 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Mon, 23 Sep 2024 22:54:17 +0200 Subject: [PATCH 280/333] Update changelog. --- Changelog.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/Changelog.md b/Changelog.md index 279ac8713..858b65041 100644 --- a/Changelog.md +++ b/Changelog.md @@ -56,6 +56,16 @@ Unstable features errors instead of logging and ignoring them. ([#362]) * Allow both glue (A/AAAA) and zone cuts at the same owner when zone parsing. ([#363]) +* `unstable-zonetree`: + * Added `ZoneUpdate`. ([#375]) + * Added `ZoneUpdater`, `ZoneDiffBuilder` and `ZoneDiff` and improved + `ZoneUpdate`. ([#376]) + * Removed / renamed references to `clean` in `zonetree::in_memory` to + `remove`. ([#376]) + * Altered the logic in `Versioned::remove_all()` (formerly + `Versioned::clean()`) as it made destructive changes to the zone that + would have impacted readers of the current zone version while the new zone + version was being created. ([#376]) Other changes @@ -77,6 +87,7 @@ Other changes [#373]: https://github.com/NLnetLabs/domain/pull/373 [#374]: https://github.com/NLnetLabs/domain/pull/374 [#375]: https://github.com/NLnetLabs/domain/pull/375 +[#376]: https://github.com/NLnetLabs/domain/pull/376 [#377]: https://github.com/NLnetLabs/domain/pull/377 [#380]: https://github.com/NLnetLabs/domain/pull/380 [@dklbreitling]: https://github.com/dklbreitling From c923bb4b24b0aa717998ddc0681c0ede486fade9 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 24 Sep 2024 20:43:53 +0200 Subject: [PATCH 281/333] Add RFC 1996 NOTIFY support. (#382) Add RFC 1996 "slave" side minimalistic NOTIFY request support, delegating the actual work to a caller supplied helper that implements a new Notifiable trait. --- src/net/server/middleware/mod.rs | 19 +++++++++++++++++++ src/net/server/middleware/notify.rs | 4 ++++ 2 files changed, 23 insertions(+) diff --git a/src/net/server/middleware/mod.rs b/src/net/server/middleware/mod.rs index b44bcfcb4..f6c3b0a91 100644 --- a/src/net/server/middleware/mod.rs +++ b/src/net/server/middleware/mod.rs @@ -12,11 +12,30 @@ //! post-processing the resulting responses and propagating them back down //! through the layers to the server. //! +//! # Middleware layering strategies +//! +//! The simplest strategy for using middleware is to use a single layered +//! stack of middleware for all incoming requests. +//! +//! If however some middleware layers impose a disproportionately high cost on +//! request processing for request types that occur rarely, an alternate +//! strategy could be to add a middleware layer that routes requests to the +//! appropriate middleware "chain" based on some property or properties of the +//! request. Rather than a liner processing "chain" one would then have a tree +//! like processing path. +//! +//! Another option that may be suitable in some cases could be to use separate +//! server instances listening on separate ports or interfaces, each with +//! their own differing middleware "chains". +//! +//! # Middleware-to-middleware communication +//! //! If needed middleware services can pass service specific data to upstream //! services for consumption, via the `RequestMeta` custom data support of //! the [`Service`] trait. An example of this can be seen in the //! [`TsigMiddlewareSvc`][tsig::TsigMiddlewareSvc]. //! +//! //! Currently the following middleware are available: //! //! [`Service`]: crate::net::server::service::Service diff --git a/src/net/server/middleware/notify.rs b/src/net/server/middleware/notify.rs index 77d0f4fb5..27f0d5800 100644 --- a/src/net/server/middleware/notify.rs +++ b/src/net/server/middleware/notify.rs @@ -248,6 +248,10 @@ where fn get_relevant_question( msg: &Message, ) -> Option>>> { + // NOTE: If this middleware is used with a server that primarily + // receives Opcode::QUERY it would be more efficient to place a + // "router" middleware in front of this middleware that routes + // requests by Opcode to separate dedicated middleware "chains". if Opcode::NOTIFY == msg.header().opcode() { if let Some(q) = msg.first_question() { if q.qtype() == Rtype::SOA { From e383cc5d2e8696c83bffb12b71316d5d3dd2220e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 25 Sep 2024 09:30:55 +0200 Subject: [PATCH 282/333] Update changelog. --- Changelog.md | 30 +++++++++++++++++------------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/Changelog.md b/Changelog.md index 858b65041..4db4400c3 100644 --- a/Changelog.md +++ b/Changelog.md @@ -27,9 +27,9 @@ Unstable features * New unstable feature `unstable-validator` that adds a DNSSEC validator. ([#328]) -* New unstable feature `unstable-xfr` that adds an XFR response - interpreter in `net::xfr` for iterating over XFR responses - as a sequence of high level `ZoneUpdate`s. ([#375]) +* New unstable feature `unstable-xfr` that adds `XfrResponseInterpreter` in + `net::xfr` for iterating over XFR responses as a sequence of high level + `ZoneUpdate`s. ([#375]) * `unstable-client-transport`: * Fixed an issue with slow responses in the `multi_stream` transport by not waiting in the first iteration if an @@ -46,26 +46,29 @@ Unstable features transport in `net::client:tsig`. ([#373]) * `unstable-server-transport` * Breaking changes to the `Service` and middleware traits. ([#369]) - * Added a TSIG request validating and response signing middleware in - `net::server::middleware::tsig`. ([#380]) - * The cookies middleware now allows requests with invalid cookies to - proceed if they are authenticated or not required to authenticate. ([#336]) + * Added `TsigMiddlewareSvc` request validating and response signing + middleware in `net::server::middleware::tsig`. ([#380]) + * Added `NotifyMiddlewareSvc` in `net::server::middleware::notify` to parse + and acknowledge SOA NOTIFY requests, for use by secondary nameservers to + detect outdated zones compared to the primary. ([#382]) + * `CookiesMiddlewareSvc` now allows requests with invalid cookies to proceed + if they are authenticated or not required to authenticate. ([#336]) * Added an `enabled` flag to `CookiesMiddlewareSvc`. ([#369]) +* `unstable-zonetree`: + * Added `ZoneUpdate`. ([#375]) + * Added `ZoneUpdater`, `ZoneDiffBuilder` and `ZoneDiff` and improved + `ZoneUpdate`. ([#376]) * Improved zonefile parsing error messages. ([#362]). * `TryFrom for Zonefile` now returns the set of errors instead of logging and ignoring them. ([#362]) * Allow both glue (A/AAAA) and zone cuts at the same owner when zone parsing. ([#363]) -* `unstable-zonetree`: - * Added `ZoneUpdate`. ([#375]) - * Added `ZoneUpdater`, `ZoneDiffBuilder` and `ZoneDiff` and improved - `ZoneUpdate`. ([#376]) - * Removed / renamed references to `clean` in `zonetree::in_memory` to - `remove`. ([#376]) * Altered the logic in `Versioned::remove_all()` (formerly `Versioned::clean()`) as it made destructive changes to the zone that would have impacted readers of the current zone version while the new zone version was being created. ([#376]) + * Removed / renamed references to `clean` in `zonetree::in_memory` to + `remove`. ([#376]) Other changes @@ -90,6 +93,7 @@ Other changes [#376]: https://github.com/NLnetLabs/domain/pull/376 [#377]: https://github.com/NLnetLabs/domain/pull/377 [#380]: https://github.com/NLnetLabs/domain/pull/380 +[#382]: https://github.com/NLnetLabs/domain/pull/382 [@dklbreitling]: https://github.com/dklbreitling ## 0.10.1 From e70f8ea6508177f2f1853bc2655f50bf492ccfdb Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 25 Sep 2024 21:39:42 +0200 Subject: [PATCH 283/333] Compilation fix for Rust 1.76.0. --- src/zonetree/traits.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index 376cc236e..2aa6619cf 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -275,7 +275,7 @@ pub trait ZoneDiff { Self: 'a; /// The type of [`Stream`] used to access the diff records. - type Stream<'a>: Stream> + type Stream<'a>: Stream> + Send where Self: 'a; From ab6a63411725a8af57aa3c1064cd15480c691def Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 00:23:12 +0200 Subject: [PATCH 284/333] Fix compilation bug with Rust 1.68.2 caused by an unnecessary HRTB (for<'a> ...) that Rust 1.69.0+ are fine with but 1.68.2 is not (perhaps due to https://github.com/rust-lang/rust/pull/103695/ which was fixed in Rust 1.69.0). Also reorder XfrMiddlewareSvc generic types to match the order used in other middleware services, i.e. add the extra type XDP at the end, not before RequestMeta. Also rename Metadata to RequestMeta as in other middleware services. (and fix a comment that is ahead of its time, there is no ZoneMaintainer yet in this branch) --- src/net/server/middleware/xfr.rs | 58 ++++++++++++++--------------- src/net/server/tests/integration.rs | 5 +-- 2 files changed, 31 insertions(+), 32 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index c72fb5fd8..04eb9133e 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -106,7 +106,7 @@ const MAX_TCP_MSG_BYTE_LEN: u16 = u16::MAX; /// /// [module documentation]: crate::net::server::middleware::xfr #[derive(Clone, Debug)] -pub struct XfrMiddlewareSvc { +pub struct XfrMiddlewareSvc { /// The upstream [`Service`] to pass requests to and receive responses /// from. next_svc: NextSvc, @@ -123,13 +123,13 @@ pub struct XfrMiddlewareSvc { /// may run concurrently. batcher_semaphore: Arc, - _phantom: PhantomData<(RequestOctets, Metadata)>, + _phantom: PhantomData<(RequestOctets, RequestMeta)>, } -impl - XfrMiddlewareSvc +impl + XfrMiddlewareSvc where - XDP: XfrDataProvider, + XDP: XfrDataProvider, { /// Creates a new instance of this middleware. /// @@ -158,8 +158,8 @@ where } } -impl - XfrMiddlewareSvc +impl + XfrMiddlewareSvc where RequestOctets: Octets + Send + Sync + 'static + Unpin, for<'a> ::Range<'a>: Send + Sync, @@ -167,7 +167,7 @@ where NextSvc::Future: Send + Sync + Unpin, NextSvc::Target: Composer + Default + Send + Sync, NextSvc::Stream: Send + Sync, - XDP: XfrDataProvider, + XDP: XfrDataProvider, XDP::Diff: Debug + 'static, for<'a> ::Stream<'a>: Send, { @@ -178,7 +178,7 @@ where pub async fn preprocess( zone_walking_semaphore: Arc, batcher_semaphore: Arc, - req: &Request, + req: &Request, xfr_data_provider: XDP, ) -> Result< ControlFlow< @@ -807,8 +807,9 @@ where //--- impl Service -impl Service - for XfrMiddlewareSvc +impl + Service + for XfrMiddlewareSvc where RequestOctets: Octets + Send + Sync + Unpin + 'static, for<'a> ::Range<'a>: Send + Sync, @@ -816,10 +817,9 @@ where NextSvc::Future: Send + Sync + Unpin, NextSvc::Target: Composer + Default + Send + Sync, NextSvc::Stream: Send + Sync, - XDP: XfrDataProvider + Clone + Sync + Send + 'static, + XDP: XfrDataProvider + Clone + Sync + Send + 'static, XDP::Diff: Debug + Sync, - for<'a> ::Stream<'a>: Send, - Metadata: Clone + Default + Sync + Send + 'static, + RequestMeta: Clone + Default + Sync + Send + 'static, { type Target = NextSvc::Target; type Stream = XfrMiddlewareStream< @@ -831,7 +831,7 @@ where fn call( &self, - request: Request, + request: Request, ) -> Self::Future { let request = request.clone(); let next_svc = self.next_svc.clone(); @@ -902,8 +902,8 @@ pub enum XfrDataProviderError { //------------ Transferable --------------------------------------------------- /// A provider of data needed for responding to XFR requests. -pub trait XfrDataProvider { - type Diff: ZoneDiff + Send; +pub trait XfrDataProvider { + type Diff: ZoneDiff + Send + Sync; /// Request data needed to respond to an XFR request. /// @@ -918,7 +918,7 @@ pub trait XfrDataProvider { #[allow(clippy::type_complexity)] fn request( &self, - req: &Request, + req: &Request, diff_from: Option, ) -> Pin< Box< @@ -938,16 +938,16 @@ pub trait XfrDataProvider { //--- impl XfrDataProvider for Deref -impl XfrDataProvider for U +impl XfrDataProvider for U where - T: XfrDataProvider + 'static, + T: XfrDataProvider + 'static, U: Deref, { type Diff = T::Diff; fn request( &self, - req: &Request, + req: &Request, diff_from: Option, ) -> Pin< Box< @@ -970,7 +970,7 @@ where //--- impl XfrDataProvider for Zone -impl XfrDataProvider for Zone { +impl XfrDataProvider for Zone { type Diff = EmptyZoneDiff; /// Request data needed to respond to an XFR request. @@ -981,7 +981,7 @@ impl XfrDataProvider for Zone { /// Returns Err if the requested zone is not this zone. fn request( &self, - req: &Request, + req: &Request, _diff_from: Option, ) -> Pin< Box< @@ -1013,7 +1013,7 @@ impl XfrDataProvider for Zone { //--- impl XfrDataProvider for ZoneTree -impl XfrDataProvider for ZoneTree { +impl XfrDataProvider for ZoneTree { type Diff = EmptyZoneDiff; /// Request data needed to respond to an XFR request. @@ -1024,7 +1024,7 @@ impl XfrDataProvider for ZoneTree { /// Returns Err if the requested zone is not this zone tree. fn request( &self, - req: &Request, + req: &Request, _diff_from: Option, ) -> Pin< Box< @@ -2057,7 +2057,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 async fn get_zone_soa(zone: &Zone) -> Soa> { let read = zone.read(); let zone_soa_answer = - XfrMiddlewareSvc::, TestNextSvc, Zone>::read_soa( + XfrMiddlewareSvc::, TestNextSvc, (), Zone>::read_soa( &read, zone.apex_name().to_owned(), ) @@ -2176,9 +2176,9 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 ) } - async fn do_preprocess>( + async fn do_preprocess>( zone: XDP, - req: &Request, Metadata>, + req: &Request, RequestMeta>, ) -> Result< ControlFlow< XfrMiddlewareStream< @@ -2193,7 +2193,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 XDP::Diff: Debug + 'static, for<'a> ::Stream<'a>: Send, { - XfrMiddlewareSvc::, TestNextSvc, XDP, Metadata>::preprocess( + XfrMiddlewareSvc::, TestNextSvc, RequestMeta, XDP>::preprocess( Arc::new(Semaphore::new(1)), Arc::new(Semaphore::new(1)), req, diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index 78ab2d9ca..db9c5f190 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -158,9 +158,8 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { let svc = EdnsMiddlewareSvc::new(svc).enable(server_config.edns_tcp_keepalive); - // 4. XFR(-in) middleware service (XFR-out is handled by the - // ZoneMaintainer). - let svc = XfrMiddlewareSvc::, _, _, Option>>::new( + // 4. XFR(-in) middleware service. + let svc = XfrMiddlewareSvc::, _, Option>, _>::new( svc, zones, 1, ); From 875e2d6e38fc249a32ac497104aa3dd89b0699a6 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 00:28:24 +0200 Subject: [PATCH 285/333] Fix RustDoc test compilation error. --- src/zonetree/answer.rs | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/zonetree/answer.rs b/src/zonetree/answer.rs index f02e1df5e..4c2cc0760 100644 --- a/src/zonetree/answer.rs +++ b/src/zonetree/answer.rs @@ -208,12 +208,15 @@ impl AnswerContent { /// This can be used to get both the data as a specific variant, and the /// associated TTL, in a single step: /// - /// ``` - /// # let answer = Answer::new(Rcode::NOERROR); + /// ```should_panic + /// # use domain::base::iana::Rcode; + /// # use domain::rdata::ZoneRecordData; + /// # use domain::zonetree::Answer; + /// # let some_answer = Answer::new(Rcode::NOERROR); /// let Some((soa_ttl, ZoneRecordData::Soa(soa))) = /// some_answer.content().first() /// else { - /// // ... + /// todo!(); /// }; /// ``` pub fn first(&self) -> Option<(Ttl, StoredRecordData)> { From 569d65d52cb2535ac243f6a07734defed318a191 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 00:29:55 +0200 Subject: [PATCH 286/333] Improve the last fix. --- src/zonetree/answer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/zonetree/answer.rs b/src/zonetree/answer.rs index 4c2cc0760..2c31927a2 100644 --- a/src/zonetree/answer.rs +++ b/src/zonetree/answer.rs @@ -216,7 +216,7 @@ impl AnswerContent { /// let Some((soa_ttl, ZoneRecordData::Soa(soa))) = /// some_answer.content().first() /// else { - /// todo!(); + /// panic!("some_answer is not a variant of AnswerContent that has data"); /// }; /// ``` pub fn first(&self) -> Option<(Ttl, StoredRecordData)> { From 4407c84d1a5fee52e667bf63610188d4b99d38c2 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 09:08:45 +0200 Subject: [PATCH 287/333] Code simplification with additional From impls. --- src/net/server/middleware/xfr.rs | 50 +++++++++++++------------------- src/net/server/service.rs | 18 ++++++++++++ 2 files changed, 38 insertions(+), 30 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 04eb9133e..23c33643f 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -456,10 +456,7 @@ where // responses. The transport should modify its behaviour to account // for the potentially slow and long running nature of a // transaction. - add_to_stream( - CallResult::feedback_only(ServiceFeedback::BeginTransaction), - &response_tx, - ); + add_to_stream(ServiceFeedback::BeginTransaction, &response_tx); } // Enqueue the zone SOA RRset for the batcher to process. @@ -501,8 +498,10 @@ where // one at a time to the batching responder. tokio::spawn(async move { if let Err(rcode) = zone_funneler.run().await { - let resp = mk_error_response(&cloned_msg, rcode); - add_to_stream(CallResult::new(resp), &cloned_response_tx); + add_to_stream( + mk_error_response(&cloned_msg, rcode), + &cloned_response_tx, + ); } }); @@ -513,16 +512,16 @@ where Ok(()) => { trace!("Ending transaction"); add_to_stream( - CallResult::feedback_only( - ServiceFeedback::EndTransaction, - ), + ServiceFeedback::EndTransaction, &response_tx, ); } Err(rcode) => { - let resp = mk_error_response(&msg, rcode); - add_to_stream(CallResult::new(resp), &response_tx); + add_to_stream( + mk_error_response(&msg, rcode), + &response_tx, + ); } } }); @@ -605,10 +604,7 @@ where // responses. The transport should modify its behaviour to account // for the potentially slow and long running nature of a // transaction. - add_to_stream( - CallResult::feedback_only(ServiceFeedback::BeginTransaction), - &sender, - ); + add_to_stream(ServiceFeedback::BeginTransaction, &sender); } // Stream the IXFR diffs in the background @@ -622,8 +618,10 @@ where error!( "Internal error: Failed to acquire XFR batcher semaphore" ); - let resp = mk_error_response(&msg, OptRcode::SERVFAIL); - add_to_stream(CallResult::new(resp), &sender); + add_to_stream( + mk_error_response(&msg, OptRcode::SERVFAIL), + &sender, + ); return; } @@ -747,12 +745,7 @@ where if !must_fit_in_single_message { trace!("Ending transaction"); - add_to_stream( - CallResult::feedback_only( - ServiceFeedback::EndTransaction, - ), - &sender, - ); + add_to_stream(ServiceFeedback::EndTransaction, &sender); } }); @@ -1447,10 +1440,7 @@ where mk_builder_for_target(), ); - add_to_stream( - CallResult::new(resp), - &self.response_tx, - ); + add_to_stream(resp, &self.response_tx); return Ok(()); } @@ -1487,11 +1477,11 @@ where } //------------ add_to_stream() ------------------------------------------------ -fn add_to_stream( - call_result: CallResult, +fn add_to_stream>>( + call_result: T, response_tx: &UnboundedSender>, ) { - if response_tx.send(Ok(call_result)).is_err() { + if response_tx.send(Ok(call_result.into())).is_err() { // We failed to write the message into the response stream. This // shouldn't happen. We can't now return an error to the client // because that would require writing to the response stream as diff --git a/src/net/server/service.rs b/src/net/server/service.rs index 4c7993248..13e177bd9 100644 --- a/src/net/server/service.rs +++ b/src/net/server/service.rs @@ -374,3 +374,21 @@ impl CallResult { (response, feedback) } } + +//--- From + +impl From>> + for CallResult +{ + fn from(response: AdditionalBuilder>) -> Self { + Self::new(response) + } +} + +//--- From + +impl From for CallResult { + fn from(feedback: ServiceFeedback) -> Self { + Self::feedback_only(feedback) + } +} From 69ea09341cc4533595b8f97e32a9e5c628844ef9 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 10:05:50 +0200 Subject: [PATCH 288/333] - Fix temporarily removed support for RFC 5936 backward compatible mode. - Handle parsing errors. - Introduce new XfrData type. - Use more functional style to improve readability. --- src/net/server/middleware/xfr.rs | 173 +++++++++++++++++++++---------- 1 file changed, 118 insertions(+), 55 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 23c33643f..9c1cf90ac 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -64,6 +64,7 @@ use crate::base::message_builder::{ AdditionalBuilder, AnswerBuilder, PushError, }; use crate::base::wire::Composer; +use crate::base::wire::ParseError; use crate::base::{ Message, Name, ParsedName, Question, RecordData, Rtype, Serial, StreamTarget, ToName, @@ -223,10 +224,20 @@ where } // Is transfer allowed for the requested zone for this requestor? - let (zone, diffs) = xfr_data_provider + let xfr_data = xfr_data_provider .request(req, ixfr_query_serial) .await .map_err(|err| match err { + XfrDataProviderError::ParseError(err) => { + debug!( + "{} for {} from {} refused: parse error: {err}", + q.qtype(), + q.qname(), + req.client_addr() + ); + OptRcode::FORMERR + } + XfrDataProviderError::UnknownZone => { // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2.1 // 2.2.1 Header Values @@ -268,7 +279,7 @@ where })?; // Read the zone SOA RR - let read = zone.read(); + let read = xfr_data.zone.read(); let Ok(zone_soa_answer) = Self::read_soa(&read, q.qname().to_name()).await else { @@ -307,8 +318,8 @@ where ))))) } - Rtype::AXFR | Rtype::IXFR if diffs.is_empty() => { - if q.qtype() == Rtype::IXFR && diffs.is_empty() { + Rtype::AXFR | Rtype::IXFR if xfr_data.diffs.is_empty() => { + if q.qtype() == Rtype::IXFR && xfr_data.diffs.is_empty() { // https://datatracker.ietf.org/doc/html/rfc1995#section-4 // 4. Response Format // "If incremental zone transfer is not available, the @@ -336,6 +347,7 @@ where q.qname().to_name(), &zone_soa_answer, read, + xfr_data.compatibility_mode, ) .await?; @@ -365,7 +377,7 @@ where req, ixfr_query_serial, &zone_soa_answer, - diffs, + xfr_data.diffs, ) .await?; @@ -385,6 +397,7 @@ where qname: StoredName, zone_soa_answer: &Answer, read: Box, + compatibility_mode: bool, ) -> Result< XfrMiddlewareStream< NextSvc::Future, @@ -405,11 +418,6 @@ where let soft_byte_limit = Self::calc_msg_bytes_available(req); - // TODO - // let compatibility_mode = xfr_config.compatibility_mode - // == CompatibilityMode::BackwardCompatible; - let compatibility_mode = false; - if compatibility_mode { trace!( "Compatibility mode enabled for client with IP address {}", @@ -885,6 +893,8 @@ pub enum XfrMode { #[derive(Copy, Clone, Debug, PartialEq, Eq)] pub enum XfrDataProviderError { + ParseError(ParseError), + UnknownZone, Refused, @@ -892,7 +902,47 @@ pub enum XfrDataProviderError { TemporarilyUnavailable, } -//------------ Transferable --------------------------------------------------- +//--- From + +impl From for XfrDataProviderError { + fn from(err: ParseError) -> Self { + Self::ParseError(err) + } +} + +//------------ XfrData -------------------------------------------------------- + +/// The data supplied by an [`XfrDataProvider`]. +pub struct XfrData { + /// The zone to transfer. + zone: Zone, + + /// The requested diffs. + /// + /// Empty if the requested diff range could not be satisfied. + diffs: Vec, + + /// Should XFR be done in RFC 5936 backward compatible mode? + /// + /// See: https://www.rfc-editor.org/rfc/rfc5936#section-7 + compatibility_mode: bool, +} + +impl XfrData { + pub fn new( + zone: Zone, + diffs: Vec, + backward_compatible: bool, + ) -> Self { + Self { + zone, + diffs, + compatibility_mode: backward_compatible, + } + } +} + +//------------ XfrDataProvider ------------------------------------------------ /// A provider of data needed for responding to XFR requests. pub trait XfrDataProvider { @@ -917,7 +967,7 @@ pub trait XfrDataProvider { Box< dyn Future< Output = Result< - (Zone, Vec), + XfrData, XfrDataProviderError, >, > + Sync @@ -946,7 +996,7 @@ where Box< dyn Future< Output = Result< - (Zone, Vec), + XfrData, XfrDataProviderError, >, > + Sync @@ -980,7 +1030,7 @@ impl XfrDataProvider for Zone { Box< dyn Future< Output = Result< - (Zone, Vec), + XfrData, XfrDataProviderError, >, > + Sync @@ -990,15 +1040,18 @@ impl XfrDataProvider for Zone { where Octs: Octets + Send + Sync, { - let res = if let Ok(q) = req.message().sole_question() { - if q.qname() == self.apex_name() && q.qclass() == self.class() { - Ok((self.clone(), vec![])) - } else { - Err(XfrDataProviderError::UnknownZone) - } - } else { - Err(XfrDataProviderError::UnknownZone) - }; + let res = req + .message() + .sole_question() + .map_err(XfrDataProviderError::ParseError) + .and_then(|q| { + if q.qname() == self.apex_name() && q.qclass() == self.class() + { + Ok(XfrData::new(self.clone(), vec![], false)) + } else { + Err(XfrDataProviderError::UnknownZone) + } + }); Box::pin(ready(res)) } @@ -1023,7 +1076,7 @@ impl XfrDataProvider for ZoneTree { Box< dyn Future< Output = Result< - (Zone, Vec), + XfrData, XfrDataProviderError, >, > + Sync @@ -1033,15 +1086,17 @@ impl XfrDataProvider for ZoneTree { where Octs: Octets + Send + Sync, { - let res = if let Ok(q) = req.message().sole_question() { - if let Some(zone) = self.find_zone(q.qname(), q.qclass()) { - Ok((zone.clone(), vec![])) - } else { - Err(XfrDataProviderError::UnknownZone) - } - } else { - Err(XfrDataProviderError::UnknownZone) - }; + let res = req + .message() + .sole_question() + .map_err(XfrDataProviderError::ParseError) + .and_then(|q| { + if let Some(zone) = self.find_zone(q.qname(), q.qclass()) { + Ok(XfrData::new(zone.clone(), vec![], false)) + } else { + Err(XfrDataProviderError::UnknownZone) + } + }); Box::pin(ready(res)) } @@ -1982,7 +2037,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 Box< dyn Future< Output = Result< - (Zone, Vec), + XfrData, XfrDataProviderError, >, > + Sync @@ -2280,6 +2335,17 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 diffs: diffs.into_iter().map(Arc::new).collect(), } } + + fn get_diffs( + &self, + diff_from: Option, + ) -> Vec> { + if self.diffs.first().map(|diff| diff.start_serial) == diff_from { + self.diffs.clone() + } else { + vec![] + } + } } impl XfrDataProvider for ZoneWithDiffs { @@ -2292,7 +2358,7 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 Box< dyn Future< Output = Result< - (Zone, Vec>), + XfrData, XfrDataProviderError, >, > + Sync @@ -2302,26 +2368,23 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 where Octs: Octets + Send + Sync, { - let res = if let Ok(q) = req.message().sole_question() { - if q.qname() == self.zone.apex_name() - && q.qclass() == self.zone.class() - { - let diffs = - if self.diffs.first().map(|diff| diff.start_serial) - == diff_from - { - self.diffs.clone() - } else { - vec![] - }; - - Ok((self.zone.clone(), diffs)) - } else { - Err(XfrDataProviderError::UnknownZone) - } - } else { - Err(XfrDataProviderError::UnknownZone) - }; + let res = req + .message() + .sole_question() + .map_err(XfrDataProviderError::ParseError) + .and_then(|q| { + if q.qname() == self.zone.apex_name() + && q.qclass() == self.zone.class() + { + Ok(XfrData::new( + self.zone.clone(), + self.get_diffs(diff_from), + false, + )) + } else { + Err(XfrDataProviderError::UnknownZone) + } + }); Box::pin(ready(res)) } From ceb66f8616361f27467c5c11a3caf99577718b22 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 11:12:43 +0200 Subject: [PATCH 289/333] Normalize how AXFR and IXFR sending are implemented. --- src/net/server/middleware/xfr.rs | 367 +++++++++++++++++++------------ src/zonetree/traits.rs | 2 +- 2 files changed, 222 insertions(+), 147 deletions(-) diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs index 9c1cf90ac..f86d1e7e9 100644 --- a/src/net/server/middleware/xfr.rs +++ b/src/net/server/middleware/xfr.rs @@ -170,12 +170,15 @@ where NextSvc::Stream: Send + Sync, XDP: XfrDataProvider, XDP::Diff: Debug + 'static, - for<'a> ::Stream<'a>: Send, { /// Pre-process received DNS XFR queries. /// /// Other types of query will be propagated unmodified to the next /// middleware or application service in the layered stack of services. + /// + /// Data to respond to the query will be requested from the given + /// [`XfrDataProvider`] which will act according to its policy concerning + /// the given [`Request`]. pub async fn preprocess( zone_walking_semaphore: Arc, batcher_semaphore: Arc, @@ -376,6 +379,7 @@ where batcher_semaphore.clone(), req, ixfr_query_serial, + q.qname().to_name(), &zone_soa_answer, xfr_data.diffs, ) @@ -384,7 +388,10 @@ where Ok(ControlFlow::Break(stream)) } - _ => Ok(ControlFlow::Continue(())), + _ => { + // Other QTYPEs should have been filtered out by get_relevant_question(). + unreachable!(); + } } } @@ -416,8 +423,6 @@ where return Err(OptRcode::SERVFAIL); }; - let soft_byte_limit = Self::calc_msg_bytes_available(req); - if compatibility_mode { trace!( "Compatibility mode enabled for client with IP address {}", @@ -444,11 +449,13 @@ where // - https://datatracker.ietf.org/doc/html/rfc1995#section-4 // - https://datatracker.ietf.org/doc/html/rfc5936#section-4.2 + let soft_byte_limit = Self::calc_msg_bytes_available(req); + // Create a stream that will be immediately returned to the caller. // Async tasks will then push DNS response messages into the stream as // they become available. - let (response_tx, receiver) = unbounded_channel(); - let stream = UnboundedReceiverStream::new(receiver); + let (response_tx, response_rx) = unbounded_channel(); + let stream = UnboundedReceiverStream::new(response_rx); // Create a bounded queue for passing RRsets found during zone walking // to a task which will batch the RRs together before pushing them @@ -548,6 +555,7 @@ where batcher_semaphore: Arc, req: &Request, query_serial: Serial, + qname: StoredName, zone_soa_answer: &Answer, diffs: Vec, ) -> Result< @@ -563,11 +571,17 @@ where { let msg = req.message(); - let Some((soa_ttl, ZoneRecordData::Soa(soa))) = - zone_soa_answer.content().first() + let AnswerContent::Data(zone_soa_rrset) = + zone_soa_answer.content().clone() else { return Err(OptRcode::SERVFAIL); }; + let Some(first_rr) = zone_soa_rrset.first() else { + return Err(OptRcode::SERVFAIL); + }; + let ZoneRecordData::Soa(soa) = first_rr.data() else { + return Err(OptRcode::SERVFAIL); + }; // Note: Unlike RFC 5936 for AXFR, neither RFC 1995 nor RFC 9103 say // anything about whether an IXFR response can consist of more than @@ -586,7 +600,7 @@ where // Errata https://www.rfc-editor.org/errata/eid3196 points out that // this is NOT "just as in AXFR" as AXFR does not do that. if query_serial >= soa.serial() { - trace!("IXFR finished because query_serial >= zone_serial"); + trace!("Responding to IXFR with single SOA because query serial >= zone serial"); let builder = mk_builder_for_target(); let response = zone_soa_answer.to_message(msg, builder); let res = Ok(CallResult::new(response)); @@ -598,11 +612,19 @@ where // the full zone) beyond which named chooses to use an AXFR response // rather than IXFR when answering zone transfer requests"? + let soft_byte_limit = Self::calc_msg_bytes_available(req); + // Create a stream that will be immediately returned to the caller. // Async tasks will then push DNS response messages into the stream as // they become available. - let (sender, receiver) = unbounded_channel(); - let stream = UnboundedReceiverStream::new(receiver); + let (response_tx, response_rx) = unbounded_channel(); + let stream = UnboundedReceiverStream::new(response_rx); + + // Create a bounded queue for passing RRsets found during diff walking + // to a task which will batch the RRs together before pushing them + // into the result stream. + let (batcher_tx, batcher_rx) = + tokio::sync::mpsc::channel::<(StoredName, SharedRrset)>(100); let must_fit_in_single_message = matches!(req.transport_ctx(), TransportSpecificContext::Udp(_)); @@ -612,151 +634,65 @@ where // responses. The transport should modify its behaviour to account // for the potentially slow and long running nature of a // transaction. - add_to_stream(ServiceFeedback::BeginTransaction, &sender); + add_to_stream(ServiceFeedback::BeginTransaction, &response_tx); } - // Stream the IXFR diffs in the background - let msg = msg.clone(); - let soft_byte_limit = Self::calc_msg_bytes_available(req); + // Stream the IXFR diffs in the background to the batcher. + let diff_funneler = + DiffFunneler::new(qname, zone_soa_rrset, diffs, batcher_tx); + + let batching_responder = BatchingRrResponder::new( + req.message().clone(), + zone_soa_answer.clone(), + batcher_rx, + response_tx.clone(), + false, + soft_byte_limit, + must_fit_in_single_message, + batcher_semaphore, + ); + let cloned_msg = msg.clone(); + let cloned_response_tx = response_tx.clone(); + + // Start the funneler. It will walk the diffs and send all of the RRs + // one at a time to the batching responder. tokio::spawn(async move { - // Limit the number of concurrently running XFR batching - // operations. - if batcher_semaphore.acquire().await.is_err() { - error!( - "Internal error: Failed to acquire XFR batcher semaphore" - ); + if let Err(rcode) = diff_funneler.run().await { add_to_stream( - mk_error_response(&msg, OptRcode::SERVFAIL), - &sender, + mk_error_response(&cloned_msg, rcode), + &cloned_response_tx, ); - return; } + }); - // https://datatracker.ietf.org/doc/html/rfc1995#section-4 - // 4. Response Format - // ... - // "If incremental zone transfer is available, one or more - // difference sequences is returned. The list of difference - // sequences is preceded and followed by a copy of the server's - // current version of the SOA. - // - // Each difference sequence represents one update to the zone - // (one SOA serial change) consisting of deleted RRs and added - // RRs. The first RR of the deleted RRs is the older SOA RR - // and the first RR of the added RRs is the newer SOA RR. - // - // Modification of an RR is performed first by removing the - // original RR and then adding the modified one. - // - // The sequences of differential information are ordered oldest - // first newest last. Thus, the differential sequences are the - // history of changes made since the version known by the IXFR - // client up to the server's current version. - // - // RRs in the incremental transfer messages may be partial. That - // is, if a single RR of multiple RRs of the same RR type changes, - // only the changed RR is transferred." - - let (owner, qclass) = { - let Ok(q) = msg.sole_question() else { - unreachable!(); - }; - (q.qname().to_name::(), q.qclass()) - }; - - let mut batcher = XfrRrBatcher::build( - msg.clone(), - sender.clone(), - Some(soft_byte_limit), - None, - must_fit_in_single_message, - ); + let cloned_msg = msg.clone(); - batcher - .push((owner.clone(), qclass, soa_ttl, &soa)) - .unwrap(); // TODO - - for diff in diffs { - // 4. Response Format - // "Each difference sequence represents one update to the - // zone (one SOA serial change) consisting of deleted RRs - // and added RRs. The first RR of the deleted RRs is the - // older SOA RR and the first RR of the added RRs is the - // newer SOA RR. - let removed_soa = diff - .get_removed(owner.clone(), Rtype::SOA) - .await - .unwrap(); // The zone MUST have a SOA record - batcher - .push(( - owner.clone(), - qclass, - removed_soa.ttl(), - &removed_soa.data()[0], - )) - .unwrap(); // TODO - - let removed_stream = diff.removed(); - pin_mut!(removed_stream); - while let Some(item) = removed_stream.next().await { - let (owner, rtype) = item.key(); - if *rtype != Rtype::SOA { - let rrset = item.value(); - for rr in rrset.data() { - batcher - .push(( - owner.clone(), - qclass, - rrset.ttl(), - rr, - )) - .unwrap(); // TODO - } - } + // Start the batching responder. It will receive RRs from the funneler + // and push them in batches into the response stream. + tokio::spawn(async move { + match batching_responder.run().await { + Ok(()) => { + trace!("Ending transaction"); + add_to_stream( + ServiceFeedback::EndTransaction, + &response_tx, + ); } - let added_soa = - diff.get_added(owner.clone(), Rtype::SOA).await.unwrap(); // The zone MUST have a SOA record - batcher - .push(( - owner.clone(), - qclass, - added_soa.ttl(), - &added_soa.data()[0], - )) - .unwrap(); // TODO - - let added_stream = diff.added(); - pin_mut!(added_stream); - while let Some(item) = added_stream.next().await { - let (owner, rtype) = item.key(); - if *rtype != Rtype::SOA { - let rrset = item.value(); - for rr in rrset.data() { - batcher - .push(( - owner.clone(), - qclass, - rrset.ttl(), - rr, - )) - .unwrap(); // TODO - } - } + Err(rcode) => { + add_to_stream( + mk_error_response(&cloned_msg, rcode), + &response_tx, + ); } } - - batcher.push((owner, qclass, soa_ttl, soa)).unwrap(); // TODO - - batcher.finish().unwrap(); // TODO - - if !must_fit_in_single_message { - trace!("Ending transaction"); - add_to_stream(ServiceFeedback::EndTransaction, &sender); - } }); + // If either the funneler or batcher responder terminate then so will + // the other as they each own half of a send <-> receive channel and + // abort if the other side of the channel is gone. + Ok(MiddlewareStream::Result(stream)) } @@ -1390,6 +1326,144 @@ impl ZoneFunneler { } } +//------------ DiffFunneler ---------------------------------------------------- + +//------------ ZoneFunneler --------------------------------------------------- + +struct DiffFunneler { + qname: StoredName, + zone_soa_rrset: SharedRrset, + diffs: Vec, + batcher_tx: Sender<(Name, SharedRrset)>, +} + +impl DiffFunneler +where + Diff: ZoneDiff, +{ + fn new( + qname: StoredName, + zone_soa_rrset: SharedRrset, + diffs: Vec, + batcher_tx: Sender<(Name, SharedRrset)>, + ) -> Self { + Self { + qname, + zone_soa_rrset, + diffs, + batcher_tx, + } + } + + async fn run(self) -> Result<(), OptRcode> { + // https://datatracker.ietf.org/doc/html/rfc1995#section-4 + // 4. Response Format + // ... + // "If incremental zone transfer is available, one or more + // difference sequences is returned. The list of difference + // sequences is preceded and followed by a copy of the server's + // current version of the SOA. + // + // Each difference sequence represents one update to the zone + // (one SOA serial change) consisting of deleted RRs and added + // RRs. The first RR of the deleted RRs is the older SOA RR + // and the first RR of the added RRs is the newer SOA RR. + // + // Modification of an RR is performed first by removing the + // original RR and then adding the modified one. + // + // The sequences of differential information are ordered oldest + // first newest last. Thus, the differential sequences are the + // history of changes made since the version known by the IXFR + // client up to the server's current version. + // + // RRs in the incremental transfer messages may be partial. That + // is, if a single RR of multiple RRs of the same RR type changes, + // only the changed RR is transferred." + + if let Err(err) = self + .batcher_tx + .send((self.qname.clone(), self.zone_soa_rrset.clone())) + .await + { + error!("Internal error: Failed to send initial IXFR SOA to batcher: {err}"); + return Err(OptRcode::SERVFAIL); + } + + let qname = self.qname.clone(); + + for diff in self.diffs { + // 4. Response Format + // "Each difference sequence represents one update to the + // zone (one SOA serial change) consisting of deleted RRs + // and added RRs. The first RR of the deleted RRs is the + // older SOA RR and the first RR of the added RRs is the + // newer SOA RR. + + let added_soa = + diff.get_added(qname.clone(), Rtype::SOA).await.unwrap(); // The diff MUST have a SOA record + Self::send_diff_section( + &qname, + &self.batcher_tx, + added_soa, + diff.added(), + ) + .await?; + + let removed_soa = + diff.get_removed(qname.clone(), Rtype::SOA).await.unwrap(); // The diff MUST have a SOA record + Self::send_diff_section( + &qname, + &self.batcher_tx, + removed_soa, + diff.removed(), + ) + .await?; + } + + if let Err(err) = self + .batcher_tx + .send((qname.clone(), self.zone_soa_rrset)) + .await + { + error!("Internal error: Failed to send final IXFR SOA to batcher: {err}"); + return Err(OptRcode::SERVFAIL); + } + + Ok(()) + } + + async fn send_diff_section( + qname: &StoredName, + batcher_tx: &Sender<(Name, SharedRrset)>, + soa: &SharedRrset, + diff_stream: ::Stream<'_>, + ) -> Result<(), OptRcode> { + if let Err(err) = batcher_tx.send((qname.clone(), soa.clone())).await + { + error!("Internal error: Failed to send SOA to batcher: {err}"); + return Err(OptRcode::SERVFAIL); + } + + pin_mut!(diff_stream); + + while let Some(item) = diff_stream.next().await { + let (owner, rtype) = item.key(); + if *rtype != Rtype::SOA { + let rrset = item.value(); + if let Err(err) = + batcher_tx.send((owner.clone(), rrset.clone())).await + { + error!("Internal error: Failed to send RRSET to batcher: {err}"); + return Err(OptRcode::SERVFAIL); + } + } + } + + Ok(()) + } +} + //------------ BatchingRrResponder --------------------------------------------- struct BatchingRrResponder { @@ -1490,10 +1564,11 @@ where debug_assert!(self.must_fit_in_single_message); debug!("Responding to IXFR with single SOA because response does not fit in a single UDP reply"); - let resp = self.zone_soa_answer.to_message( - &self.msg, - mk_builder_for_target(), - ); + let builder = mk_builder_for_target(); + + let resp = self + .zone_soa_answer + .to_message(&self.msg, builder); add_to_stream(resp, &self.response_tx); @@ -1532,6 +1607,7 @@ where } //------------ add_to_stream() ------------------------------------------------ + fn add_to_stream>>( call_result: T, response_tx: &UnboundedSender>, @@ -2236,7 +2312,6 @@ JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 > where XDP::Diff: Debug + 'static, - for<'a> ::Stream<'a>: Send, { XfrMiddlewareSvc::, TestNextSvc, RequestMeta, XDP>::preprocess( Arc::new(Semaphore::new(1)), diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index 2aa6619cf..0605480a9 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -270,7 +270,7 @@ pub trait ZoneDiffItem { /// [`Stream`]s. pub trait ZoneDiff { /// A single item in the diff. - type Item<'a>: ZoneDiffItem + type Item<'a>: ZoneDiffItem + Send where Self: 'a; From 609ed598cb73ca300048ac38fcd78550346188e9 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 11:42:41 +0200 Subject: [PATCH 290/333] Split the large xfr module into separate sub-modules. --- src/net/server/middleware/xfr.rs | 2467 ----------------- src/net/server/middleware/xfr/axfr.rs | 94 + src/net/server/middleware/xfr/batcher.rs | 224 ++ .../server/middleware/xfr/data_provider.rs | 243 ++ src/net/server/middleware/xfr/ixfr.rs | 146 + src/net/server/middleware/xfr/mod.rs | 52 + src/net/server/middleware/xfr/responder.rs | 161 ++ src/net/server/middleware/xfr/service.rs | 754 +++++ src/net/server/middleware/xfr/tests.rs | 824 ++++++ src/net/server/middleware/xfr/util.rs | 39 + 10 files changed, 2537 insertions(+), 2467 deletions(-) delete mode 100644 src/net/server/middleware/xfr.rs create mode 100644 src/net/server/middleware/xfr/axfr.rs create mode 100644 src/net/server/middleware/xfr/batcher.rs create mode 100644 src/net/server/middleware/xfr/data_provider.rs create mode 100644 src/net/server/middleware/xfr/ixfr.rs create mode 100644 src/net/server/middleware/xfr/mod.rs create mode 100644 src/net/server/middleware/xfr/responder.rs create mode 100644 src/net/server/middleware/xfr/service.rs create mode 100644 src/net/server/middleware/xfr/tests.rs create mode 100644 src/net/server/middleware/xfr/util.rs diff --git a/src/net/server/middleware/xfr.rs b/src/net/server/middleware/xfr.rs deleted file mode 100644 index f86d1e7e9..000000000 --- a/src/net/server/middleware/xfr.rs +++ /dev/null @@ -1,2467 +0,0 @@ -//! RFC 5936 AXFR and RFC 1995 IXFR request handling middleware. -//! -//! This module provides the [`XfrMiddlewareSvc`] service which responds to -//! [RFC 5936] AXFR and [RFC 1995] IXFR requests to perform entire or -//! incremental difference based zone transfers. -//! -//! Determining which requests to honour and with what data is delegated to a -//! caller supplied implementation of the [`XfrDataProvider`] trait. -//! [`XfrDataProvider`] implementations for [`Zone`] and [`ZoneTree`] are -//! provided allowing those types to be used as-is as XFR data providers with -//! this middleware service. -//! -//! [`XfrRrBatcher`], primarily intended for internal use by -//! [`XfrMiddlewareSvc`], handles splitting of large zone transfer replies -//! into batches with as many resource records per response as will fit. -//! -//! # Requiring TSIG authenticated XFR requests -//! -//! To require XFR requests to be TSIG authenticated, implement -//! [`XfrDataProvider>`], extract the key data using -//! [`Request::metadata()`] and verify that a TSIG key was used to sign the -//! request, and that the name and algorithm of the used key are acceptable to -//! you. -//! -//! You can then use your [`XfrDataProvider`] impl with [`XfrMiddlewareSvc`], -//! and add [`TsigMiddlewareSvc`] directly before [`XfrMiddlewareSvc`] in the -//! middleware layer stack so that the used `Key` is made available from the -//! TSIG middleware to the XFR middleware. -//! -//! # Limitations -//! -//! * RFC 1995 2 Brief Description of the Protocol states: _"To ensure -//! integrity, servers should use UDP checksums for all UDP responses."_. -//! This is not implemented. -//! -//! [RFC 5936]: https://www.rfc-editor.org/info/rfc5936 -//! [RFC 1995]: https://www.rfc-editor.org/info/rfc1995 -//! [`TsigMiddlewareSvc`]: -//! crate::net::server::middleware::tsig::TsigMiddlewareSvc - -use core::future::{ready, Future, Ready}; -use core::marker::PhantomData; -use core::ops::{ControlFlow, Deref}; - -use std::boxed::Box; -use std::fmt::Debug; -use std::pin::Pin; -use std::sync::Arc; -use std::vec::Vec; - -use bytes::Bytes; -use futures_util::stream::{once, Once, Stream}; -use futures_util::{pin_mut, StreamExt}; -use octseq::Octets; -use tokio::sync::mpsc::{ - unbounded_channel, Receiver, Sender, UnboundedSender, -}; -use tokio::sync::Semaphore; -use tokio_stream::wrappers::UnboundedReceiverStream; -use tracing::{debug, error, info, trace, warn}; - -use crate::base::iana::{Opcode, OptRcode, Rcode}; -use crate::base::message_builder::{ - AdditionalBuilder, AnswerBuilder, PushError, -}; -use crate::base::wire::Composer; -use crate::base::wire::ParseError; -use crate::base::{ - Message, Name, ParsedName, Question, RecordData, Rtype, Serial, - StreamTarget, ToName, -}; -use crate::net::server::batcher::{ - CallbackBatcher, Callbacks, ResourceRecordBatcher, -}; -use crate::net::server::message::{Request, TransportSpecificContext}; -use crate::net::server::middleware::stream::MiddlewareStream; -use crate::net::server::service::{ - CallResult, Service, ServiceFeedback, ServiceResult, -}; -use crate::net::server::util::{mk_builder_for_target, mk_error_response}; -use crate::rdata::{Soa, ZoneRecordData}; -use crate::zonetree::error::OutOfZone; -use crate::zonetree::types::EmptyZoneDiff; -use crate::zonetree::{ - Answer, AnswerContent, ReadableZone, SharedRrset, StoredName, Zone, - ZoneDiff, ZoneDiffItem, ZoneTree, -}; - -//------------ Constants ----------------------------------------------------- - -/// https://datatracker.ietf.org/doc/html/rfc1035#section-2.3.4 -/// 2.3.4. Size limits -/// "UDP messages 512 octets or less" -const MAX_UDP_MSG_BYTE_LEN: u16 = 512; - -/// https://datatracker.ietf.org/doc/html/rfc1035#section-4.2.2 -/// 4.2.2. TCP usage -/// "The message is prefixed with a two byte length field which gives the -/// message length, excluding the two byte length field" -const MAX_TCP_MSG_BYTE_LEN: u16 = u16::MAX; - -//------------ XfrMiddlewareSvc ---------------------------------------------- - -/// RFC 5936 AXFR and RFC 1995 IXFR request handling middleware. -/// -/// See the [module documentation] for a high level introduction. -/// -/// [module documentation]: crate::net::server::middleware::xfr -#[derive(Clone, Debug)] -pub struct XfrMiddlewareSvc { - /// The upstream [`Service`] to pass requests to and receive responses - /// from. - next_svc: NextSvc, - - /// A caller supplied implementation of [`XfrDataProvider`] for - /// determining which requests to answer and with which data. - xfr_data_provider: XDP, - - /// A limit on the number of XFR related zone walking operations - /// that may run concurrently. - zone_walking_semaphore: Arc, - - /// A limit on the number of XFR related response batching operations that - /// may run concurrently. - batcher_semaphore: Arc, - - _phantom: PhantomData<(RequestOctets, RequestMeta)>, -} - -impl - XfrMiddlewareSvc -where - XDP: XfrDataProvider, -{ - /// Creates a new instance of this middleware. - /// - /// Takes an implementation of [`XfrDataProvider`] as a parameter to - /// determine which requests to honour and with which data. - /// - /// The `max_concurrency` parameter limits the number of simultaneous zone - /// transfer operations that may occur concurrently without blocking. - #[must_use] - pub fn new( - next_svc: NextSvc, - xfr_data_provider: XDP, - max_concurrency: usize, - ) -> Self { - let zone_walking_semaphore = - Arc::new(Semaphore::new(max_concurrency)); - let batcher_semaphore = Arc::new(Semaphore::new(max_concurrency)); - - Self { - next_svc, - xfr_data_provider, - zone_walking_semaphore, - batcher_semaphore, - _phantom: PhantomData, - } - } -} - -impl - XfrMiddlewareSvc -where - RequestOctets: Octets + Send + Sync + 'static + Unpin, - for<'a> ::Range<'a>: Send + Sync, - NextSvc: Service + Clone + Send + Sync + 'static, - NextSvc::Future: Send + Sync + Unpin, - NextSvc::Target: Composer + Default + Send + Sync, - NextSvc::Stream: Send + Sync, - XDP: XfrDataProvider, - XDP::Diff: Debug + 'static, -{ - /// Pre-process received DNS XFR queries. - /// - /// Other types of query will be propagated unmodified to the next - /// middleware or application service in the layered stack of services. - /// - /// Data to respond to the query will be requested from the given - /// [`XfrDataProvider`] which will act according to its policy concerning - /// the given [`Request`]. - pub async fn preprocess( - zone_walking_semaphore: Arc, - batcher_semaphore: Arc, - req: &Request, - xfr_data_provider: XDP, - ) -> Result< - ControlFlow< - XfrMiddlewareStream< - NextSvc::Future, - NextSvc::Stream, - ::Item, - >, - >, - OptRcode, - > { - let msg = req.message(); - - // Do we support this type of request? - let Some(q) = Self::get_relevant_question(msg) else { - return Ok(ControlFlow::Continue(())); - }; - - // https://datatracker.ietf.org/doc/html/rfc1995#section-3 - // 3. Query Format - // "The IXFR query packet format is the same as that of a normal DNS - // query, but with the query type being IXFR and the authority - // section containing the SOA record of client's version of the - // zone." - let ixfr_query_serial = if let Ok(Some(Ok(query_soa))) = msg - .authority() - .map(|section| section.limit_to::>>().next()) - { - Some(query_soa.data().serial()) - } else { - None - }; - - if q.qtype() == Rtype::IXFR && ixfr_query_serial.is_none() { - warn!( - "{} for {} from {} refused: IXFR request lacks authority section SOA", - q.qtype(), - q.qname(), - req.client_addr() - ); - return Err(OptRcode::FORMERR); - } - - // Is transfer allowed for the requested zone for this requestor? - let xfr_data = xfr_data_provider - .request(req, ixfr_query_serial) - .await - .map_err(|err| match err { - XfrDataProviderError::ParseError(err) => { - debug!( - "{} for {} from {} refused: parse error: {err}", - q.qtype(), - q.qname(), - req.client_addr() - ); - OptRcode::FORMERR - } - - XfrDataProviderError::UnknownZone => { - // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2.1 - // 2.2.1 Header Values - // "If a server is not authoritative for the queried - // zone, the server SHOULD set the value to NotAuth(9)" - debug!( - "{} for {} from {} refused: unknown zone", - q.qtype(), - q.qname(), - req.client_addr() - ); - OptRcode::NOTAUTH - } - - XfrDataProviderError::TemporarilyUnavailable => { - // The zone is not yet loaded or has expired, both of - // which are presumably transient conditions and thus - // SERVFAIL is the appropriate response, not NOTAUTH, as - // we know we are supposed to be authoritative for the - // zone but we just don't have the data right now. - warn!( - "{} for {} from {} refused: zone not currently available", - q.qtype(), - q.qname(), - req.client_addr() - ); - OptRcode::SERVFAIL - } - - XfrDataProviderError::Refused => { - warn!( - "{} for {} from {} refused: access denied", - q.qtype(), - q.qname(), - req.client_addr() - ); - OptRcode::REFUSED - } - })?; - - // Read the zone SOA RR - let read = xfr_data.zone.read(); - let Ok(zone_soa_answer) = - Self::read_soa(&read, q.qname().to_name()).await - else { - debug!( - "{} for {} from {} refused: name is outside the zone", - q.qtype(), - q.qname(), - req.client_addr() - ); - return Err(OptRcode::SERVFAIL); - }; - - match q.qtype() { - Rtype::AXFR if req.transport_ctx().is_udp() => { - // https://datatracker.ietf.org/doc/html/rfc5936#section-4.2 - // 4.2. UDP - // "With the addition of EDNS0 and applications that require - // many small zones, such as in web hosting and some ENUM - // scenarios, AXFR sessions on UDP would now seem - // desirable. However, there are still some aspects of - // AXFR sessions that are not easily translated to UDP. - // - // Therefore, this document does not update RFC 1035 in - // this respect: AXFR sessions over UDP transport are not - // defined." - warn!( - "{} for {} from {} refused: AXFR not supported over UDP", - q.qtype(), - q.qname(), - req.client_addr() - ); - let response = mk_error_response(msg, OptRcode::NOTIMP); - let res = Ok(CallResult::new(response)); - Ok(ControlFlow::Break(MiddlewareStream::Map(once(ready( - res, - ))))) - } - - Rtype::AXFR | Rtype::IXFR if xfr_data.diffs.is_empty() => { - if q.qtype() == Rtype::IXFR && xfr_data.diffs.is_empty() { - // https://datatracker.ietf.org/doc/html/rfc1995#section-4 - // 4. Response Format - // "If incremental zone transfer is not available, the - // entire zone is returned. The first and the last RR of - // the response is the SOA record of the zone. I.e. the - // behavior is the same as an AXFR response except the - // query type is IXFR." - info!( - "IXFR for {} (serial {} from {}: diffs not available, falling back to AXFR", - q.qname(), - ixfr_query_serial.unwrap(), // SAFETY: Always Some() if IXFR - req.client_addr() - ); - } else { - info!( - "AXFR for {} from {}", - q.qname(), - req.client_addr() - ); - } - let stream = Self::respond_to_axfr_query( - zone_walking_semaphore, - batcher_semaphore, - req, - q.qname().to_name(), - &zone_soa_answer, - read, - xfr_data.compatibility_mode, - ) - .await?; - - Ok(ControlFlow::Break(stream)) - } - - Rtype::IXFR => { - // SAFETY: Always Some() if IXFR - let ixfr_query_serial = ixfr_query_serial.unwrap(); - info!( - "IXFR for {} (serial {ixfr_query_serial}) from {}", - q.qname(), - req.client_addr() - ); - - // https://datatracker.ietf.org/doc/html/rfc1995#section-2 - // 2. Brief Description of the Protocol - // "Transport of a query may be by either UDP or TCP. If an - // IXFR query is via UDP, the IXFR server may attempt to - // reply using UDP if the entire response can be contained - // in a single DNS packet. If the UDP reply does not fit, - // the query is responded to with a single SOA record of - // the server's current version to inform the client that a - // TCP query should be initiated." - let stream = Self::respond_to_ixfr_query( - batcher_semaphore.clone(), - req, - ixfr_query_serial, - q.qname().to_name(), - &zone_soa_answer, - xfr_data.diffs, - ) - .await?; - - Ok(ControlFlow::Break(stream)) - } - - _ => { - // Other QTYPEs should have been filtered out by get_relevant_question(). - unreachable!(); - } - } - } - - /// Generate and send an AXFR response for a given request and zone. - #[allow(clippy::too_many_arguments)] - async fn respond_to_axfr_query( - zone_walk_semaphore: Arc, - batcher_semaphore: Arc, - req: &Request, - qname: StoredName, - zone_soa_answer: &Answer, - read: Box, - compatibility_mode: bool, - ) -> Result< - XfrMiddlewareStream< - NextSvc::Future, - NextSvc::Stream, - ::Item, - >, - OptRcode, - > { - let AnswerContent::Data(zone_soa_rrset) = - zone_soa_answer.content().clone() - else { - error!( - "AXFR for {qname} from {} refused: zone lacks SOA RR", - req.client_addr() - ); - return Err(OptRcode::SERVFAIL); - }; - - if compatibility_mode { - trace!( - "Compatibility mode enabled for client with IP address {}", - req.client_addr().ip() - ); - } - - // Return a stream of response messages containing: - // - SOA - // - RRSETs, one or more per response message - // - SOA - // - // Neither RFC 5936 nor RFC 1035 defined AXFR for UDP, only for TCP. - // However, RFC 1995 says that for IXFR if no diffs are available the - // full zone should be served just as with AXFR, and that UDP is - // supported as long as the entire XFR response fits in a single - // datagram. Thus we don't check for UDP or TCP here, except to abort - // if the response is too large to fit in a single UDP datagram, - // instead we let the caller that has the context decide whether AXFR - // is supported or not. - // - // References: - // - https://datatracker.ietf.org/doc/html/rfc1995#section-2 - // - https://datatracker.ietf.org/doc/html/rfc1995#section-4 - // - https://datatracker.ietf.org/doc/html/rfc5936#section-4.2 - - let soft_byte_limit = Self::calc_msg_bytes_available(req); - - // Create a stream that will be immediately returned to the caller. - // Async tasks will then push DNS response messages into the stream as - // they become available. - let (response_tx, response_rx) = unbounded_channel(); - let stream = UnboundedReceiverStream::new(response_rx); - - // Create a bounded queue for passing RRsets found during zone walking - // to a task which will batch the RRs together before pushing them - // into the result stream. - let (batcher_tx, batcher_rx) = - tokio::sync::mpsc::channel::<(StoredName, SharedRrset)>(100); - - let must_fit_in_single_message = - matches!(req.transport_ctx(), TransportSpecificContext::Udp(_)); - - if !must_fit_in_single_message { - // Notify the underlying transport to expect a stream of related - // responses. The transport should modify its behaviour to account - // for the potentially slow and long running nature of a - // transaction. - add_to_stream(ServiceFeedback::BeginTransaction, &response_tx); - } - - // Enqueue the zone SOA RRset for the batcher to process. - if batcher_tx - .send((qname.clone(), zone_soa_rrset.clone())) - .await - .is_err() - { - return Err(OptRcode::SERVFAIL); - } - - let msg = req.message().clone(); - - // Stream the remaining non-SOA zone RRsets in the background to the - // batcher. - let zone_funneler = ZoneFunneler::new( - read, - qname, - zone_soa_rrset, - batcher_tx, - zone_walk_semaphore, - ); - - let batching_responder = BatchingRrResponder::new( - req.message().clone(), - zone_soa_answer.clone(), - batcher_rx, - response_tx.clone(), - compatibility_mode, - soft_byte_limit, - must_fit_in_single_message, - batcher_semaphore, - ); - - let cloned_msg = msg.clone(); - let cloned_response_tx = response_tx.clone(); - - // Start the funneler. It will walk the zone and send all of the RRs - // one at a time to the batching responder. - tokio::spawn(async move { - if let Err(rcode) = zone_funneler.run().await { - add_to_stream( - mk_error_response(&cloned_msg, rcode), - &cloned_response_tx, - ); - } - }); - - // Start the batching responder. It will receive RRs from the funneler - // and push them in batches into the response stream. - tokio::spawn(async move { - match batching_responder.run().await { - Ok(()) => { - trace!("Ending transaction"); - add_to_stream( - ServiceFeedback::EndTransaction, - &response_tx, - ); - } - - Err(rcode) => { - add_to_stream( - mk_error_response(&msg, rcode), - &response_tx, - ); - } - } - }); - - // If either the funneler or batcher responder terminate then so will - // the other as they each own half of a send <-> receive channel and - // abort if the other side of the channel is gone. - - Ok(MiddlewareStream::Result(stream)) - } - - // Generate and send an IXFR response for the given request and zone - // diffs. - #[allow(clippy::too_many_arguments)] - async fn respond_to_ixfr_query( - batcher_semaphore: Arc, - req: &Request, - query_serial: Serial, - qname: StoredName, - zone_soa_answer: &Answer, - diffs: Vec, - ) -> Result< - XfrMiddlewareStream< - NextSvc::Future, - NextSvc::Stream, - ::Item, - >, - OptRcode, - > - where - XDP::Diff: Send + 'static, - { - let msg = req.message(); - - let AnswerContent::Data(zone_soa_rrset) = - zone_soa_answer.content().clone() - else { - return Err(OptRcode::SERVFAIL); - }; - let Some(first_rr) = zone_soa_rrset.first() else { - return Err(OptRcode::SERVFAIL); - }; - let ZoneRecordData::Soa(soa) = first_rr.data() else { - return Err(OptRcode::SERVFAIL); - }; - - // Note: Unlike RFC 5936 for AXFR, neither RFC 1995 nor RFC 9103 say - // anything about whether an IXFR response can consist of more than - // one response message, but given the 2^16 byte maximum response size - // of a TCP DNS message and the 2^16 maximum number of ANSWER RRs - // allowed per DNS response, large zones may not fit in a single - // response message and will have to be split into multiple response - // messages. - - // https://datatracker.ietf.org/doc/html/rfc1995#section-2 - // 2. Brief Description of the Protocol - // "If an IXFR query with the same or newer version number than that - // of the server is received, it is replied to with a single SOA - // record of the server's current version, just as in AXFR." - // ^^^^^^^^^^^^^^^ - // Errata https://www.rfc-editor.org/errata/eid3196 points out that - // this is NOT "just as in AXFR" as AXFR does not do that. - if query_serial >= soa.serial() { - trace!("Responding to IXFR with single SOA because query serial >= zone serial"); - let builder = mk_builder_for_target(); - let response = zone_soa_answer.to_message(msg, builder); - let res = Ok(CallResult::new(response)); - return Ok(MiddlewareStream::Map(once(ready(res)))); - } - - // TODO: Add something like the Bind `max-ixfr-ratio` option that - // "sets the size threshold (expressed as a percentage of the size of - // the full zone) beyond which named chooses to use an AXFR response - // rather than IXFR when answering zone transfer requests"? - - let soft_byte_limit = Self::calc_msg_bytes_available(req); - - // Create a stream that will be immediately returned to the caller. - // Async tasks will then push DNS response messages into the stream as - // they become available. - let (response_tx, response_rx) = unbounded_channel(); - let stream = UnboundedReceiverStream::new(response_rx); - - // Create a bounded queue for passing RRsets found during diff walking - // to a task which will batch the RRs together before pushing them - // into the result stream. - let (batcher_tx, batcher_rx) = - tokio::sync::mpsc::channel::<(StoredName, SharedRrset)>(100); - - let must_fit_in_single_message = - matches!(req.transport_ctx(), TransportSpecificContext::Udp(_)); - - if !must_fit_in_single_message { - // Notify the underlying transport to expect a stream of related - // responses. The transport should modify its behaviour to account - // for the potentially slow and long running nature of a - // transaction. - add_to_stream(ServiceFeedback::BeginTransaction, &response_tx); - } - - // Stream the IXFR diffs in the background to the batcher. - let diff_funneler = - DiffFunneler::new(qname, zone_soa_rrset, diffs, batcher_tx); - - let batching_responder = BatchingRrResponder::new( - req.message().clone(), - zone_soa_answer.clone(), - batcher_rx, - response_tx.clone(), - false, - soft_byte_limit, - must_fit_in_single_message, - batcher_semaphore, - ); - - let cloned_msg = msg.clone(); - let cloned_response_tx = response_tx.clone(); - - // Start the funneler. It will walk the diffs and send all of the RRs - // one at a time to the batching responder. - tokio::spawn(async move { - if let Err(rcode) = diff_funneler.run().await { - add_to_stream( - mk_error_response(&cloned_msg, rcode), - &cloned_response_tx, - ); - } - }); - - let cloned_msg = msg.clone(); - - // Start the batching responder. It will receive RRs from the funneler - // and push them in batches into the response stream. - tokio::spawn(async move { - match batching_responder.run().await { - Ok(()) => { - trace!("Ending transaction"); - add_to_stream( - ServiceFeedback::EndTransaction, - &response_tx, - ); - } - - Err(rcode) => { - add_to_stream( - mk_error_response(&cloned_msg, rcode), - &response_tx, - ); - } - } - }); - - // If either the funneler or batcher responder terminate then so will - // the other as they each own half of a send <-> receive channel and - // abort if the other side of the channel is gone. - - Ok(MiddlewareStream::Result(stream)) - } - - #[allow(clippy::borrowed_box)] - async fn read_soa( - read: &Box, - qname: Name, - ) -> Result { - match read.is_async() { - true => read.query_async(qname, Rtype::SOA).await, - false => read.query(qname, Rtype::SOA), - } - } - - /// Is this message for us? - /// - /// Returns `Some(Question)` if the given query uses OPCODE QUERYY and has - /// a first question with a QTYPE of `AXFR` or `IXFR`, `None` otherwise. - fn get_relevant_question( - msg: &Message, - ) -> Option>>> { - if Opcode::QUERY == msg.header().opcode() && !msg.header().qr() { - if let Ok(q) = msg.sole_question() { - if matches!(q.qtype(), Rtype::AXFR | Rtype::IXFR) { - return Some(q); - } - } - } - - None - } - - fn calc_msg_bytes_available(req: &Request) -> usize { - let bytes_available = match req.transport_ctx() { - TransportSpecificContext::Udp(ctx) => { - let max_msg_size = ctx - .max_response_size_hint() - .unwrap_or(MAX_UDP_MSG_BYTE_LEN); - max_msg_size - req.num_reserved_bytes() - } - TransportSpecificContext::NonUdp(_) => { - MAX_TCP_MSG_BYTE_LEN - req.num_reserved_bytes() - } - }; - - bytes_available as usize - } -} - -//--- impl Service - -impl - Service - for XfrMiddlewareSvc -where - RequestOctets: Octets + Send + Sync + Unpin + 'static, - for<'a> ::Range<'a>: Send + Sync, - NextSvc: Service + Clone + Send + Sync + 'static, - NextSvc::Future: Send + Sync + Unpin, - NextSvc::Target: Composer + Default + Send + Sync, - NextSvc::Stream: Send + Sync, - XDP: XfrDataProvider + Clone + Sync + Send + 'static, - XDP::Diff: Debug + Sync, - RequestMeta: Clone + Default + Sync + Send + 'static, -{ - type Target = NextSvc::Target; - type Stream = XfrMiddlewareStream< - NextSvc::Future, - NextSvc::Stream, - ::Item, - >; - type Future = Pin + Send + Sync>>; - - fn call( - &self, - request: Request, - ) -> Self::Future { - let request = request.clone(); - let next_svc = self.next_svc.clone(); - let xfr_data_provider = self.xfr_data_provider.clone(); - let zone_walking_semaphore = self.zone_walking_semaphore.clone(); - let batcher_semaphore = self.batcher_semaphore.clone(); - Box::pin(async move { - match Self::preprocess( - zone_walking_semaphore, - batcher_semaphore, - &request, - xfr_data_provider, - ) - .await - { - Ok(ControlFlow::Continue(())) => { - let request = request.with_new_metadata(()); - let stream = next_svc.call(request).await; - MiddlewareStream::IdentityStream(stream) - } - - Ok(ControlFlow::Break(stream)) => stream, - - Err(rcode) => { - let response = - mk_error_response(request.message(), rcode); - let res = Ok(CallResult::new(response)); - MiddlewareStream::Map(once(ready(res))) - } - } - }) - } -} - -//------------ XfrMapStream --------------------------------------------------- - -pub type XfrResultStream = UnboundedReceiverStream; - -//------------ XfrMiddlewareStream -------------------------------------------- - -pub type XfrMiddlewareStream = MiddlewareStream< - Future, - Stream, - Once>, - XfrResultStream, - StreamItem, ->; - -//------------ XfrMode -------------------------------------------------------- - -#[derive(Copy, Clone, Debug, PartialEq)] -pub enum XfrMode { - AxfrAndIxfr, - AxfrOnly, -} - -//------------ XfrDataProviderError ------------------------------------------- - -#[derive(Copy, Clone, Debug, PartialEq, Eq)] -pub enum XfrDataProviderError { - ParseError(ParseError), - - UnknownZone, - - Refused, - - TemporarilyUnavailable, -} - -//--- From - -impl From for XfrDataProviderError { - fn from(err: ParseError) -> Self { - Self::ParseError(err) - } -} - -//------------ XfrData -------------------------------------------------------- - -/// The data supplied by an [`XfrDataProvider`]. -pub struct XfrData { - /// The zone to transfer. - zone: Zone, - - /// The requested diffs. - /// - /// Empty if the requested diff range could not be satisfied. - diffs: Vec, - - /// Should XFR be done in RFC 5936 backward compatible mode? - /// - /// See: https://www.rfc-editor.org/rfc/rfc5936#section-7 - compatibility_mode: bool, -} - -impl XfrData { - pub fn new( - zone: Zone, - diffs: Vec, - backward_compatible: bool, - ) -> Self { - Self { - zone, - diffs, - compatibility_mode: backward_compatible, - } - } -} - -//------------ XfrDataProvider ------------------------------------------------ - -/// A provider of data needed for responding to XFR requests. -pub trait XfrDataProvider { - type Diff: ZoneDiff + Send + Sync; - - /// Request data needed to respond to an XFR request. - /// - /// Returns Ok if the request is allowed and the requested data is - /// available. - /// - /// Returns Err otherwise. - /// - /// Pass `Some` zone SOA serial number in the `diff_from` parameter to - /// request `ZoneDiff`s from the specified serial to the current SOA - /// serial number of the zone, inclusive, if available. - #[allow(clippy::type_complexity)] - fn request( - &self, - req: &Request, - diff_from: Option, - ) -> Pin< - Box< - dyn Future< - Output = Result< - XfrData, - XfrDataProviderError, - >, - > + Sync - + Send - + '_, - >, - > - where - Octs: Octets + Send + Sync; -} - -//--- impl XfrDataProvider for Deref - -impl XfrDataProvider for U -where - T: XfrDataProvider + 'static, - U: Deref, -{ - type Diff = T::Diff; - - fn request( - &self, - req: &Request, - diff_from: Option, - ) -> Pin< - Box< - dyn Future< - Output = Result< - XfrData, - XfrDataProviderError, - >, - > + Sync - + Send - + '_, - >, - > - where - Octs: Octets + Send + Sync, - { - (**self).request(req, diff_from) - } -} - -//--- impl XfrDataProvider for Zone - -impl XfrDataProvider for Zone { - type Diff = EmptyZoneDiff; - - /// Request data needed to respond to an XFR request. - /// - /// Returns Ok(Self, vec![]) if the given apex name and class match this - /// zone, irrespective of the given request or diff range. - /// - /// Returns Err if the requested zone is not this zone. - fn request( - &self, - req: &Request, - _diff_from: Option, - ) -> Pin< - Box< - dyn Future< - Output = Result< - XfrData, - XfrDataProviderError, - >, - > + Sync - + Send, - >, - > - where - Octs: Octets + Send + Sync, - { - let res = req - .message() - .sole_question() - .map_err(XfrDataProviderError::ParseError) - .and_then(|q| { - if q.qname() == self.apex_name() && q.qclass() == self.class() - { - Ok(XfrData::new(self.clone(), vec![], false)) - } else { - Err(XfrDataProviderError::UnknownZone) - } - }); - - Box::pin(ready(res)) - } -} - -//--- impl XfrDataProvider for ZoneTree - -impl XfrDataProvider for ZoneTree { - type Diff = EmptyZoneDiff; - - /// Request data needed to respond to an XFR request. - /// - /// Returns Ok(zone, vec![]) if the given apex name and class match a zone - /// in this zone tree, irrespective of the given request or diff range. - /// - /// Returns Err if the requested zone is not this zone tree. - fn request( - &self, - req: &Request, - _diff_from: Option, - ) -> Pin< - Box< - dyn Future< - Output = Result< - XfrData, - XfrDataProviderError, - >, - > + Sync - + Send, - >, - > - where - Octs: Octets + Send + Sync, - { - let res = req - .message() - .sole_question() - .map_err(XfrDataProviderError::ParseError) - .and_then(|q| { - if let Some(zone) = self.find_zone(q.qname(), q.qclass()) { - Ok(XfrData::new(zone.clone(), vec![], false)) - } else { - Err(XfrDataProviderError::UnknownZone) - } - }); - - Box::pin(ready(res)) - } -} - -//------------ BatchReadyError ------------------------------------------------ - -#[derive(Clone, Copy, Debug)] -pub enum BatchReadyError { - PushError(PushError), - - SendError, - - MustFitInSingleMessage, -} - -//--- Display - -impl std::fmt::Display for BatchReadyError { - fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { - match self { - BatchReadyError::MustFitInSingleMessage => { - f.write_str("MustFitInSingleMessage") - } - BatchReadyError::PushError(err) => { - f.write_fmt(format_args!("PushError: {err}")) - } - BatchReadyError::SendError => f.write_str("SendError"), - } - } -} - -//--- From - -impl From for BatchReadyError { - fn from(err: PushError) -> Self { - Self::PushError(err) - } -} - -//------------ XfrRrBatcher --------------------------------------------------- - -pub struct XfrRrBatcher { - _phantom: PhantomData<(RequestOctets, Target)>, -} - -impl XfrRrBatcher -where - RequestOctets: Octets + Sync + Send + 'static, - Target: Composer + Default + Send + 'static, -{ - pub fn build( - req_msg: Arc>, - sender: UnboundedSender>, - soft_byte_limit: Option, - hard_rr_limit: Option, - must_fit_in_single_message: bool, - ) -> impl ResourceRecordBatcher - { - let cb_state = CallbackState::new( - req_msg.clone(), - sender, - soft_byte_limit, - hard_rr_limit, - must_fit_in_single_message, - ); - - CallbackBatcher::< - RequestOctets, - Target, - Self, - CallbackState, - >::new(req_msg, cb_state) - } -} - -impl XfrRrBatcher -where - RequestOctets: Octets, - Target: Composer + Default, -{ - fn set_axfr_header( - msg: &Message, - additional: &mut AdditionalBuilder>, - ) { - // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2.1 - // 2.2.1: Header Values - // - // "These are the DNS message header values for AXFR responses. - // - // ID MUST be copied from request -- see Note a) - // - // QR MUST be 1 (Response) - // - // OPCODE MUST be 0 (Standard Query) - // - // Flags: - // AA normally 1 -- see Note b) - // TC MUST be 0 (Not truncated) - // RD RECOMMENDED: copy request's value; MAY be set to 0 - // RA SHOULD be 0 -- see Note c) - // Z "mbz" -- see Note d) - // AD "mbz" -- see Note d) - // CD "mbz" -- see Note d)" - let header = additional.header_mut(); - - // Note: MandatoryMiddlewareSvc will also "fix" ID and QR, so strictly - // speaking this isn't necessary, but as a caller might not use - // MandatoryMiddlewareSvc we do it anyway to try harder to conform to - // the RFC. - header.set_id(msg.header().id()); - header.set_qr(true); - - header.set_opcode(Opcode::QUERY); - header.set_aa(true); - header.set_tc(false); - header.set_rd(msg.header().rd()); - header.set_ra(false); - header.set_z(false); - header.set_ad(false); - header.set_cd(false); - } -} - -//--- Callbacks - -impl - Callbacks> - for XfrRrBatcher -where - RequestOctets: Octets, - Target: Composer + Default, -{ - type Error = BatchReadyError; - - fn batch_started( - cb_state: &CallbackState, - msg: &Message, - ) -> Result>, PushError> { - let mut builder = mk_builder_for_target(); - if let Some(limit) = cb_state.soft_byte_limit { - builder.set_push_limit(limit); - } - let answer = builder.start_answer(msg, Rcode::NOERROR)?; - Ok(answer) - } - - fn batch_ready( - cb_state: &CallbackState, - builder: AnswerBuilder>, - finished: bool, - ) -> Result<(), Self::Error> { - if !finished && cb_state.must_fit_in_single_message { - return Err(BatchReadyError::MustFitInSingleMessage); - } - - trace!("Sending RR batch"); - let mut additional = builder.additional(); - Self::set_axfr_header(&cb_state.req_msg, &mut additional); - let call_result = Ok(CallResult::new(additional)); - cb_state - .sender - .send(call_result) - .map_err(|_unsent_msg| BatchReadyError::SendError) - } - - fn record_pushed( - cb_state: &CallbackState, - answer: &AnswerBuilder>, - ) -> bool { - if let Some(hard_rr_limit) = cb_state.hard_rr_limit { - let ancount = answer.counts().ancount(); - let limit_reached = ancount == hard_rr_limit; - trace!( - "ancount={ancount}, hard_rr_limit={hard_rr_limit}, limit_reached={limit_reached}"); - limit_reached - } else { - false - } - } -} - -//------------ CallbackState -------------------------------------------------- - -struct CallbackState { - req_msg: Arc>, - sender: UnboundedSender>, - soft_byte_limit: Option, - hard_rr_limit: Option, - must_fit_in_single_message: bool, -} - -impl CallbackState { - fn new( - req_msg: Arc>, - sender: UnboundedSender>, - soft_byte_limit: Option, - hard_rr_limit: Option, - must_fit_in_single_message: bool, - ) -> Self { - Self { - req_msg, - sender, - soft_byte_limit, - hard_rr_limit, - must_fit_in_single_message, - } - } -} - -//------------ ZoneFunneler --------------------------------------------------- - -struct ZoneFunneler { - read: Box, - qname: StoredName, - zone_soa_rrset: SharedRrset, - batcher_tx: Sender<(Name, SharedRrset)>, - zone_walk_semaphore: Arc, -} - -impl ZoneFunneler { - fn new( - read: Box, - qname: StoredName, - zone_soa_rrset: SharedRrset, - batcher_tx: Sender<(Name, SharedRrset)>, - zone_walk_semaphore: Arc, - ) -> Self { - Self { - read, - qname, - zone_soa_rrset, - batcher_tx, - zone_walk_semaphore, - } - } - - async fn run(self) -> Result<(), OptRcode> { - // Limit the number of concurrently running XFR related zone walking - // operations. - if self.zone_walk_semaphore.acquire().await.is_err() { - error!("Internal error: Failed to acquire XFR zone walking semaphore"); - return Err(OptRcode::SERVFAIL); - } - - let cloned_batcher_tx = self.batcher_tx.clone(); - let op = Box::new(move |owner: StoredName, rrset: &SharedRrset| { - if rrset.rtype() != Rtype::SOA { - let _ = cloned_batcher_tx - .blocking_send((owner.clone(), rrset.clone())); - // If the blocking send fails it means that the - // batcher is no longer available. This can happen if - // it was no longer able to pass messages back to the - // underlying transport, which can happen if the - // client closed the connection. We don't log this - // because we can't stop the tree walk and so will - // keep hitting this error until the tree walk is - // complete, causing a lot of noise if we were to log - // this. - } - }); - - // Walk the zone tree, invoking our operation for each leaf. - match self.read.is_async() { - true => { - self.read.walk_async(op).await; - if let Err(err) = self - .batcher_tx - .send((self.qname, self.zone_soa_rrset)) - .await - { - error!("Internal error: Failed to send final AXFR SOA to batcher: {err}"); - return Err(OptRcode::SERVFAIL); - } - } - false => { - tokio::task::spawn_blocking(move || { - self.read.walk(op); - if let Err(err) = self - .batcher_tx - .blocking_send((self.qname, self.zone_soa_rrset)) - { - error!("Internal error: Failed to send final AXFR SOA to batcher: {err}"); - // Note: The lack of the final SOA will be detected by the batcher. - } - }); - } - } - - Ok(()) - } -} - -//------------ DiffFunneler ---------------------------------------------------- - -//------------ ZoneFunneler --------------------------------------------------- - -struct DiffFunneler { - qname: StoredName, - zone_soa_rrset: SharedRrset, - diffs: Vec, - batcher_tx: Sender<(Name, SharedRrset)>, -} - -impl DiffFunneler -where - Diff: ZoneDiff, -{ - fn new( - qname: StoredName, - zone_soa_rrset: SharedRrset, - diffs: Vec, - batcher_tx: Sender<(Name, SharedRrset)>, - ) -> Self { - Self { - qname, - zone_soa_rrset, - diffs, - batcher_tx, - } - } - - async fn run(self) -> Result<(), OptRcode> { - // https://datatracker.ietf.org/doc/html/rfc1995#section-4 - // 4. Response Format - // ... - // "If incremental zone transfer is available, one or more - // difference sequences is returned. The list of difference - // sequences is preceded and followed by a copy of the server's - // current version of the SOA. - // - // Each difference sequence represents one update to the zone - // (one SOA serial change) consisting of deleted RRs and added - // RRs. The first RR of the deleted RRs is the older SOA RR - // and the first RR of the added RRs is the newer SOA RR. - // - // Modification of an RR is performed first by removing the - // original RR and then adding the modified one. - // - // The sequences of differential information are ordered oldest - // first newest last. Thus, the differential sequences are the - // history of changes made since the version known by the IXFR - // client up to the server's current version. - // - // RRs in the incremental transfer messages may be partial. That - // is, if a single RR of multiple RRs of the same RR type changes, - // only the changed RR is transferred." - - if let Err(err) = self - .batcher_tx - .send((self.qname.clone(), self.zone_soa_rrset.clone())) - .await - { - error!("Internal error: Failed to send initial IXFR SOA to batcher: {err}"); - return Err(OptRcode::SERVFAIL); - } - - let qname = self.qname.clone(); - - for diff in self.diffs { - // 4. Response Format - // "Each difference sequence represents one update to the - // zone (one SOA serial change) consisting of deleted RRs - // and added RRs. The first RR of the deleted RRs is the - // older SOA RR and the first RR of the added RRs is the - // newer SOA RR. - - let added_soa = - diff.get_added(qname.clone(), Rtype::SOA).await.unwrap(); // The diff MUST have a SOA record - Self::send_diff_section( - &qname, - &self.batcher_tx, - added_soa, - diff.added(), - ) - .await?; - - let removed_soa = - diff.get_removed(qname.clone(), Rtype::SOA).await.unwrap(); // The diff MUST have a SOA record - Self::send_diff_section( - &qname, - &self.batcher_tx, - removed_soa, - diff.removed(), - ) - .await?; - } - - if let Err(err) = self - .batcher_tx - .send((qname.clone(), self.zone_soa_rrset)) - .await - { - error!("Internal error: Failed to send final IXFR SOA to batcher: {err}"); - return Err(OptRcode::SERVFAIL); - } - - Ok(()) - } - - async fn send_diff_section( - qname: &StoredName, - batcher_tx: &Sender<(Name, SharedRrset)>, - soa: &SharedRrset, - diff_stream: ::Stream<'_>, - ) -> Result<(), OptRcode> { - if let Err(err) = batcher_tx.send((qname.clone(), soa.clone())).await - { - error!("Internal error: Failed to send SOA to batcher: {err}"); - return Err(OptRcode::SERVFAIL); - } - - pin_mut!(diff_stream); - - while let Some(item) = diff_stream.next().await { - let (owner, rtype) = item.key(); - if *rtype != Rtype::SOA { - let rrset = item.value(); - if let Err(err) = - batcher_tx.send((owner.clone(), rrset.clone())).await - { - error!("Internal error: Failed to send RRSET to batcher: {err}"); - return Err(OptRcode::SERVFAIL); - } - } - } - - Ok(()) - } -} - -//------------ BatchingRrResponder --------------------------------------------- - -struct BatchingRrResponder { - msg: Arc>, - zone_soa_answer: Answer, - batcher_rx: Receiver<(Name, SharedRrset)>, - response_tx: UnboundedSender>, - compatibility_mode: bool, - soft_byte_limit: usize, - must_fit_in_single_message: bool, - batcher_semaphore: Arc, -} - -impl BatchingRrResponder -where - RequestOctets: Octets + Send + Sync + 'static + Unpin, - Target: Composer + Default + Send + Sync + 'static, -{ - #[allow(clippy::too_many_arguments)] - fn new( - msg: Arc>, - zone_soa_answer: Answer, - batcher_rx: Receiver<(Name, SharedRrset)>, - response_tx: UnboundedSender>, - compatibility_mode: bool, - soft_byte_limit: usize, - must_fit_in_single_message: bool, - batcher_semaphore: Arc, - ) -> Self { - Self { - msg, - zone_soa_answer, - batcher_rx, - response_tx, - compatibility_mode, - soft_byte_limit, - must_fit_in_single_message, - batcher_semaphore, - } - } - - async fn run(mut self) -> Result<(), OptRcode> { - // Limit the number of concurrently running XFR batching - // operations. - if self.batcher_semaphore.acquire().await.is_err() { - error!("Internal error: Failed to acquire XFR batcher semaphore"); - return Err(OptRcode::SERVFAIL); - } - - // SAFETY: msg.sole_question() was already checked in - // get_relevant_question(). - let qclass = self.msg.sole_question().unwrap().qclass(); - - // Note: NSD apparently uses name compresson on AXFR responses - // because AXFR responses they typically contain lots of - // alphabetically ordered duplicate names which compress well. NSD - // limits AXFR responses to 16,383 RRs because DNS name - // compression uses a 14-bit offset (2^14-1=16383) from the start - // of the message to the first occurence of a name instead of - // repeating the name, and name compression is less effective - // over 16383 bytes. (Credit: Wouter Wijngaards) - // - // TODO: Once we start supporting name compression in responses decide - // if we want to behave the same way. - - let hard_rr_limit = match self.compatibility_mode { - true => Some(1), - false => None, - }; - - let mut batcher = XfrRrBatcher::build( - self.msg.clone(), - self.response_tx.clone(), - Some(self.soft_byte_limit), - hard_rr_limit, - self.must_fit_in_single_message, - ); - - let mut last_rr_rtype = None; - - while let Some((owner, rrset)) = self.batcher_rx.recv().await { - for rr in rrset.data() { - last_rr_rtype = Some(rr.rtype()); - - if let Err(err) = - batcher.push((owner.clone(), qclass, rrset.ttl(), rr)) - { - match err { - BatchReadyError::MustFitInSingleMessage => { - // https://datatracker.ietf.org/doc/html/rfc1995#section-2 - // 2. Brief Description of the Protocol - // .. - // "If the UDP reply does not fit, the - // query is responded to with a single SOA - // record of the server's current version - // to inform the client that a TCP query - // should be initiated." - debug_assert!(self.must_fit_in_single_message); - debug!("Responding to IXFR with single SOA because response does not fit in a single UDP reply"); - - let builder = mk_builder_for_target(); - - let resp = self - .zone_soa_answer - .to_message(&self.msg, builder); - - add_to_stream(resp, &self.response_tx); - - return Ok(()); - } - - BatchReadyError::PushError(err) => { - error!("Internal error: Failed to send RR to batcher: {err}"); - return Err(OptRcode::SERVFAIL); - } - - BatchReadyError::SendError => { - debug!("Batcher was unable to send completed batch. Was the receiver dropped?"); - return Err(OptRcode::SERVFAIL); - } - } - } - } - } - - if let Err(err) = batcher.finish() { - debug!("Batcher was unable to finish: {err}"); - return Err(OptRcode::SERVFAIL); - } - - if last_rr_rtype != Some(Rtype::SOA) { - error!( - "Internal error: Last RR was {}, expected SOA", - last_rr_rtype.unwrap() - ); - return Err(OptRcode::SERVFAIL); - } - - Ok(()) - } -} - -//------------ add_to_stream() ------------------------------------------------ - -fn add_to_stream>>( - call_result: T, - response_tx: &UnboundedSender>, -) { - if response_tx.send(Ok(call_result.into())).is_err() { - // We failed to write the message into the response stream. This - // shouldn't happen. We can't now return an error to the client - // because that would require writing to the response stream as - // well. We don't want to panic and take down the entire - // application, so instead just log. - error!("Failed to send DNS message to the internal response stream"); - } -} - -//------------ Tests ---------------------------------------------------------- - -#[cfg(test)] -mod tests { - use core::str::FromStr; - use core::sync::atomic::{AtomicBool, Ordering}; - - use std::borrow::ToOwned; - - use futures_util::StreamExt; - use tokio::time::Instant; - - use crate::base::iana::Class; - use crate::base::{MessageBuilder, Ttl}; - use crate::net::server::message::{ - NonUdpTransportContext, UdpTransportContext, - }; - use crate::net::server::service::ServiceError; - use crate::rdata::{Aaaa, AllRecordData, Cname, Mx, Ns, Txt, A}; - use crate::tsig::{Algorithm, Key, KeyName}; - use crate::zonefile::inplace::Zonefile; - use crate::zonetree::types::Rrset; - use crate::zonetree::{InMemoryZoneDiff, InMemoryZoneDiffBuilder}; - - use super::*; - - type ExpectedRecords = - Vec<(Name, AllRecordData>)>; - - #[tokio::test] - async fn axfr_with_example_zone() { - let zone = load_zone(include_bytes!( - "../../../../test-data/zonefiles/nsd-example.txt" - )); - - let req = mk_axfr_request(zone.apex_name(), ()); - - let res = do_preprocess(zone.clone(), &req).await.unwrap(); - - let ControlFlow::Break(mut stream) = res else { - panic!("AXFR failed"); - }; - - let zone_soa = get_zone_soa(&zone).await; - - let mut expected_records: ExpectedRecords = vec![ - (n("example.com"), zone_soa.clone().into()), - (n("example.com"), Ns::new(n("example.com")).into()), - (n("example.com"), A::new(p("192.0.2.1")).into()), - (n("example.com"), Aaaa::new(p("2001:db8::3")).into()), - (n("www.example.com"), Cname::new(n("example.com")).into()), - (n("mail.example.com"), Mx::new(10, n("example.com")).into()), - (n("example.com"), zone_soa.into()), - ]; - - let msg = stream.next().await.unwrap().unwrap(); - assert!(matches!( - msg.feedback(), - Some(ServiceFeedback::BeginTransaction) - )); - - let stream = assert_stream_eq( - req.message(), - &mut stream, - &mut expected_records, - ) - .await; - - let msg = stream.next().await.unwrap().unwrap(); - assert!(matches!( - msg.feedback(), - Some(ServiceFeedback::EndTransaction) - )); - } - - #[tokio::test] - async fn axfr_multi_response() { - let zone = load_zone(include_bytes!( - "../../../../test-data/zonefiles/big.example.com.txt" - )); - - let req = mk_axfr_request(zone.apex_name(), ()); - - let res = do_preprocess(zone.clone(), &req).await.unwrap(); - - let ControlFlow::Break(mut stream) = res else { - panic!("AXFR failed"); - }; - - let zone_soa = get_zone_soa(&zone).await; - - let mut expected_records: ExpectedRecords = vec![ - (n("example.com"), zone_soa.clone().into()), - (n("example.com"), Ns::new(n("ns1.example.com")).into()), - (n("example.com"), Ns::new(n("ns2.example.com")).into()), - (n("example.com"), Mx::new(10, n("mail.example.com")).into()), - (n("example.com"), A::new(p("192.0.2.1")).into()), - (n("example.com"), Aaaa::new(p("2001:db8:10::1")).into()), - (n("ns1.example.com"), A::new(p("192.0.2.2")).into()), - (n("ns1.example.com"), Aaaa::new(p("2001:db8:10::2")).into()), - (n("ns2.example.com"), A::new(p("192.0.2.3")).into()), - (n("ns2.example.com"), Aaaa::new(p("2001:db8:10::3")).into()), - (n("mail.example.com"), A::new(p("192.0.2.4")).into()), - (n("mail.example.com"), Aaaa::new(p("2001:db8:10::4")).into()), - ]; - - for i in 1..=10000 { - expected_records.push(( - n(&format!("host-{i}.example.com")), - Txt::build_from_slice(b"text").unwrap().into(), - )); - } - - expected_records.push((n("example.com"), zone_soa.into())); - - let msg = stream.next().await.unwrap().unwrap(); - assert!(matches!( - msg.feedback(), - Some(ServiceFeedback::BeginTransaction) - )); - - let stream = assert_stream_eq( - req.message(), - &mut stream, - &mut expected_records, - ) - .await; - - let msg = stream.next().await.unwrap().unwrap(); - assert!(matches!( - msg.feedback(), - Some(ServiceFeedback::EndTransaction) - )); - } - - #[tokio::test] - async fn axfr_delegation_records() { - // https://datatracker.ietf.org/doc/html/rfc5936#section-3.2 - } - - #[tokio::test] - async fn axfr_glue_records() { - // https://datatracker.ietf.org/doc/html/rfc5936#section-3.3 - } - - #[tokio::test] - async fn axfr_name_compression_not_yet_supported() { - // https://datatracker.ietf.org/doc/html/rfc5936#section-3.4 - } - - #[tokio::test] - async fn axfr_occluded_names() { - // https://datatracker.ietf.org/doc/html/rfc5936#section-3.5 - } - - #[tokio::test] - async fn axfr_not_allowed_over_udp() { - // https://datatracker.ietf.org/doc/html/rfc5936#section-4.2 - let zone = load_zone(include_bytes!( - "../../../../test-data/zonefiles/nsd-example.txt" - )); - - let req = mk_udp_axfr_request(zone.apex_name(), ()); - - let res = do_preprocess(zone, &req).await.unwrap(); - - let ControlFlow::Break(mut stream) = res else { - panic!("AXFR failed"); - }; - - let msg = stream.next().await.unwrap().unwrap(); - let resp_builder = msg.into_inner().0.unwrap(); - let resp = resp_builder.as_message(); - - assert_eq!(resp.header().rcode(), Rcode::NOTIMP); - } - - #[tokio::test] - async fn ixfr_rfc1995_section7_full_zone_reply() { - // Based on https://datatracker.ietf.org/doc/html/rfc1995#section-7 - - // initial zone content: - // JAIN.AD.JP. IN SOA NS.JAIN.AD.JP. mohta.jain.ad.jp. ( - // 1 600 600 3600000 604800) - // IN NS NS.JAIN.AD.JP. - // NS.JAIN.AD.JP. IN A 133.69.136.1 - // NEZU.JAIN.AD.JP. IN A 133.69.136.5 - - // Final zone content: - let rfc_1995_zone = r#" -JAIN.AD.JP. IN SOA NS.JAIN.AD.JP. mohta.jain.ad.jp. ( - 3 600 600 3600000 604800) - IN NS NS.JAIN.AD.JP. -NS.JAIN.AD.JP. IN A 133.69.136.1 -JAIN-BB.JAIN.AD.JP. IN A 133.69.136.3 -JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 - "#; - let zone = load_zone(rfc_1995_zone.as_bytes()); - - // Create an object that knows how to provide zone and diff data for - // our zone and no diffs. - let zone_with_diffs = ZoneWithDiffs::new(zone.clone(), vec![]); - - // The following IXFR query - let req = mk_udp_ixfr_request(zone.apex_name(), Serial(1), ()); - - let res = do_preprocess(zone_with_diffs, &req).await.unwrap(); - - let ControlFlow::Break(mut stream) = res else { - panic!("IXFR failed"); - }; - - // could be replied to with the following full zone transfer message: - let zone_soa = get_zone_soa(&zone).await; - - let mut expected_records: ExpectedRecords = vec![ - (n("JAIN.AD.JP."), zone_soa.clone().into()), - (n("JAIN.AD.JP."), Ns::new(n("NS.JAIN.AD.JP.")).into()), - (n("NS.JAIN.AD.JP."), A::new(p("133.69.136.1")).into()), - (n("JAIN-BB.JAIN.AD.JP."), A::new(p("133.69.136.3")).into()), - (n("JAIN-BB.JAIN.AD.JP."), A::new(p("192.41.197.2")).into()), - (n("JAIN.AD.JP."), zone_soa.into()), - ]; - - assert_stream_eq(req.message(), &mut stream, &mut expected_records) - .await; - } - - #[tokio::test] - async fn ixfr_rfc1995_section7_incremental_reply() { - // Based on https://datatracker.ietf.org/doc/html/rfc1995#section-7 - let mut diffs = Vec::new(); - - // initial zone content: - // JAIN.AD.JP. IN SOA NS.JAIN.AD.JP. mohta.jain.ad.jp. ( - // 1 600 600 3600000 604800) - // IN NS NS.JAIN.AD.JP. - // NS.JAIN.AD.JP. IN A 133.69.136.1 - // NEZU.JAIN.AD.JP. IN A 133.69.136.5 - - // Final zone content: - let rfc_1995_zone = r#" -JAIN.AD.JP. IN SOA NS.JAIN.AD.JP. mohta.jain.ad.jp. ( - 3 600 600 3600000 604800) - IN NS NS.JAIN.AD.JP. -NS.JAIN.AD.JP. IN A 133.69.136.1 -JAIN-BB.JAIN.AD.JP. IN A 133.69.136.3 -JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 - "#; - let zone = load_zone(rfc_1995_zone.as_bytes()); - - // Diff 1: NEZU.JAIN.AD.JP. is removed and JAIN-BB.JAIN.AD.JP. is added. - let mut diff = InMemoryZoneDiffBuilder::new(); - - // -- Remove the old SOA. - let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); - let soa = Soa::new( - n("NS.JAIN.AD.JP."), - n("mohta.jain.ad.jp."), - Serial(1), - Ttl::from_secs(600), - Ttl::from_secs(600), - Ttl::from_secs(3600000), - Ttl::from_secs(604800), - ); - rrset.push_data(soa.into()); - diff.remove(n("JAIN.AD.JP"), Rtype::SOA, SharedRrset::new(rrset)); - - // -- Remove the A record. - let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); - rrset.push_data(A::new(p("133.69.136.5")).into()); - diff.remove(n("NEZU.JAIN.AD.JP"), Rtype::A, SharedRrset::new(rrset)); - - // -- Add the new SOA. - let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); - let soa = Soa::new( - n("NS.JAIN.AD.JP."), - n("mohta.jain.ad.jp."), - Serial(2), - Ttl::from_secs(600), - Ttl::from_secs(600), - Ttl::from_secs(3600000), - Ttl::from_secs(604800), - ); - rrset.push_data(soa.into()); - diff.add(n("JAIN.AD.JP"), Rtype::SOA, SharedRrset::new(rrset)); - - // -- Add the new A records. - let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); - rrset.push_data(A::new(p("133.69.136.4")).into()); - rrset.push_data(A::new(p("192.41.197.2")).into()); - diff.add(n("JAIN-BB.JAIN.AD.JP"), Rtype::A, SharedRrset::new(rrset)); - - diffs.push(diff.build().unwrap()); - - // Diff 2: One of the IP addresses of JAIN-BB.JAIN.AD.JP. is changed. - let mut diff = InMemoryZoneDiffBuilder::new(); - - // -- Remove the old SOA. - let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); - let soa = Soa::new( - n("NS.JAIN.AD.JP."), - n("mohta.jain.ad.jp."), - Serial(2), - Ttl::from_secs(600), - Ttl::from_secs(600), - Ttl::from_secs(3600000), - Ttl::from_secs(604800), - ); - rrset.push_data(soa.into()); - diff.remove(n("JAIN.AD.JP"), Rtype::SOA, SharedRrset::new(rrset)); - - // Remove the outdated IP address. - let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); - rrset.push_data(A::new(p("133.69.136.4")).into()); - diff.remove( - n("JAIN-BB.JAIN.AD.JP"), - Rtype::A, - SharedRrset::new(rrset), - ); - - // -- Add the new SOA. - let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); - let soa = Soa::new( - n("NS.JAIN.AD.JP."), - n("mohta.jain.ad.jp."), - Serial(3), - Ttl::from_secs(600), - Ttl::from_secs(600), - Ttl::from_secs(3600000), - Ttl::from_secs(604800), - ); - rrset.push_data(soa.into()); - diff.add(n("JAIN.AD.JP"), Rtype::SOA, SharedRrset::new(rrset)); - - // Add the updated IP address. - let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); - rrset.push_data(A::new(p("133.69.136.3")).into()); - diff.add(n("JAIN-BB.JAIN.AD.JP"), Rtype::A, SharedRrset::new(rrset)); - - diffs.push(diff.build().unwrap()); - - // Create an object that knows how to provide zone and diff data for - // our zone and diffs. - let zone_with_diffs = ZoneWithDiffs::new(zone.clone(), diffs); - - // The following IXFR query - let req = mk_ixfr_request(zone.apex_name(), Serial(1), ()); - - let res = do_preprocess(zone_with_diffs, &req).await.unwrap(); - - let ControlFlow::Break(mut stream) = res else { - panic!("IXFR failed"); - }; - - let zone_soa = get_zone_soa(&zone).await; - - // could be replied to with the following incremental message: - let mut expected_records: ExpectedRecords = vec![ - (n("JAIN.AD.JP."), zone_soa.clone().into()), - ( - n("JAIN.AD.JP."), - Soa::new( - n("NS.JAIN.AD.JP."), - n("mohta.jain.ad.jp."), - Serial(1), - Ttl::from_secs(600), - Ttl::from_secs(600), - Ttl::from_secs(3600000), - Ttl::from_secs(604800), - ) - .into(), - ), - (n("NEZU.JAIN.AD.JP."), A::new(p("133.69.136.5")).into()), - ( - n("JAIN.AD.JP."), - Soa::new( - n("NS.JAIN.AD.JP."), - n("mohta.jain.ad.jp."), - Serial(2), - Ttl::from_secs(600), - Ttl::from_secs(600), - Ttl::from_secs(3600000), - Ttl::from_secs(604800), - ) - .into(), - ), - (n("JAIN-BB.JAIN.AD.JP."), A::new(p("133.69.136.4")).into()), - (n("JAIN-BB.JAIN.AD.JP."), A::new(p("192.41.197.2")).into()), - ( - n("JAIN.AD.JP."), - Soa::new( - n("NS.JAIN.AD.JP."), - n("mohta.jain.ad.jp."), - Serial(2), - Ttl::from_secs(600), - Ttl::from_secs(600), - Ttl::from_secs(3600000), - Ttl::from_secs(604800), - ) - .into(), - ), - (n("JAIN-BB.JAIN.AD.JP."), A::new(p("133.69.136.4")).into()), - ( - n("JAIN.AD.JP."), - Soa::new( - n("NS.JAIN.AD.JP."), - n("mohta.jain.ad.jp."), - Serial(3), - Ttl::from_secs(600), - Ttl::from_secs(600), - Ttl::from_secs(3600000), - Ttl::from_secs(604800), - ) - .into(), - ), - (n("JAIN-BB.JAIN.AD.JP."), A::new(p("133.69.136.3")).into()), - (n("JAIN.AD.JP."), zone_soa.into()), - ]; - - let msg = stream.next().await.unwrap().unwrap(); - assert!(matches!( - msg.feedback(), - Some(ServiceFeedback::BeginTransaction) - )); - - let stream = assert_stream_eq( - req.message(), - &mut stream, - &mut expected_records, - ) - .await; - - let msg = stream.next().await.unwrap().unwrap(); - assert!(matches!( - msg.feedback(), - Some(ServiceFeedback::EndTransaction) - )); - } - - #[tokio::test] - async fn ixfr_rfc1995_section7_udp_packet_overflow() { - // Based on https://datatracker.ietf.org/doc/html/rfc1995#section-7 - let zone = load_zone(include_bytes!( - "../../../../test-data/zonefiles/big.example.com.txt" - )); - - let req = mk_udp_ixfr_request(zone.apex_name(), Serial(0), ()); - - let res = do_preprocess(zone.clone(), &req).await.unwrap(); - - let ControlFlow::Break(mut stream) = res else { - panic!("IXFR failed"); - }; - - let zone_soa = get_zone_soa(&zone).await; - - let mut expected_records: ExpectedRecords = - vec![(n("example.com"), zone_soa.into())]; - - assert_stream_eq(req.message(), &mut stream, &mut expected_records) - .await; - } - - #[tokio::test] - async fn ixfr_multi_response_tcp() {} - - #[tokio::test] - async fn axfr_with_tsig_key() { - // Define an XfrDataProvider that expects to receive a Request that is - // generic over a type that we specify: Authentication. This is the - // type over which the Request produced by TsigMiddlewareSvc is generic. - // When the XfrMiddlewareSvc receives a Request it - // passes it to the XfrDataProvider which in turn can inspect it. - struct KeyReceivingXfrDataProvider { - key: Arc, - checked: Arc, - } - - impl XfrDataProvider>> for KeyReceivingXfrDataProvider { - type Diff = EmptyZoneDiff; - - #[allow(clippy::type_complexity)] - fn request( - &self, - req: &Request>>, - _diff_from: Option, - ) -> Pin< - Box< - dyn Future< - Output = Result< - XfrData, - XfrDataProviderError, - >, - > + Sync - + Send, - >, - > - where - Octs: Octets + Send + Sync, - { - let key = req.metadata().as_ref().unwrap(); - assert_eq!(key.name(), self.key.name()); - self.checked.store(true, Ordering::SeqCst); - Box::pin(ready(Err(XfrDataProviderError::Refused))) - } - } - - let key_name = KeyName::from_str("some_tsig_key_name").unwrap(); - let secret = crate::utils::base64::decode::>( - "zlCZbVJPIhobIs1gJNQfrsS3xCxxsR9pMUrGwG8OgG8=", - ) - .unwrap(); - let key = Arc::new( - Key::new(Algorithm::Sha256, &secret, key_name, None, None) - .unwrap(), - ); - - let metadata = Some(key.clone()); - let req = mk_axfr_request(n("example.com"), metadata); - let checked = Arc::new(AtomicBool::new(false)); - let xdp = KeyReceivingXfrDataProvider { - key, - checked: checked.clone(), - }; - - // Invoke XfrMiddlewareSvc with our custom XfrDataProvidedr. - let _ = do_preprocess(xdp, &req).await; - - // Veirfy that our XfrDataProvider was invoked and received the expected - // TSIG key name data. - assert!(checked.load(Ordering::SeqCst)); - } - - //------------ Helper functions ------------------------------------------- - - fn n(name: &str) -> Name { - Name::from_str(name).unwrap() - } - - fn p(txt: &str) -> T - where - ::Err: std::fmt::Debug, - { - txt.parse().unwrap() - } - - fn load_zone(bytes: &[u8]) -> Zone { - let mut zone_bytes = std::io::BufReader::new(bytes); - let reader = Zonefile::load(&mut zone_bytes).unwrap(); - Zone::try_from(reader).unwrap() - } - - async fn get_zone_soa(zone: &Zone) -> Soa> { - let read = zone.read(); - let zone_soa_answer = - XfrMiddlewareSvc::, TestNextSvc, (), Zone>::read_soa( - &read, - zone.apex_name().to_owned(), - ) - .await - .unwrap(); - let AnswerContent::Data(zone_soa_rrset) = - zone_soa_answer.content().clone() - else { - unreachable!() - }; - let first_rr = zone_soa_rrset.first().unwrap(); - let ZoneRecordData::Soa(soa) = first_rr.data() else { - unreachable!() - }; - soa.clone() - } - - fn mk_axfr_request( - qname: impl ToName, - metadata: T, - ) -> Request, T> { - mk_axfr_request_for_transport( - qname, - metadata, - TransportSpecificContext::NonUdp(NonUdpTransportContext::new( - None, - )), - ) - } - - fn mk_udp_axfr_request( - qname: impl ToName, - metadata: T, - ) -> Request, T> { - mk_axfr_request_for_transport( - qname, - metadata, - TransportSpecificContext::Udp(UdpTransportContext::new(None)), - ) - } - - fn mk_axfr_request_for_transport( - qname: impl ToName, - metadata: T, - transport_specific: TransportSpecificContext, - ) -> Request, T> { - let client_addr = "127.0.0.1:12345".parse().unwrap(); - let received_at = Instant::now(); - let msg = MessageBuilder::new_vec(); - let mut msg = msg.question(); - msg.push((qname, Rtype::AXFR)).unwrap(); - let msg = msg.into_message(); - - Request::new( - client_addr, - received_at, - msg, - transport_specific, - metadata, - ) - } - - fn mk_ixfr_request( - qname: impl ToName + Clone, - serial: Serial, - metadata: T, - ) -> Request, T> { - mk_ixfr_request_for_transport( - qname, - serial, - metadata, - TransportSpecificContext::NonUdp(NonUdpTransportContext::new( - None, - )), - ) - } - - fn mk_udp_ixfr_request( - qname: impl ToName + Clone, - serial: Serial, - metadata: T, - ) -> Request, T> { - mk_ixfr_request_for_transport( - qname, - serial, - metadata, - TransportSpecificContext::Udp(UdpTransportContext::new(None)), - ) - } - - fn mk_ixfr_request_for_transport( - qname: impl ToName + Clone, - serial: Serial, - metadata: T, - transport_specific: TransportSpecificContext, - ) -> Request, T> { - let client_addr = "127.0.0.1:12345".parse().unwrap(); - let received_at = Instant::now(); - let msg = MessageBuilder::new_vec(); - let mut msg = msg.question(); - msg.push((qname.clone(), Rtype::IXFR)).unwrap(); - - let mut msg = msg.authority(); - let ttl = Ttl::from_secs(0); - let soa = Soa::new(n("name"), n("rname"), serial, ttl, ttl, ttl, ttl); - msg.push((qname, Class::IN, Ttl::from_secs(0), soa)) - .unwrap(); - let msg = msg.into_message(); - - Request::new( - client_addr, - received_at, - msg, - transport_specific, - metadata, - ) - } - - async fn do_preprocess>( - zone: XDP, - req: &Request, RequestMeta>, - ) -> Result< - ControlFlow< - XfrMiddlewareStream< - ::Future, - ::Stream, - <::Stream as Stream>::Item, - >, - >, - OptRcode, - > - where - XDP::Diff: Debug + 'static, - { - XfrMiddlewareSvc::, TestNextSvc, RequestMeta, XDP>::preprocess( - Arc::new(Semaphore::new(1)), - Arc::new(Semaphore::new(1)), - req, - zone, - ) - .await - } - - async fn assert_stream_eq< - O: octseq::Octets, - S: Stream>, ServiceError>> + Unpin, - >( - req: &Message, - mut stream: S, - expected_records: &mut ExpectedRecords, - ) -> S { - while !expected_records.is_empty() { - let msg = stream.next().await.unwrap().unwrap(); - - let resp_builder = msg.into_inner().0.unwrap(); - let resp = resp_builder.as_message(); - assert!(resp.is_answer(req)); - let mut records = resp.answer().unwrap().peekable(); - - for (idx, rec) in records.by_ref().enumerate() { - let rec = rec.unwrap(); - - let rec = rec - .into_record::>>() - .unwrap() - .unwrap(); - - eprintln!( - "XFR record {idx} {} {} {} {}", - rec.owner(), - rec.class(), - rec.rtype(), - rec.data(), - ); - - let pos = expected_records - .iter() - .position(|(name, data)| { - name == rec.owner() && data == rec.data() - }) - .unwrap_or_else(|| { - panic!( - "XFR record {idx} {} {} {} {} was not expected", - rec.owner(), - rec.class(), - rec.rtype(), - rec.data(), - ) - }); - - let _ = expected_records.remove(pos); - - eprintln!( - "Found {} {} {}", - rec.owner(), - rec.class(), - rec.rtype() - ) - } - - assert!(records.next().is_none()); - } - - stream - } - - #[derive(Clone)] - struct TestNextSvc; - - impl Service, ()> for TestNextSvc { - type Target = Vec; - type Stream = Once>>; - type Future = Ready; - - fn call(&self, _request: Request, ()>) -> Self::Future { - todo!() - } - } - - struct ZoneWithDiffs { - zone: Zone, - diffs: Vec>, - } - - impl ZoneWithDiffs { - fn new(zone: Zone, diffs: Vec) -> Self { - Self { - zone, - diffs: diffs.into_iter().map(Arc::new).collect(), - } - } - - fn get_diffs( - &self, - diff_from: Option, - ) -> Vec> { - if self.diffs.first().map(|diff| diff.start_serial) == diff_from { - self.diffs.clone() - } else { - vec![] - } - } - } - - impl XfrDataProvider for ZoneWithDiffs { - type Diff = Arc; - fn request( - &self, - req: &Request, - diff_from: Option, - ) -> Pin< - Box< - dyn Future< - Output = Result< - XfrData, - XfrDataProviderError, - >, - > + Sync - + Send, - >, - > - where - Octs: Octets + Send + Sync, - { - let res = req - .message() - .sole_question() - .map_err(XfrDataProviderError::ParseError) - .and_then(|q| { - if q.qname() == self.zone.apex_name() - && q.qclass() == self.zone.class() - { - Ok(XfrData::new( - self.zone.clone(), - self.get_diffs(diff_from), - false, - )) - } else { - Err(XfrDataProviderError::UnknownZone) - } - }); - - Box::pin(ready(res)) - } - } -} diff --git a/src/net/server/middleware/xfr/axfr.rs b/src/net/server/middleware/xfr/axfr.rs new file mode 100644 index 000000000..755b85035 --- /dev/null +++ b/src/net/server/middleware/xfr/axfr.rs @@ -0,0 +1,94 @@ +use std::boxed::Box; +use std::sync::Arc; + +use bytes::Bytes; +use tokio::sync::mpsc::Sender; +use tokio::sync::Semaphore; +use tracing::error; + +use crate::base::iana::OptRcode; +use crate::base::{Name, Rtype}; +use crate::zonetree::{ReadableZone, SharedRrset, StoredName}; + +//------------ ZoneFunneler --------------------------------------------------- + +pub struct ZoneFunneler { + read: Box, + qname: StoredName, + zone_soa_rrset: SharedRrset, + batcher_tx: Sender<(Name, SharedRrset)>, + zone_walk_semaphore: Arc, +} + +impl ZoneFunneler { + pub fn new( + read: Box, + qname: StoredName, + zone_soa_rrset: SharedRrset, + batcher_tx: Sender<(Name, SharedRrset)>, + zone_walk_semaphore: Arc, + ) -> Self { + Self { + read, + qname, + zone_soa_rrset, + batcher_tx, + zone_walk_semaphore, + } + } + + pub async fn run(self) -> Result<(), OptRcode> { + // Limit the number of concurrently running XFR related zone walking + // operations. + if self.zone_walk_semaphore.acquire().await.is_err() { + error!("Internal error: Failed to acquire XFR zone walking semaphore"); + return Err(OptRcode::SERVFAIL); + } + + let cloned_batcher_tx = self.batcher_tx.clone(); + let op = Box::new(move |owner: StoredName, rrset: &SharedRrset| { + if rrset.rtype() != Rtype::SOA { + let _ = cloned_batcher_tx + .blocking_send((owner.clone(), rrset.clone())); + // If the blocking send fails it means that the + // batcher is no longer available. This can happen if + // it was no longer able to pass messages back to the + // underlying transport, which can happen if the + // client closed the connection. We don't log this + // because we can't stop the tree walk and so will + // keep hitting this error until the tree walk is + // complete, causing a lot of noise if we were to log + // this. + } + }); + + // Walk the zone tree, invoking our operation for each leaf. + match self.read.is_async() { + true => { + self.read.walk_async(op).await; + if let Err(err) = self + .batcher_tx + .send((self.qname, self.zone_soa_rrset)) + .await + { + error!("Internal error: Failed to send final AXFR SOA to batcher: {err}"); + return Err(OptRcode::SERVFAIL); + } + } + false => { + tokio::task::spawn_blocking(move || { + self.read.walk(op); + if let Err(err) = self + .batcher_tx + .blocking_send((self.qname, self.zone_soa_rrset)) + { + error!("Internal error: Failed to send final AXFR SOA to batcher: {err}"); + // Note: The lack of the final SOA will be detected by the batcher. + } + }); + } + } + + Ok(()) + } +} diff --git a/src/net/server/middleware/xfr/batcher.rs b/src/net/server/middleware/xfr/batcher.rs new file mode 100644 index 000000000..ae83d5d67 --- /dev/null +++ b/src/net/server/middleware/xfr/batcher.rs @@ -0,0 +1,224 @@ +use core::marker::PhantomData; + +use std::sync::Arc; + +use octseq::Octets; +use tokio::sync::mpsc::UnboundedSender; +use tracing::trace; + +use crate::base::iana::{Opcode, Rcode}; +use crate::base::message_builder::{ + AdditionalBuilder, AnswerBuilder, PushError, +}; +use crate::base::wire::Composer; +use crate::base::{Message, StreamTarget}; +use crate::net::server::batcher::{ + CallbackBatcher, Callbacks, ResourceRecordBatcher, +}; +use crate::net::server::service::{CallResult, ServiceResult}; +use crate::net::server::util::mk_builder_for_target; + +//------------ BatchReadyError ------------------------------------------------ + +#[derive(Clone, Copy, Debug)] +pub enum BatchReadyError { + PushError(PushError), + + SendError, + + MustFitInSingleMessage, +} + +//--- Display + +impl std::fmt::Display for BatchReadyError { + fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { + match self { + BatchReadyError::MustFitInSingleMessage => { + f.write_str("MustFitInSingleMessage") + } + BatchReadyError::PushError(err) => { + f.write_fmt(format_args!("PushError: {err}")) + } + BatchReadyError::SendError => f.write_str("SendError"), + } + } +} + +//--- From + +impl From for BatchReadyError { + fn from(err: PushError) -> Self { + Self::PushError(err) + } +} + +//------------ XfrRrBatcher --------------------------------------------------- + +pub struct XfrRrBatcher { + _phantom: PhantomData<(RequestOctets, Target)>, +} + +impl XfrRrBatcher +where + RequestOctets: Octets + Sync + Send + 'static, + Target: Composer + Default + Send + 'static, +{ + pub fn build( + req_msg: Arc>, + sender: UnboundedSender>, + soft_byte_limit: Option, + hard_rr_limit: Option, + must_fit_in_single_message: bool, + ) -> impl ResourceRecordBatcher + { + let cb_state = CallbackState::new( + req_msg.clone(), + sender, + soft_byte_limit, + hard_rr_limit, + must_fit_in_single_message, + ); + + CallbackBatcher::< + RequestOctets, + Target, + Self, + CallbackState, + >::new(req_msg, cb_state) + } +} + +impl XfrRrBatcher +where + RequestOctets: Octets, + Target: Composer + Default, +{ + fn set_axfr_header( + msg: &Message, + additional: &mut AdditionalBuilder>, + ) { + // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2.1 + // 2.2.1: Header Values + // + // "These are the DNS message header values for AXFR responses. + // + // ID MUST be copied from request -- see Note a) + // + // QR MUST be 1 (Response) + // + // OPCODE MUST be 0 (Standard Query) + // + // Flags: + // AA normally 1 -- see Note b) + // TC MUST be 0 (Not truncated) + // RD RECOMMENDED: copy request's value; MAY be set to 0 + // RA SHOULD be 0 -- see Note c) + // Z "mbz" -- see Note d) + // AD "mbz" -- see Note d) + // CD "mbz" -- see Note d)" + let header = additional.header_mut(); + + // Note: MandatoryMiddlewareSvc will also "fix" ID and QR, so strictly + // speaking this isn't necessary, but as a caller might not use + // MandatoryMiddlewareSvc we do it anyway to try harder to conform to + // the RFC. + header.set_id(msg.header().id()); + header.set_qr(true); + + header.set_opcode(Opcode::QUERY); + header.set_aa(true); + header.set_tc(false); + header.set_rd(msg.header().rd()); + header.set_ra(false); + header.set_z(false); + header.set_ad(false); + header.set_cd(false); + } +} + +//--- Callbacks + +impl + Callbacks> + for XfrRrBatcher +where + RequestOctets: Octets, + Target: Composer + Default, +{ + type Error = BatchReadyError; + + fn batch_started( + cb_state: &CallbackState, + msg: &Message, + ) -> Result>, PushError> { + let mut builder = mk_builder_for_target(); + if let Some(limit) = cb_state.soft_byte_limit { + builder.set_push_limit(limit); + } + let answer = builder.start_answer(msg, Rcode::NOERROR)?; + Ok(answer) + } + + fn batch_ready( + cb_state: &CallbackState, + builder: AnswerBuilder>, + finished: bool, + ) -> Result<(), Self::Error> { + if !finished && cb_state.must_fit_in_single_message { + return Err(BatchReadyError::MustFitInSingleMessage); + } + + trace!("Sending RR batch"); + let mut additional = builder.additional(); + Self::set_axfr_header(&cb_state.req_msg, &mut additional); + let call_result = Ok(CallResult::new(additional)); + cb_state + .sender + .send(call_result) + .map_err(|_unsent_msg| BatchReadyError::SendError) + } + + fn record_pushed( + cb_state: &CallbackState, + answer: &AnswerBuilder>, + ) -> bool { + if let Some(hard_rr_limit) = cb_state.hard_rr_limit { + let ancount = answer.counts().ancount(); + let limit_reached = ancount == hard_rr_limit; + trace!( + "ancount={ancount}, hard_rr_limit={hard_rr_limit}, limit_reached={limit_reached}"); + limit_reached + } else { + false + } + } +} + +//------------ CallbackState -------------------------------------------------- + +struct CallbackState { + req_msg: Arc>, + sender: UnboundedSender>, + soft_byte_limit: Option, + hard_rr_limit: Option, + must_fit_in_single_message: bool, +} + +impl CallbackState { + fn new( + req_msg: Arc>, + sender: UnboundedSender>, + soft_byte_limit: Option, + hard_rr_limit: Option, + must_fit_in_single_message: bool, + ) -> Self { + Self { + req_msg, + sender, + soft_byte_limit, + hard_rr_limit, + must_fit_in_single_message, + } + } +} diff --git a/src/net/server/middleware/xfr/data_provider.rs b/src/net/server/middleware/xfr/data_provider.rs new file mode 100644 index 000000000..65aae2619 --- /dev/null +++ b/src/net/server/middleware/xfr/data_provider.rs @@ -0,0 +1,243 @@ +use core::future::{ready, Future}; +use core::ops::Deref; +use core::pin::Pin; + +use octseq::Octets; + +use std::boxed::Box; +use std::vec::Vec; + +use crate::base::wire::ParseError; +use crate::base::Serial; +use crate::net::server::message::Request; +use crate::zonetree::types::EmptyZoneDiff; +use crate::zonetree::{Zone, ZoneDiff, ZoneTree}; + +//------------ XfrDataProviderError ------------------------------------------- + +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +pub enum XfrDataProviderError { + ParseError(ParseError), + + UnknownZone, + + Refused, + + TemporarilyUnavailable, +} + +//--- From + +impl From for XfrDataProviderError { + fn from(err: ParseError) -> Self { + Self::ParseError(err) + } +} + +//------------ XfrData -------------------------------------------------------- + +/// The data supplied by an [`XfrDataProvider`]. +pub struct XfrData { + /// The zone to transfer. + zone: Zone, + + /// The requested diffs. + /// + /// Empty if the requested diff range could not be satisfied. + diffs: Vec, + + /// Should XFR be done in RFC 5936 backward compatible mode? + /// + /// See: https://www.rfc-editor.org/rfc/rfc5936#section-7 + compatibility_mode: bool, +} + +impl XfrData { + pub fn new( + zone: Zone, + diffs: Vec, + backward_compatible: bool, + ) -> Self { + Self { + zone, + diffs, + compatibility_mode: backward_compatible, + } + } + + pub fn zone(&self) -> &Zone { + &self.zone + } + + pub fn diffs(&self) -> &[Diff] { + &self.diffs + } + + pub fn into_diffs(self) -> Vec { + self.diffs + } + + pub fn compatibility_mode(&self) -> bool { + self.compatibility_mode + } +} + +//------------ XfrDataProvider ------------------------------------------------ + +/// A provider of data needed for responding to XFR requests. +pub trait XfrDataProvider { + type Diff: ZoneDiff + Send + Sync; + + /// Request data needed to respond to an XFR request. + /// + /// Returns Ok if the request is allowed and the requested data is + /// available. + /// + /// Returns Err otherwise. + /// + /// Pass `Some` zone SOA serial number in the `diff_from` parameter to + /// request `ZoneDiff`s from the specified serial to the current SOA + /// serial number of the zone, inclusive, if available. + #[allow(clippy::type_complexity)] + fn request( + &self, + req: &Request, + diff_from: Option, + ) -> Pin< + Box< + dyn Future< + Output = Result< + XfrData, + XfrDataProviderError, + >, + > + Sync + + Send + + '_, + >, + > + where + Octs: Octets + Send + Sync; +} + +//--- impl XfrDataProvider for Deref + +impl XfrDataProvider for U +where + T: XfrDataProvider + 'static, + U: Deref, +{ + type Diff = T::Diff; + + fn request( + &self, + req: &Request, + diff_from: Option, + ) -> Pin< + Box< + dyn Future< + Output = Result< + XfrData, + XfrDataProviderError, + >, + > + Sync + + Send + + '_, + >, + > + where + Octs: Octets + Send + Sync, + { + (**self).request(req, diff_from) + } +} + +//--- impl XfrDataProvider for Zone + +impl XfrDataProvider for Zone { + type Diff = EmptyZoneDiff; + + /// Request data needed to respond to an XFR request. + /// + /// Returns Ok(Self, vec![]) if the given apex name and class match this + /// zone, irrespective of the given request or diff range. + /// + /// Returns Err if the requested zone is not this zone. + fn request( + &self, + req: &Request, + _diff_from: Option, + ) -> Pin< + Box< + dyn Future< + Output = Result< + XfrData, + XfrDataProviderError, + >, + > + Sync + + Send, + >, + > + where + Octs: Octets + Send + Sync, + { + let res = req + .message() + .sole_question() + .map_err(XfrDataProviderError::ParseError) + .and_then(|q| { + if q.qname() == self.apex_name() && q.qclass() == self.class() + { + Ok(XfrData::new(self.clone(), vec![], false)) + } else { + Err(XfrDataProviderError::UnknownZone) + } + }); + + Box::pin(ready(res)) + } +} + +//--- impl XfrDataProvider for ZoneTree + +impl XfrDataProvider for ZoneTree { + type Diff = EmptyZoneDiff; + + /// Request data needed to respond to an XFR request. + /// + /// Returns Ok(zone, vec![]) if the given apex name and class match a zone + /// in this zone tree, irrespective of the given request or diff range. + /// + /// Returns Err if the requested zone is not this zone tree. + fn request( + &self, + req: &Request, + _diff_from: Option, + ) -> Pin< + Box< + dyn Future< + Output = Result< + XfrData, + XfrDataProviderError, + >, + > + Sync + + Send, + >, + > + where + Octs: Octets + Send + Sync, + { + let res = req + .message() + .sole_question() + .map_err(XfrDataProviderError::ParseError) + .and_then(|q| { + if let Some(zone) = self.find_zone(q.qname(), q.qclass()) { + Ok(XfrData::new(zone.clone(), vec![], false)) + } else { + Err(XfrDataProviderError::UnknownZone) + } + }); + + Box::pin(ready(res)) + } +} diff --git a/src/net/server/middleware/xfr/ixfr.rs b/src/net/server/middleware/xfr/ixfr.rs new file mode 100644 index 000000000..262d22a2c --- /dev/null +++ b/src/net/server/middleware/xfr/ixfr.rs @@ -0,0 +1,146 @@ +use std::vec::Vec; + +use bytes::Bytes; +use futures_util::{pin_mut, StreamExt}; +use tokio::sync::mpsc::Sender; +use tracing::error; + +use crate::base::iana::OptRcode; +use crate::base::{Name, Rtype}; +use crate::zonetree::{SharedRrset, StoredName, ZoneDiff, ZoneDiffItem}; + +//------------ DiffFunneler ---------------------------------------------------- + +pub struct DiffFunneler { + qname: StoredName, + zone_soa_rrset: SharedRrset, + diffs: Vec, + batcher_tx: Sender<(Name, SharedRrset)>, +} + +impl DiffFunneler +where + Diff: ZoneDiff, +{ + pub fn new( + qname: StoredName, + zone_soa_rrset: SharedRrset, + diffs: Vec, + batcher_tx: Sender<(Name, SharedRrset)>, + ) -> Self { + Self { + qname, + zone_soa_rrset, + diffs, + batcher_tx, + } + } + + pub async fn run(self) -> Result<(), OptRcode> { + // https://datatracker.ietf.org/doc/html/rfc1995#section-4 + // 4. Response Format + // ... + // "If incremental zone transfer is available, one or more + // difference sequences is returned. The list of difference + // sequences is preceded and followed by a copy of the server's + // current version of the SOA. + // + // Each difference sequence represents one update to the zone + // (one SOA serial change) consisting of deleted RRs and added + // RRs. The first RR of the deleted RRs is the older SOA RR + // and the first RR of the added RRs is the newer SOA RR. + // + // Modification of an RR is performed first by removing the + // original RR and then adding the modified one. + // + // The sequences of differential information are ordered oldest + // first newest last. Thus, the differential sequences are the + // history of changes made since the version known by the IXFR + // client up to the server's current version. + // + // RRs in the incremental transfer messages may be partial. That + // is, if a single RR of multiple RRs of the same RR type changes, + // only the changed RR is transferred." + + if let Err(err) = self + .batcher_tx + .send((self.qname.clone(), self.zone_soa_rrset.clone())) + .await + { + error!("Internal error: Failed to send initial IXFR SOA to batcher: {err}"); + return Err(OptRcode::SERVFAIL); + } + + let qname = self.qname.clone(); + + for diff in self.diffs { + // 4. Response Format + // "Each difference sequence represents one update to the + // zone (one SOA serial change) consisting of deleted RRs + // and added RRs. The first RR of the deleted RRs is the + // older SOA RR and the first RR of the added RRs is the + // newer SOA RR. + + let added_soa = + diff.get_added(qname.clone(), Rtype::SOA).await.unwrap(); // The diff MUST have a SOA record + Self::send_diff_section( + &qname, + &self.batcher_tx, + added_soa, + diff.added(), + ) + .await?; + + let removed_soa = + diff.get_removed(qname.clone(), Rtype::SOA).await.unwrap(); // The diff MUST have a SOA record + Self::send_diff_section( + &qname, + &self.batcher_tx, + removed_soa, + diff.removed(), + ) + .await?; + } + + if let Err(err) = self + .batcher_tx + .send((qname.clone(), self.zone_soa_rrset)) + .await + { + error!("Internal error: Failed to send final IXFR SOA to batcher: {err}"); + return Err(OptRcode::SERVFAIL); + } + + Ok(()) + } + + async fn send_diff_section( + qname: &StoredName, + batcher_tx: &Sender<(Name, SharedRrset)>, + soa: &SharedRrset, + diff_stream: ::Stream<'_>, + ) -> Result<(), OptRcode> { + if let Err(err) = batcher_tx.send((qname.clone(), soa.clone())).await + { + error!("Internal error: Failed to send SOA to batcher: {err}"); + return Err(OptRcode::SERVFAIL); + } + + pin_mut!(diff_stream); + + while let Some(item) = diff_stream.next().await { + let (owner, rtype) = item.key(); + if *rtype != Rtype::SOA { + let rrset = item.value(); + if let Err(err) = + batcher_tx.send((owner.clone(), rrset.clone())).await + { + error!("Internal error: Failed to send RRSET to batcher: {err}"); + return Err(OptRcode::SERVFAIL); + } + } + } + + Ok(()) + } +} diff --git a/src/net/server/middleware/xfr/mod.rs b/src/net/server/middleware/xfr/mod.rs new file mode 100644 index 000000000..cac4f3846 --- /dev/null +++ b/src/net/server/middleware/xfr/mod.rs @@ -0,0 +1,52 @@ +//! RFC 5936 AXFR and RFC 1995 IXFR request handling middleware. +//! +//! This module provides the [`XfrMiddlewareSvc`] service which responds to +//! [RFC 5936] AXFR and [RFC 1995] IXFR requests to perform entire or +//! incremental difference based zone transfers. +//! +//! Determining which requests to honour and with what data is delegated to a +//! caller supplied implementation of the [`XfrDataProvider`] trait. +//! [`XfrDataProvider`] implementations for [`Zone`] and [`ZoneTree`] are +//! provided allowing those types to be used as-is as XFR data providers with +//! this middleware service. +//! +//! [`XfrRrBatcher`], primarily intended for internal use by +//! [`XfrMiddlewareSvc`], handles splitting of large zone transfer replies +//! into batches with as many resource records per response as will fit. +//! +//! # Requiring TSIG authenticated XFR requests +//! +//! To require XFR requests to be TSIG authenticated, implement +//! [`XfrDataProvider>`], extract the key data using +//! [`Request::metadata()`] and verify that a TSIG key was used to sign the +//! request, and that the name and algorithm of the used key are acceptable to +//! you. +//! +//! You can then use your [`XfrDataProvider`] impl with [`XfrMiddlewareSvc`], +//! and add [`TsigMiddlewareSvc`] directly before [`XfrMiddlewareSvc`] in the +//! middleware layer stack so that the used `Key` is made available from the +//! TSIG middleware to the XFR middleware. +//! +//! # Limitations +//! +//! * RFC 1995 2 Brief Description of the Protocol states: _"To ensure +//! integrity, servers should use UDP checksums for all UDP responses."_. +//! This is not implemented. +//! +//! [RFC 5936]: https://www.rfc-editor.org/info/rfc5936 +//! [RFC 1995]: https://www.rfc-editor.org/info/rfc1995 +//! [`TsigMiddlewareSvc`]: +//! crate::net::server::middleware::tsig::TsigMiddlewareSvc +mod axfr; +mod batcher; +mod ixfr; +mod responder; +mod util; + +pub mod data_provider; +pub mod service; + +pub use service::XfrMiddlewareSvc; + +#[cfg(test)] +mod tests; diff --git a/src/net/server/middleware/xfr/responder.rs b/src/net/server/middleware/xfr/responder.rs new file mode 100644 index 000000000..d604aaccd --- /dev/null +++ b/src/net/server/middleware/xfr/responder.rs @@ -0,0 +1,161 @@ +use std::sync::Arc; + +use bytes::Bytes; +use octseq::Octets; +use tokio::sync::mpsc::{Receiver, UnboundedSender}; +use tokio::sync::Semaphore; +use tracing::{debug, error}; + +use crate::base::iana::OptRcode; +use crate::base::rdata::RecordData; +use crate::base::wire::Composer; +use crate::base::{Message, Name, Rtype}; +use crate::net::server::batcher::ResourceRecordBatcher; +use crate::net::server::middleware::xfr::util::add_to_stream; +use crate::net::server::service::ServiceResult; +use crate::net::server::util::mk_builder_for_target; +use crate::zonetree::{Answer, SharedRrset}; + +use super::batcher::{BatchReadyError, XfrRrBatcher}; + +//------------ BatchingRrResponder --------------------------------------------- + +pub struct BatchingRrResponder { + msg: Arc>, + zone_soa_answer: Answer, + batcher_rx: Receiver<(Name, SharedRrset)>, + response_tx: UnboundedSender>, + compatibility_mode: bool, + soft_byte_limit: usize, + must_fit_in_single_message: bool, + batcher_semaphore: Arc, +} + +impl BatchingRrResponder +where + RequestOctets: Octets + Send + Sync + 'static + Unpin, + Target: Composer + Default + Send + Sync + 'static, +{ + #[allow(clippy::too_many_arguments)] + pub fn new( + msg: Arc>, + zone_soa_answer: Answer, + batcher_rx: Receiver<(Name, SharedRrset)>, + response_tx: UnboundedSender>, + compatibility_mode: bool, + soft_byte_limit: usize, + must_fit_in_single_message: bool, + batcher_semaphore: Arc, + ) -> Self { + Self { + msg, + zone_soa_answer, + batcher_rx, + response_tx, + compatibility_mode, + soft_byte_limit, + must_fit_in_single_message, + batcher_semaphore, + } + } + + pub async fn run(mut self) -> Result<(), OptRcode> { + // Limit the number of concurrently running XFR batching + // operations. + if self.batcher_semaphore.acquire().await.is_err() { + error!("Internal error: Failed to acquire XFR batcher semaphore"); + return Err(OptRcode::SERVFAIL); + } + + // SAFETY: msg.sole_question() was already checked in + // get_relevant_question(). + let qclass = self.msg.sole_question().unwrap().qclass(); + + // Note: NSD apparently uses name compresson on AXFR responses + // because AXFR responses they typically contain lots of + // alphabetically ordered duplicate names which compress well. NSD + // limits AXFR responses to 16,383 RRs because DNS name + // compression uses a 14-bit offset (2^14-1=16383) from the start + // of the message to the first occurence of a name instead of + // repeating the name, and name compression is less effective + // over 16383 bytes. (Credit: Wouter Wijngaards) + // + // TODO: Once we start supporting name compression in responses decide + // if we want to behave the same way. + + let hard_rr_limit = match self.compatibility_mode { + true => Some(1), + false => None, + }; + + let mut batcher = XfrRrBatcher::build( + self.msg.clone(), + self.response_tx.clone(), + Some(self.soft_byte_limit), + hard_rr_limit, + self.must_fit_in_single_message, + ); + + let mut last_rr_rtype = None; + + while let Some((owner, rrset)) = self.batcher_rx.recv().await { + for rr in rrset.data() { + last_rr_rtype = Some(rr.rtype()); + + if let Err(err) = + batcher.push((owner.clone(), qclass, rrset.ttl(), rr)) + { + match err { + BatchReadyError::MustFitInSingleMessage => { + // https://datatracker.ietf.org/doc/html/rfc1995#section-2 + // 2. Brief Description of the Protocol + // .. + // "If the UDP reply does not fit, the + // query is responded to with a single SOA + // record of the server's current version + // to inform the client that a TCP query + // should be initiated." + debug_assert!(self.must_fit_in_single_message); + debug!("Responding to IXFR with single SOA because response does not fit in a single UDP reply"); + + let builder = mk_builder_for_target(); + + let resp = self + .zone_soa_answer + .to_message(&self.msg, builder); + + add_to_stream(resp, &self.response_tx); + + return Ok(()); + } + + BatchReadyError::PushError(err) => { + error!("Internal error: Failed to send RR to batcher: {err}"); + return Err(OptRcode::SERVFAIL); + } + + BatchReadyError::SendError => { + debug!("Batcher was unable to send completed batch. Was the receiver dropped?"); + return Err(OptRcode::SERVFAIL); + } + } + } + } + } + + if let Err(err) = batcher.finish() { + debug!("Batcher was unable to finish: {err}"); + return Err(OptRcode::SERVFAIL); + } + + if last_rr_rtype != Some(Rtype::SOA) { + error!( + "Internal error: Last RR was {}, expected SOA", + last_rr_rtype.unwrap() + ); + return Err(OptRcode::SERVFAIL); + } + + Ok(()) + } +} diff --git a/src/net/server/middleware/xfr/service.rs b/src/net/server/middleware/xfr/service.rs new file mode 100644 index 000000000..d0862f5e7 --- /dev/null +++ b/src/net/server/middleware/xfr/service.rs @@ -0,0 +1,754 @@ +use core::future::{ready, Future, Ready}; +use core::marker::PhantomData; +use core::ops::ControlFlow; + +use std::boxed::Box; +use std::fmt::Debug; +use std::pin::Pin; +use std::sync::Arc; +use std::vec::Vec; + +use futures_util::stream::{once, Once, Stream}; +use octseq::Octets; +use tokio::sync::mpsc::unbounded_channel; +use tokio::sync::Semaphore; +use tokio_stream::wrappers::UnboundedReceiverStream; +use tracing::{debug, error, info, trace, warn}; + +use crate::base::iana::{Opcode, OptRcode}; +use crate::base::wire::Composer; +use crate::base::{Message, ParsedName, Question, Rtype, Serial, ToName}; +use crate::net::server::message::{Request, TransportSpecificContext}; +use crate::net::server::middleware::stream::MiddlewareStream; +use crate::net::server::middleware::xfr::axfr::ZoneFunneler; +use crate::net::server::middleware::xfr::data_provider::XfrDataProvider; +use crate::net::server::middleware::xfr::data_provider::XfrDataProviderError; +use crate::net::server::middleware::xfr::ixfr::DiffFunneler; +use crate::net::server::middleware::xfr::responder::BatchingRrResponder; +use crate::net::server::service::{CallResult, Service, ServiceFeedback}; +use crate::net::server::util::{mk_builder_for_target, mk_error_response}; +use crate::rdata::{Soa, ZoneRecordData}; +use crate::zonetree::{ + Answer, AnswerContent, ReadableZone, SharedRrset, StoredName, +}; + +use super::util::{add_to_stream, read_soa}; + +//------------ Constants ----------------------------------------------------- + +/// https://datatracker.ietf.org/doc/html/rfc1035#section-2.3.4 +/// 2.3.4. Size limits +/// "UDP messages 512 octets or less" +const MAX_UDP_MSG_BYTE_LEN: u16 = 512; + +/// https://datatracker.ietf.org/doc/html/rfc1035#section-4.2.2 +/// 4.2.2. TCP usage +/// "The message is prefixed with a two byte length field which gives the +/// message length, excluding the two byte length field" +const MAX_TCP_MSG_BYTE_LEN: u16 = u16::MAX; + +//------------ XfrMiddlewareSvc ---------------------------------------------- + +/// RFC 5936 AXFR and RFC 1995 IXFR request handling middleware. +/// +/// See the [module documentation] for a high level introduction. +/// +/// [module documentation]: crate::net::server::middleware::xfr +#[derive(Clone, Debug)] +pub struct XfrMiddlewareSvc { + /// The upstream [`Service`] to pass requests to and receive responses + /// from. + next_svc: NextSvc, + + /// A caller supplied implementation of [`XfrDataProvider`] for + /// determining which requests to answer and with which data. + xfr_data_provider: XDP, + + /// A limit on the number of XFR related zone walking operations + /// that may run concurrently. + zone_walking_semaphore: Arc, + + /// A limit on the number of XFR related response batching operations that + /// may run concurrently. + batcher_semaphore: Arc, + + _phantom: PhantomData<(RequestOctets, RequestMeta)>, +} + +impl + XfrMiddlewareSvc +where + XDP: XfrDataProvider, +{ + /// Creates a new instance of this middleware. + /// + /// Takes an implementation of [`XfrDataProvider`] as a parameter to + /// determine which requests to honour and with which data. + /// + /// The `max_concurrency` parameter limits the number of simultaneous zone + /// transfer operations that may occur concurrently without blocking. + #[must_use] + pub fn new( + next_svc: NextSvc, + xfr_data_provider: XDP, + max_concurrency: usize, + ) -> Self { + let zone_walking_semaphore = + Arc::new(Semaphore::new(max_concurrency)); + let batcher_semaphore = Arc::new(Semaphore::new(max_concurrency)); + + Self { + next_svc, + xfr_data_provider, + zone_walking_semaphore, + batcher_semaphore, + _phantom: PhantomData, + } + } +} + +impl + XfrMiddlewareSvc +where + RequestOctets: Octets + Send + Sync + 'static + Unpin, + for<'a> ::Range<'a>: Send + Sync, + NextSvc: Service + Clone + Send + Sync + 'static, + NextSvc::Future: Send + Sync + Unpin, + NextSvc::Target: Composer + Default + Send + Sync, + NextSvc::Stream: Send + Sync, + XDP: XfrDataProvider, + XDP::Diff: Debug + 'static, +{ + /// Pre-process received DNS XFR queries. + /// + /// Other types of query will be propagated unmodified to the next + /// middleware or application service in the layered stack of services. + /// + /// Data to respond to the query will be requested from the given + /// [`XfrDataProvider`] which will act according to its policy concerning + /// the given [`Request`]. + pub async fn preprocess( + zone_walking_semaphore: Arc, + batcher_semaphore: Arc, + req: &Request, + xfr_data_provider: XDP, + ) -> Result< + ControlFlow< + XfrMiddlewareStream< + NextSvc::Future, + NextSvc::Stream, + ::Item, + >, + >, + OptRcode, + > { + let msg = req.message(); + + // Do we support this type of request? + let Some(q) = Self::get_relevant_question(msg) else { + return Ok(ControlFlow::Continue(())); + }; + + // https://datatracker.ietf.org/doc/html/rfc1995#section-3 + // 3. Query Format + // "The IXFR query packet format is the same as that of a normal DNS + // query, but with the query type being IXFR and the authority + // section containing the SOA record of client's version of the + // zone." + let ixfr_query_serial = if let Ok(Some(Ok(query_soa))) = msg + .authority() + .map(|section| section.limit_to::>>().next()) + { + Some(query_soa.data().serial()) + } else { + None + }; + + if q.qtype() == Rtype::IXFR && ixfr_query_serial.is_none() { + warn!( + "{} for {} from {} refused: IXFR request lacks authority section SOA", + q.qtype(), + q.qname(), + req.client_addr() + ); + return Err(OptRcode::FORMERR); + } + + // Is transfer allowed for the requested zone for this requestor? + let xfr_data = xfr_data_provider + .request(req, ixfr_query_serial) + .await + .map_err(|err| match err { + XfrDataProviderError::ParseError(err) => { + debug!( + "{} for {} from {} refused: parse error: {err}", + q.qtype(), + q.qname(), + req.client_addr() + ); + OptRcode::FORMERR + } + + XfrDataProviderError::UnknownZone => { + // https://datatracker.ietf.org/doc/html/rfc5936#section-2.2.1 + // 2.2.1 Header Values + // "If a server is not authoritative for the queried + // zone, the server SHOULD set the value to NotAuth(9)" + debug!( + "{} for {} from {} refused: unknown zone", + q.qtype(), + q.qname(), + req.client_addr() + ); + OptRcode::NOTAUTH + } + + XfrDataProviderError::TemporarilyUnavailable => { + // The zone is not yet loaded or has expired, both of + // which are presumably transient conditions and thus + // SERVFAIL is the appropriate response, not NOTAUTH, as + // we know we are supposed to be authoritative for the + // zone but we just don't have the data right now. + warn!( + "{} for {} from {} refused: zone not currently available", + q.qtype(), + q.qname(), + req.client_addr() + ); + OptRcode::SERVFAIL + } + + XfrDataProviderError::Refused => { + warn!( + "{} for {} from {} refused: access denied", + q.qtype(), + q.qname(), + req.client_addr() + ); + OptRcode::REFUSED + } + })?; + + // Read the zone SOA RR + let read = xfr_data.zone().read(); + let Ok(zone_soa_answer) = read_soa(&read, q.qname().to_name()).await + else { + debug!( + "{} for {} from {} refused: name is outside the zone", + q.qtype(), + q.qname(), + req.client_addr() + ); + return Err(OptRcode::SERVFAIL); + }; + + match q.qtype() { + Rtype::AXFR if req.transport_ctx().is_udp() => { + // https://datatracker.ietf.org/doc/html/rfc5936#section-4.2 + // 4.2. UDP + // "With the addition of EDNS0 and applications that require + // many small zones, such as in web hosting and some ENUM + // scenarios, AXFR sessions on UDP would now seem + // desirable. However, there are still some aspects of + // AXFR sessions that are not easily translated to UDP. + // + // Therefore, this document does not update RFC 1035 in + // this respect: AXFR sessions over UDP transport are not + // defined." + warn!( + "{} for {} from {} refused: AXFR not supported over UDP", + q.qtype(), + q.qname(), + req.client_addr() + ); + let response = mk_error_response(msg, OptRcode::NOTIMP); + let res = Ok(CallResult::new(response)); + Ok(ControlFlow::Break(MiddlewareStream::Map(once(ready( + res, + ))))) + } + + Rtype::AXFR | Rtype::IXFR if xfr_data.diffs().is_empty() => { + if q.qtype() == Rtype::IXFR && xfr_data.diffs().is_empty() { + // https://datatracker.ietf.org/doc/html/rfc1995#section-4 + // 4. Response Format + // "If incremental zone transfer is not available, the + // entire zone is returned. The first and the last RR of + // the response is the SOA record of the zone. I.e. the + // behavior is the same as an AXFR response except the + // query type is IXFR." + info!( + "IXFR for {} (serial {} from {}: diffs not available, falling back to AXFR", + q.qname(), + ixfr_query_serial.unwrap(), // SAFETY: Always Some() if IXFR + req.client_addr() + ); + } else { + info!( + "AXFR for {} from {}", + q.qname(), + req.client_addr() + ); + } + let stream = Self::respond_to_axfr_query( + zone_walking_semaphore, + batcher_semaphore, + req, + q.qname().to_name(), + &zone_soa_answer, + read, + xfr_data.compatibility_mode(), + ) + .await?; + + Ok(ControlFlow::Break(stream)) + } + + Rtype::IXFR => { + // SAFETY: Always Some() if IXFR + let ixfr_query_serial = ixfr_query_serial.unwrap(); + info!( + "IXFR for {} (serial {ixfr_query_serial}) from {}", + q.qname(), + req.client_addr() + ); + + // https://datatracker.ietf.org/doc/html/rfc1995#section-2 + // 2. Brief Description of the Protocol + // "Transport of a query may be by either UDP or TCP. If an + // IXFR query is via UDP, the IXFR server may attempt to + // reply using UDP if the entire response can be contained + // in a single DNS packet. If the UDP reply does not fit, + // the query is responded to with a single SOA record of + // the server's current version to inform the client that a + // TCP query should be initiated." + let stream = Self::respond_to_ixfr_query( + batcher_semaphore.clone(), + req, + ixfr_query_serial, + q.qname().to_name(), + &zone_soa_answer, + xfr_data.into_diffs(), + ) + .await?; + + Ok(ControlFlow::Break(stream)) + } + + _ => { + // Other QTYPEs should have been filtered out by get_relevant_question(). + unreachable!(); + } + } + } + + /// Generate and send an AXFR response for a given request and zone. + #[allow(clippy::too_many_arguments)] + async fn respond_to_axfr_query( + zone_walk_semaphore: Arc, + batcher_semaphore: Arc, + req: &Request, + qname: StoredName, + zone_soa_answer: &Answer, + read: Box, + compatibility_mode: bool, + ) -> Result< + XfrMiddlewareStream< + NextSvc::Future, + NextSvc::Stream, + ::Item, + >, + OptRcode, + > { + let AnswerContent::Data(zone_soa_rrset) = + zone_soa_answer.content().clone() + else { + error!( + "AXFR for {qname} from {} refused: zone lacks SOA RR", + req.client_addr() + ); + return Err(OptRcode::SERVFAIL); + }; + + if compatibility_mode { + trace!( + "Compatibility mode enabled for client with IP address {}", + req.client_addr().ip() + ); + } + + // Return a stream of response messages containing: + // - SOA + // - RRSETs, one or more per response message + // - SOA + // + // Neither RFC 5936 nor RFC 1035 defined AXFR for UDP, only for TCP. + // However, RFC 1995 says that for IXFR if no diffs are available the + // full zone should be served just as with AXFR, and that UDP is + // supported as long as the entire XFR response fits in a single + // datagram. Thus we don't check for UDP or TCP here, except to abort + // if the response is too large to fit in a single UDP datagram, + // instead we let the caller that has the context decide whether AXFR + // is supported or not. + // + // References: + // - https://datatracker.ietf.org/doc/html/rfc1995#section-2 + // - https://datatracker.ietf.org/doc/html/rfc1995#section-4 + // - https://datatracker.ietf.org/doc/html/rfc5936#section-4.2 + + let soft_byte_limit = Self::calc_msg_bytes_available(req); + + // Create a stream that will be immediately returned to the caller. + // Async tasks will then push DNS response messages into the stream as + // they become available. + let (response_tx, response_rx) = unbounded_channel(); + let stream = UnboundedReceiverStream::new(response_rx); + + // Create a bounded queue for passing RRsets found during zone walking + // to a task which will batch the RRs together before pushing them + // into the result stream. + let (batcher_tx, batcher_rx) = + tokio::sync::mpsc::channel::<(StoredName, SharedRrset)>(100); + + let must_fit_in_single_message = + matches!(req.transport_ctx(), TransportSpecificContext::Udp(_)); + + if !must_fit_in_single_message { + // Notify the underlying transport to expect a stream of related + // responses. The transport should modify its behaviour to account + // for the potentially slow and long running nature of a + // transaction. + add_to_stream(ServiceFeedback::BeginTransaction, &response_tx); + } + + // Enqueue the zone SOA RRset for the batcher to process. + if batcher_tx + .send((qname.clone(), zone_soa_rrset.clone())) + .await + .is_err() + { + return Err(OptRcode::SERVFAIL); + } + + let msg = req.message().clone(); + + // Stream the remaining non-SOA zone RRsets in the background to the + // batcher. + let zone_funneler = ZoneFunneler::new( + read, + qname, + zone_soa_rrset, + batcher_tx, + zone_walk_semaphore, + ); + + let batching_responder = BatchingRrResponder::new( + req.message().clone(), + zone_soa_answer.clone(), + batcher_rx, + response_tx.clone(), + compatibility_mode, + soft_byte_limit, + must_fit_in_single_message, + batcher_semaphore, + ); + + let cloned_msg = msg.clone(); + let cloned_response_tx = response_tx.clone(); + + // Start the funneler. It will walk the zone and send all of the RRs + // one at a time to the batching responder. + tokio::spawn(async move { + if let Err(rcode) = zone_funneler.run().await { + add_to_stream( + mk_error_response(&cloned_msg, rcode), + &cloned_response_tx, + ); + } + }); + + // Start the batching responder. It will receive RRs from the funneler + // and push them in batches into the response stream. + tokio::spawn(async move { + match batching_responder.run().await { + Ok(()) => { + trace!("Ending transaction"); + add_to_stream( + ServiceFeedback::EndTransaction, + &response_tx, + ); + } + + Err(rcode) => { + add_to_stream( + mk_error_response(&msg, rcode), + &response_tx, + ); + } + } + }); + + // If either the funneler or batcher responder terminate then so will + // the other as they each own half of a send <-> receive channel and + // abort if the other side of the channel is gone. + + Ok(MiddlewareStream::Result(stream)) + } + + // Generate and send an IXFR response for the given request and zone + // diffs. + #[allow(clippy::too_many_arguments)] + async fn respond_to_ixfr_query( + batcher_semaphore: Arc, + req: &Request, + query_serial: Serial, + qname: StoredName, + zone_soa_answer: &Answer, + diffs: Vec, + ) -> Result< + XfrMiddlewareStream< + NextSvc::Future, + NextSvc::Stream, + ::Item, + >, + OptRcode, + > + where + XDP::Diff: Send + 'static, + { + let msg = req.message(); + + let AnswerContent::Data(zone_soa_rrset) = + zone_soa_answer.content().clone() + else { + return Err(OptRcode::SERVFAIL); + }; + let Some(first_rr) = zone_soa_rrset.first() else { + return Err(OptRcode::SERVFAIL); + }; + let ZoneRecordData::Soa(soa) = first_rr.data() else { + return Err(OptRcode::SERVFAIL); + }; + + // Note: Unlike RFC 5936 for AXFR, neither RFC 1995 nor RFC 9103 say + // anything about whether an IXFR response can consist of more than + // one response message, but given the 2^16 byte maximum response size + // of a TCP DNS message and the 2^16 maximum number of ANSWER RRs + // allowed per DNS response, large zones may not fit in a single + // response message and will have to be split into multiple response + // messages. + + // https://datatracker.ietf.org/doc/html/rfc1995#section-2 + // 2. Brief Description of the Protocol + // "If an IXFR query with the same or newer version number than that + // of the server is received, it is replied to with a single SOA + // record of the server's current version, just as in AXFR." + // ^^^^^^^^^^^^^^^ + // Errata https://www.rfc-editor.org/errata/eid3196 points out that + // this is NOT "just as in AXFR" as AXFR does not do that. + if query_serial >= soa.serial() { + trace!("Responding to IXFR with single SOA because query serial >= zone serial"); + let builder = mk_builder_for_target(); + let response = zone_soa_answer.to_message(msg, builder); + let res = Ok(CallResult::new(response)); + return Ok(MiddlewareStream::Map(once(ready(res)))); + } + + // TODO: Add something like the Bind `max-ixfr-ratio` option that + // "sets the size threshold (expressed as a percentage of the size of + // the full zone) beyond which named chooses to use an AXFR response + // rather than IXFR when answering zone transfer requests"? + + let soft_byte_limit = Self::calc_msg_bytes_available(req); + + // Create a stream that will be immediately returned to the caller. + // Async tasks will then push DNS response messages into the stream as + // they become available. + let (response_tx, response_rx) = unbounded_channel(); + let stream = UnboundedReceiverStream::new(response_rx); + + // Create a bounded queue for passing RRsets found during diff walking + // to a task which will batch the RRs together before pushing them + // into the result stream. + let (batcher_tx, batcher_rx) = + tokio::sync::mpsc::channel::<(StoredName, SharedRrset)>(100); + + let must_fit_in_single_message = + matches!(req.transport_ctx(), TransportSpecificContext::Udp(_)); + + if !must_fit_in_single_message { + // Notify the underlying transport to expect a stream of related + // responses. The transport should modify its behaviour to account + // for the potentially slow and long running nature of a + // transaction. + add_to_stream(ServiceFeedback::BeginTransaction, &response_tx); + } + + // Stream the IXFR diffs in the background to the batcher. + let diff_funneler = + DiffFunneler::new(qname, zone_soa_rrset, diffs, batcher_tx); + + let batching_responder = BatchingRrResponder::new( + req.message().clone(), + zone_soa_answer.clone(), + batcher_rx, + response_tx.clone(), + false, + soft_byte_limit, + must_fit_in_single_message, + batcher_semaphore, + ); + + let cloned_msg = msg.clone(); + let cloned_response_tx = response_tx.clone(); + + // Start the funneler. It will walk the diffs and send all of the RRs + // one at a time to the batching responder. + tokio::spawn(async move { + if let Err(rcode) = diff_funneler.run().await { + add_to_stream( + mk_error_response(&cloned_msg, rcode), + &cloned_response_tx, + ); + } + }); + + let cloned_msg = msg.clone(); + + // Start the batching responder. It will receive RRs from the funneler + // and push them in batches into the response stream. + tokio::spawn(async move { + match batching_responder.run().await { + Ok(()) => { + trace!("Ending transaction"); + add_to_stream( + ServiceFeedback::EndTransaction, + &response_tx, + ); + } + + Err(rcode) => { + add_to_stream( + mk_error_response(&cloned_msg, rcode), + &response_tx, + ); + } + } + }); + + // If either the funneler or batcher responder terminate then so will + // the other as they each own half of a send <-> receive channel and + // abort if the other side of the channel is gone. + + Ok(MiddlewareStream::Result(stream)) + } + + /// Is this message for us? + /// + /// Returns `Some(Question)` if the given query uses OPCODE QUERYY and has + /// a first question with a QTYPE of `AXFR` or `IXFR`, `None` otherwise. + fn get_relevant_question( + msg: &Message, + ) -> Option>>> { + if Opcode::QUERY == msg.header().opcode() && !msg.header().qr() { + if let Ok(q) = msg.sole_question() { + if matches!(q.qtype(), Rtype::AXFR | Rtype::IXFR) { + return Some(q); + } + } + } + + None + } + + fn calc_msg_bytes_available(req: &Request) -> usize { + let bytes_available = match req.transport_ctx() { + TransportSpecificContext::Udp(ctx) => { + let max_msg_size = ctx + .max_response_size_hint() + .unwrap_or(MAX_UDP_MSG_BYTE_LEN); + max_msg_size - req.num_reserved_bytes() + } + TransportSpecificContext::NonUdp(_) => { + MAX_TCP_MSG_BYTE_LEN - req.num_reserved_bytes() + } + }; + + bytes_available as usize + } +} + +//--- impl Service + +impl + Service + for XfrMiddlewareSvc +where + RequestOctets: Octets + Send + Sync + Unpin + 'static, + for<'a> ::Range<'a>: Send + Sync, + NextSvc: Service + Clone + Send + Sync + 'static, + NextSvc::Future: Send + Sync + Unpin, + NextSvc::Target: Composer + Default + Send + Sync, + NextSvc::Stream: Send + Sync, + XDP: XfrDataProvider + Clone + Sync + Send + 'static, + XDP::Diff: Debug + Sync, + RequestMeta: Clone + Default + Sync + Send + 'static, +{ + type Target = NextSvc::Target; + type Stream = XfrMiddlewareStream< + NextSvc::Future, + NextSvc::Stream, + ::Item, + >; + type Future = Pin + Send + Sync>>; + + fn call( + &self, + request: Request, + ) -> Self::Future { + let request = request.clone(); + let next_svc = self.next_svc.clone(); + let xfr_data_provider = self.xfr_data_provider.clone(); + let zone_walking_semaphore = self.zone_walking_semaphore.clone(); + let batcher_semaphore = self.batcher_semaphore.clone(); + Box::pin(async move { + match Self::preprocess( + zone_walking_semaphore, + batcher_semaphore, + &request, + xfr_data_provider, + ) + .await + { + Ok(ControlFlow::Continue(())) => { + let request = request.with_new_metadata(()); + let stream = next_svc.call(request).await; + MiddlewareStream::IdentityStream(stream) + } + + Ok(ControlFlow::Break(stream)) => stream, + + Err(rcode) => { + let response = + mk_error_response(request.message(), rcode); + let res = Ok(CallResult::new(response)); + MiddlewareStream::Map(once(ready(res))) + } + } + }) + } +} + +//------------ XfrMapStream --------------------------------------------------- + +pub type XfrResultStream = UnboundedReceiverStream; + +//------------ XfrMiddlewareStream -------------------------------------------- + +pub type XfrMiddlewareStream = MiddlewareStream< + Future, + Stream, + Once>, + XfrResultStream, + StreamItem, +>; diff --git a/src/net/server/middleware/xfr/tests.rs b/src/net/server/middleware/xfr/tests.rs new file mode 100644 index 000000000..ec87646a2 --- /dev/null +++ b/src/net/server/middleware/xfr/tests.rs @@ -0,0 +1,824 @@ +use core::future::{ready, Future, Ready}; +use core::ops::ControlFlow; +use core::pin::Pin; +use core::str::FromStr; +use core::sync::atomic::{AtomicBool, Ordering}; + +use std::borrow::ToOwned; +use std::boxed::Box; +use std::fmt::Debug; +use std::sync::Arc; +use std::vec::Vec; + +use bytes::Bytes; +use futures_util::stream::Once; +use futures_util::{Stream, StreamExt}; +use octseq::Octets; +use tokio::sync::Semaphore; +use tokio::time::Instant; + +use crate::base::iana::{Class, OptRcode, Rcode}; +use crate::base::{ + Message, MessageBuilder, Name, ParsedName, Rtype, Serial, ToName, Ttl, +}; +use crate::net::server::message::{ + NonUdpTransportContext, Request, TransportSpecificContext, + UdpTransportContext, +}; +use crate::net::server::middleware::xfr::data_provider::{ + XfrData, XfrDataProvider, XfrDataProviderError, +}; +use crate::net::server::service::{ + CallResult, Service, ServiceError, ServiceFeedback, ServiceResult, +}; +use crate::rdata::{ + Aaaa, AllRecordData, Cname, Mx, Ns, Soa, Txt, ZoneRecordData, A, +}; +use crate::tsig::{Algorithm, Key, KeyName}; +use crate::zonefile::inplace::Zonefile; +use crate::zonetree::types::{EmptyZoneDiff, Rrset}; +use crate::zonetree::{ + AnswerContent, InMemoryZoneDiff, InMemoryZoneDiffBuilder, SharedRrset, + Zone, +}; + +use super::service::{XfrMiddlewareStream, XfrMiddlewareSvc}; +use super::util::read_soa; + +//------------ ExpectedRecords ------------------------------------------------ + +type ExpectedRecords = Vec<(Name, AllRecordData>)>; + +//------------ Tests ---------------------------------------------------------- + +#[tokio::test] +async fn axfr_with_example_zone() { + let zone = load_zone(include_bytes!( + "../../../../../test-data/zonefiles/nsd-example.txt" + )); + + let req = mk_axfr_request(zone.apex_name(), ()); + + let res = do_preprocess(zone.clone(), &req).await.unwrap(); + + let ControlFlow::Break(mut stream) = res else { + panic!("AXFR failed"); + }; + + let zone_soa = get_zone_soa(&zone).await; + + let mut expected_records: ExpectedRecords = vec![ + (n("example.com"), zone_soa.clone().into()), + (n("example.com"), Ns::new(n("example.com")).into()), + (n("example.com"), A::new(p("192.0.2.1")).into()), + (n("example.com"), Aaaa::new(p("2001:db8::3")).into()), + (n("www.example.com"), Cname::new(n("example.com")).into()), + (n("mail.example.com"), Mx::new(10, n("example.com")).into()), + (n("example.com"), zone_soa.into()), + ]; + + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::BeginTransaction) + )); + + let stream = + assert_stream_eq(req.message(), &mut stream, &mut expected_records) + .await; + + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::EndTransaction) + )); +} + +#[tokio::test] +async fn axfr_multi_response() { + let zone = load_zone(include_bytes!( + "../../../../../test-data/zonefiles/big.example.com.txt" + )); + + let req = mk_axfr_request(zone.apex_name(), ()); + + let res = do_preprocess(zone.clone(), &req).await.unwrap(); + + let ControlFlow::Break(mut stream) = res else { + panic!("AXFR failed"); + }; + + let zone_soa = get_zone_soa(&zone).await; + + let mut expected_records: ExpectedRecords = vec![ + (n("example.com"), zone_soa.clone().into()), + (n("example.com"), Ns::new(n("ns1.example.com")).into()), + (n("example.com"), Ns::new(n("ns2.example.com")).into()), + (n("example.com"), Mx::new(10, n("mail.example.com")).into()), + (n("example.com"), A::new(p("192.0.2.1")).into()), + (n("example.com"), Aaaa::new(p("2001:db8:10::1")).into()), + (n("ns1.example.com"), A::new(p("192.0.2.2")).into()), + (n("ns1.example.com"), Aaaa::new(p("2001:db8:10::2")).into()), + (n("ns2.example.com"), A::new(p("192.0.2.3")).into()), + (n("ns2.example.com"), Aaaa::new(p("2001:db8:10::3")).into()), + (n("mail.example.com"), A::new(p("192.0.2.4")).into()), + (n("mail.example.com"), Aaaa::new(p("2001:db8:10::4")).into()), + ]; + + for i in 1..=10000 { + expected_records.push(( + n(&format!("host-{i}.example.com")), + Txt::build_from_slice(b"text").unwrap().into(), + )); + } + + expected_records.push((n("example.com"), zone_soa.into())); + + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::BeginTransaction) + )); + + let stream = + assert_stream_eq(req.message(), &mut stream, &mut expected_records) + .await; + + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::EndTransaction) + )); +} + +#[tokio::test] +async fn axfr_delegation_records() { + // https://datatracker.ietf.org/doc/html/rfc5936#section-3.2 +} + +#[tokio::test] +async fn axfr_glue_records() { + // https://datatracker.ietf.org/doc/html/rfc5936#section-3.3 +} + +#[tokio::test] +async fn axfr_name_compression_not_yet_supported() { + // https://datatracker.ietf.org/doc/html/rfc5936#section-3.4 +} + +#[tokio::test] +async fn axfr_occluded_names() { + // https://datatracker.ietf.org/doc/html/rfc5936#section-3.5 +} + +#[tokio::test] +async fn axfr_not_allowed_over_udp() { + // https://datatracker.ietf.org/doc/html/rfc5936#section-4.2 + let zone = load_zone(include_bytes!( + "../../../../../test-data/zonefiles/nsd-example.txt" + )); + + let req = mk_udp_axfr_request(zone.apex_name(), ()); + + let res = do_preprocess(zone, &req).await.unwrap(); + + let ControlFlow::Break(mut stream) = res else { + panic!("AXFR failed"); + }; + + let msg = stream.next().await.unwrap().unwrap(); + let resp_builder = msg.into_inner().0.unwrap(); + let resp = resp_builder.as_message(); + + assert_eq!(resp.header().rcode(), Rcode::NOTIMP); +} + +#[tokio::test] +async fn ixfr_rfc1995_section7_full_zone_reply() { + // Based on https://datatracker.ietf.org/doc/html/rfc1995#section-7 + + // initial zone content: + // JAIN.AD.JP. IN SOA NS.JAIN.AD.JP. mohta.jain.ad.jp. ( + // 1 600 600 3600000 604800) + // IN NS NS.JAIN.AD.JP. + // NS.JAIN.AD.JP. IN A 133.69.136.1 + // NEZU.JAIN.AD.JP. IN A 133.69.136.5 + + // Final zone content: + let rfc_1995_zone = r#" +JAIN.AD.JP. IN SOA NS.JAIN.AD.JP. mohta.jain.ad.jp. ( + 3 600 600 3600000 604800) + IN NS NS.JAIN.AD.JP. +NS.JAIN.AD.JP. IN A 133.69.136.1 +JAIN-BB.JAIN.AD.JP. IN A 133.69.136.3 +JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 + "#; + let zone = load_zone(rfc_1995_zone.as_bytes()); + + // Create an object that knows how to provide zone and diff data for + // our zone and no diffs. + let zone_with_diffs = ZoneWithDiffs::new(zone.clone(), vec![]); + + // The following IXFR query + let req = mk_udp_ixfr_request(zone.apex_name(), Serial(1), ()); + + let res = do_preprocess(zone_with_diffs, &req).await.unwrap(); + + let ControlFlow::Break(mut stream) = res else { + panic!("IXFR failed"); + }; + + // could be replied to with the following full zone transfer message: + let zone_soa = get_zone_soa(&zone).await; + + let mut expected_records: ExpectedRecords = vec![ + (n("JAIN.AD.JP."), zone_soa.clone().into()), + (n("JAIN.AD.JP."), Ns::new(n("NS.JAIN.AD.JP.")).into()), + (n("NS.JAIN.AD.JP."), A::new(p("133.69.136.1")).into()), + (n("JAIN-BB.JAIN.AD.JP."), A::new(p("133.69.136.3")).into()), + (n("JAIN-BB.JAIN.AD.JP."), A::new(p("192.41.197.2")).into()), + (n("JAIN.AD.JP."), zone_soa.into()), + ]; + + assert_stream_eq(req.message(), &mut stream, &mut expected_records).await; +} + +#[tokio::test] +async fn ixfr_rfc1995_section7_incremental_reply() { + // Based on https://datatracker.ietf.org/doc/html/rfc1995#section-7 + let mut diffs = Vec::new(); + + // initial zone content: + // JAIN.AD.JP. IN SOA NS.JAIN.AD.JP. mohta.jain.ad.jp. ( + // 1 600 600 3600000 604800) + // IN NS NS.JAIN.AD.JP. + // NS.JAIN.AD.JP. IN A 133.69.136.1 + // NEZU.JAIN.AD.JP. IN A 133.69.136.5 + + // Final zone content: + let rfc_1995_zone = r#" +JAIN.AD.JP. IN SOA NS.JAIN.AD.JP. mohta.jain.ad.jp. ( + 3 600 600 3600000 604800) + IN NS NS.JAIN.AD.JP. +NS.JAIN.AD.JP. IN A 133.69.136.1 +JAIN-BB.JAIN.AD.JP. IN A 133.69.136.3 +JAIN-BB.JAIN.AD.JP. IN A 192.41.197.2 + "#; + let zone = load_zone(rfc_1995_zone.as_bytes()); + + // Diff 1: NEZU.JAIN.AD.JP. is removed and JAIN-BB.JAIN.AD.JP. is added. + let mut diff = InMemoryZoneDiffBuilder::new(); + + // -- Remove the old SOA. + let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); + let soa = Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(1), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ); + rrset.push_data(soa.into()); + diff.remove(n("JAIN.AD.JP"), Rtype::SOA, SharedRrset::new(rrset)); + + // -- Remove the A record. + let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); + rrset.push_data(A::new(p("133.69.136.5")).into()); + diff.remove(n("NEZU.JAIN.AD.JP"), Rtype::A, SharedRrset::new(rrset)); + + // -- Add the new SOA. + let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); + let soa = Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(2), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ); + rrset.push_data(soa.into()); + diff.add(n("JAIN.AD.JP"), Rtype::SOA, SharedRrset::new(rrset)); + + // -- Add the new A records. + let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); + rrset.push_data(A::new(p("133.69.136.4")).into()); + rrset.push_data(A::new(p("192.41.197.2")).into()); + diff.add(n("JAIN-BB.JAIN.AD.JP"), Rtype::A, SharedRrset::new(rrset)); + + diffs.push(diff.build().unwrap()); + + // Diff 2: One of the IP addresses of JAIN-BB.JAIN.AD.JP. is changed. + let mut diff = InMemoryZoneDiffBuilder::new(); + + // -- Remove the old SOA. + let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); + let soa = Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(2), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ); + rrset.push_data(soa.into()); + diff.remove(n("JAIN.AD.JP"), Rtype::SOA, SharedRrset::new(rrset)); + + // Remove the outdated IP address. + let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); + rrset.push_data(A::new(p("133.69.136.4")).into()); + diff.remove(n("JAIN-BB.JAIN.AD.JP"), Rtype::A, SharedRrset::new(rrset)); + + // -- Add the new SOA. + let mut rrset = Rrset::new(Rtype::SOA, Ttl::from_secs(0)); + let soa = Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(3), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ); + rrset.push_data(soa.into()); + diff.add(n("JAIN.AD.JP"), Rtype::SOA, SharedRrset::new(rrset)); + + // Add the updated IP address. + let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(0)); + rrset.push_data(A::new(p("133.69.136.3")).into()); + diff.add(n("JAIN-BB.JAIN.AD.JP"), Rtype::A, SharedRrset::new(rrset)); + + diffs.push(diff.build().unwrap()); + + // Create an object that knows how to provide zone and diff data for + // our zone and diffs. + let zone_with_diffs = ZoneWithDiffs::new(zone.clone(), diffs); + + // The following IXFR query + let req = mk_ixfr_request(zone.apex_name(), Serial(1), ()); + + let res = do_preprocess(zone_with_diffs, &req).await.unwrap(); + + let ControlFlow::Break(mut stream) = res else { + panic!("IXFR failed"); + }; + + let zone_soa = get_zone_soa(&zone).await; + + // could be replied to with the following incremental message: + let mut expected_records: ExpectedRecords = vec![ + (n("JAIN.AD.JP."), zone_soa.clone().into()), + ( + n("JAIN.AD.JP."), + Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(1), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ) + .into(), + ), + (n("NEZU.JAIN.AD.JP."), A::new(p("133.69.136.5")).into()), + ( + n("JAIN.AD.JP."), + Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(2), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ) + .into(), + ), + (n("JAIN-BB.JAIN.AD.JP."), A::new(p("133.69.136.4")).into()), + (n("JAIN-BB.JAIN.AD.JP."), A::new(p("192.41.197.2")).into()), + ( + n("JAIN.AD.JP."), + Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(2), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ) + .into(), + ), + (n("JAIN-BB.JAIN.AD.JP."), A::new(p("133.69.136.4")).into()), + ( + n("JAIN.AD.JP."), + Soa::new( + n("NS.JAIN.AD.JP."), + n("mohta.jain.ad.jp."), + Serial(3), + Ttl::from_secs(600), + Ttl::from_secs(600), + Ttl::from_secs(3600000), + Ttl::from_secs(604800), + ) + .into(), + ), + (n("JAIN-BB.JAIN.AD.JP."), A::new(p("133.69.136.3")).into()), + (n("JAIN.AD.JP."), zone_soa.into()), + ]; + + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::BeginTransaction) + )); + + let stream = + assert_stream_eq(req.message(), &mut stream, &mut expected_records) + .await; + + let msg = stream.next().await.unwrap().unwrap(); + assert!(matches!( + msg.feedback(), + Some(ServiceFeedback::EndTransaction) + )); +} + +#[tokio::test] +async fn ixfr_rfc1995_section7_udp_packet_overflow() { + // Based on https://datatracker.ietf.org/doc/html/rfc1995#section-7 + let zone = load_zone(include_bytes!( + "../../../../../test-data/zonefiles/big.example.com.txt" + )); + + let req = mk_udp_ixfr_request(zone.apex_name(), Serial(0), ()); + + let res = do_preprocess(zone.clone(), &req).await.unwrap(); + + let ControlFlow::Break(mut stream) = res else { + panic!("IXFR failed"); + }; + + let zone_soa = get_zone_soa(&zone).await; + + let mut expected_records: ExpectedRecords = + vec![(n("example.com"), zone_soa.into())]; + + assert_stream_eq(req.message(), &mut stream, &mut expected_records).await; +} + +#[tokio::test] +async fn ixfr_multi_response_tcp() {} + +#[tokio::test] +async fn axfr_with_tsig_key() { + // Define an XfrDataProvider that expects to receive a Request that is + // generic over a type that we specify: Authentication. This is the + // type over which the Request produced by TsigMiddlewareSvc is generic. + // When the XfrMiddlewareSvc receives a Request it + // passes it to the XfrDataProvider which in turn can inspect it. + struct KeyReceivingXfrDataProvider { + key: Arc, + checked: Arc, + } + + impl XfrDataProvider>> for KeyReceivingXfrDataProvider { + type Diff = EmptyZoneDiff; + + #[allow(clippy::type_complexity)] + fn request( + &self, + req: &Request>>, + _diff_from: Option, + ) -> Pin< + Box< + dyn Future< + Output = Result< + XfrData, + XfrDataProviderError, + >, + > + Sync + + Send, + >, + > + where + Octs: Octets + Send + Sync, + { + let key = req.metadata().as_ref().unwrap(); + assert_eq!(key.name(), self.key.name()); + self.checked.store(true, Ordering::SeqCst); + Box::pin(ready(Err(XfrDataProviderError::Refused))) + } + } + + let key_name = KeyName::from_str("some_tsig_key_name").unwrap(); + let secret = crate::utils::base64::decode::>( + "zlCZbVJPIhobIs1gJNQfrsS3xCxxsR9pMUrGwG8OgG8=", + ) + .unwrap(); + let key = Arc::new( + Key::new(Algorithm::Sha256, &secret, key_name, None, None).unwrap(), + ); + + let metadata = Some(key.clone()); + let req = mk_axfr_request(n("example.com"), metadata); + let checked = Arc::new(AtomicBool::new(false)); + let xdp = KeyReceivingXfrDataProvider { + key, + checked: checked.clone(), + }; + + // Invoke XfrMiddlewareSvc with our custom XfrDataProvidedr. + let _ = do_preprocess(xdp, &req).await; + + // Veirfy that our XfrDataProvider was invoked and received the expected + // TSIG key name data. + assert!(checked.load(Ordering::SeqCst)); +} + +//------------ Helper functions ------------------------------------------- + +fn n(name: &str) -> Name { + Name::from_str(name).unwrap() +} + +fn p(txt: &str) -> T +where + ::Err: Debug, +{ + txt.parse().unwrap() +} + +fn load_zone(bytes: &[u8]) -> Zone { + let mut zone_bytes = std::io::BufReader::new(bytes); + let reader = Zonefile::load(&mut zone_bytes).unwrap(); + Zone::try_from(reader).unwrap() +} + +async fn get_zone_soa(zone: &Zone) -> Soa> { + let read = zone.read(); + let zone_soa_answer = + read_soa(&read, zone.apex_name().to_owned()).await.unwrap(); + let AnswerContent::Data(zone_soa_rrset) = + zone_soa_answer.content().clone() + else { + unreachable!() + }; + let first_rr = zone_soa_rrset.first().unwrap(); + let ZoneRecordData::Soa(soa) = first_rr.data() else { + unreachable!() + }; + soa.clone() +} + +fn mk_axfr_request( + qname: impl ToName, + metadata: T, +) -> Request, T> { + mk_axfr_request_for_transport( + qname, + metadata, + TransportSpecificContext::NonUdp(NonUdpTransportContext::new(None)), + ) +} + +fn mk_udp_axfr_request( + qname: impl ToName, + metadata: T, +) -> Request, T> { + mk_axfr_request_for_transport( + qname, + metadata, + TransportSpecificContext::Udp(UdpTransportContext::new(None)), + ) +} + +fn mk_axfr_request_for_transport( + qname: impl ToName, + metadata: T, + transport_specific: TransportSpecificContext, +) -> Request, T> { + let client_addr = "127.0.0.1:12345".parse().unwrap(); + let received_at = Instant::now(); + let msg = MessageBuilder::new_vec(); + let mut msg = msg.question(); + msg.push((qname, Rtype::AXFR)).unwrap(); + let msg = msg.into_message(); + + Request::new(client_addr, received_at, msg, transport_specific, metadata) +} + +fn mk_ixfr_request( + qname: impl ToName + Clone, + serial: Serial, + metadata: T, +) -> Request, T> { + mk_ixfr_request_for_transport( + qname, + serial, + metadata, + TransportSpecificContext::NonUdp(NonUdpTransportContext::new(None)), + ) +} + +fn mk_udp_ixfr_request( + qname: impl ToName + Clone, + serial: Serial, + metadata: T, +) -> Request, T> { + mk_ixfr_request_for_transport( + qname, + serial, + metadata, + TransportSpecificContext::Udp(UdpTransportContext::new(None)), + ) +} + +fn mk_ixfr_request_for_transport( + qname: impl ToName + Clone, + serial: Serial, + metadata: T, + transport_specific: TransportSpecificContext, +) -> Request, T> { + let client_addr = "127.0.0.1:12345".parse().unwrap(); + let received_at = Instant::now(); + let msg = MessageBuilder::new_vec(); + let mut msg = msg.question(); + msg.push((qname.clone(), Rtype::IXFR)).unwrap(); + + let mut msg = msg.authority(); + let ttl = Ttl::from_secs(0); + let soa = Soa::new(n("name"), n("rname"), serial, ttl, ttl, ttl, ttl); + msg.push((qname, Class::IN, Ttl::from_secs(0), soa)) + .unwrap(); + let msg = msg.into_message(); + + Request::new(client_addr, received_at, msg, transport_specific, metadata) +} + +async fn do_preprocess>( + zone: XDP, + req: &Request, RequestMeta>, +) -> Result< + ControlFlow< + XfrMiddlewareStream< + ::Future, + ::Stream, + <::Stream as Stream>::Item, + >, + >, + OptRcode, +> +where + XDP::Diff: Debug + 'static, +{ + XfrMiddlewareSvc::, TestNextSvc, RequestMeta, XDP>::preprocess( + Arc::new(Semaphore::new(1)), + Arc::new(Semaphore::new(1)), + req, + zone, + ) + .await +} + +async fn assert_stream_eq< + O: octseq::Octets, + S: Stream>, ServiceError>> + Unpin, +>( + req: &Message, + mut stream: S, + expected_records: &mut ExpectedRecords, +) -> S { + while !expected_records.is_empty() { + let msg = stream.next().await.unwrap().unwrap(); + + let resp_builder = msg.into_inner().0.unwrap(); + let resp = resp_builder.as_message(); + assert!(resp.is_answer(req)); + let mut records = resp.answer().unwrap().peekable(); + + for (idx, rec) in records.by_ref().enumerate() { + let rec = rec.unwrap(); + + let rec = rec + .into_record::>>() + .unwrap() + .unwrap(); + + eprintln!( + "XFR record {idx} {} {} {} {}", + rec.owner(), + rec.class(), + rec.rtype(), + rec.data(), + ); + + let pos = expected_records + .iter() + .position(|(name, data)| { + name == rec.owner() && data == rec.data() + }) + .unwrap_or_else(|| { + panic!( + "XFR record {idx} {} {} {} {} was not expected", + rec.owner(), + rec.class(), + rec.rtype(), + rec.data(), + ) + }); + + let _ = expected_records.remove(pos); + + eprintln!("Found {} {} {}", rec.owner(), rec.class(), rec.rtype()) + } + + assert!(records.next().is_none()); + } + + stream +} + +#[derive(Clone)] +struct TestNextSvc; + +impl Service, ()> for TestNextSvc { + type Target = Vec; + type Stream = Once>>; + type Future = Ready; + + fn call(&self, _request: Request, ()>) -> Self::Future { + todo!() + } +} + +struct ZoneWithDiffs { + zone: Zone, + diffs: Vec>, +} + +impl ZoneWithDiffs { + fn new(zone: Zone, diffs: Vec) -> Self { + Self { + zone, + diffs: diffs.into_iter().map(Arc::new).collect(), + } + } + + fn get_diffs( + &self, + diff_from: Option, + ) -> Vec> { + if self.diffs.first().map(|diff| diff.start_serial) == diff_from { + self.diffs.clone() + } else { + vec![] + } + } +} + +impl XfrDataProvider for ZoneWithDiffs { + type Diff = Arc; + fn request( + &self, + req: &Request, + diff_from: Option, + ) -> Pin< + Box< + dyn Future< + Output = Result< + XfrData, + XfrDataProviderError, + >, + > + Sync + + Send, + >, + > + where + Octs: Octets + Send + Sync, + { + let res = req + .message() + .sole_question() + .map_err(XfrDataProviderError::ParseError) + .and_then(|q| { + if q.qname() == self.zone.apex_name() + && q.qclass() == self.zone.class() + { + Ok(XfrData::new( + self.zone.clone(), + self.get_diffs(diff_from), + false, + )) + } else { + Err(XfrDataProviderError::UnknownZone) + } + }); + + Box::pin(ready(res)) + } +} diff --git a/src/net/server/middleware/xfr/util.rs b/src/net/server/middleware/xfr/util.rs new file mode 100644 index 000000000..f404bcbb2 --- /dev/null +++ b/src/net/server/middleware/xfr/util.rs @@ -0,0 +1,39 @@ +use std::boxed::Box; + +use bytes::Bytes; +use tokio::sync::mpsc::UnboundedSender; +use tracing::error; + +use crate::base::{Name, Rtype}; +use crate::net::server::service::{CallResult, ServiceResult}; +use crate::zonetree::error::OutOfZone; +use crate::zonetree::{Answer, ReadableZone}; + +//------------ read_soa() ----------------------------------------------------- + +#[allow(clippy::borrowed_box)] +pub async fn read_soa( + read: &Box, + qname: Name, +) -> Result { + match read.is_async() { + true => read.query_async(qname, Rtype::SOA).await, + false => read.query(qname, Rtype::SOA), + } +} + +//------------ add_to_stream() ------------------------------------------------ + +pub fn add_to_stream>>( + call_result: T, + response_tx: &UnboundedSender>, +) { + if response_tx.send(Ok(call_result.into())).is_err() { + // We failed to write the message into the response stream. This + // shouldn't happen. We can't now return an error to the client + // because that would require writing to the response stream as well. + // We don't want to panic and take down the entire application, so + // instead just log. + error!("Failed to send DNS message to the internal response stream"); + } +} From 84950e28fd3b8ef4c815373b593d09e261a82d40 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 11:55:57 +0200 Subject: [PATCH 291/333] Remove unrelated change. --- src/net/server/middleware/tsig.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/net/server/middleware/tsig.rs b/src/net/server/middleware/tsig.rs index fd5da6415..759328965 100644 --- a/src/net/server/middleware/tsig.rs +++ b/src/net/server/middleware/tsig.rs @@ -161,6 +161,7 @@ where } Err(err) => { + // Message is incorrectly signed or signed with an unknown key. warn!( "{} from {} refused: {err}", req.message().header().opcode(), @@ -168,6 +169,7 @@ where ); let builder = mk_builder_for_target(); + let res = match err.build_message(req.message(), builder) { Ok(additional) => Ok(ControlFlow::Break(additional)), Err(err) => { From b8173389a3fbba29b1766eef2dd4c08577048d12 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 12:18:29 +0200 Subject: [PATCH 292/333] Minor comment change. --- src/net/server/tests/integration.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index db9c5f190..af378a961 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -158,7 +158,7 @@ async fn server_tests(#[files("test-data/server/*.rpl")] rpl_file: PathBuf) { let svc = EdnsMiddlewareSvc::new(svc).enable(server_config.edns_tcp_keepalive); - // 4. XFR(-in) middleware service. + // 4. RFC 5936 AXFR and RFC 1995 IXFR middleware service. let svc = XfrMiddlewareSvc::, _, Option>, _>::new( svc, zones, 1, ); From 863636c2bab18c9dc6f7cea553b6f90b405ffd42 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 12:23:07 +0200 Subject: [PATCH 293/333] RustDoc fixes. --- src/net/server/middleware/xfr/mod.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/net/server/middleware/xfr/mod.rs b/src/net/server/middleware/xfr/mod.rs index cac4f3846..ab9d3c121 100644 --- a/src/net/server/middleware/xfr/mod.rs +++ b/src/net/server/middleware/xfr/mod.rs @@ -10,14 +10,10 @@ //! provided allowing those types to be used as-is as XFR data providers with //! this middleware service. //! -//! [`XfrRrBatcher`], primarily intended for internal use by -//! [`XfrMiddlewareSvc`], handles splitting of large zone transfer replies -//! into batches with as many resource records per response as will fit. -//! //! # Requiring TSIG authenticated XFR requests //! //! To require XFR requests to be TSIG authenticated, implement -//! [`XfrDataProvider>`], extract the key data using +//! `XfrDataProvider>`, extract the key data using //! [`Request::metadata()`] and verify that a TSIG key was used to sign the //! request, and that the name and algorithm of the used key are acceptable to //! you. @@ -35,8 +31,12 @@ //! //! [RFC 5936]: https://www.rfc-editor.org/info/rfc5936 //! [RFC 1995]: https://www.rfc-editor.org/info/rfc1995 +//! [`Request::metadata()`]: crate::net::server::message::Request::metadata //! [`TsigMiddlewareSvc`]: //! crate::net::server::middleware::tsig::TsigMiddlewareSvc +//! [`XfrDataProvider`]: super::data_provider::XfrDataProvider +//! [`Zone`]: crate::net::zonetree::Zone +//! [`ZoneTree`]: crate::net::zonetree::ZoneTree mod axfr; mod batcher; mod ixfr; From d4423ecc06b6a0c20e9820a5ad8e43bb60d6241f Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 12:52:59 +0200 Subject: [PATCH 294/333] Add some RustDocs about combining domain features to act as primary or secondary and doing zone maintenance. --- src/net/server/middleware/xfr/mod.rs | 1 + src/net/server/mod.rs | 44 ++++++++++++++++++++++++++++ 2 files changed, 45 insertions(+) diff --git a/src/net/server/middleware/xfr/mod.rs b/src/net/server/middleware/xfr/mod.rs index ab9d3c121..e9acc914d 100644 --- a/src/net/server/middleware/xfr/mod.rs +++ b/src/net/server/middleware/xfr/mod.rs @@ -46,6 +46,7 @@ mod util; pub mod data_provider; pub mod service; +pub use data_provider::{XfrData, XfrDataProvider, XfrDataProviderError}; pub use service::XfrMiddlewareSvc; #[cfg(test)] diff --git a/src/net/server/mod.rs b/src/net/server/mod.rs index 7a2508104..3842e3f6c 100644 --- a/src/net/server/mod.rs +++ b/src/net/server/mod.rs @@ -105,6 +105,50 @@ //! [`Service`] trait yourself and passing an instance of that service to the //! server or middleware service as input. //! +//! ## Zone maintenance and zone transfers +//! +//! This crate provides everything you need to do zone maintenance, i.e. +//! serving entire zones to clients and keeping your own zones synchronized +//! with those of a primary server. +//! +//! If acting as a primary nameserver: +//! - Use [`XfrMiddlewareSvc`] to respond to AXFR and IXFR requests from +//! secondary nameservers. +//! - Implement [`XfrDataProvider`] to define your XFR access policy. +//! - Create [`ZoneDiff`]s when making changes to [`Zone`] content and make +//! those diffs available via your [`XfrDataProvider`] implementation. +//! - Use [`TsigMiddlewareSvc`] to authenticate transfer requests from +//! secondary nameservers. +//! - Use the UDP client support in `net::client` to send out NOTIFY messages +//! on zone change. +//! +//! If acting as a secondary nameserver: +//! - Use [`NotifyMiddlewareSvc`] to detect changes at the primary to zones +//! that you are mirroring. +//! - Use the TCP client support in [`net::client`] to make outbound XFR +//! requests on SOA timer expiration or NOTIFY to fetch changes to zone +//! content. +//! - Use [`net::client::tsig`] to authenticate your transfer requests to +//! primary nameservers. +//! - Use [`XfrResponseInterpreter`] and [`ZoneUpdater`] to parse transfer +//! responses and apply the changes to your zones. +//! +//! Additionally you may wish to use [`ZoneTree`] to simplify serving multiple +//! zones. +//! +//! [`net::client`]: crate::net::client +//! [`net::client::tsig`]: crate::net::client::tsig +//! [`NotifyMiddlewareSvc`]: middleware::notify::NotifyMiddlewareSvc +//! [`TsigMiddlewareSvc`]: middleware::tsig::TsigMiddlewareSvc +//! [`XfrMiddlewareSvc`]: middleware::xfr::XfrMiddlewareSvc +//! [`XfrDataProvider`]: middleware::xfr::XfrDataProvider +//! [`XfrResponseInterpreter`]: +//! crate::net::xfr::protocol::XfrResponseInterpreter +//! [`Zone`]: crate::zonetree::Zone +//! [`ZoneDiff`]: crate::zonetree::ZoneDiff +//! [`ZoneTree`]: crate::zonetree::ZoneTree +//! [`ZoneUpdater`]: crate::zonetree::update::ZoneUpdater +//! //! # Advanced //! //! ## Memory allocation From d54782aca3cc20800c0de9824a32b23b7867761f Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 14:01:06 +0200 Subject: [PATCH 295/333] Simplify test key store Arc wrapper. --- src/net/server/tests/integration.rs | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index af378a961..1be3cf4e1 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -53,6 +53,7 @@ use crate::utils::base16; use crate::zonefile::inplace::Zonefile; use crate::zonetree::{Answer, Zone}; use crate::zonetree::{StoredName, ZoneBuilder, ZoneTree}; +use core::ops::Deref; //----------- Tests ---------------------------------------------------------- @@ -582,10 +583,6 @@ impl KeyStore for Arc { name: &N, algorithm: Algorithm, ) -> Option { - if let Ok(name) = name.try_to_name() { - self.get(&(name, algorithm)).cloned() - } else { - None - } + Arc::deref(&self).get_key(name, algorithm) } } From 2912ec30af63c303ef9ceae0826c4f96a64b8e4a Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 14:01:46 +0200 Subject: [PATCH 296/333] Organize imports. --- src/net/server/tests/integration.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index 1be3cf4e1..0fabfedd5 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -1,4 +1,5 @@ use core::future::{ready, Future}; +use core::ops::Deref; use core::pin::Pin; use core::str::FromStr; @@ -53,7 +54,6 @@ use crate::utils::base16; use crate::zonefile::inplace::Zonefile; use crate::zonetree::{Answer, Zone}; use crate::zonetree::{StoredName, ZoneBuilder, ZoneTree}; -use core::ops::Deref; //----------- Tests ---------------------------------------------------------- From e2f921e0aa6cc8d58a775c6494917cb36849942e Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 14:04:45 +0200 Subject: [PATCH 297/333] Clippy. --- src/net/server/tests/integration.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/net/server/tests/integration.rs b/src/net/server/tests/integration.rs index 0fabfedd5..6d63adcca 100644 --- a/src/net/server/tests/integration.rs +++ b/src/net/server/tests/integration.rs @@ -583,6 +583,6 @@ impl KeyStore for Arc { name: &N, algorithm: Algorithm, ) -> Option { - Arc::deref(&self).get_key(name, algorithm) + Arc::deref(self).get_key(name, algorithm) } } From 185d99c23531eebf84bee51801a19489bf307fd6 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 14:24:24 +0200 Subject: [PATCH 298/333] Extend the serve-zone example with minimal AXFR out with support for a hard-coded TSIG key and printing of received NOTIFY triggers. --- examples/serve-zone.rs | 68 ++++++++++++++++++++++++++++++++++++++---- 1 file changed, 63 insertions(+), 5 deletions(-) diff --git a/examples/serve-zone.rs b/examples/serve-zone.rs index db9c986e0..98e19d09e 100644 --- a/examples/serve-zone.rs +++ b/examples/serve-zone.rs @@ -21,7 +21,11 @@ use std::process::exit; use std::sync::Arc; use std::time::Duration; -use domain::base::iana::Rcode; +use core::future::{ready, Future}; +use core::net::IpAddr; +use core::pin::Pin; +use core::str::FromStr; +use domain::base::iana::{Class, Rcode}; use domain::base::ToName; use domain::net::server::buf::VecBufSource; use domain::net::server::dgram::DgramServer; @@ -30,12 +34,19 @@ use domain::net::server::message::Request; use domain::net::server::middleware::cookies::CookiesMiddlewareSvc; use domain::net::server::middleware::edns::EdnsMiddlewareSvc; use domain::net::server::middleware::mandatory::MandatoryMiddlewareSvc; +use domain::net::server::middleware::notify::{ + Notifiable, NotifyError, NotifyMiddlewareSvc, +}; +use domain::net::server::middleware::tsig::TsigMiddlewareSvc; +use domain::net::server::middleware::xfr::XfrMiddlewareSvc; use domain::net::server::service::{CallResult, ServiceResult}; use domain::net::server::stream::StreamServer; use domain::net::server::util::{mk_builder_for_target, service_fn}; +use domain::tsig::{Algorithm, Key, KeyName}; use domain::zonefile::inplace; -use domain::zonetree::Answer; +use domain::zonetree::{Answer, StoredName}; use domain::zonetree::{Zone, ZoneTree}; +use std::collections::HashMap; use tokio::net::{TcpListener, UdpSocket}; use tracing_subscriber::EnvFilter; @@ -50,6 +61,18 @@ async fn main() { .try_init() .ok(); + // Create a TSIG key store with a demo key. + let mut key_store = HashMap::new(); + let key_name = KeyName::from_str("demo-key").unwrap(); + let secret = domain::utils::base64::decode::>( + "zlCZbVJPIhobIs1gJNQfrsS3xCxxsR9pMUrGwG8OgG8=", + ) + .unwrap(); + let key = + Key::new(Algorithm::Sha256, &secret, key_name.clone(), None, None) + .unwrap(); + key_store.insert((key_name, Algorithm::Sha256), key.into()); + // Populate a zone tree with test data let zone_bytes = include_bytes!("../test-data/zonefiles/nsd-example.txt"); let mut zone_bytes = BufReader::new(&zone_bytes[..]); @@ -77,15 +100,20 @@ async fn main() { let zones = Arc::new(zones); let addr = "127.0.0.1:8053"; - let svc = service_fn(my_service, zones); + let svc = service_fn(my_service, zones.clone()); #[cfg(feature = "siphasher")] let svc = CookiesMiddlewareSvc::, _, _>::with_random_secret(svc); let svc = EdnsMiddlewareSvc::, _, _>::new(svc); + let svc = XfrMiddlewareSvc::, _, Option>, _>::new( + svc, zones, 1, + ); + let svc = NotifyMiddlewareSvc::new(svc, DemoNotifyTarget); let svc = MandatoryMiddlewareSvc::, _, _>::new(svc); + let svc = TsigMiddlewareSvc::new(svc, key_store); let svc = Arc::new(svc); - let sock = UdpSocket::bind(addr).await.unwrap(); + let sock = UdpSocket::bind(&addr).await.unwrap(); let sock = Arc::new(sock); let mut udp_metrics = vec![]; let num_cores = std::thread::available_parallelism().unwrap().get(); @@ -103,7 +131,13 @@ async fn main() { tokio::spawn(async move { tcp_srv.run().await }); - eprintln!("Ready"); + eprintln!("Listening on {addr}"); + eprintln!("Try:"); + eprintln!(" dig @127.0.0.1 -p 8053 example.com"); + eprintln!(" dig @127.0.0.1 -p 8053 -y hmac-sha256:demo-key:zlCZbVJPIhobIs1gJNQfrsS3xCxxsR9pMUrGwG8OgG8= example.com AXFR"); + eprintln!(" dig @127.0.0.1 -p 8053 +opcode=notify example.com SOA"); + eprintln!(""); + eprintln!("Tip: set env var RUST_LOG=info (or debug or trace) for more log output."); tokio::spawn(async move { loop { @@ -163,3 +197,27 @@ fn my_service( let additional = answer.to_message(request.message(), builder); Ok(CallResult::new(additional)) } + +#[derive(Copy, Clone, Default, Debug)] +struct DemoNotifyTarget; + +impl Notifiable for DemoNotifyTarget { + fn notify_zone_changed( + &self, + class: Class, + apex_name: &StoredName, + source: IpAddr, + ) -> Pin< + Box> + Sync + Send + '_>, + > { + eprintln!("Notify received from {source} of change to zone {apex_name} in class {class}"); + + let res = match apex_name.to_string().to_lowercase().as_str() { + "example.com" => Ok(()), + "othererror.com" => Err(NotifyError::Other), + _ => Err(NotifyError::NotAuthForZone), + }; + + Box::pin(ready(res)) + } +} From a1a92130e37d9b420d8564b4d44343037d392886 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 14:27:36 +0200 Subject: [PATCH 299/333] Clippy. --- examples/serve-zone.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/serve-zone.rs b/examples/serve-zone.rs index 98e19d09e..1931b6748 100644 --- a/examples/serve-zone.rs +++ b/examples/serve-zone.rs @@ -136,7 +136,7 @@ async fn main() { eprintln!(" dig @127.0.0.1 -p 8053 example.com"); eprintln!(" dig @127.0.0.1 -p 8053 -y hmac-sha256:demo-key:zlCZbVJPIhobIs1gJNQfrsS3xCxxsR9pMUrGwG8OgG8= example.com AXFR"); eprintln!(" dig @127.0.0.1 -p 8053 +opcode=notify example.com SOA"); - eprintln!(""); + eprintln!(); eprintln!("Tip: set env var RUST_LOG=info (or debug or trace) for more log output."); tokio::spawn(async move { From a26221e8ef4ef0b83afcec402d12b3ebe7900f0b Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 16:17:08 +0200 Subject: [PATCH 300/333] IXFR fix: removes before adds. --- src/net/server/middleware/xfr/ixfr.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/net/server/middleware/xfr/ixfr.rs b/src/net/server/middleware/xfr/ixfr.rs index 262d22a2c..26361ba92 100644 --- a/src/net/server/middleware/xfr/ixfr.rs +++ b/src/net/server/middleware/xfr/ixfr.rs @@ -81,23 +81,23 @@ where // older SOA RR and the first RR of the added RRs is the // newer SOA RR. - let added_soa = - diff.get_added(qname.clone(), Rtype::SOA).await.unwrap(); // The diff MUST have a SOA record + let removed_soa = + diff.get_removed(qname.clone(), Rtype::SOA).await.unwrap(); // The diff MUST have a SOA record Self::send_diff_section( &qname, &self.batcher_tx, - added_soa, - diff.added(), + removed_soa, + diff.removed(), ) .await?; - let removed_soa = - diff.get_removed(qname.clone(), Rtype::SOA).await.unwrap(); // The diff MUST have a SOA record + let added_soa = + diff.get_added(qname.clone(), Rtype::SOA).await.unwrap(); // The diff MUST have a SOA record Self::send_diff_section( &qname, &self.batcher_tx, - removed_soa, - diff.removed(), + added_soa, + diff.added(), ) .await?; } From dceb29960a16fe473ec18afe03ed5397f04e3a14 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 16:17:33 +0200 Subject: [PATCH 301/333] Fix bump SOA serial if not bumped to actually properly detect not bumped. --- src/zonetree/in_memory/write.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/zonetree/in_memory/write.rs b/src/zonetree/in_memory/write.rs index c4241f438..d99631cab 100644 --- a/src/zonetree/in_memory/write.rs +++ b/src/zonetree/in_memory/write.rs @@ -305,10 +305,14 @@ impl WritableZone for WriteZone { // that case be a SOA record in the new version of the zone anyway. let old_soa_rr = self.apex.get_soa(self.last_published_version()); - let new_soa_rr = self.apex.get_soa(self.new_version); + let mut new_soa_rr = self.apex.get_soa(self.new_version); - if bump_soa_serial && old_soa_rr.is_some() && new_soa_rr.is_none() { + if bump_soa_serial + && old_soa_rr.is_some() + && (new_soa_rr.is_none() || new_soa_rr == old_soa_rr) + { self.bump_soa_serial(&old_soa_rr); + new_soa_rr = self.apex.get_soa(self.new_version); } // Extract (and finish) the created diff, if any. From ab6bba10e20a150e962d2bd695bb670ab6926555 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 16:17:58 +0200 Subject: [PATCH 302/333] Extend serve-zone example to demonstrate AXFR. TSIG and NOTIFY. --- examples/serve-zone.rs | 164 +++++++++++++++++++++++++++++++++++++---- 1 file changed, 150 insertions(+), 14 deletions(-) diff --git a/examples/serve-zone.rs b/examples/serve-zone.rs index 1931b6748..a72ffddb6 100644 --- a/examples/serve-zone.rs +++ b/examples/serve-zone.rs @@ -15,18 +15,24 @@ //! //! dig @127.0.0.1 -p 8053 AXFR example.com +use core::future::{ready, Future}; +use core::pin::Pin; +use core::str::FromStr; + +use std::collections::HashMap; use std::future::pending; use std::io::BufReader; use std::process::exit; -use std::sync::Arc; +use std::sync::{Arc, Mutex}; use std::time::Duration; -use core::future::{ready, Future}; -use core::net::IpAddr; -use core::pin::Pin; -use core::str::FromStr; +use tokio::net::{TcpListener, UdpSocket}; +use tracing_subscriber::EnvFilter; + use domain::base::iana::{Class, Rcode}; -use domain::base::ToName; +use domain::base::name::OwnedLabel; +use domain::base::net::IpAddr; +use domain::base::{Name, Rtype, Serial, ToName, Ttl}; use domain::net::server::buf::VecBufSource; use domain::net::server::dgram::DgramServer; use domain::net::server::message::Request; @@ -38,17 +44,21 @@ use domain::net::server::middleware::notify::{ Notifiable, NotifyError, NotifyMiddlewareSvc, }; use domain::net::server::middleware::tsig::TsigMiddlewareSvc; -use domain::net::server::middleware::xfr::XfrMiddlewareSvc; +use domain::net::server::middleware::xfr::{ + XfrData, XfrDataProvider, XfrDataProviderError, XfrMiddlewareSvc, +}; use domain::net::server::service::{CallResult, ServiceResult}; use domain::net::server::stream::StreamServer; use domain::net::server::util::{mk_builder_for_target, service_fn}; use domain::tsig::{Algorithm, Key, KeyName}; use domain::zonefile::inplace; -use domain::zonetree::{Answer, StoredName}; +use domain::zonetree::{ + Answer, InMemoryZoneDiff, Rrset, SharedRrset, StoredName, +}; use domain::zonetree::{Zone, ZoneTree}; -use std::collections::HashMap; -use tokio::net::{TcpListener, UdpSocket}; -use tracing_subscriber::EnvFilter; +use octseq::Octets; +use rand::distributions::Alphanumeric; +use rand::Rng; #[tokio::main()] async fn main() { @@ -62,7 +72,7 @@ async fn main() { .ok(); // Create a TSIG key store with a demo key. - let mut key_store = HashMap::new(); + let mut key_store = HashMap::<(KeyName, Algorithm), Key>::new(); let key_name = KeyName::from_str("demo-key").unwrap(); let secret = domain::utils::base64::decode::>( "zlCZbVJPIhobIs1gJNQfrsS3xCxxsR9pMUrGwG8OgG8=", @@ -102,11 +112,15 @@ async fn main() { let addr = "127.0.0.1:8053"; let svc = service_fn(my_service, zones.clone()); + let zones_and_diffs = ZoneTreeWithDiffs::new(zones.clone()); + #[cfg(feature = "siphasher")] let svc = CookiesMiddlewareSvc::, _, _>::with_random_secret(svc); let svc = EdnsMiddlewareSvc::, _, _>::new(svc); - let svc = XfrMiddlewareSvc::, _, Option>, _>::new( - svc, zones, 1, + let svc = XfrMiddlewareSvc::, _, _, _>::new( + svc, + zones_and_diffs.clone(), + 1, ); let svc = NotifyMiddlewareSvc::new(svc, DemoNotifyTarget); let svc = MandatoryMiddlewareSvc::, _, _>::new(svc); @@ -134,8 +148,10 @@ async fn main() { eprintln!("Listening on {addr}"); eprintln!("Try:"); eprintln!(" dig @127.0.0.1 -p 8053 example.com"); + eprintln!(" dig @127.0.0.1 -p 8053 example.com AXFR"); eprintln!(" dig @127.0.0.1 -p 8053 -y hmac-sha256:demo-key:zlCZbVJPIhobIs1gJNQfrsS3xCxxsR9pMUrGwG8OgG8= example.com AXFR"); eprintln!(" dig @127.0.0.1 -p 8053 +opcode=notify example.com SOA"); + eprintln!(" cargo run --example ixfr-client --all-features -- 127.0.0.1:8053 example.com 2020080302"); eprintln!(); eprintln!("Tip: set env var RUST_LOG=info (or debug or trace) for more log output."); @@ -172,6 +188,49 @@ async fn main() { } }); + tokio::spawn(async move { + let zone_name = Name::>::from_str("example.com").unwrap(); + let mut label: Option = None; + + loop { + tokio::time::sleep(Duration::from_millis(10000)).await; + + let zone = zones.get_zone(&zone_name, Class::IN).unwrap(); + let mut writer = zone.write().await; + { + let node = writer.open(true).await.unwrap(); + + if let Some(old_label) = label { + let node = node.update_child(&old_label).await.unwrap(); + node.remove_rrset(Rtype::A).await.unwrap(); + } + + let random_string: String = rand::thread_rng() + .sample_iter(&Alphanumeric) + .take(7) + .map(char::from) + .collect(); + let new_label = OwnedLabel::from_str(&random_string).unwrap(); + + let node = node.update_child(&new_label).await.unwrap(); + let mut rrset = Rrset::new(Rtype::A, Ttl::from_secs(60)); + let rec = domain::rdata::A::new("127.0.0.1".parse().unwrap()); + rrset.push_data(rec.into()); + node.update_rrset(SharedRrset::new(rrset)).await.unwrap(); + + label = Some(new_label); + } + let diff = writer.commit(true).await.unwrap(); + if let Some(diff) = diff { + zones_and_diffs.add_diff(diff); + } + eprintln!( + "Added {} A record to zone example.com", + label.unwrap() + ); + } + }); + pending::<()>().await; } @@ -221,3 +280,80 @@ impl Notifiable for DemoNotifyTarget { Box::pin(ready(res)) } } + +#[derive(Clone)] +struct ZoneTreeWithDiffs { + zones: Arc, + diffs: Arc>>, +} + +impl ZoneTreeWithDiffs { + fn new(zones: Arc) -> Self { + Self { + zones, + diffs: Default::default(), + } + } + + fn add_diff(&self, diff: InMemoryZoneDiff) { + self.diffs.lock().unwrap().push(diff); + } + + fn get_diffs(&self, diff_from: Option) -> Vec { + eprintln!("Looking up diffs from {diff_from:?}.."); + let diffs = self.diffs.lock().unwrap(); + if let Some(idx) = diffs + .iter() + .position(|diff| Some(diff.start_serial) == diff_from) + { + eprintln!("Returning diffs from {idx}.."); + diffs[idx..].to_vec() + } else { + vec![] + } + } +} + +impl XfrDataProvider for ZoneTreeWithDiffs { + type Diff = InMemoryZoneDiff; + + fn request( + &self, + req: &Request, + diff_from: Option, + ) -> Pin< + Box< + dyn Future< + Output = Result< + XfrData, + XfrDataProviderError, + >, + > + Sync + + Send, + >, + > + where + Octs: Octets + Send + Sync, + { + eprintln!("XFR data requested: {diff_from:?}"); + let res = req + .message() + .sole_question() + .map_err(XfrDataProviderError::ParseError) + .and_then(|q| { + if let Some(zone) = + self.zones.find_zone(q.qname(), q.qclass()) + { + Ok(XfrData::new( + zone.clone(), + self.get_diffs(diff_from), + false, + )) + } else { + Err(XfrDataProviderError::UnknownZone) + } + }); + + Box::pin(ready(res)) + } +} From dd18900562d9973c0415fccae90590a1106918bc Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 16:18:07 +0200 Subject: [PATCH 303/333] Add an ixfr-client example. --- Cargo.toml | 4 +++ examples/common/serve-utils.rs | 10 ++++-- examples/ixfr-client.rs | 65 ++++++++++++++++++++++++++++++++++ 3 files changed, 76 insertions(+), 3 deletions(-) create mode 100644 examples/ixfr-client.rs diff --git a/Cargo.toml b/Cargo.toml index 82108c642..c0bd217f8 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -127,6 +127,10 @@ required-features = ["zonefile", "unstable-zonetree"] name = "serve-zone" required-features = ["zonefile", "net", "unstable-server-transport", "unstable-zonetree"] +[[example]] +name = "ixfr-client" +required-features = ["zonefile", "net", "unstable-client-transport", "unstable-zonetree"] + # This example is commented out because it is difficult, if not impossible, # when including the sqlx dependency, to make the dependency tree compatible # with both `cargo +nightly update -Z minimal versions` and the crate minimum diff --git a/examples/common/serve-utils.rs b/examples/common/serve-utils.rs index 0978481c2..e4d287c1b 100644 --- a/examples/common/serve-utils.rs +++ b/examples/common/serve-utils.rs @@ -2,7 +2,9 @@ use bytes::Bytes; use domain::base::{Message, MessageBuilder, Name, ParsedName, Rtype}; use domain::rdata::ZoneRecordData; use domain::zonetree::Answer; +use octseq::Octets; +#[allow(dead_code)] pub fn generate_wire_query( qname: &Name, qtype: Rtype, @@ -13,6 +15,7 @@ pub fn generate_wire_query( query.into() } +#[allow(dead_code)] pub fn generate_wire_response( wire_query: &Message>, zone_answer: Answer, @@ -22,9 +25,10 @@ pub fn generate_wire_response( response.into() } -pub fn print_dig_style_response( - query: &Message>, - response: &Message>, +#[allow(dead_code)] +pub fn print_dig_style_response( + query: &Message, + response: &Message, short: bool, ) { if !short { diff --git a/examples/ixfr-client.rs b/examples/ixfr-client.rs new file mode 100644 index 000000000..8d22b1dc9 --- /dev/null +++ b/examples/ixfr-client.rs @@ -0,0 +1,65 @@ +/// Using the `domain::net::client` module for sending a query. +use std::vec::Vec; + +use tokio::net::TcpStream; + +use core::str::FromStr; +use domain::base::Name; +use domain::base::Rtype; +use domain::base::{MessageBuilder, Serial, Ttl}; +use domain::net::client::request::SendRequestMulti; +use domain::net::client::request::{RequestMessage, RequestMessageMulti}; +use domain::net::client::stream; +use domain::rdata::Soa; + +#[path = "common/serve-utils.rs"] +mod common; + +#[tokio::main] +async fn main() { + let args: Vec = std::env::args().collect(); + + if args.len() != 4 { + eprintln!("Usage: {} ", args[0]); + eprintln!("E.g.: {} 127.0.0.1:8053 example.com 2020080302", args[0]); + std::process::exit(1); + } + + let server_addr = &args[1]; + let qname = Name::>::from_str(&args[2]).unwrap(); + let soa_serial: u32 = args[3].parse().unwrap(); + + eprintln!("Requesting IXFR from {server_addr} for zone {qname} from serial {soa_serial}"); + + let tcp_conn = TcpStream::connect(server_addr).await.unwrap(); + let (tcp, transport) = stream::Connection::< + RequestMessage>, + RequestMessageMulti>, + >::new(tcp_conn); + tokio::spawn(async move { + transport.run().await; + println!("single TSIG TCP run terminated"); + }); + + let mname = Name::>::from_str("mname").unwrap(); + let rname = Name::>::from_str("rname").unwrap(); + let ttl = Ttl::from_secs(3600); + let soa = Soa::new(mname, rname, Serial(soa_serial), ttl, ttl, ttl, ttl); + + let mut msg = MessageBuilder::new_vec(); + msg.header_mut().set_rd(true); + msg.header_mut().set_ad(true); + let mut msg = msg.question(); + msg.push((&qname, Rtype::IXFR)).unwrap(); + let mut msg = msg.authority(); + msg.push((&qname, 3600, soa)).unwrap(); + let req = RequestMessageMulti::new(msg.clone()).unwrap(); + + let mut request = SendRequestMulti::send_request(&tcp, req); + + // Get the reply + let mock_req = msg.into_message(); + while let Some(reply) = request.get_response().await.unwrap() { + common::print_dig_style_response(&mock_req, &reply, false); + } +} From 2225031a16fb05a1b38c5e2d24962e6fd3b5a8cc Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 16:21:26 +0200 Subject: [PATCH 304/333] Clippy. --- examples/serve-zone.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/serve-zone.rs b/examples/serve-zone.rs index a72ffddb6..7543d8310 100644 --- a/examples/serve-zone.rs +++ b/examples/serve-zone.rs @@ -81,7 +81,7 @@ async fn main() { let key = Key::new(Algorithm::Sha256, &secret, key_name.clone(), None, None) .unwrap(); - key_store.insert((key_name, Algorithm::Sha256), key.into()); + key_store.insert((key_name, Algorithm::Sha256), key); // Populate a zone tree with test data let zone_bytes = include_bytes!("../test-data/zonefiles/nsd-example.txt"); From c30bf68e11f04afb8da6a5b8ebc661401ffcb740 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 16:23:53 +0200 Subject: [PATCH 305/333] Added some comments. --- examples/serve-zone.rs | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/examples/serve-zone.rs b/examples/serve-zone.rs index 7543d8310..57481dc3d 100644 --- a/examples/serve-zone.rs +++ b/examples/serve-zone.rs @@ -109,11 +109,14 @@ async fn main() { zones.insert_zone(zone).unwrap(); let zones = Arc::new(zones); + // Create an XFR data provider that can serve diffs for our zone. + let zones_and_diffs = ZoneTreeWithDiffs::new(zones.clone()); + + // Create a server with middleware layers and an application service + // listening on localhost port 8053. let addr = "127.0.0.1:8053"; let svc = service_fn(my_service, zones.clone()); - let zones_and_diffs = ZoneTreeWithDiffs::new(zones.clone()); - #[cfg(feature = "siphasher")] let svc = CookiesMiddlewareSvc::, _, _>::with_random_secret(svc); let svc = EdnsMiddlewareSvc::, _, _>::new(svc); @@ -155,6 +158,7 @@ async fn main() { eprintln!(); eprintln!("Tip: set env var RUST_LOG=info (or debug or trace) for more log output."); + // Print some status information every 5 seconds tokio::spawn(async move { loop { tokio::time::sleep(Duration::from_millis(5000)).await; @@ -188,6 +192,7 @@ async fn main() { } }); + // Mutate our own zone every 10 seconds. tokio::spawn(async move { let zone_name = Name::>::from_str("example.com").unwrap(); let mut label: Option = None; From c66895d5944a48a28ff4093413dc653aa4782ae3 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 16:24:19 +0200 Subject: [PATCH 306/333] Cargo fmt. --- examples/ixfr-client.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/examples/ixfr-client.rs b/examples/ixfr-client.rs index 8d22b1dc9..ec86bb8a5 100644 --- a/examples/ixfr-client.rs +++ b/examples/ixfr-client.rs @@ -20,7 +20,10 @@ async fn main() { let args: Vec = std::env::args().collect(); if args.len() != 4 { - eprintln!("Usage: {} ", args[0]); + eprintln!( + "Usage: {} ", + args[0] + ); eprintln!("E.g.: {} 127.0.0.1:8053 example.com 2020080302", args[0]); std::process::exit(1); } From b89f08146c213875c2741fb50954d7d1ea8beedf Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 16:25:08 +0200 Subject: [PATCH 307/333] Organize imports. --- examples/ixfr-client.rs | 3 ++- examples/serve-zone.rs | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/examples/ixfr-client.rs b/examples/ixfr-client.rs index ec86bb8a5..1e0613b49 100644 --- a/examples/ixfr-client.rs +++ b/examples/ixfr-client.rs @@ -1,9 +1,10 @@ /// Using the `domain::net::client` module for sending a query. +use core::str::FromStr; + use std::vec::Vec; use tokio::net::TcpStream; -use core::str::FromStr; use domain::base::Name; use domain::base::Rtype; use domain::base::{MessageBuilder, Serial, Ttl}; diff --git a/examples/serve-zone.rs b/examples/serve-zone.rs index 57481dc3d..8b61090ae 100644 --- a/examples/serve-zone.rs +++ b/examples/serve-zone.rs @@ -26,6 +26,9 @@ use std::process::exit; use std::sync::{Arc, Mutex}; use std::time::Duration; +use octseq::Octets; +use rand::distributions::Alphanumeric; +use rand::Rng; use tokio::net::{TcpListener, UdpSocket}; use tracing_subscriber::EnvFilter; @@ -56,9 +59,6 @@ use domain::zonetree::{ Answer, InMemoryZoneDiff, Rrset, SharedRrset, StoredName, }; use domain::zonetree::{Zone, ZoneTree}; -use octseq::Octets; -use rand::distributions::Alphanumeric; -use rand::Rng; #[tokio::main()] async fn main() { From 7c2e2fe73c21689031a295fbc7fdaae5141a7783 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 16:30:02 +0200 Subject: [PATCH 308/333] Remove some temporary diagnostic logging. --- examples/serve-zone.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/examples/serve-zone.rs b/examples/serve-zone.rs index 8b61090ae..f06dae9d8 100644 --- a/examples/serve-zone.rs +++ b/examples/serve-zone.rs @@ -305,13 +305,11 @@ impl ZoneTreeWithDiffs { } fn get_diffs(&self, diff_from: Option) -> Vec { - eprintln!("Looking up diffs from {diff_from:?}.."); let diffs = self.diffs.lock().unwrap(); if let Some(idx) = diffs .iter() .position(|diff| Some(diff.start_serial) == diff_from) { - eprintln!("Returning diffs from {idx}.."); diffs[idx..].to_vec() } else { vec![] From 7043421c42b48590ed1b7ce8d1c38f30829f973b Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 16:30:20 +0200 Subject: [PATCH 309/333] Remove some temporary diagnostic logging. --- examples/serve-zone.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/examples/serve-zone.rs b/examples/serve-zone.rs index f06dae9d8..c82b30ceb 100644 --- a/examples/serve-zone.rs +++ b/examples/serve-zone.rs @@ -338,7 +338,6 @@ impl XfrDataProvider for ZoneTreeWithDiffs { where Octs: Octets + Send + Sync, { - eprintln!("XFR data requested: {diff_from:?}"); let res = req .message() .sole_question() From d396301e07b663dbb86d69f01f272a93d035d436 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 26 Sep 2024 22:33:24 +0200 Subject: [PATCH 310/333] More XFR RustDoc. --- src/net/xfr/mod.rs | 65 +++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 64 insertions(+), 1 deletion(-) diff --git a/src/net/xfr/mod.rs b/src/net/xfr/mod.rs index 22b877ab5..ea7a8ae5b 100644 --- a/src/net/xfr/mod.rs +++ b/src/net/xfr/mod.rs @@ -4,5 +4,68 @@ )] // #![warn(missing_docs)] // #![warn(clippy::missing_docs_in_private_items)] -//! XFR protocol related functionality. +//! XFR related functionality. +//! +//! # What is XFR? +//! +//! XFR refers to the protocols used to transfer entire zones between +//! nameservers. +//! +//! There are two XFR protocols and a couple of protocols often used in +//! combination with XFR: +//! +//! - AXFR defined by [RFC 5936] "DNS Zone Transfer Protocol (AXFR)" +//! - IXFR defined by [RFC 1995] "Incremental Zone Transfer in DNS" +//! - NOTIFY defined by [RFC 1996] "A Mechanism for Prompt Notification of +//! Zone Changes" +//! - TSIG defined by [RFC 8945] "Secret Key Transaction Authentication for +//! DNS (TSIG)" +//! +//! AXFR is used to transfer a complete zone via one or more DNS responses. +//! +//! IXFR is used to incrementally apply the changes that occur to a zone on +//! one nameserver to the same zone on another server, assuming that the +//! latter server has a reasonably up-to-date copy of the zone. +//! +//! NOTIFY allows the server that holds the primary copy of a zone to notify +//! interested servers that the zone has changed and should be re-fetched. +//! +//! TSIG can be used to sign XFR requests and responses to authenticate the +//! servers involved to each other. +//! +//! # XFR support available in this crate +//! +//! Sending requests & handling responses: +//! - [`net::client::stream`] supports sending of XFR requests and receiving +//! one or more responses via [`RequestMessageMulti`]. +//! - [`net::client::tsig`] can be wrapped around another transport to add +//! TSIG request signing and response validation. +//! - [`net::xfr::protocol::XfrResponseInterpreter`] can be used to parse +//! those XFR responses into [`ZoneUpdate`]s. +//! - [`zonetree::update::ZoneUpdater`] can then be used to apply those +//! updates to a [`Zone`]. +//! +//! Responding to requests: +//! - [`net::server::middleware::xfr::XfrMiddlewareSvc`] can respond to +//! XFR requests with zone transfer responses. +//! - [`net::server::middleware::tsig::TsigMiddlewareSvc`] can validate +//! request signatures and sign transer responses. +//! - [`net::server::middleware::notify::NotifyMiddlewareSvc`] can invoke +//! a user supplied callback when a NOTIFY request is received. +//! +//! [RFC 5936]: https://www.rfc-editor.org/info/rfc5936 +//! [RFC 1995]: https://www.rfc-editor.org/info/rfc1995 +//! [RFC 1996]: https://www.rfc-editor.org/info/rfc1996 +//! [RFC 8945]: https://www.rfc-editor.org/info/rfc8945 +//! [`net::client::stream`]: crate::net::client::stream +//! [`RequestMessageMulti`]: crate::net::client::request::RequestMessageMulti +//! [`net::client::tsig`]: crate::net::client::tsig +//! [`net::xfr::protocol::XfrResponseInterpreter`]: crate::net::xfr::protocol::XfrResponseInterpreter +//! [`ZoneUpdate`]: crate::zonetree::types::ZoneUpdate +//! [`zonetree::update::ZoneUpdater`]: crate::zonetree::update::ZoneUpdater +//! [`Zone`]: crate::zonetree::Zone +//! [`net::server::middleware::xfr::XfrMiddlewareSvc`]: crate::net::server::middleware::xfr::XfrMiddlewareSvc +//! [`net::server::middleware::tsig::TsigMiddlewareSvc`]: crate::net::server::middleware::tsig::TsigMiddlewareSvc +//! [`net::server::middleware::notify::NotifyMiddlewareSvc`]: crate::net::server::middleware::notify::NotifyMiddlewareSvc + pub mod protocol; From ee1b0f436ff880cee0fac5c395fd86e6e5588557 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 27 Sep 2024 13:48:07 +0200 Subject: [PATCH 311/333] Document some known limitations. --- src/net/server/middleware/xfr/mod.rs | 7 +++++++ src/zonetree/in_memory/mod.rs | 9 ++++++++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/src/net/server/middleware/xfr/mod.rs b/src/net/server/middleware/xfr/mod.rs index e9acc914d..9eb7f74d5 100644 --- a/src/net/server/middleware/xfr/mod.rs +++ b/src/net/server/middleware/xfr/mod.rs @@ -28,6 +28,13 @@ //! * RFC 1995 2 Brief Description of the Protocol states: _"To ensure //! integrity, servers should use UDP checksums for all UDP responses."_. //! This is not implemented. +//! * RFC 1995 5 Purging Strategy states: _"Information about older versions +//! should be purged if the total length of an IXFR response would be longer +//! than that of an AXFR response."_. This is not implemented. +//! * RFC 1995 6 Optional Condensation of Multiple Versions states: _"An IXFR +//! server may optionally condense multiple difference sequences into a +//! single difference sequence, thus, dropping information on intermediate +//! versions."_. This is not implemented. //! //! [RFC 5936]: https://www.rfc-editor.org/info/rfc5936 //! [RFC 1995]: https://www.rfc-editor.org/info/rfc1995 diff --git a/src/zonetree/in_memory/mod.rs b/src/zonetree/in_memory/mod.rs index 59c36b45b..703bcd9c9 100644 --- a/src/zonetree/in_memory/mod.rs +++ b/src/zonetree/in_memory/mod.rs @@ -1,5 +1,12 @@ -//! An in-memory backing store for [`Zone`]s. +//! A versioned in-memory backing store for [`Zone`]s. //! +//! # Limitations +//! +//! * There is currently no support for removing old versions of zone data +//! stored in the tree. The only options are to [`rollback()`] the newest +//! version or [`walk()`] the [`Zone`] cloning the current version into a +//! new [`Zone`] then dropping the old [`Zone`]. +//! //! [`Zone`]: super::Zone mod builder; mod nodes; From 030c68e747756c8beec9a1cb7b2f09b518c86afd Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 27 Sep 2024 13:50:45 +0200 Subject: [PATCH 312/333] Fix RustDoc broken links and clarify limitation note. --- src/zonetree/in_memory/mod.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/zonetree/in_memory/mod.rs b/src/zonetree/in_memory/mod.rs index 703bcd9c9..6d7d75375 100644 --- a/src/zonetree/in_memory/mod.rs +++ b/src/zonetree/in_memory/mod.rs @@ -3,11 +3,11 @@ //! # Limitations //! //! * There is currently no support for removing old versions of zone data -//! stored in the tree. The only options are to [`rollback()`] the newest -//! version or [`walk()`] the [`Zone`] cloning the current version into a -//! new [`Zone`] then dropping the old [`Zone`]. -//! -//! [`Zone`]: super::Zone +//! stored in the tree. The only optionis to [`walk()`] the [`Zone`] cloning +//! the current version into a new [`Zone`] then dropping the old [`Zone`]. +//! +//! [`Zone`]: crate::zonetree::Zone +//! [`walk()`]: crate::zoneree::ReadableZone::walk() mod builder; mod nodes; mod read; From 72b5d76aa67ab9afd2eebb6faf7f7251e21d5bb3 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 27 Sep 2024 13:52:22 +0200 Subject: [PATCH 313/333] Fix line wrapping. --- src/zonetree/traits.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index 0605480a9..a54787249 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -231,7 +231,8 @@ pub trait WritableZoneNode: Send + Sync { cname: SharedRr, ) -> Pin> + Send + Sync>>; - /// Recursively make all content at and below this point appear to be removed. + /// Recursively make all content at and below this point appear to be + /// removed. fn remove_all( &self, ) -> Pin> + Send + Sync>>; From 181b5331043fdc48480acc70c4fffe3e8794d019 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 27 Sep 2024 13:58:22 +0200 Subject: [PATCH 314/333] Document the `create_diff` argument to `WritableZone::open()`. --- src/zonetree/traits.rs | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index a54787249..acab293e8 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -130,6 +130,14 @@ pub trait ReadableZone: Send + Sync { /// [`Zone`]: super::Zone pub trait WritableZone: Send + Sync { /// Start a write operation for the zone. + /// + /// If `create_diff` is true the zone backing store is requested to create + /// an [`InMemoryZoneDiff`] which will accumulate entries as changes are + /// made to the zone and will be returned finally when [`commit()`] is + /// invoked. + /// + /// Creating a diff is optional. If the backing store doesn't support + /// diff creation [`commit()`] will return `None`. #[allow(clippy::type_complexity)] fn open( &self, @@ -150,6 +158,11 @@ pub trait WritableZone: Send + Sync { /// made since the last commit. Only clients who obtain a [`ReadableZone`] /// _after_ invoking this function will be able to see the changes made /// since [`open()`] was called. + /// + /// If `create_diff` was set to `true` when [`open()`] was invoked then + /// this function _may_ return `Some` if a diff was created. `None` may be + /// returned if the zone backing store does not support creation of diffs + /// or was unable to create a diff for some reason. /// /// [`open()`]: Self::open fn commit( From 351b2271d586215d3c0de73fb69c1405abe094e8 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Fri, 27 Sep 2024 13:59:35 +0200 Subject: [PATCH 315/333] Fix RustDoc broken links. --- src/zonetree/traits.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index acab293e8..a7e26b875 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -130,14 +130,16 @@ pub trait ReadableZone: Send + Sync { /// [`Zone`]: super::Zone pub trait WritableZone: Send + Sync { /// Start a write operation for the zone. - /// + /// /// If `create_diff` is true the zone backing store is requested to create /// an [`InMemoryZoneDiff`] which will accumulate entries as changes are /// made to the zone and will be returned finally when [`commit()`] is /// invoked. - /// + /// /// Creating a diff is optional. If the backing store doesn't support /// diff creation [`commit()`] will return `None`. + /// + /// [`commit()`]: Self::commit #[allow(clippy::type_complexity)] fn open( &self, @@ -158,7 +160,7 @@ pub trait WritableZone: Send + Sync { /// made since the last commit. Only clients who obtain a [`ReadableZone`] /// _after_ invoking this function will be able to see the changes made /// since [`open()`] was called. - /// + /// /// If `create_diff` was set to `true` when [`open()`] was invoked then /// this function _may_ return `Some` if a diff was created. `None` may be /// returned if the zone backing store does not support creation of diffs From 9d3cbcf9044e817f9e5b138edf4ef122bab90e55 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Sun, 29 Sep 2024 11:59:29 +0200 Subject: [PATCH 316/333] Add some small Zone extensions from the xfr branch. --- src/zonetree/tree.rs | 6 +++--- src/zonetree/zone.rs | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/src/zonetree/tree.rs b/src/zonetree/tree.rs index 12915783c..d8c46c8aa 100644 --- a/src/zonetree/tree.rs +++ b/src/zonetree/tree.rs @@ -15,7 +15,7 @@ use super::zone::Zone; /// A multi-rooted [`Zone`] hierarchy. /// /// [`Zone`]: crate::zonetree::Zone -#[derive(Default)] +#[derive(Clone, Default, Debug)] pub struct ZoneTree { roots: Roots, } @@ -82,7 +82,7 @@ impl ZoneTree { //------------ Roots --------------------------------------------------------- -#[derive(Default)] +#[derive(Clone, Default, Debug)] struct Roots { in_: ZoneSetNode, others: HashMap, @@ -116,7 +116,7 @@ impl Roots { //------------ ZoneSetNode --------------------------------------------------- -#[derive(Default)] +#[derive(Clone, Default, Debug)] struct ZoneSetNode { zone: Option, children: HashMap, diff --git a/src/zonetree/zone.rs b/src/zonetree/zone.rs index ca3058a74..9ba476f63 100644 --- a/src/zonetree/zone.rs +++ b/src/zonetree/zone.rs @@ -13,6 +13,15 @@ use super::traits::WritableZone; use super::types::StoredName; use super::{parsed, ReadableZone, ZoneStore}; +//------------ ZoneKey ------------------------------------------------------- + +/// A key that uniquely identifies a zone. +/// +/// A zone is identified by the owner name of the apex and its class. Every +/// record in a zone must be at or under the apex owner name and be of the +/// same class. +pub type ZoneKey = (StoredName, Class); + //------------ Zone ---------------------------------------------------------- /// A single DNS zone. @@ -51,6 +60,14 @@ impl Zone { { self.store.clone().write() } + + /// Gets a key that uniquely identifies this zone. + /// + /// Note: Assumes that there is only ever one instance of a zone with a + /// given apex name and class in a set of zones. + pub fn key(&self) -> ZoneKey { + (self.apex_name().clone(), self.class()) + } } //--- TryFrom From a112b1e151d115a5875d7a1fb1a841ae2861b807 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Sun, 29 Sep 2024 12:03:44 +0200 Subject: [PATCH 317/333] Expose ZoneKey. --- src/zonetree/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/zonetree/mod.rs b/src/zonetree/mod.rs index 5556f234e..dfee66496 100644 --- a/src/zonetree/mod.rs +++ b/src/zonetree/mod.rs @@ -124,7 +124,7 @@ pub use self::types::{ StoredName, StoredRecord, }; pub use self::walk::WalkOp; -pub use self::zone::Zone; +pub use self::zone::{Zone, ZoneKey}; /// Zone related utilities. pub mod util { From 317acd00501125c9f5cc372cbd7a4decf9e06bef Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Sun, 29 Sep 2024 12:10:02 +0200 Subject: [PATCH 318/333] Add as_any(). --- src/zonetree/in_memory/nodes.rs | 6 ++++++ src/zonetree/traits.rs | 3 +++ 2 files changed, 9 insertions(+) diff --git a/src/zonetree/in_memory/nodes.rs b/src/zonetree/in_memory/nodes.rs index 864682fbb..856075670 100644 --- a/src/zonetree/in_memory/nodes.rs +++ b/src/zonetree/in_memory/nodes.rs @@ -1,5 +1,7 @@ //! The resource record tree nodes of an in-memory zone. +use core::any::Any; + use std::boxed::Box; use std::collections::{hash_map, HashMap}; use std::future::Future; @@ -145,6 +147,10 @@ impl ZoneStore for ZoneApex { as Box }) } + + fn as_any(&self) -> &dyn Any { + self as &dyn Any + } } //--- impl From<&'a ZoneApex> diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index a7e26b875..97b34f3b4 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -59,6 +59,9 @@ pub trait ZoneStore: Debug + Sync + Send + Any { + 'static), >, >; + + /// TODO + fn as_any(&self) -> &dyn Any; } //------------ ReadableZone -------------------------------------------------- From 74c69b1bddc5fdbc3d428b6f55422b30b3ca9c13 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Sun, 29 Sep 2024 12:17:26 +0200 Subject: [PATCH 319/333] impl Arc for KeyStore. --- src/tsig/mod.rs | 34 ++++++++++++++++++++++++++++------ 1 file changed, 28 insertions(+), 6 deletions(-) diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index d88362f76..8f3619763 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -56,6 +56,17 @@ mod interop; +use core::fmt::Display; +use core::{cmp, fmt, mem, str}; + +use std::sync::Arc; +#[cfg(feature = "std")] +use std::collections::HashMap; + +use bytes::{Bytes, BytesMut}; +use octseq::octets::Octets; +use ring::{constant_time, hkdf::KeyType, hmac, rand}; + use crate::base::header::HeaderSection; use crate::base::iana::{Class, Rcode, TsigRcode}; use crate::base::message::Message; @@ -66,12 +77,6 @@ use crate::base::name::{Label, Name, ParsedName, ToLabelIter, ToName}; use crate::base::record::Record; use crate::base::wire::{Composer, ParseError}; use crate::rdata::tsig::{Time48, Tsig}; -use bytes::{Bytes, BytesMut}; -use core::{cmp, fmt, mem, str}; -use octseq::octets::Octets; -use ring::{constant_time, hkdf::KeyType, hmac, rand}; -#[cfg(feature = "std")] -use std::collections::HashMap; //------------ KeyName ------------------------------------------------------- @@ -460,6 +465,23 @@ where } } +#[cfg(feature = "std")] +impl KeyStore for Arc> { + type Key = Key; + + fn get_key( + &self, + name: &N, + algorithm: Algorithm, + ) -> Option { + if let Ok(name) = name.try_to_name() { + self.get(&(name, algorithm)).cloned() + } else { + None + } + } +} + //------------ ClientTransaction --------------------------------------------- /// TSIG Client transaction state. From 4d3899946c94626c0f62051d57655dbcd3b1d1a5 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 1 Oct 2024 10:42:37 +0200 Subject: [PATCH 320/333] Impl SendRequest for Boxed types. --- src/net/client/request.rs | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/src/net/client/request.rs b/src/net/client/request.rs index 9bb0e3af8..33c2194e7 100644 --- a/src/net/client/request.rs +++ b/src/net/client/request.rs @@ -119,6 +119,17 @@ pub trait SendRequest { ) -> Box; } +impl> + ?Sized, Octs: Octets> + SendRequest> for Box +{ + fn send_request( + &self, + request_msg: RequestMessage, + ) -> Box { + (**self).send_request(request_msg) + } +} + //------------ SendRequestMulti ----------------------------------------------- /// Trait for starting a DNS request based on a request composer. @@ -133,6 +144,17 @@ pub trait SendRequestMulti { ) -> Box; } +impl> + ?Sized, Octs: Octets> + SendRequestMulti> for Box +{ + fn send_request( + &self, + request_msg: RequestMessage, + ) -> Box { + (**self).send_request(request_msg) + } +} + //------------ GetResponse --------------------------------------------------- /// Trait for getting the result of a DNS query. From bb254bfba39c94c3b856c1bc18694e1d4b77ddcf Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 1 Oct 2024 10:42:56 +0200 Subject: [PATCH 321/333] Remove unused import. --- src/tsig/mod.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index 8f3619763..9526e531d 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -56,12 +56,11 @@ mod interop; -use core::fmt::Display; use core::{cmp, fmt, mem, str}; -use std::sync::Arc; #[cfg(feature = "std")] use std::collections::HashMap; +use std::sync::Arc; use bytes::{Bytes, BytesMut}; use octseq::octets::Octets; From 51facf6b53821662b8a9b155db418676bc48df57 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 1 Oct 2024 11:09:26 +0200 Subject: [PATCH 322/333] Allow a Zone to be exchanged for its underlying ZoneStore type. --- src/zonetree/zone.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/zonetree/zone.rs b/src/zonetree/zone.rs index 9ba476f63..df15bf0c5 100644 --- a/src/zonetree/zone.rs +++ b/src/zonetree/zone.rs @@ -38,6 +38,12 @@ impl Zone { } } + /// Exchange this [`Zone`] wrapper for the actual underlying backing store + /// implementation. + pub fn into_inner(self) -> Arc { + self.store + } + /// Gets the CLASS of this zone. pub fn class(&self) -> Class { self.store.class() From d4d7dbf88db992c25e5803172ebf1e8cce9bf191 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 1 Oct 2024 11:21:27 +0200 Subject: [PATCH 323/333] Impl Display for tsig::Key. --- src/tsig/mod.rs | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index 9526e531d..2e31d3103 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -393,6 +393,15 @@ impl AsRef for Key { } } +//--- Display + +#[cfg(feature = "std")] +impl fmt::Display for Key { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.write_fmt(format_args!("{}", self.name)) + } +} + //------------ KeyStore ------------------------------------------------------ /// A type that stores TSIG secret keys. From def789acd5b5aa0e23e39b4e9e97dd00f90a82de Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 1 Oct 2024 11:40:44 +0200 Subject: [PATCH 324/333] Impl AsRef to get a reference to the underlying ZoneStore type. --- src/zonetree/zone.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/zonetree/zone.rs b/src/zonetree/zone.rs index df15bf0c5..64f5f3fee 100644 --- a/src/zonetree/zone.rs +++ b/src/zonetree/zone.rs @@ -76,6 +76,12 @@ impl Zone { } } +impl AsRef for Zone { + fn as_ref(&self) -> &dyn ZoneStore { + self.store.as_ref() + } +} + //--- TryFrom impl TryFrom for Zone { From 67d7200665b9fb8a775d586af3c481169cae32f6 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 1 Oct 2024 11:44:17 +0200 Subject: [PATCH 325/333] Impl Display for update::Error. --- src/zonetree/update.rs | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index 62d90e7fd..b035e7872 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -1309,6 +1309,21 @@ pub enum Error { Finished, } +//--- Display + +impl std::fmt::Display for Error { + fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { + match self { + Error::OutOfZone => f.write_str("OutOfZone"), + Error::NotSoaRecord => f.write_str("NotSoaRecord"), + Error::IoError(err) => { + f.write_fmt(format_args!("I/O error: {err}")) + } + Error::Finished => f.write_str("Finished"), + } + } +} + //--- From impl From for Error { From e2f85a031fe995177effdd24b20aeadedda25364 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 1 Oct 2024 16:37:57 +0200 Subject: [PATCH 326/333] Document `ZoneStore::as_any()` and the purpose behind it. --- src/zonetree/traits.rs | 6 +++++- src/zonetree/zone.rs | 29 +++++++++++++++++++++++++++++ 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index 97b34f3b4..e38f2e722 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -60,7 +60,11 @@ pub trait ZoneStore: Debug + Sync + Send + Any { >, >; - /// TODO + /// Returns an [`Any`] interface to the store. + /// + /// This can be used to obtain access to methods on the specific + /// [`ZoneStore`] implementation. See [`Zone`] for how this can used to + /// layer functionality on top of a zone. fn as_any(&self) -> &dyn Any; } diff --git a/src/zonetree/zone.rs b/src/zonetree/zone.rs index 64f5f3fee..84ea2aa55 100644 --- a/src/zonetree/zone.rs +++ b/src/zonetree/zone.rs @@ -25,6 +25,35 @@ pub type ZoneKey = (StoredName, Class); //------------ Zone ---------------------------------------------------------- /// A single DNS zone. +/// +/// # Abstract backing store +/// +/// The actual backing store implementation used by a [`Zone`] is determined +/// by the [`ZoeStore`] impl it wraps. In this way one can treat in-memory +/// zone implementations and other backing store types (for example a database +/// backed zone) in the same way, and even to store zones with different +/// backing stores together in the same [`ZoneTree`]. +/// +/// # Layering functionality +/// +/// The functionality of [`Zone`]s can be extended by creating a [`ZoneStore`] +/// implementation that wraps another [`ZoneStore`] implementation with the +/// purpose of wrapping the original zone with additional state and +/// functionality. +/// +/// This could be used to detect changes to the [`Zone`] via your own +/// [`WritableZone`] impl e.g. to sign it or persist it, or to detect updated +/// SOA timers, and so on. +/// +/// To layer [`ZoneStore`] implementations on top of one another, use +/// [`Zone::into_inner()`] to obtain backing store implementation of a +/// [`Zone`] then store that (via [`Arc, From 36bd95b5166540c61e34a0064fa92f1270b66580 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Tue, 1 Oct 2024 16:40:22 +0200 Subject: [PATCH 327/333] Cargo fmt. --- src/zonetree/traits.rs | 2 +- src/zonetree/zone.rs | 14 +++++++------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/zonetree/traits.rs b/src/zonetree/traits.rs index e38f2e722..e2fc12ef1 100644 --- a/src/zonetree/traits.rs +++ b/src/zonetree/traits.rs @@ -61,7 +61,7 @@ pub trait ZoneStore: Debug + Sync + Send + Any { >; /// Returns an [`Any`] interface to the store. - /// + /// /// This can be used to obtain access to methods on the specific /// [`ZoneStore`] implementation. See [`Zone`] for how this can used to /// layer functionality on top of a zone. diff --git a/src/zonetree/zone.rs b/src/zonetree/zone.rs index 84ea2aa55..2441c1325 100644 --- a/src/zonetree/zone.rs +++ b/src/zonetree/zone.rs @@ -25,32 +25,32 @@ pub type ZoneKey = (StoredName, Class); //------------ Zone ---------------------------------------------------------- /// A single DNS zone. -/// +/// /// # Abstract backing store -/// +/// /// The actual backing store implementation used by a [`Zone`] is determined /// by the [`ZoeStore`] impl it wraps. In this way one can treat in-memory /// zone implementations and other backing store types (for example a database /// backed zone) in the same way, and even to store zones with different /// backing stores together in the same [`ZoneTree`]. -/// +/// /// # Layering functionality -/// +/// /// The functionality of [`Zone`]s can be extended by creating a [`ZoneStore`] /// implementation that wraps another [`ZoneStore`] implementation with the /// purpose of wrapping the original zone with additional state and /// functionality. -/// +/// /// This could be used to detect changes to the [`Zone`] via your own /// [`WritableZone`] impl e.g. to sign it or persist it, or to detect updated /// SOA timers, and so on. -/// +/// /// To layer [`ZoneStore`] implementations on top of one another, use /// [`Zone::into_inner()`] to obtain backing store implementation of a /// [`Zone`] then store that (via [`Arc Date: Wed, 2 Oct 2024 20:53:19 +0200 Subject: [PATCH 328/333] Review feedback: Use a less confusing and more accurate name: ZoneTuple instead of ZoneKey. --- src/zonetree/mod.rs | 2 +- src/zonetree/zone.rs | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/zonetree/mod.rs b/src/zonetree/mod.rs index dfee66496..b953d6cbf 100644 --- a/src/zonetree/mod.rs +++ b/src/zonetree/mod.rs @@ -124,7 +124,7 @@ pub use self::types::{ StoredName, StoredRecord, }; pub use self::walk::WalkOp; -pub use self::zone::{Zone, ZoneKey}; +pub use self::zone::{Zone, ZoneTuple}; /// Zone related utilities. pub mod util { diff --git a/src/zonetree/zone.rs b/src/zonetree/zone.rs index 2441c1325..4fd5a5caf 100644 --- a/src/zonetree/zone.rs +++ b/src/zonetree/zone.rs @@ -13,16 +13,16 @@ use super::traits::WritableZone; use super::types::StoredName; use super::{parsed, ReadableZone, ZoneStore}; -//------------ ZoneKey ------------------------------------------------------- +//------------ ZoneTuple ------------------------------------------------------ -/// A key that uniquely identifies a zone. +/// A tuple that identifies a zone. /// /// A zone is identified by the owner name of the apex and its class. Every /// record in a zone must be at or under the apex owner name and be of the /// same class. -pub type ZoneKey = (StoredName, Class); +pub type ZoneTuple = (StoredName, Class); -//------------ Zone ---------------------------------------------------------- +//------------ Zone ----------------------------------------------------------- /// A single DNS zone. /// @@ -100,7 +100,7 @@ impl Zone { /// /// Note: Assumes that there is only ever one instance of a zone with a /// given apex name and class in a set of zones. - pub fn key(&self) -> ZoneKey { + pub fn key(&self) -> ZoneTuple { (self.apex_name().clone(), self.class()) } } From 7de9492c1de27e1bee2713c7094b6751b0a55abe Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 2 Oct 2024 20:54:36 +0200 Subject: [PATCH 329/333] Review feedback: Use `fmt() `directly. --- src/tsig/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index 2e31d3103..c9898f4d9 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -398,7 +398,7 @@ impl AsRef for Key { #[cfg(feature = "std")] impl fmt::Display for Key { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.write_fmt(format_args!("{}", self.name)) + self.name.fmt(f) } } From 3a8ff03fe88778fedaebae75c67363fd626117de Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Wed, 2 Oct 2024 20:55:57 +0200 Subject: [PATCH 330/333] Review feedback: Use write!. --- src/zonetree/update.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/zonetree/update.rs b/src/zonetree/update.rs index b035e7872..1fa9d1cb6 100644 --- a/src/zonetree/update.rs +++ b/src/zonetree/update.rs @@ -1316,9 +1316,8 @@ impl std::fmt::Display for Error { match self { Error::OutOfZone => f.write_str("OutOfZone"), Error::NotSoaRecord => f.write_str("NotSoaRecord"), - Error::IoError(err) => { - f.write_fmt(format_args!("I/O error: {err}")) - } + Error::IoError(err) => write!(f, "I/O error: {err}"), + Error::Finished => f.write_str("Finished"), } } From 6fe1493e0d8f56431e25edf604fa42961c7c6f23 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 3 Oct 2024 10:33:46 +0200 Subject: [PATCH 331/333] Merge branch 'main' into zone-tweaks --- Changelog.md | 34 ++++++-- src/base/iana/exterr.rs | 26 +++--- src/base/iana/rtype.rs | 5 ++ src/base/name/parsed.rs | 13 +++ src/base/opt/algsig.rs | 16 ++++ src/base/opt/chain.rs | 1 + src/base/opt/cookie.rs | 21 +++++ src/base/opt/expire.rs | 1 + src/base/opt/exterr.rs | 23 +++++ src/base/opt/keepalive.rs | 2 + src/base/opt/keytag.rs | 18 ++++ src/base/opt/macros.rs | 10 +++ src/base/opt/mod.rs | 30 +++++++ src/base/opt/nsid.rs | 9 ++ src/base/opt/padding.rs | 12 +++ src/base/opt/subnet.rs | 1 + src/lib.rs | 1 - src/net/server/connection.rs | 2 +- src/net/server/dgram.rs | 2 +- src/net/server/middleware/mandatory.rs | 23 ++++- src/net/server/middleware/xfr/responder.rs | 2 +- src/rdata/macros.rs | 11 +++ src/test/cargo.rs | 3 - src/test/utils.rs | 12 --- src/tsig/mod.rs | 97 +++++++++++++++++----- src/zonetree/in_memory/read.rs | 3 - {src/test => tests/common}/mod.rs | 4 +- {src/test => tests/common}/nsd.rs | 5 +- {src/tsig => tests}/interop.rs | 36 ++++---- 29 files changed, 333 insertions(+), 90 deletions(-) delete mode 100644 src/test/cargo.rs delete mode 100644 src/test/utils.rs rename {src/test => tests/common}/mod.rs (83%) rename {src/test => tests/common}/nsd.rs (99%) rename {src/tsig => tests}/interop.rs (96%) diff --git a/Changelog.md b/Changelog.md index fcf8dd158..009077e15 100644 --- a/Changelog.md +++ b/Changelog.md @@ -14,6 +14,9 @@ New * Added `MessageBuilder::start_error`, like `start_answer` but infallible. ([#369]) * Added `AnswerBuilder::push_ref`, like `push` but takes the record by reference. ([#383]) +* Added `Rtype::NXNAME` and `ExtendedErrorCode::INVALID_QUERY_TYPE`. ([#392]) +* Added a `Serialize` impl to `AllRecordData` and as a consequence to + the OPT record and all OPT options, as well as `ParsedName`. ([#343]) Bug fixes @@ -24,26 +27,29 @@ Bug fixes * Fixes the way the `Txt<_> `record data implements comparison-related traits. They now directly compare the underlying octets, i.e., the wire format bytes. ([#374] by [@dklbreitling]) +* Fix the `tsig` module to reject messages with multiple TSIG records + ([#334]) Unstable features * New unstable feature `unstable-validator` that adds a DNSSEC validator. ([#328]) -* New unstable feature `unstable-xfr` that adds `XfrResponseInterpreter` in - `net::xfr` for iterating over XFR responses as a sequence of high level - `ZoneUpdate`s. ([#375]) +* New unstable feature `unstable-xfr` that adds `XfrResponseInterpreter` for + iterating over XFR responses as a sequence of high level `ZoneUpdate`s, and + `XfrMiddlewareSvc` and `XfrDataProvider` for responding to received XFR + requests. ([#375], [#384]) * `unstable-client-transport`: * Fixed an issue with slow responses in the `multi_stream` transport by not waiting in the first iteration if an underlying stream reports its connection being closed. ([#338]) - * Added an option called idle_timeout to stream that allows a TCP or + * Added an option called `idle_timeout` to stream that allows a TCP or TLS connection to stay open even if no TcpKeepalive option is received from the server. ([#341]) * Fixed an off-by-one error in Dgram client retry count checking. ([#354]) * Add support for requests that may result in multiple responses. This - adds ComposeRequestMulti and other *Multi types. The main change is to + adds `ComposeRequestMulti` and other `*Multi` types. The main change is to the stream transport, which is the only transport that implements - SendRequestMulti. ([#377]) + `SendRequestMulti`. ([#377]) * Added a TSIG request signing and response validating passthrough transport in `net::client:tsig`. ([#373]) * `unstable-server-transport` @@ -59,10 +65,13 @@ Unstable features * Added trait `ResourceRecordBatcher` and impl `CallbackBatcher` in `net::server::batcher` for pushing as many records into a response as will fit according to defined limits. ([#383]) + * Enforce dgram max response size limit. ([#398]) + * Extend MandatoryMiddlewareSvc with an RFC 9619 check for opcode QUERY with + QDCOUNT > 1. ([#365]) * `unstable-zonetree`: * Added `ZoneUpdate`. ([#375]) - * Added `ZoneUpdater`, `ZoneDiffBuilder` and `ZoneDiff` and improved - `ZoneUpdate`. ([#376]) + * Added `ZoneUpdater`, `ZoneDiff`, `InMemoryZoneDiffBuilder`, + `InMemoryZoneDiff` and improved `ZoneUpdate`. ([#376], [#384]) * Improved zonefile parsing error messages. ([#362]). * `TryFrom for Zonefile` now returns the set of errors instead of logging and ignoring them. ([#362]) @@ -74,6 +83,8 @@ Unstable features version was being created. ([#376]) * Removed / renamed references to `clean` in `zonetree::in_memory` to `remove`. ([#376]) + * Fix zone walking to include non-leaf CNAMEs. ([#352]) + * Fix zone walking to pass the correct owner name to the callback. ([#384]) Other changes @@ -81,16 +92,20 @@ Other changes [#328]: https://github.com/NLnetLabs/domain/pull/328 [#333]: https://github.com/NLnetLabs/domain/pull/333 +[#334]: https://github.com/NLnetLabs/domain/pull/334 [#336]: https://github.com/NLnetLabs/domain/pull/336 [#338]: https://github.com/NLnetLabs/domain/pull/338 [#341]: https://github.com/NLnetLabs/domain/pull/341 +[#343]: https://github.com/NLnetLabs/domain/pull/343 [#348]: https://github.com/NLnetLabs/domain/pull/348 +[#352]: https://github.com/NLnetLabs/domain/pull/352 [#354]: https://github.com/NLnetLabs/domain/pull/354 [#357]: https://github.com/NLnetLabs/domain/pull/357 [#358]: https://github.com/NLnetLabs/domain/pull/358 [#360]: https://github.com/NLnetLabs/domain/pull/360 [#362]: https://github.com/NLnetLabs/domain/pull/362 [#363]: https://github.com/NLnetLabs/domain/pull/363 +[#365]: https://github.com/NLnetLabs/domain/pull/365 [#369]: https://github.com/NLnetLabs/domain/pull/369 [#373]: https://github.com/NLnetLabs/domain/pull/373 [#374]: https://github.com/NLnetLabs/domain/pull/374 @@ -100,6 +115,9 @@ Other changes [#380]: https://github.com/NLnetLabs/domain/pull/380 [#382]: https://github.com/NLnetLabs/domain/pull/382 [#383]: https://github.com/NLnetLabs/domain/pull/383 +[#384]: https://github.com/NLnetLabs/domain/pull/384 +[#392]: https://github.com/NLnetLabs/domain/pull/392 +[#398]: https://github.com/NLnetLabs/domain/pull/398 [@dklbreitling]: https://github.com/dklbreitling ## 0.10.1 diff --git a/src/base/iana/exterr.rs b/src/base/iana/exterr.rs index 4e302e8bd..1307c2d15 100644 --- a/src/base/iana/exterr.rs +++ b/src/base/iana/exterr.rs @@ -149,6 +149,9 @@ int_enum! { /// is otherwise configured to support. Examples of this include /// its most recent zone being too old or having expired. (INVALID_DATA => 24, b"Invalid Data") + + /// The requested resource record type should not appear in a query. + (INVALID_QUERY_TYPE => 30, b"Invalid Query Type") } /// Start of the private range for EDE codes. @@ -160,19 +163,10 @@ int_enum! { /// ``` pub const EDE_PRIVATE_RANGE_BEGIN: u16 = 49152; -// Only implement `Display` for `ExtendedErrorCode`, as the `FromStr` -// bundled by the `int_enum_*` macros is not very useful. -impl core::fmt::Display for ExtendedErrorCode { - fn fmt(&self, f: &mut core::fmt::Formatter) -> core::fmt::Result { - use core::fmt::Write; - match self.to_mnemonic() { - Some(m) => { - for ch in m { - f.write_char(*ch as char)? - } - Ok(()) - } - None => write!(f, "EDE{}", self.to_int()), - } - } -} +int_enum_str_with_prefix!( + ExtendedErrorCode, + "EDE", + b"EDE", + u16, + "unknown error code" +); diff --git a/src/base/iana/rtype.rs b/src/base/iana/rtype.rs index ab031066b..c38c204b7 100644 --- a/src/base/iana/rtype.rs +++ b/src/base/iana/rtype.rs @@ -364,6 +364,11 @@ int_enum! { /// See RFC 7043. (EUI64 => 109, b"EUI64") + /// NXNAME. + /// + /// IANA-Reserved. + (NXNAME => 128, b"NXNAME") + /// Transaction key. /// /// See RFC 2930. diff --git a/src/base/name/parsed.rs b/src/base/name/parsed.rs index 50a25fd5d..5f77b99c7 100644 --- a/src/base/name/parsed.rs +++ b/src/base/name/parsed.rs @@ -518,6 +518,19 @@ impl> fmt::Debug for ParsedName { } } +//--- Serialize + +#[cfg(feature = "serde")] +impl> serde::Serialize for ParsedName { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + use std::string::ToString; + self.to_string().serialize(serializer) + } +} + //------------ ParsedNameIter ----------------------------------------------- /// An iterator over the labels in a parsed domain name. diff --git a/src/base/opt/algsig.rs b/src/base/opt/algsig.rs index 03810ca52..a4f6172b3 100644 --- a/src/base/opt/algsig.rs +++ b/src/base/opt/algsig.rs @@ -432,6 +432,22 @@ impl + ?Sized> fmt::Debug for Understood { } } +//--- Serialize + +#[cfg(feature = "serde")] +impl> serde::Serialize for Understood { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer { + use serde::ser::SerializeSeq; + let mut list = serializer.serialize_seq(None)?; + for item in self.iter() { + list.serialize_element(&item)?; + } + list.end() + } +} + //--- Extended Opt and OptBuilder impl Opt { diff --git a/src/base/opt/chain.rs b/src/base/opt/chain.rs index 455f43b89..a5b83c5c8 100644 --- a/src/base/opt/chain.rs +++ b/src/base/opt/chain.rs @@ -28,6 +28,7 @@ use octseq::parse::Parser; /// furthest away from the root to which the requesting resolver already has /// all necessary records. #[derive(Clone, Copy)] +#[cfg_attr(feature = "serde", derive(serde::Serialize))] #[repr(transparent)] pub struct Chain { /// The start name AKA ‘closest trust point.’ diff --git a/src/base/opt/cookie.rs b/src/base/opt/cookie.rs index e1d5e21bb..6c373a19b 100644 --- a/src/base/opt/cookie.rs +++ b/src/base/opt/cookie.rs @@ -61,6 +61,7 @@ use super::{Opt, OptData, ComposeOptData, ParseOptData}; /// cookie to be included in a response. #[derive(Clone, Debug, Eq, Hash, Ord, PartialEq, PartialOrd)] #[cfg_attr(feature = "rand", derive(Default))] +#[cfg_attr(feature = "serde", derive(serde::Serialize))] pub struct Cookie { /// The client cookie. client: ClientCookie, @@ -281,6 +282,16 @@ impl<'a, Target: Composer> OptBuilder<'a, Target> { #[derive(Clone, Copy, Debug, Eq, PartialEq, Ord, PartialOrd)] pub struct ClientCookie([u8; 8]); +#[cfg(feature = "serde")] +impl serde::Serialize for ClientCookie { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer { + use octseq::serde::SerializeOctets; + self.0.serialize_octets(serializer) + } +} + impl ClientCookie { /// Creates a new client cookie from the given octets. #[must_use] @@ -399,6 +410,16 @@ impl fmt::Display for ClientCookie { #[derive(Clone, Debug, Eq, Hash, Ord, PartialEq, PartialOrd)] pub struct ServerCookie(Array<32>); +#[cfg(feature = "serde")] +impl serde::Serialize for ServerCookie { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer { + use octseq::serde::SerializeOctets; + self.0.serialize_octets(serializer) + } +} + impl ServerCookie { /// Creates a new server cookie from the given octets. /// diff --git a/src/base/opt/expire.rs b/src/base/opt/expire.rs index 7414f6f59..573c5e4e3 100644 --- a/src/base/opt/expire.rs +++ b/src/base/opt/expire.rs @@ -29,6 +29,7 @@ use octseq::parse::Parser; /// /// See [RFC 7314](https://tools.ietf.org/html/rfc7314) for details. #[derive(Clone, Copy, Debug, Eq, Hash, Ord, PartialEq, PartialOrd)] +#[cfg_attr(feature = "serde", derive(serde::Serialize))] pub struct Expire(Option); impl Expire { diff --git a/src/base/opt/exterr.rs b/src/base/opt/exterr.rs index 2da1d0dae..fa9e0cde6 100644 --- a/src/base/opt/exterr.rs +++ b/src/base/opt/exterr.rs @@ -30,6 +30,13 @@ use core::convert::Infallible; /// standardized [`ExtendedErrorCode`] for machines and an optional UTF-8 /// error text for humans. #[derive(Clone)] +#[cfg_attr( + feature = "serde", + derive(serde::Serialize), + serde(bound( + serialize = "Octs: AsRef<[u8]>" + )) +)] pub struct ExtendedError { /// The extended error code. code: ExtendedErrorCode, @@ -37,9 +44,25 @@ pub struct ExtendedError { /// Optional human-readable error information. /// /// See `text` for the interpretation of the result. + #[cfg_attr(feature = "serde", serde(serialize_with = "lossy_text"))] text: Option, Octs>>, } +#[cfg(feature = "serde")] +fn lossy_text>( + text: &Option, Octs>>, + serializer: S, +) -> Result +where + S: serde::Serializer, +{ + match text { + Some(Ok(text)) => serializer.serialize_str(text), + Some(Err(text)) => serializer.serialize_str(&std::string::String::from_utf8_lossy(text.as_ref())), + None => serializer.serialize_none(), + } +} + impl ExtendedError<()> { /// The option code for this option. pub(super) const CODE: OptionCode = OptionCode::EXTENDED_ERROR; diff --git a/src/base/opt/keepalive.rs b/src/base/opt/keepalive.rs index 43190bb7e..28ab443f9 100644 --- a/src/base/opt/keepalive.rs +++ b/src/base/opt/keepalive.rs @@ -30,6 +30,7 @@ use octseq::parse::Parser; /// its response, including a 16-bit value that provides the idle time in /// units of 100 milliseconds. #[derive(Clone, Copy, Debug, Eq, Hash, Ord, PartialEq, PartialOrd)] +#[cfg_attr(feature = "serde", derive(serde::Serialize))] pub struct TcpKeepalive(Option); impl TcpKeepalive { @@ -147,6 +148,7 @@ impl<'a, Target: Composer> OptBuilder<'a, Target> { /// type provides means to conver the value into its raw `u16` value or into /// a [`Duration`] value. #[derive(Clone, Copy, Debug, Eq, Hash, Ord, PartialEq, PartialOrd)] +#[cfg_attr(feature = "serde", derive(serde::Serialize))] pub struct IdleTimeout(u16); impl IdleTimeout { diff --git a/src/base/opt/keytag.rs b/src/base/opt/keytag.rs index 025abf485..79316252c 100644 --- a/src/base/opt/keytag.rs +++ b/src/base/opt/keytag.rs @@ -335,6 +335,24 @@ impl<'a, Target: Composer> OptBuilder<'a, Target> { } } +//--- Serialize + +#[cfg(feature = "serde")] +impl> serde::Serialize for KeyTag { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer { + use serde::ser::SerializeSeq; + + let mut list = serializer.serialize_seq(None)?; + + for i in self.as_ref().iter() { + list.serialize_element(i)?; + } + + list.end() + } +} //------------ KeyTagIter ---------------------------------------------------- diff --git a/src/base/opt/macros.rs b/src/base/opt/macros.rs index d17474f3e..5f1d422c4 100644 --- a/src/base/opt/macros.rs +++ b/src/base/opt/macros.rs @@ -17,6 +17,16 @@ macro_rules! opt_types { //------------ AllOptData -------------------------------------------- #[derive(Clone)] + #[cfg_attr( + feature = "serde", + derive(serde::Serialize), + serde(bound( + serialize = " + Octs: AsRef<[u8]> + octseq::serde::SerializeOctets, + Name: serde::Serialize, + ", + )) + )] #[non_exhaustive] pub enum AllOptData { $( $( diff --git a/src/base/opt/mod.rs b/src/base/opt/mod.rs index 49c9929b4..c99b2ce1d 100644 --- a/src/base/opt/mod.rs +++ b/src/base/opt/mod.rs @@ -77,6 +77,26 @@ pub struct Opt { octets: Octs, } +#[cfg(feature = "serde")] +impl> serde::Serialize for Opt { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + use serde::ser::SerializeSeq; + let mut list = serializer.serialize_seq(None)?; + + for rec in self.for_slice_ref().iter::>() { + let Ok(rec) = rec else { + continue; + }; + list.serialize_element(&rec)?; + } + + list.end() + } +} + impl Opt<()> { /// The rtype of this record data type. pub(crate) const RTYPE: Rtype = Rtype::OPT; @@ -866,11 +886,21 @@ pub trait ComposeOptData: OptData { /// /// This type accepts any option type via its option code and raw data. #[derive(Clone)] +#[cfg_attr(feature = "serde", derive(serde::Serialize))] pub struct UnknownOptData { /// The option code for the option. code: OptionCode, /// The raw option data. + #[cfg_attr( + feature = "serde", + serde( + serialize_with = "crate::utils::base16::serde::serialize", + bound( + serialize = "Octs: AsRef<[u8]> + octseq::serde::SerializeOctets", + ) + ) + )] data: Octs, } diff --git a/src/base/opt/nsid.rs b/src/base/opt/nsid.rs index 715ec829f..4711e7873 100644 --- a/src/base/opt/nsid.rs +++ b/src/base/opt/nsid.rs @@ -42,6 +42,15 @@ impl Nsid<()> { /// The option code for this option. pub(super) const CODE: OptionCode = OptionCode::NSID; } + +#[cfg(feature = "serde")] +impl serde::Serialize for Nsid { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer { + self.octets.serialize_octets(serializer) + } +} impl Nsid { /// Creates a value from the ocets of the name server identifier. diff --git a/src/base/opt/padding.rs b/src/base/opt/padding.rs index 0a16f5365..8d0c05d1b 100644 --- a/src/base/opt/padding.rs +++ b/src/base/opt/padding.rs @@ -210,3 +210,15 @@ impl<'a, Target: Composer> OptBuilder<'a, Target> { } } + +//--- Serialize + +#[cfg(feature = "serde")] +impl> serde::Serialize for Padding { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer { + use octseq::serde::SerializeOctets; + self.octets.as_ref().serialize_octets(serializer) + } +} diff --git a/src/base/opt/subnet.rs b/src/base/opt/subnet.rs index b01fcfb79..47f3c5521 100644 --- a/src/base/opt/subnet.rs +++ b/src/base/opt/subnet.rs @@ -38,6 +38,7 @@ use octseq::parse::Parser; /// The option is defined in [RFC 7871](https://tools.ietf.org/html/rfc7871) /// which also includes some guidance on its use. #[derive(Clone, Copy, Debug, Eq, Hash, Ord, PartialEq, PartialOrd)] +#[cfg_attr(feature = "serde", derive(serde::Serialize))] pub struct ClientSubnet { /// The source prefix length. source_prefix_len: u8, diff --git a/src/lib.rs b/src/lib.rs index 8e06f2d4c..fcc82d2d0 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -167,7 +167,6 @@ pub mod rdata; pub mod resolv; pub mod sign; pub mod stelline; -pub mod test; pub mod tsig; pub mod utils; pub mod validate; diff --git a/src/net/server/connection.rs b/src/net/server/connection.rs index 7fce0c141..ec1f0a389 100644 --- a/src/net/server/connection.rs +++ b/src/net/server/connection.rs @@ -113,7 +113,7 @@ pub struct Config { /// `tcp-idle-timeout` setting. response_write_timeout: Duration, - /// Limit on the number of DNS responses queued for wriing to the client. + /// Limit on the number of DNS responses queued for writing to the client. max_queued_responses: usize, } diff --git a/src/net/server/dgram.rs b/src/net/server/dgram.rs index e44488290..cac9fc25e 100644 --- a/src/net/server/dgram.rs +++ b/src/net/server/dgram.rs @@ -118,7 +118,7 @@ impl Config { /// [RFC 6891]: /// https://datatracker.ietf.org/doc/html/rfc6891#section-6.2.5 pub fn set_max_response_size(&mut self, value: Option) { - self.max_response_size = value; + self.max_response_size = value.map(|v| MAX_RESPONSE_SIZE.limit(v)); } /// Sets the time to wait for a complete message to be written to the diff --git a/src/net/server/middleware/mandatory.rs b/src/net/server/middleware/mandatory.rs index 21ff82762..fffea530d 100644 --- a/src/net/server/middleware/mandatory.rs +++ b/src/net/server/middleware/mandatory.rs @@ -35,9 +35,11 @@ pub const MINIMUM_RESPONSE_BYTE_LEN: u16 = 512; /// |--------|---------| /// | [1035] | TBD | /// | [2181] | TBD | +/// | [9619] | TBD | /// /// [1035]: https://datatracker.ietf.org/doc/html/rfc1035 /// [2181]: https://datatracker.ietf.org/doc/html/rfc2181 +/// [9619]: https://datatracker.ietf.org/doc/html/rfc9619 #[derive(Clone, Debug)] pub struct MandatoryMiddlewareSvc { /// The upstream [`Service`] to pass requests to and receive responses @@ -203,12 +205,29 @@ where // "Therefore IQUERY is now obsolete, and name servers SHOULD return // a "Not Implemented" error when an IQUERY request is received." if self.strict && msg.header().opcode() == Opcode::IQUERY { + debug!("RFC 3425 violation: request opcode IQUERY is obsolete."); + return ControlFlow::Break(mk_error_response( + msg, + OptRcode::NOTIMP, + )); + } + + // https://datatracker.ietf.org/doc/html/rfc9619#section-4 + // 4. Updates to RFC 1035 + // ... + // "A DNS message with OPCODE = 0 and QDCOUNT > 1 MUST be treated as + // an incorrectly formatted message. The value of the RCODE + // parameter in the response message MUST be set to 1 (FORMERR)." + if self.strict + && msg.header().opcode() == Opcode::QUERY + && msg.header_counts().qdcount() > 1 + { debug!( - "RFC 3425 3 violation: request opcode IQUERY is obsolete." + "RFC 9619 violation: request opcode QUERY with QDCOUNT > 1." ); return ControlFlow::Break(mk_error_response( msg, - OptRcode::NOTIMP, + OptRcode::FORMERR, )); } diff --git a/src/net/server/middleware/xfr/responder.rs b/src/net/server/middleware/xfr/responder.rs index d604aaccd..3b0f64b97 100644 --- a/src/net/server/middleware/xfr/responder.rs +++ b/src/net/server/middleware/xfr/responder.rs @@ -74,7 +74,7 @@ where // Note: NSD apparently uses name compresson on AXFR responses // because AXFR responses they typically contain lots of // alphabetically ordered duplicate names which compress well. NSD - // limits AXFR responses to 16,383 RRs because DNS name + // limits AXFR responses to 16,383 bytes because DNS name // compression uses a 14-bit offset (2^14-1=16383) from the start // of the message to the first occurence of a name instead of // repeating the name, and name compression is less effective diff --git a/src/rdata/macros.rs b/src/rdata/macros.rs index 5f855701e..55c0f4029 100644 --- a/src/rdata/macros.rs +++ b/src/rdata/macros.rs @@ -506,6 +506,17 @@ macro_rules! rdata_types { /// This enum collects the record data types for all currently /// implemented record types. #[derive(Clone)] + #[cfg_attr( + feature = "serde", + derive(serde::Serialize), + serde(bound( + serialize = " + O: AsRef<[u8]> + octseq::serde::SerializeOctets, + N: serde::Serialize, + ", + )), + serde(rename_all = "UPPERCASE") + )] #[non_exhaustive] pub enum AllRecordData { $( $( $( diff --git a/src/test/cargo.rs b/src/test/cargo.rs deleted file mode 100644 index 546a487c9..000000000 --- a/src/test/cargo.rs +++ /dev/null @@ -1,3 +0,0 @@ -//! Stuff from Cargo. - -pub struct Environ {} diff --git a/src/test/utils.rs b/src/test/utils.rs deleted file mode 100644 index 29a3abee0..000000000 --- a/src/test/utils.rs +++ /dev/null @@ -1,12 +0,0 @@ -//! Various utility functions. - -use std::fs::File; -use std::io; -use std::path::Path; - -//------------ Functions ----------------------------------------------------- - -pub fn save>(path: P, content: &str) -> Result<(), io::Error> { - let mut file = File::create(path)?; - io::Write::write_all(&mut file, content.as_bytes()) -} diff --git a/src/tsig/mod.rs b/src/tsig/mod.rs index c9898f4d9..fffec9c1d 100644 --- a/src/tsig/mod.rs +++ b/src/tsig/mod.rs @@ -54,8 +54,6 @@ #![cfg(feature = "tsig")] #![cfg_attr(docsrs, doc(cfg(feature = "tsig")))] -mod interop; - use core::{cmp, fmt, mem, str}; #[cfg(feature = "std")] @@ -1135,8 +1133,24 @@ impl> SigningContext { // // First, do we have a valid TSIG? let tsig = match MessageTsig::from_message(message) { - Some(tsig) => tsig, - None => return Ok(None), + Ok(tsig) => tsig, + // RFC 8945, section 5.2: + // > If multiple TSIG records are detected or a TSIG record is present + // > in any other position, the DNS message is dropped and a response + // > with RCODE 1 (FORMERR) MUST be returned. + Err(TsigError::Position) => { + return Err(ServerError::unsigned(TsigRcode::FORMERR)); + } + // RFC 8945, section 5.2: + // > If the TSIG RR cannot be interpreted, the server MUST regard + // > the message as corrupt and return a FORMERR to the server. + Err(TsigError::Invalid) => { + return Err(ServerError::unsigned(TsigRcode::FORMERR)) + } + Err(TsigError::ParseError) => { + return Err(ServerError::unsigned(TsigRcode::FORMERR)) + } + Err(TsigError::Missing) => return Ok(None), }; // 4.5.1. KEY check and error handling @@ -1224,8 +1238,20 @@ impl> SigningContext { { // Extract TSIG or bail out. let tsig = match MessageTsig::from_message(message) { - Some(tsig) => tsig, - None => return Ok(None), + Ok(tsig) => tsig, + // RFC 8945, section 5.2: + // > If multiple TSIG records are detected or a TSIG record is present + // > in any other position, the DNS message is dropped and a response + // > with RCODE 1 (FORMERR) MUST be returned. + Err(TsigError::Position) => return Err(ValidationError::FormErr), + // RFC 8945, section 5.2: + // > If the TSIG RR cannot be interpreted, the server MUST regard + // > the message as corrupt and return a FORMERR to the server. + Err(TsigError::Invalid) => return Err(ValidationError::FormErr), + Err(TsigError::ParseError) => { + return Err(ValidationError::FormErr) + } + Err(TsigError::Missing) => return Ok(None), }; // Check for unsigned errors. @@ -1449,23 +1475,34 @@ impl<'a, Octs: Octets + ?Sized> MessageTsig<'a, Octs> { /// Checks that there is exactly one TSIG record in the additional /// section, that it is the last record in this section. If that is true, /// returns the parsed TSIG records. - fn from_message(msg: &'a Message) -> Option { - let mut section = msg.additional().ok()?; - let mut start = section.pos(); - let mut record = section.next()?; + fn from_message(msg: &'a Message) -> Result { + let mut section = + msg.additional().map_err(|_| TsigError::ParseError)?; + + // Find the first TSIG record, which we will assert to be the last + // one to verify that it is the only one. loop { - let record_start = section.pos(); - record = match section.next() { - Some(record) => record, - None => break, + let start = section.pos(); + + let Some(record) = section.next() else { + return Err(TsigError::Missing); }; - start = record_start; + + let record = record + .map_err(|_| TsigError::ParseError)? + .into_record::>() + .map_err(|_| TsigError::Invalid)?; + + // If it's None, then it's some other record type, and we just + // continue. + if let Some(record) = record { + // We got a valid TSIG, now assert that it's the last record: + if section.next().is_some() { + return Err(TsigError::Position); + } + return Ok(MessageTsig { record, start }); + } } - record - .ok()? - .into_record::>() - .ok()? - .map(|record| MessageTsig { record, start }) } fn variables(&self) -> Variables { @@ -1794,7 +1831,7 @@ impl> ServerError { tsig.record.owner(), tsig.record.class(), tsig.record.ttl(), - // The TSIG record data can never ever be to long. + // The TSIG record data can never ever be too long. Tsig::new( tsig.record.data().algorithm(), tsig.record.data().time_signed(), @@ -2008,3 +2045,21 @@ impl fmt::Display for ValidationError { #[cfg(feature = "std")] impl std::error::Error for ValidationError {} + +//------------ TsigError ----------------------------------------------------- + +/// Error while retrieving a TSIG from a message +#[derive(Debug)] +enum TsigError { + /// The TSIG record was present but malformed + Invalid, + + /// A TSIG record was found that was **not** the last record in the message + Position, + + /// The TSIG record was missing + Missing, + + /// An error occurred while parsing the message + ParseError, +} diff --git a/src/zonetree/in_memory/read.rs b/src/zonetree/in_memory/read.rs index e9eb45772..d1828f39d 100644 --- a/src/zonetree/in_memory/read.rs +++ b/src/zonetree/in_memory/read.rs @@ -112,9 +112,6 @@ impl ReadZone { walk.op(&SharedRrset::new(rrset)); } - // As we are querying for a qname below this node this - // means that the CNAME at this node cannot be a match, so - // ignore it and look for a match in the children. self.query_children( node.children(), label, diff --git a/src/test/mod.rs b/tests/common/mod.rs similarity index 83% rename from src/test/mod.rs rename to tests/common/mod.rs index e32522215..d7bc42f96 100644 --- a/src/test/mod.rs +++ b/tests/common/mod.rs @@ -7,8 +7,6 @@ //! `interop`. Since they require additional software packages to be available //! and can be expensive, they are tagged as ignored by default. //! -#![cfg(all(test, feature = "bytes", feature = "std"))] +#![cfg(all(feature = "bytes", feature = "std"))] -pub mod cargo; pub mod nsd; -pub mod utils; diff --git a/src/test/nsd.rs b/tests/common/nsd.rs similarity index 99% rename from src/test/nsd.rs rename to tests/common/nsd.rs index 551088139..e874ebb2a 100644 --- a/src/test/nsd.rs +++ b/tests/common/nsd.rs @@ -1,7 +1,5 @@ //! Configuring and running NSD. -use crate::utils::base64; -use bytes::Bytes; use std::fs::File; use std::net::{IpAddr, SocketAddr}; use std::path::{Path, PathBuf}; @@ -9,6 +7,9 @@ use std::string::String; use std::vec::Vec; use std::{fmt, io}; +use bytes::Bytes; +use domain::utils::base64; + //------------ Config -------------------------------------------------------- /// NSD configuration. diff --git a/src/tsig/interop.rs b/tests/interop.rs similarity index 96% rename from src/tsig/interop.rs rename to tests/interop.rs index be614e829..e19d63339 100644 --- a/src/tsig/interop.rs +++ b/tests/interop.rs @@ -1,20 +1,8 @@ -//! Tests the TSIG implementation. -#![cfg(test)] +//! TSIG interop testing with other DNS implementations. +#![cfg(all(feature = "bytes", feature = "std"))] + +mod common; -use crate::base::iana::{Rcode, Rtype}; -use crate::base::message::Message; -use crate::base::message_builder::{ - AdditionalBuilder, AnswerBuilder, MessageBuilder, StreamTarget, -}; -use crate::base::name::Name; -use crate::base::opt::TcpKeepalive; -use crate::base::record::Ttl; -use crate::rdata::tsig::Time48; -use crate::rdata::{Soa, A}; -use crate::test::nsd; -use crate::tsig; -use crate::utils::base64; -use ring::rand::SystemRandom; use std::io::{Read, Write}; use std::net::{IpAddr, SocketAddr, TcpListener, TcpStream, UdpSocket}; use std::process::Command; @@ -24,6 +12,22 @@ use std::time::Duration; use std::vec::Vec; use std::{env, fs, io, path::PathBuf, thread}; +use domain::base::iana::{Rcode, Rtype}; +use domain::base::message::Message; +use domain::base::message_builder::{ + AdditionalBuilder, AnswerBuilder, MessageBuilder, StreamTarget, +}; +use domain::base::name::Name; +use domain::base::opt::TcpKeepalive; +use domain::base::record::Ttl; +use domain::rdata::tsig::Time48; +use domain::rdata::{Soa, A}; +use domain::tsig; +use domain::utils::base64; +use ring::rand::SystemRandom; + +use common::nsd; + type TestMessage = Message>; type TestBuilder = MessageBuilder>>; type TestAnswer = AnswerBuilder>>; From 736671994fd9e8caa17ccaec346d28a2d9a64018 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 3 Oct 2024 11:16:41 +0200 Subject: [PATCH 332/333] Remove ZoneTuple and Zone::key(). They are confusing and hard to name well and not currently really needed. --- src/zonetree/zone.rs | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/src/zonetree/zone.rs b/src/zonetree/zone.rs index 4fd5a5caf..536a22cee 100644 --- a/src/zonetree/zone.rs +++ b/src/zonetree/zone.rs @@ -13,17 +13,6 @@ use super::traits::WritableZone; use super::types::StoredName; use super::{parsed, ReadableZone, ZoneStore}; -//------------ ZoneTuple ------------------------------------------------------ - -/// A tuple that identifies a zone. -/// -/// A zone is identified by the owner name of the apex and its class. Every -/// record in a zone must be at or under the apex owner name and be of the -/// same class. -pub type ZoneTuple = (StoredName, Class); - -//------------ Zone ----------------------------------------------------------- - /// A single DNS zone. /// /// # Abstract backing store @@ -95,14 +84,6 @@ impl Zone { { self.store.clone().write() } - - /// Gets a key that uniquely identifies this zone. - /// - /// Note: Assumes that there is only ever one instance of a zone with a - /// given apex name and class in a set of zones. - pub fn key(&self) -> ZoneTuple { - (self.apex_name().clone(), self.class()) - } } impl AsRef for Zone { From fa2eeffe73df80d0ef93bd15fbdac3a9328ba2d4 Mon Sep 17 00:00:00 2001 From: Ximon Eighteen <3304436+ximon18@users.noreply.github.com> Date: Thu, 3 Oct 2024 11:52:53 +0200 Subject: [PATCH 333/333] Compilation fix. --- src/zonetree/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/zonetree/mod.rs b/src/zonetree/mod.rs index b953d6cbf..5556f234e 100644 --- a/src/zonetree/mod.rs +++ b/src/zonetree/mod.rs @@ -124,7 +124,7 @@ pub use self::types::{ StoredName, StoredRecord, }; pub use self::walk::WalkOp; -pub use self::zone::{Zone, ZoneTuple}; +pub use self::zone::Zone; /// Zone related utilities. pub mod util {