"README.md" did not exist on "9fbbc3d9e54dbe0a1ce71907b4844ba41cec741b"
Newer
Older
use crate::{client_server, utils, ConduitResult, Database, Error, PduEvent, Result, Ruma};
use http::header::{HeaderValue, AUTHORIZATION, HOST};
Timo Kösters
committed
use regex::Regex;
client::error::ErrorKind,
directory::{get_public_rooms, get_public_rooms_filtered},
get_remote_server_keys, get_server_keys,
get_server_version::v1 as get_server_version, ServerSigningKeys, VerifyKey,
event::{get_event, get_missing_events, get_room_state_ids},
directory::{IncomingFilter, IncomingRoomNetwork},
Timo Kösters
committed
events::EventType,
identifiers::{KeyId, KeyName},
serde::to_canonical_value,
signatures::{CanonicalJsonObject, CanonicalJsonValue, PublicKeyMap},
EventId, RoomId, RoomVersionId, ServerName, ServerSigningKeyId, SigningKeyAlgorithm, UserId,
use state_res::{Event, EventMap, StateMap};
collections::{BTreeMap, BTreeSet, HashMap},
future::Future,
net::{IpAddr, SocketAddr},
pin::Pin,
result::Result as StdResult,
#[cfg(feature = "conduit_bin")]
use rocket::{get, post, put};
pub async fn send_request<T: OutgoingRequest>(
globals: &crate::database::globals::Globals,
) -> Result<T::IncomingResponse>
where
T: Debug,
{
return Err(Error::bad_config("Federation is disabled."));
let maybe_result = globals
.actual_destination_cache
.read()
.unwrap()
let (actual_destination, host) = if let Some(result) = maybe_result {
result
} else {
let result = find_actual_destination(globals, &destination).await;
globals
.actual_destination_cache
.write()
.unwrap()
.insert(Box::<ServerName>::from(destination), result.clone());
let mut http_request = request
.try_into_http_request(&actual_destination, Some(""))
warn!("Failed to find destination {}: {}", actual_destination, e);
Error::BadServerResponse("Invalid destination")
})?;
let mut request_map = serde_json::Map::new();
if !http_request.body().is_empty() {
request_map.insert(
"content".to_owned(),
serde_json::from_slice(http_request.body())
.expect("body is valid json, we just created it"),
request_map.insert("method".to_owned(), T::METADATA.method.to_string().into());
request_map.insert(
"uri".to_owned(),
http_request
.uri()
.path_and_query()
.expect("all requests have a path")
.to_string()
.into(),
);
request_map.insert("origin".to_owned(), globals.server_name().as_str().into());
request_map.insert("destination".to_owned(), destination.as_str().into());
let mut request_json =
serde_json::from_value(request_map.into()).expect("valid JSON is valid BTreeMap");
globals.server_name().as_str(),
globals.keypair(),
.expect("our request json is what ruma expects");
let request_json: serde_json::Map<String, serde_json::Value> =
serde_json::from_slice(&serde_json::to_vec(&request_json).unwrap()).unwrap();
let signatures = request_json["signatures"]
.as_object()
.unwrap()
.values()
.map(|v| {
v.as_object()
.unwrap()
.iter()
.map(|(k, v)| (k, v.as_str().unwrap()))
});
for signature_server in signatures {
for s in signature_server {
http_request.headers_mut().insert(
AUTHORIZATION,
HeaderValue::from_str(&format!(
"X-Matrix origin={},key=\"{}\",sig=\"{}\"",
s.0,
s.1
))
.unwrap(),
);
}
http_request
.headers_mut()
.insert(HOST, HeaderValue::from_str(&host).unwrap());
let mut reqwest_request = reqwest::Request::try_from(http_request)
.expect("all http requests are valid reqwest requests");
*reqwest_request.timeout_mut() = Some(Duration::from_secs(30));
let url = reqwest_request.url().clone();
let reqwest_response = globals.reqwest_client().execute(reqwest_request).await;
// Because reqwest::Response -> http::Response is complicated:
match reqwest_response {
Ok(mut reqwest_response) => {
let status = reqwest_response.status();
let mut http_response = http::Response::builder().status(status);
let headers = http_response.headers_mut().unwrap();
for (k, v) in reqwest_response.headers_mut().drain() {
if let Some(key) = k {
headers.insert(key, v);
}
}
let status = reqwest_response.status();
let body = reqwest_response
.bytes()
.await
Vec::new().into()
}) // TODO: handle timeout
.collect::<Vec<_>>();
if status != 200 {
info!(
"Server returned bad response {} {}\n{}\n{:?}",
destination,
status,
utils::string_from_bytes(&body)
);
}
let response = T::IncomingResponse::try_from(
http_response
.body(body)
.expect("reqwest body is valid http body"),
);
response.map_err(|_| {
info!(
"Server returned invalid response bytes {}\n{}",
destination, url
Error::BadServerResponse("Server returned bad response.")
})
fn get_ip_with_port(destination_str: String) -> Option<String> {
if destination_str.parse::<SocketAddr>().is_ok() {
Some(destination_str)
} else if let Ok(ip_addr) = destination_str.parse::<IpAddr>() {
Some(SocketAddr::new(ip_addr, 8448).to_string())
} else {
None
}
}
fn add_port_to_hostname(destination_str: String) -> String {
match destination_str.find(':') {
None => destination_str.to_owned() + ":8448",
Some(_) => destination_str.to_string(),
}
}
/// Returns: actual_destination, host header
/// Implemented according to the specification at https://matrix.org/docs/spec/server_server/r0.1.4#resolving-server-names
/// Numbers in comments below refer to bullet points in linked section of specification
async fn find_actual_destination(
globals: &crate::database::globals::Globals,
) -> (String, String) {
let destination_str = destination.as_str().to_owned();
let mut host = destination_str.clone();
let actual_destination = "https://".to_owned()
+ &match get_ip_with_port(destination_str.clone()) {
Some(host_port) => {
// 1: IP literal with provided or default port
host_port
None => {
if destination_str.find(':').is_some() {
// 2: Hostname with included port
destination_str
} else {
match request_well_known(globals, &destination.as_str()).await {
// 3: A .well-known file is available
Some(delegated_hostname) => {
host = delegated_hostname.clone();
match get_ip_with_port(delegated_hostname.clone()) {
Some(host_and_port) => host_and_port, // 3.1: IP literal in .well-known file
None => {
if destination_str.find(':').is_some() {
// 3.2: Hostname with port in .well-known file
destination_str
} else {
match query_srv_record(globals, &delegated_hostname).await {
// 3.3: SRV lookup successful
Some(hostname) => hostname,
// 3.4: No SRV records, just use the hostname from .well-known
None => add_port_to_hostname(delegated_hostname),
}
}
}
}
}
// 4: No .well-known or an error occured
None => {
match query_srv_record(globals, &destination_str).await {
// 4: SRV record found
Some(hostname) => hostname,
// 5: No SRV record found
None => add_port_to_hostname(destination_str.to_string()),
}
}
}
}
}
};
(actual_destination, host)
}
globals: &crate::database::globals::Globals,
) -> Option<String> {
if let Ok(Some(host_port)) = globals
.dns_resolver()
.srv_lookup(format!("_matrix._tcp.{}", hostname))
.await
.map(|srv| {
srv.iter().next().map(|result| {
format!(
"{}:{}",
result.target().to_string().trim_end_matches('.'),
result.port().to_string()
)
})
})
{
Some(host_port)
} else {
None
}
}
pub async fn request_well_known(
globals: &crate::database::globals::Globals,
destination: &str,
) -> Option<String> {
let body: serde_json::Value = serde_json::from_str(
&globals
.reqwest_client()
.get(&format!(
"https://{}/.well-known/matrix/server",
destination
))
.send()
.await
.ok()?
.text()
.await
.ok()?,
)
.ok()?;
Some(body.get("m.server")?.as_str()?.to_owned())
}
#[cfg_attr(feature = "conduit_bin", get("/_matrix/federation/v1/version"))]
pub fn get_server_version_route(
db: State<'_, Database>,
) -> ConduitResult<get_server_version::Response> {
return Err(Error::bad_config("Federation is disabled."));
Ok(get_server_version::Response {
name: Some("Conduit".to_owned()),
version: Some(env!("CARGO_PKG_VERSION").to_owned()),
#[cfg_attr(feature = "conduit_bin", get("/_matrix/key/v2/server"))]
pub fn get_server_keys_route(db: State<'_, Database>) -> Json<String> {
// TODO: Use proper types
return Json("Federation is disabled.".to_owned());
}
let mut verify_keys = BTreeMap::new();
verify_keys.insert(
ServerSigningKeyId::try_from(
format!("ed25519:{}", db.globals.keypair().version()).as_str(),
)
.expect("found invalid server signing keys in DB"),
timokoesters
committed
key: base64::encode_config(db.globals.keypair().public_key(), base64::STANDARD_NO_PAD),
},
);
let mut response = serde_json::from_slice(
http::Response::try_from(get_server_keys::v2::Response {
server_key: ServerSigningKeys {
server_name: db.globals.server_name().to_owned(),
verify_keys,
old_verify_keys: BTreeMap::new(),
signatures: BTreeMap::new(),
valid_until_ts: SystemTime::now() + Duration::from_secs(60 * 2),
},
})
.unwrap()
.body(),
)
.unwrap();
db.globals.server_name().as_str(),
db.globals.keypair(),
&mut response,
)
.unwrap();
Json(ruma::serde::to_canonical_json_string(&response).expect("JSON is canonical"))
#[cfg_attr(feature = "conduit_bin", get("/_matrix/key/v2/server/<_>"))]
pub fn get_server_keys_deprecated_route(db: State<'_, Database>) -> Json<String> {
get_server_keys_route(db)
#[cfg_attr(
feature = "conduit_bin",
post("/_matrix/federation/v1/publicRooms", data = "<body>")
)]
#[tracing::instrument(skip(db, body))]
pub async fn get_public_rooms_filtered_route(
db: State<'_, Database>,
body: Ruma<get_public_rooms_filtered::v1::Request<'_>>,
) -> ConduitResult<get_public_rooms_filtered::v1::Response> {
return Err(Error::bad_config("Federation is disabled."));
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
let response = client_server::get_public_rooms_filtered_helper(
&db,
None,
body.limit,
body.since.as_deref(),
&body.filter,
&body.room_network,
)
.await?
.0;
Ok(get_public_rooms_filtered::v1::Response {
chunk: response
.chunk
.into_iter()
.map(|c| {
// Convert ruma::api::federation::directory::get_public_rooms::v1::PublicRoomsChunk
// to ruma::api::client::r0::directory::PublicRoomsChunk
Ok::<_, Error>(
serde_json::from_str(
&serde_json::to_string(&c)
.expect("PublicRoomsChunk::to_string always works"),
)
.expect("federation and client-server PublicRoomsChunk are the same type"),
)
})
.filter_map(|r| r.ok())
.collect(),
prev_batch: response.prev_batch,
next_batch: response.next_batch,
total_room_count_estimate: response.total_room_count_estimate,
}
.into())
}
#[cfg_attr(
feature = "conduit_bin",
get("/_matrix/federation/v1/publicRooms", data = "<body>")
)]
#[tracing::instrument(skip(db, body))]
pub async fn get_public_rooms_route(
db: State<'_, Database>,
) -> ConduitResult<get_public_rooms::v1::Response> {
return Err(Error::bad_config("Federation is disabled."));
let response = client_server::get_public_rooms_filtered_helper(
&db,
None,
body.limit,
body.since.as_deref(),
&IncomingFilter::default(),
&IncomingRoomNetwork::Matrix,
)
.await?
.0;
Ok(get_public_rooms::v1::Response {
.into_iter()
.map(|c| {
// Convert ruma::api::federation::directory::get_public_rooms::v1::PublicRoomsChunk
// to ruma::api::client::r0::directory::PublicRoomsChunk
Ok::<_, Error>(
serde_json::from_str(
&serde_json::to_string(&c)
.expect("PublicRoomsChunk::to_string always works"),
)
.expect("federation and client-server PublicRoomsChunk are the same type"),
)
})
.filter_map(|r| r.ok())
.collect(),
prev_batch: response.prev_batch,
next_batch: response.next_batch,
total_room_count_estimate: response.total_room_count_estimate,
}
.into())
}
#[cfg_attr(
feature = "conduit_bin",
put("/_matrix/federation/v1/send/<_>", data = "<body>")
)]
#[tracing::instrument(skip(db, body))]
pub async fn send_transaction_message_route<'a>(
body: Ruma<send_transaction_message::v1::Request<'_>>,
) -> ConduitResult<send_transaction_message::v1::Response> {
return Err(Error::bad_config("Federation is disabled."));
info!("Incoming PDUs: {:?}", &body.pdus);
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
for edu in &body.edus {
match serde_json::from_str::<send_transaction_message::v1::Edu>(edu.json().get()) {
Ok(edu) => match edu.edu_type.as_str() {
"m.typing" => {
if let Some(typing) = edu.content.get("typing") {
if typing.as_bool().unwrap_or_default() {
db.rooms.edus.typing_add(
&UserId::try_from(edu.content["user_id"].as_str().unwrap())
.unwrap(),
&RoomId::try_from(edu.content["room_id"].as_str().unwrap())
.unwrap(),
3000 + utils::millis_since_unix_epoch(),
&db.globals,
)?;
} else {
db.rooms.edus.typing_remove(
&UserId::try_from(edu.content["user_id"].as_str().unwrap())
.unwrap(),
&RoomId::try_from(edu.content["room_id"].as_str().unwrap())
.unwrap(),
&db.globals,
)?;
}
}
}
"m.presence" => {}
"m.receipt" => {}
"m.device_list_update" => {}
Err(_err) => {
let mut resolved_map = BTreeMap::new();
let pdus_to_resolve = body
.pdus
.iter()
.filter_map(|pdu| {
// 1. Is a valid event, otherwise it is dropped.
// Ruma/PduEvent/StateEvent satisfies this
// We do not add the event_id field to the pdu here because of signature and hashes checks
let (event_id, value) = match crate::pdu::gen_event_id_canonical_json(pdu) {
Ok(t) => t,
Err(_) => {
// Event could not be converted to canonical json
return None;
}
};
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
// If we have no idea about this room skip the PDU
let room_id = match value
.get("room_id")
.map(|id| match id {
CanonicalJsonValue::String(id) => RoomId::try_from(id.as_str()).ok(),
_ => None,
})
.flatten()
{
Some(id) => id,
None => {
resolved_map.insert(event_id, Err("Event needs a valid RoomId".to_string()));
return None;
}
};
// 1. check the server is in the room (optional)
match db.rooms.exists(&room_id) {
Ok(true) => {}
_ => {
resolved_map
.insert(event_id, Err("Room is unknown to this server".to_string()));
return None;
}
}
// If we know of this pdu we don't need to continue processing it
if let Ok(Some(_)) = db.rooms.get_pdu_id(&event_id) {
return None;
}
Some((event_id, room_id, value))
})
.collect::<Vec<_>>();
// TODO: For RoomVersion6 we must check that Raw<..> is canonical do we anywhere?
// SPEC:
// Servers MUST strictly enforce the JSON format specified in the appendices.
// This translates to a 400 M_BAD_JSON error on most endpoints, or discarding of
// events over federation. For example, the Federation API's /send endpoint would
// discard the event whereas the Client Server API's /send/{eventType} endpoint
// would return a M_BAD_JSON error.
'main_pdu_loop: for (event_id, _room_id, value) in pdus_to_resolve {
info!("Working on incoming pdu: {:?}", value);
let server_name = &body.body.origin;
let mut pub_key_map = BTreeMap::new();
// TODO: make this persist but not a DB Tree...
// This is all the auth_events that have been recursively fetched so they don't have to be
// deserialized over and over again. This could potentially also be some sort of trie (suffix tree)
// like structure so that once an auth event is known it would know (using indexes maybe) all of
// the auth events that it references.
let mut auth_cache = EventMap::new();
// 2. check content hash, redact if doesn't match
// 3. fetch any missing auth events doing all checks listed here starting at 1. These are not timeline events
// 4. reject "due to auth events" if can't get all the auth events or some of the auth events are also rejected "due to auth events"
// 5. reject "due to auth events" if the event doesn't pass auth based on the auth events
// 7. if not timeline event: stop
// TODO; 8. fetch any missing prev events doing all checks listed here starting at 1. These are timeline events
// the events found in step 8 can be authed/resolved and appended to the DB
let (pdu, previous_create): (Arc<PduEvent>, Option<Arc<PduEvent>>) = match validate_event(
&db,
value,
event_id.clone(),
server_name,
// All the auth events gathered will be here
&mut auth_cache,
)
.await
{
Ok(pdu) => pdu,
Err(e) => {
resolved_map.insert(event_id, Err(e));
continue;
}
};
// 6. persist the event as an outlier.
db.rooms.add_pdu_outlier(&pdu)?;
info!("Added pdu as outlier.");
// Step 9. fetch missing state by calling /state_ids at backwards extremities doing all
// the checks in this list starting at 1. These are not timeline events.
//
// Step 10. check the auth of the event passes based on the calculated state of the event
//
// TODO: if we know the prev_events of the incoming event we can avoid the request and build
// the state from a known point and resolve if > 1 prev_event
debug!("Requesting state at event.");
let (state_at_event, incoming_auth_events): (StateMap<Arc<PduEvent>>, Vec<Arc<PduEvent>>) =
match db
.sending
.send_federation_request(
&db.globals,
get_room_state_ids::v1::Request {
room_id: pdu.room_id(),
event_id: pdu.event_id(),
},
.await
{
Ok(res) => {
debug!("Fetching state events at event.");
let state = match fetch_events(
&db,
server_name,
&mut auth_cache,
)
.await
{
Ok(state) => state,
Err(_) => continue,
};
// Sanity check: there are no conflicting events in the state we received
let mut seen = BTreeSet::new();
for ev in &state {
// If the key is already present
if !seen.insert((&ev.kind, &ev.state_key)) {
error!("Server sent us an invalid state");
continue;
}
}
let state = state
.into_iter()
.map(|pdu| ((pdu.kind.clone(), pdu.state_key.clone()), pdu))
.collect();
let incoming_auth_events = match fetch_events(
&db,
server_name,
&mut pub_key_map,
&res.auth_chain_ids,
&mut auth_cache,
.await
{
Ok(state) => state,
Err(_) => continue,
};
debug!("Fetching auth events of state events at event.");
}
Err(_) => {
resolved_map.insert(
pdu.event_id().clone(),
Err("Fetching state for event failed".into()),
);
continue;
}
};
// 10. This is the actual auth check for state at the event
if !state_res::event_auth::auth_check(
&RoomVersionId::Version6,
&pdu,
&state_at_event,
None, // TODO: third party invite
)
.map_err(|_e| Error::Conflict("Auth check failed"))?
{
// Event failed auth with state_at
resolved_map.insert(
event_id,
Err("Event has failed auth check with state at the event".into()),
);
continue;
// End of step 10.
// 12. check if the event passes auth based on the "current state" of the room, if not "soft fail" it
let current_state = db
.rooms
.room_state_full(pdu.room_id())?
.into_iter()
.map(|(k, v)| ((k.0, Some(k.1)), Arc::new(v)))
.collect();
if !state_res::event_auth::auth_check(
&RoomVersionId::Version6,
&pdu,
¤t_state,
None,
)
.map_err(|_e| Error::Conflict("Auth check failed"))?
{
// Soft fail, we add the event as an outlier.
resolved_map.insert(
pdu.event_id().clone(),
Err("Event has been soft failed".into()),
);
debug!("Auth check with current state succeeded.");
// Step 11. Ensure that the state is derived from the previous current state (i.e. we calculated by doing state res
// where one of the inputs was a previously trusted set of state, don't just trust a set of state we got from a remote)
//
// calculate_forward_extremities takes care of adding the current state if not already in the state sets
// it also calculates the new pdu leaves for the `roomid_pduleaves` DB Tree.
let extremities = match calculate_forward_extremities(&db, &pdu).await {
debug!("Calculated new forward extremities: {:?}", fork_ids);
Err(_) => {
resolved_map.insert(event_id, Err("Failed to gather forward extremities".into()));
continue;
}
};
// This will create the state after any state snapshot it builds
// So current_state will have the incoming event inserted to it
let mut fork_states = match build_forward_extremity_snapshots(
server_name,
current_state,
&extremities,
&pub_key_map,
&mut auth_cache,
)
.await
{
Ok(states) => states,
Err(_) => {
resolved_map.insert(event_id, Err("Failed to gather forward extremities".into()));
continue;
};
let mut state_after = state_at_event.clone();
state_after.insert((pdu.kind(), pdu.state_key()), pdu.clone());
// Add the incoming event to the mix of state snapshots
// Since we are using a BTreeSet (yea this may be overkill) we guarantee unique state sets
fork_states.insert(state_after.clone());
let fork_states = fork_states.into_iter().collect::<Vec<_>>();
let mut update_state = false;
// 13. start state-res with all previous forward extremities minus the ones that are in
// the prev_events of this event plus the new one created by this event and use
// the result as the new room state
let state_at_forks = if fork_states.is_empty() {
// State is empty
Default::default()
} else if fork_states.len() == 1 {
fork_states[0].clone()
} else {
// We do need to force an update to this rooms state
update_state = true;
let mut auth_events = vec![];
for map in &fork_states {
let mut state_auth = vec![];
for auth_id in map.values().flat_map(|pdu| &pdu.auth_events) {
match fetch_events(
&db,
server_name,
&mut pub_key_map,
&[auth_id.clone()],
&mut auth_cache,
)
.await
{
// This should always contain exactly one element when Ok
Ok(events) => state_auth.push(events[0].clone()),
Err(e) => {
debug!("Event was not present: {}", e);
}
auth_events.push(state_auth);
}
// Add everything we will need to event_map
auth_cache.extend(
auth_events
.iter()
.map(|pdus| pdus.iter().map(|pdu| (pdu.event_id().clone(), pdu.clone())))
.flatten(),
);
auth_cache.extend(
incoming_auth_events
.into_iter()
.map(|pdu| (pdu.event_id().clone(), pdu)),
auth_cache.extend(
.into_iter()
.map(|(_, pdu)| (pdu.event_id().clone(), pdu)),
);
let res = match state_res::StateResolution::resolve(
pdu.room_id(),
&RoomVersionId::Version6,
&fork_states
.into_iter()
.map(|map| {
map.into_iter()
.map(|(k, v)| (k, v.event_id.clone()))
.collect::<StateMap<_>>()
})
.collect::<Vec<_>>(),
auth_events
.into_iter()
.map(|pdus| pdus.into_iter().map(|pdu| pdu.event_id().clone()).collect())
.collect(),
&mut auth_cache,
Ok(res) => res,
Err(_) => {
resolved_map.insert(
pdu.event_id().clone(),
Err("State resolution failed, either an event could not be found or deserialization".into()),
);
continue 'main_pdu_loop;
}
let mut resolved = BTreeMap::new();
for (k, id) in res {
// We should know of the event but just incase
let pdu = match auth_cache.get(&id) {
Some(pdu) => pdu.clone(),
None => {
error!("Event was not present in auth_cache {}", id);
resolved_map.insert(
event_id.clone(),
Err("Event was not present in auth cache".into()),
);
continue 'main_pdu_loop;
}
};
resolved.insert(k, pdu);
resolved
// Now that the event has passed all auth it is added into the timeline.
// We use the `state_at_event` instead of `state_after` so we accurately
// represent the state for this event.
append_incoming_pdu(&db, &pdu, &extremities, &state_at_event)?;
info!("Appended incoming pdu.");
// Set the new room state to the resolved state
update_resolved_state(
pdu.room_id(),
if update_state {
Some(state_at_forks)
} else {
None
},
)?;
// Event has passed all auth/stateres checks
if !resolved_map.is_empty() {
warn!("These PDU's failed {:?}", resolved_map);
Ok(send_transaction_message::v1::Response { pdus: resolved_map }.into())
/// An async function that can recursively calls itself.
type AsyncRecursiveResult<'a, T> = Pin<Box<dyn Future<Output = StdResult<T, String>> + 'a + Send>>;
/// TODO: don't add as outlier if event is fetched as a result of gathering auth_events
/// Validate any event that is given to us by another server.
///
/// 1. Is a valid event, otherwise it is dropped (PduEvent deserialization satisfies this).
/// 2. check content hash, redact if doesn't match
/// 3. fetch any missing auth events doing all checks listed here starting at 1. These are not timeline events
/// 4. reject "due to auth events" if can't get all the auth events or some of the auth events are also rejected "due to auth events"
/// 5. reject "due to auth events" if the event doesn't pass auth based on the auth events
/// 7. if not timeline event: stop
/// 8. fetch any missing prev events doing all checks listed here starting at 1. These are timeline events
#[tracing::instrument(skip(db, pub_key_map, auth_cache))]
fn validate_event<'a>(
db: &'a Database,
value: CanonicalJsonObject,
event_id: EventId,
pub_key_map: &'a mut PublicKeyMap,
origin: &'a ServerName,
auth_cache: &'a mut EventMap<Arc<PduEvent>>,
) -> AsyncRecursiveResult<'a, (Arc<PduEvent>, Option<Arc<PduEvent>>)> {
Box::pin(async move {
for (signature_server, signature) in match value
.get("signatures")
.ok_or_else(|| "No signatures in server response pdu.".to_string())?
{
CanonicalJsonValue::Object(map) => map,
_ => return Err("Invalid signatures object in server response pdu.".to_string()),
} {
let signature_object = match signature {
CanonicalJsonValue::Object(map) => map,
_ => {
return Err(
"Invalid signatures content object in server response pdu.".to_string()
)
}
};
let signature_ids = signature_object.keys().collect::<Vec<_>>();
debug!("Fetching signing keys for {}", signature_server);
let keys = match fetch_signing_keys(
&db,
&Box::<ServerName>::try_from(&**signature_server).map_err(|_| {
"Invalid servername in signatures of server response pdu.".to_string()
})?,
Err(_) => {
return Err(
"Signature verification failed: Could not fetch signing key.".to_string(),