From d3c4df41915b3eddc777903234df25a5cbdcf92f Mon Sep 17 00:00:00 2001 From: Sam Andreae Date: Wed, 29 Jun 2022 15:10:07 +0100 Subject: [PATCH 01/13] Fix bug when materialising documents containing (non-existent) pinned relations (#177) * Add logging * Pretty print input * Introduce send_to_store helper method * Some more comments * Update CHANGELOG * Make env_logger a dev-dependency * Failing materializer tests * Logging in dependency task * More logging in reduce task * Don't critically fail when document for requested view not found * Tests * Small clippy happy * fmt * Update CHANGELOG * Remove env_logger * Wait a little longer... * Wait after both reduce tasks * Fix method visibility after merge Co-authored-by: Andreas Dzialocha Co-authored-by: Andreas Dzialocha --- CHANGELOG.md | 1 + aquadoggo/src/db/stores/test_utils.rs | 7 +- aquadoggo/src/materializer/service.rs | 202 +++++++++++++++++- .../src/materializer/tasks/dependency.rs | 64 ++++-- aquadoggo/src/materializer/tasks/reduce.rs | 125 +++++------ 5 files changed, 313 insertions(+), 86 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 46f0a2331..154804350 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -48,6 +48,7 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - Improve CI, track test coverage [#139](https://github.com/p2panda/aquadoggo/pull/139) - Fix compatibility with PostgreSQL, change sqlx runtime to `tokio` [#170](https://github.com/p2panda/aquadoggo/pull/170) - Use UPSERT for inserting or updating documents [#173](https://github.com/p2panda/aquadoggo/pull/173) +- Don't critically fail reduce task when document missing [#177](https://github.com/p2panda/aquadoggo/pull/177) ## [0.2.0] diff --git a/aquadoggo/src/db/stores/test_utils.rs b/aquadoggo/src/db/stores/test_utils.rs index 3dd98f294..5b2514bc2 100644 --- a/aquadoggo/src/db/stores/test_utils.rs +++ b/aquadoggo/src/db/stores/test_utils.rs @@ -12,9 +12,7 @@ use p2panda_rs::operation::{ PinnedRelation, PinnedRelationList, Relation, RelationList, VerifiedOperation, }; use p2panda_rs::schema::SchemaId; -use p2panda_rs::storage_provider::traits::{ - AsStorageEntry, AsStorageLog, EntryStore, LogStore, OperationStore, StorageProvider, -}; +use p2panda_rs::storage_provider::traits::{OperationStore, StorageProvider}; use p2panda_rs::test_utils::constants::{DEFAULT_PRIVATE_KEY, TEST_SCHEMA_ID}; use p2panda_rs::test_utils::fixtures::{operation, operation_fields}; use rstest::fixture; @@ -23,7 +21,6 @@ use sqlx::Any; use tokio::runtime::Builder; use crate::db::provider::SqlStorage; -use crate::db::stores::{StorageEntry, StorageLog}; use crate::db::traits::DocumentStore; use crate::db::{connection_pool, create_database, run_pending_migrations, Pool}; use crate::graphql::client::{EntryArgsRequest, PublishEntryRequest, PublishEntryResponse}; @@ -378,7 +375,7 @@ async fn create_test_db( } } -async fn send_to_store( +pub async fn send_to_store( store: &SqlStorage, operation: &Operation, document_id: Option<&DocumentId>, diff --git a/aquadoggo/src/materializer/service.rs b/aquadoggo/src/materializer/service.rs index c7b5ceb5f..f2bb8eb2e 100644 --- a/aquadoggo/src/materializer/service.rs +++ b/aquadoggo/src/materializer/service.rs @@ -132,15 +132,24 @@ pub async fn materializer_service( mod tests { use std::time::Duration; - use p2panda_rs::operation::{AsVerifiedOperation, OperationValue}; + use p2panda_rs::document::DocumentViewId; + use p2panda_rs::identity::KeyPair; + use p2panda_rs::operation::{ + AsVerifiedOperation, Operation, OperationValue, PinnedRelation, PinnedRelationList, + Relation, RelationList, + }; use p2panda_rs::storage_provider::traits::OperationStore; use p2panda_rs::test_utils::constants::TEST_SCHEMA_ID; + use p2panda_rs::test_utils::fixtures::{ + key_pair, operation, operation_fields, random_document_id, random_document_view_id, + random_operation_id, + }; use rstest::rstest; use tokio::sync::broadcast; use tokio::task; use crate::context::Context; - use crate::db::stores::test_utils::{test_db, TestDatabase, TestDatabaseRunner}; + use crate::db::stores::test_utils::{send_to_store, test_db, TestDatabase, TestDatabaseRunner}; use crate::db::traits::DocumentStore; use crate::materializer::{Task, TaskInput}; use crate::Configuration; @@ -282,4 +291,193 @@ mod tests { ); }); } + + #[rstest] + fn materialize_update_document( + #[from(test_db)] + #[with(1, 1, false, TEST_SCHEMA_ID.parse().unwrap(), vec![("name", OperationValue::Text("panda".into()))])] + runner: TestDatabaseRunner, + ) { + // Prepare database which inserts data for one document + runner.with_db_teardown(|db: TestDatabase| async move { + // Identify key_[air, document and operation which was inserted for testing + let key_pair = db.key_pairs.first().unwrap(); + let document_id = db.documents.first().unwrap(); + let verified_operation = db + .store + .get_operations_by_document_id(document_id) + .await + .unwrap() + .first() + .unwrap() + .to_owned(); + + // Prepare arguments for service + let context = Context::new(db.store.clone(), Configuration::default()); + let shutdown = task::spawn(async { + loop { + // Do this forever .. this means that the shutdown handler will never resolve + tokio::time::sleep(Duration::from_millis(100)).await; + } + }); + let (tx, _) = broadcast::channel(1024); + + // Start materializer service + let tx_clone = tx.clone(); + let handle = tokio::spawn(async move { + materializer_service(context, shutdown, tx_clone) + .await + .unwrap(); + }); + + // Wait for service to be ready .. + tokio::time::sleep(Duration::from_millis(50)).await; + + // Send a message over the bus which kicks in materialization + tx.send(crate::bus::ServiceMessage::NewOperation( + verified_operation.operation_id().to_owned(), + )) + .unwrap(); + + // Wait a little bit for work being done .. + tokio::time::sleep(Duration::from_millis(50)).await; + + // Then straight away publish an UPDATE on this document and send it over the bus too. + let (entry_encoded, _) = send_to_store( + &db.store, + &operation( + Some(operation_fields(vec![( + "name", + OperationValue::Text("panda123".into()), + )])), + Some(verified_operation.operation_id().to_owned().into()), + Some(TEST_SCHEMA_ID.parse().unwrap()), + ), + Some(document_id), + key_pair, + ) + .await; + + // Send a message over the bus which kicks in materialization + tx.send(crate::bus::ServiceMessage::NewOperation( + entry_encoded.hash().into(), + )) + .unwrap(); + + // Wait a little bit for work being done .. + tokio::time::sleep(Duration::from_millis(50)).await; + + // Make sure the service did not crash and is still running + assert_eq!(handle.is_finished(), false); + + // Check database for materialized documents + let document = db + .store + .get_document_by_id(document_id) + .await + .unwrap() + .expect("We expect that the document is `Some`"); + assert_eq!(document.id(), &entry_encoded.hash().into()); + assert_eq!( + document.fields().get("name").unwrap().value().to_owned(), + OperationValue::Text("panda123".into()) + ); + }); + } + + #[rstest] + #[case( + operation(Some(operation_fields(vec![( + "relation", + OperationValue::Relation(Relation::new( + random_document_id(), + )), + )])), None, None) + )] + #[case( + operation(Some(operation_fields(vec![( + "pinned_relation", + OperationValue::PinnedRelation(PinnedRelation::new( + DocumentViewId::new(&[random_operation_id(), random_operation_id()]).unwrap(), + )), + )])), None, None) + )] + #[case( + operation(Some(operation_fields(vec![( + "relation_list", + OperationValue::RelationList(RelationList::new( + vec![ + random_document_id(), + random_document_id(), + random_document_id() + ], + )), + )])), None, None) + )] + #[case( + operation(Some(operation_fields(vec![( + "pinned_relation_list", + OperationValue::PinnedRelationList(PinnedRelationList::new( + vec![ + DocumentViewId::new(&[random_operation_id(), random_operation_id()]).unwrap(), + DocumentViewId::new(&[random_operation_id(), random_operation_id(), random_operation_id()]).unwrap() + ], + )), + )])), None, None) + )] + fn materialize_complex_documents( + #[from(test_db)] + #[with(0, 0)] + runner: TestDatabaseRunner, + #[case] operation: Operation, + key_pair: KeyPair, + ) { + // Prepare empty database + runner.with_db_teardown(|db: TestDatabase| async move { + // Prepare arguments for service + let context = Context::new(db.store.clone(), Configuration::default()); + let shutdown = task::spawn(async { + loop { + // Do this forever .. this means that the shutdown handler will never resolve + tokio::time::sleep(Duration::from_millis(100)).await; + } + }); + let (tx, _) = broadcast::channel(1024); + + // Start materializer service + let tx_clone = tx.clone(); + let handle = tokio::spawn(async move { + materializer_service(context, shutdown, tx_clone) + .await + .unwrap(); + }); + + // Wait for service to be ready .. + tokio::time::sleep(Duration::from_millis(50)).await; + + // Then straight away publish a CREATE operation and send it to the bus. + let (entry_encoded, _) = send_to_store(&db.store, &operation, None, &key_pair).await; + + // Send a message over the bus which kicks in materialization + tx.send(crate::bus::ServiceMessage::NewOperation( + entry_encoded.hash().into(), + )) + .unwrap(); + + // Wait a little bit for work being done .. + tokio::time::sleep(Duration::from_millis(100)).await; + + // Make sure the service did not crash and is still running + assert_eq!(handle.is_finished(), false); + + // Check database for materialized documents + let document = db + .store + .get_document_by_id(&entry_encoded.hash().into()) + .await + .unwrap() + .expect("We expect that the document is `Some`"); + assert_eq!(document.id(), &entry_encoded.hash().into()); + }); + } } diff --git a/aquadoggo/src/materializer/tasks/dependency.rs b/aquadoggo/src/materializer/tasks/dependency.rs index 5f8fb929b..3d0577b11 100644 --- a/aquadoggo/src/materializer/tasks/dependency.rs +++ b/aquadoggo/src/materializer/tasks/dependency.rs @@ -1,5 +1,6 @@ // SPDX-License-Identifier: AGPL-3.0-or-later +use log::debug; use p2panda_rs::document::DocumentViewId; use crate::context::Context; @@ -19,25 +20,40 @@ use crate::materializer::TaskInput; /// Expects a _reduce_ task to have completed successfully for the given document view itself and /// returns a critical error otherwise. pub async fn dependency_task(context: Context, input: TaskInput) -> TaskResult { + debug!("Working on dependency task {:#?}", input); + // Here we retrive the document view by document view id. let document_view = match input.document_view_id { Some(view_id) => context .store .get_document_view_by_id(&view_id) .await - .map_err(|_| TaskError::Critical) - , + .map_err(|err| { + debug!("Fatal error getting document view from storage"); + debug!("{}", err); + TaskError::Critical + }) + , // We expect to handle document_view_ids in a dependency task. None => Err(TaskError::Critical), }?; let document_view = match document_view { - Some(document_view) => Ok(document_view), + Some(document_view) => { + debug!( + "Document view retrieved from storage with id: {}", + document_view.id() + ); + Ok(document_view) + } // If no document view for the id passed into this task could be retrieved then this // document has been deleted or the document view id was invalid. As "dependency" tasks // are only dispatched after a successful "reduce" task, neither `None` case should // happen, so this is a critical error. - None => Err(TaskError::Critical), + None => { + debug!("Expected document view not found in the store."); + Err(TaskError::Critical) + } }?; let mut next_tasks = Vec::new(); @@ -51,15 +67,21 @@ pub async fn dependency_task(context: Context, input: TaskInput) -> TaskResult { // same as above... + debug!("Relation list field found, no action required."); } p2panda_rs::operation::OperationValue::PinnedRelation(pinned_relation) => { // These are pinned relations. We may have the operations for these views in the db, // but this view wasn't pinned yet, so hasn't been materialised. To make sure it is // materialised when possible, we dispatch a "reduce" task for any pinned relations // which aren't found. + debug!( + "Pinned relation field found refering to view id: {}", + pinned_relation.view_id() + ); next_tasks.push( construct_relation_task(&context, pinned_relation.view_id().clone()).await?, ); @@ -67,6 +89,10 @@ pub async fn dependency_task(context: Context, input: TaskInput) -> TaskResult { // same as above... for document_view_id in pinned_relation_list.iter() { + debug!( + "Pinned relation list field found containing view id: {}", + document_view_id + ); next_tasks .push(construct_relation_task(&context, document_view_id.clone()).await?); } @@ -75,7 +101,11 @@ pub async fn dependency_task(context: Context, input: TaskInput) -> TaskResult> = next_tasks.into_iter().flatten().collect(); + + debug!("Dispatching {} reduce task(s)", next_tasks.len()); + + Ok(Some(next_tasks)) } /// Returns a _reduce_ task for a given document view only if that view does not yet exist in the @@ -84,17 +114,27 @@ async fn construct_relation_task( context: &Context, document_view_id: DocumentViewId, ) -> Result>, TaskError> { + debug!("Get view for pinned relation with id: {}", document_view_id); match context .store .get_document_view_by_id(&document_view_id) .await - .map_err(|_| TaskError::Critical)? - { - Some(_) => Ok(None), - None => Ok(Some(Task::new( - "reduce", - TaskInput::new(None, Some(document_view_id)), - ))), + .map_err(|err| { + debug!("Fatal error getting document view from storage"); + debug!("{}", err); + TaskError::Critical + })? { + Some(_) => { + debug!("View found for pinned relation: {}", document_view_id); + Ok(None) + } + None => { + debug!("No view found for pinned relation: {}", document_view_id); + Ok(Some(Task::new( + "reduce", + TaskInput::new(None, Some(document_view_id)), + ))) + } } } diff --git a/aquadoggo/src/materializer/tasks/reduce.rs b/aquadoggo/src/materializer/tasks/reduce.rs index 730fae9c2..baa8ffc10 100644 --- a/aquadoggo/src/materializer/tasks/reduce.rs +++ b/aquadoggo/src/materializer/tasks/reduce.rs @@ -21,7 +21,13 @@ pub async fn reduce_task(context: Context, input: TaskInput) -> TaskResult Ok(document_id), + None => { + debug!("No document found for this view, exit without dispatching any other tasks"); + return Ok(None); + } + }?; // Get all operations for the requested document let operations = context @@ -65,27 +71,27 @@ pub async fn reduce_task(context: Context, input: TaskInput) -> TaskResult Result { +) -> Result, TaskError> { match (&input.document_id, &input.document_view_id) { // The `DocumentId` is already given, we don't have to do anything - (Some(document_id), None) => Ok(document_id.to_owned()), + (Some(document_id), None) => Ok(Some(document_id.to_owned())), // A `DocumentViewId` is given, let's find out its document id (None, Some(document_view_id)) => { // @TODO: We can skip this step if we implement: // https://github.com/p2panda/aquadoggo/issues/148 + debug!("Find document for view with id: {}", document_view_id); let operation_id = document_view_id.clone().into_iter().next().unwrap(); - match context + context .store .get_document_by_operation_id(&operation_id) .await - .map_err(|_| TaskError::Critical)? - { - Some(document_id) => Ok(document_id), - None => Err(TaskError::Critical), - } + .map_err(|err| { + debug!("Fatal error getting document_id from storage"); + debug!("{}", err); + TaskError::Critical + }) } - // None or both have been provided which smells like a bug (_, _) => Err(TaskError::Critical), } @@ -189,27 +195,19 @@ async fn reduce_document( #[cfg(test)] mod tests { - use std::convert::TryFrom; - use p2panda_rs::document::{DocumentBuilder, DocumentId, DocumentViewId}; - use p2panda_rs::entry::{sign_and_encode, Entry}; - use p2panda_rs::identity::Author; - use p2panda_rs::operation::{ - AsVerifiedOperation, OperationEncoded, OperationValue, VerifiedOperation, - }; - use p2panda_rs::storage_provider::traits::{ - AsStorageEntry, EntryStore, OperationStore, StorageProvider, - }; + use p2panda_rs::operation::{AsVerifiedOperation, OperationValue}; + use p2panda_rs::storage_provider::traits::OperationStore; use p2panda_rs::test_utils::constants::TEST_SCHEMA_ID; - use p2panda_rs::test_utils::fixtures::{operation, operation_fields}; + use p2panda_rs::test_utils::fixtures::{ + operation, operation_fields, random_document_id, random_document_view_id, + }; use rstest::rstest; use crate::config::Configuration; use crate::context::Context; - use crate::db::stores::test_utils::{test_db, TestDatabase, TestDatabaseRunner}; - use crate::db::stores::StorageEntry; + use crate::db::stores::test_utils::{send_to_store, test_db, TestDatabase, TestDatabaseRunner}; use crate::db::traits::DocumentStore; - use crate::graphql::client::EntryArgsRequest; use crate::materializer::tasks::reduce_task; use crate::materializer::TaskInput; @@ -218,7 +216,7 @@ mod tests { #[from(test_db)] #[with( 2, - 20, + 5, false, TEST_SCHEMA_ID.parse().unwrap(), vec![("username", OperationValue::Text("panda".into()))], @@ -254,7 +252,6 @@ mod tests { runner.with_db_teardown(|db: TestDatabase| async move { let document_id = db.documents.first().unwrap(); let key_pair = db.key_pairs.first().unwrap(); - let author = Author::try_from(key_pair.public_key().to_owned()).unwrap(); let context = Context::new(db.store.clone(), Configuration::default()); let input = TaskInput::new(Some(document_id.clone()), None); @@ -264,48 +261,20 @@ mod tests { assert!(reduce_task(context.clone(), input.clone()).await.is_ok()); // Now we create and insert an UPDATE operation for this document. - let entry_args = db - .store - .get_entry_args(&EntryArgsRequest { - author, - document: Some(document_id.clone()), - }) - .await - .unwrap(); - - let operation = operation( - Some(operation_fields(vec![( - "username", - OperationValue::Text("meeeeeee".to_string()), - )])), - Some(document_id.as_str().parse().unwrap()), - None, - ); - - let entry = Entry::new( - &entry_args.log_id, - Some(&operation), - entry_args.skiplink.as_ref(), - entry_args.backlink.as_ref(), - &entry_args.seq_num, + let (_, _) = send_to_store( + &db.store, + &operation( + Some(operation_fields(vec![( + "username", + OperationValue::Text("meeeeeee".to_string()), + )])), + Some(document_id.as_str().parse().unwrap()), + None, + ), + Some(document_id), + key_pair, ) - .unwrap(); - - let entry_signed = sign_and_encode(&entry, key_pair).unwrap(); - let operation_encoded = OperationEncoded::try_from(&operation).unwrap(); - - db.store - .insert_entry(StorageEntry::new(&entry_signed, &operation_encoded).unwrap()) - .await - .unwrap(); - - let verified_operation = - VerifiedOperation::new_from_entry(&entry_signed, &operation_encoded).unwrap(); - - db.store - .insert_operation(&verified_operation, document_id) - .await - .unwrap(); + .await; // This should now find the new UPDATE operation and perform an update on the document // in the documents table. @@ -472,4 +441,26 @@ mod tests { assert!(reduce_task(context.clone(), input).await.is_err()); }); } + + #[rstest] + fn does_not_error_when_document_missing( + #[from(test_db)] + #[with(0, 0)] + runner: TestDatabaseRunner, + #[from(random_document_id)] document_id: DocumentId, + #[from(random_document_view_id)] document_view_id: DocumentViewId, + ) { + // Prepare empty database. + runner.with_db_teardown(|db: TestDatabase| async move { + let context = Context::new(db.store.clone(), Configuration::default()); + + // Dispatch a reduce task for a document which doesn't exist by it's document id. + let input = TaskInput::new(Some(document_id), None); + assert!(reduce_task(context.clone(), input).await.is_ok()); + + // Dispatch a reduce task for a document which doesn't exist by it's document view id. + let input = TaskInput::new(None, Some(document_view_id)); + assert!(reduce_task(context.clone(), input).await.is_ok()); + }); + } } From cc48e413bbf2e427f16128a3fa354f59cbdd0942 Mon Sep 17 00:00:00 2001 From: Sam Andreae Date: Thu, 30 Jun 2022 14:39:05 +0100 Subject: [PATCH 02/13] Update naming in `publishEntry` query and `nextEntryArgs` response (#181) * Update naming in entry args response * Update publishEntry parameter names according to spec change * Add entry to CHANGELOG.md Co-authored-by: Andreas Dzialocha --- CHANGELOG.md | 3 ++- aquadoggo/src/graphql/client/mutation.rs | 17 ++++++++--------- aquadoggo/src/graphql/client/response.rs | 11 +++-------- .../graphql/replication/client/schema.graphql | 4 ++-- 4 files changed, 15 insertions(+), 20 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 154804350..8158fe0bc 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -40,7 +40,8 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - Refactor tests to use fixtures exported from `p2panda-rs` [#147](https://github.com/p2panda/aquadoggo/pull/147) - Use `DocumentViewId` for previous operations [#147](https://github.com/p2panda/aquadoggo/pull/147) - Use `VerifiedOperation` [#158](https://github.com/p2panda/aquadoggo/pull/158) -- Refactor test_db creation [#176](https://github.com/p2panda/aquadoggo/pull/176) +- Refactor `test_db` helper method [#176](https://github.com/p2panda/aquadoggo/pull/176) +- Update `publishEntry` params and `nextEntryArgs` response fields [#181](https://github.com/p2panda/aquadoggo/pull/181) ### Fixed diff --git a/aquadoggo/src/graphql/client/mutation.rs b/aquadoggo/src/graphql/client/mutation.rs index 4c1d91b61..ececb276e 100644 --- a/aquadoggo/src/graphql/client/mutation.rs +++ b/aquadoggo/src/graphql/client/mutation.rs @@ -21,10 +21,9 @@ impl ClientMutationRoot { async fn publish_entry( &self, ctx: &Context<'_>, - #[graphql(name = "entryEncoded", desc = "Encoded entry to publish")] - entry_encoded_param: String, + #[graphql(name = "entry", desc = "Encoded entry to publish")] entry_encoded_param: String, #[graphql( - name = "operationEncoded", + name = "operation", desc = "Encoded entry payload, which contains a p2panda operation matching the \ provided encoded entry." )] @@ -122,8 +121,8 @@ mod tests { 6d79206669727374206d65737361676521"; const PUBLISH_ENTRY_QUERY: &str = r#" - mutation TestPublishEntry($entryEncoded: String!, $operationEncoded: String!) { - publishEntry(entryEncoded: $entryEncoded, operationEncoded: $operationEncoded) { + mutation TestPublishEntry($entry: String!, $operation: String!) { + publishEntry(entry: $entry, operation: $operation) { logId, seqNum, backlink, @@ -144,8 +143,8 @@ mod tests { ) -> Request { // Prepare GraphQL mutation publishing an entry let parameters = Variables::from_value(value!({ - "entryEncoded": entry_encoded, - "operationEncoded": operation_encoded, + "entry": entry_encoded, + "operation": operation_encoded, })); Request::new(PUBLISH_ENTRY_QUERY).variables(parameters) @@ -210,8 +209,8 @@ mod tests { let context = HttpServiceContext::new(db.store, tx); let parameters = Variables::from_value(value!({ - "entryEncoded": ENTRY_ENCODED, - "operationEncoded": "".to_string() + "entry": ENTRY_ENCODED, + "operation": "".to_string() })); let request = Request::new(PUBLISH_ENTRY_QUERY).variables(parameters); let response = context.schema.execute(request).await; diff --git a/aquadoggo/src/graphql/client/response.rs b/aquadoggo/src/graphql/client/response.rs index 3481ca33f..3c6dbbd98 100644 --- a/aquadoggo/src/graphql/client/response.rs +++ b/aquadoggo/src/graphql/client/response.rs @@ -105,15 +105,10 @@ impl PublishEntryResponse { } impl AsPublishEntryResponse for PublishEntryResponse { - fn new( - entry_hash_backlink: Option, - entry_hash_skiplink: Option, - seq_num: SeqNum, - log_id: LogId, - ) -> Self { + fn new(backlink: Option, skiplink: Option, seq_num: SeqNum, log_id: LogId) -> Self { PublishEntryResponse { - backlink: entry_hash_backlink, - skiplink: entry_hash_skiplink, + backlink, + skiplink, seq_num, log_id, } diff --git a/aquadoggo/src/graphql/replication/client/schema.graphql b/aquadoggo/src/graphql/replication/client/schema.graphql index d9cc861ae..466641704 100644 --- a/aquadoggo/src/graphql/replication/client/schema.graphql +++ b/aquadoggo/src/graphql/replication/client/schema.graphql @@ -82,10 +82,10 @@ All of the graphql mutation sub modules merged into one top level root type MutationRoot { """ Publish an entry using parameters obtained through `nextEntryArgs` query. - + Returns arguments for publishing the next entry in the same log. """ - publishEntry(entryEncoded: String!, operationEncoded: String!): PublishEntryResponse! + publishEntry(entry: String!, operation: String!): PublishEntryResponse! } """ Information about pagination in a connection From 27652772e49eaf693240c49ce6f02568699a26b8 Mon Sep 17 00:00:00 2001 From: Sam Andreae Date: Fri, 1 Jul 2022 11:31:58 +0100 Subject: [PATCH 03/13] Remove unused import --- aquadoggo/src/materializer/service.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/aquadoggo/src/materializer/service.rs b/aquadoggo/src/materializer/service.rs index f2bb8eb2e..c638e6786 100644 --- a/aquadoggo/src/materializer/service.rs +++ b/aquadoggo/src/materializer/service.rs @@ -141,7 +141,7 @@ mod tests { use p2panda_rs::storage_provider::traits::OperationStore; use p2panda_rs::test_utils::constants::TEST_SCHEMA_ID; use p2panda_rs::test_utils::fixtures::{ - key_pair, operation, operation_fields, random_document_id, random_document_view_id, + key_pair, operation, operation_fields, random_document_id, random_operation_id, }; use rstest::rstest; From a0af183deba424810872e85abd75a50df0440673 Mon Sep 17 00:00:00 2001 From: Sam Andreae Date: Fri, 1 Jul 2022 11:41:22 +0100 Subject: [PATCH 04/13] p2panda-rs v0.4.0 --- aquadoggo/Cargo.toml | 8 ++------ aquadoggo/src/materializer/service.rs | 3 +-- 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/aquadoggo/Cargo.toml b/aquadoggo/Cargo.toml index f15029934..4490bd767 100644 --- a/aquadoggo/Cargo.toml +++ b/aquadoggo/Cargo.toml @@ -35,9 +35,7 @@ lru = "0.7.5" mockall = "0.11.0" mockall_double = "0.3.0" openssl-probe = "0.1.4" -# We can not publish the `aquadoggo` crate yet, since `p2panda-rs` is an -# unpublished dependency. -p2panda-rs = { git = "https://github.com/p2panda/p2panda", branch = "main" } +p2panda-rs = "0.4.0" rand = "0.8.4" serde = { version = "1.0.130", features = ["derive"] } serde_json = "1.0.67" @@ -74,6 +72,4 @@ rstest = "0.12.0" rstest_reuse = "0.1.3" tower-service = "0.3.1" tower = "0.4.12" -p2panda-rs = { git = "https://github.com/p2panda/p2panda", branch = "main", features = [ - "testing", -] } +p2panda-rs = { version = "0.4.0", features = ["testing"] } diff --git a/aquadoggo/src/materializer/service.rs b/aquadoggo/src/materializer/service.rs index c638e6786..d15d8b6a7 100644 --- a/aquadoggo/src/materializer/service.rs +++ b/aquadoggo/src/materializer/service.rs @@ -141,8 +141,7 @@ mod tests { use p2panda_rs::storage_provider::traits::OperationStore; use p2panda_rs::test_utils::constants::TEST_SCHEMA_ID; use p2panda_rs::test_utils::fixtures::{ - key_pair, operation, operation_fields, random_document_id, - random_operation_id, + key_pair, operation, operation_fields, random_document_id, random_operation_id, }; use rstest::rstest; use tokio::sync::broadcast; From 0daf8ab19a5d37e6624c6c10d587865526660203 Mon Sep 17 00:00:00 2001 From: Sam Andreae Date: Fri, 1 Jul 2022 11:42:15 +0100 Subject: [PATCH 05/13] Commit Cargo.lock --- Cargo.lock | 71 ++++++++++++++++++++++++++++++++++++++++++++++-------- 1 file changed, 61 insertions(+), 10 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index dcf6d930c..e5311cc59 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -126,8 +126,8 @@ dependencies = [ "p2panda-rs", "rand 0.8.5", "reqwest", - "rstest", - "rstest_reuse", + "rstest 0.12.0", + "rstest_reuse 0.1.3", "serde", "serde_json", "sqlformat", @@ -407,7 +407,7 @@ dependencies = [ "serde_derive", "snafu", "static_assertions 0.3.4", - "varu64", + "varu64 0.6.2", "yasmf-hash", ] @@ -1257,6 +1257,12 @@ version = "0.3.21" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "57c66a976bf5909d801bbef33416c41372779507e7a6b3a5e25e4749c58f776a" +[[package]] +name = "futures-timer" +version = "3.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e64b03909df88034c26dc1547e8970b91f98bdb65165d6a4e9110d94263dbb2c" + [[package]] name = "futures-util" version = "0.3.21" @@ -2087,8 +2093,9 @@ dependencies = [ [[package]] name = "p2panda-rs" -version = "0.3.0" -source = "git+https://github.com/p2panda/p2panda?branch=main#da6cb0d41047261e23d043a45f204ba37cbbfad2" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "10c037a99f38662073ac86f44a13f8be1def058cb10624d094205649fe6f3b43" dependencies = [ "arrayvec 0.5.2", "async-trait", @@ -2103,21 +2110,20 @@ dependencies = [ "lazy_static", "lipmaa-link 0.2.2", "log", - "mockall", "openmls", "openmls_memory_keystore", "openmls_rust_crypto", "openmls_traits", "rand 0.7.3", "regex", - "rstest", - "rstest_reuse", + "rstest 0.15.0", + "rstest_reuse 0.3.0", "serde", "serde-wasm-bindgen", "serde_repr", "thiserror", "tls_codec", - "varu64", + "varu64 0.7.0", "wasm-bindgen", "yasmf-hash", ] @@ -2610,6 +2616,31 @@ dependencies = [ "syn", ] +[[package]] +name = "rstest" +version = "0.15.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e9c9dc66cc29792b663ffb5269be669f1613664e69ad56441fdb895c2347b930" +dependencies = [ + "futures", + "futures-timer", + "rstest_macros", + "rustc_version 0.4.0", +] + +[[package]] +name = "rstest_macros" +version = "0.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5015e68a0685a95ade3eee617ff7101ab6a3fc689203101ca16ebc16f2b89c66" +dependencies = [ + "cfg-if", + "proc-macro2", + "quote", + "rustc_version 0.4.0", + "syn", +] + [[package]] name = "rstest_reuse" version = "0.1.3" @@ -2621,6 +2652,17 @@ dependencies = [ "syn", ] +[[package]] +name = "rstest_reuse" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b29d3117bce27ea307d1fb7ce12c64ba11b3fd04311a42d32bc5f0072e6e3d4d" +dependencies = [ + "quote", + "rustc_version 0.4.0", + "syn", +] + [[package]] name = "rustc-demangle" version = "0.1.21" @@ -3624,6 +3666,15 @@ version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f4042fddf572f474ead4fa3df751f80f630087f40cafeed73ec58535fea59bb4" +[[package]] +name = "varu64" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd0b4bcb210ab3a048eda9a938508b072e474af2838994e77976c817e51af1e3" +dependencies = [ + "snafu", +] + [[package]] name = "vcpkg" version = "0.2.15" @@ -3883,7 +3934,7 @@ dependencies = [ "serde_derive", "snafu", "static_assertions 0.3.4", - "varu64", + "varu64 0.6.2", ] [[package]] From d45a9ff138035e5e795bd93dacc277b0b7957a41 Mon Sep 17 00:00:00 2001 From: Sam Andreae Date: Fri, 1 Jul 2022 11:46:41 +0100 Subject: [PATCH 06/13] Prepare CHANGELOG for v0.3.0 release --- CHANGELOG.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8158fe0bc..0a3384d43 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,8 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 ## [Unreleased] +## [0.3.0] + ### Added - Introduce GraphQL endpoint [#81](https://github.com/p2panda/aquadoggo/pull/81) @@ -86,6 +88,7 @@ Released on 2021-10-25: :package: [`crate`](https://crates.io/crates/aquadoggo/0 - Use p2panda-rs 0.2.1 with fixed linter setting [#41](https://github.com/p2panda/aquadoggo/41) - Use `tide` for HTTP server and `jsonrpc-v2` for JSON RPC [#29](https://github.com/p2panda/aquadoggo/29) -[unreleased]: https://github.com/p2panda/aquadoggo/compare/v0.2.0...HEAD +[unreleased]: https://github.com/p2panda/aquadoggo/compare/v0.3.0...HEAD +[0.3.0]: https://github.com/p2panda/aquadoggo/releases/tag/v0.3.0 [0.2.0]: https://github.com/p2panda/aquadoggo/releases/tag/v0.2.0 [0.1.0]: https://github.com/p2panda/aquadoggo/releases/tag/v0.1.0 From b3d2dfaa61d4ebbb0c2a60f5a783adc627fef378 Mon Sep 17 00:00:00 2001 From: Sam Andreae Date: Fri, 1 Jul 2022 11:49:22 +0100 Subject: [PATCH 07/13] v0.3.0 --- aquadoggo/Cargo.toml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/aquadoggo/Cargo.toml b/aquadoggo/Cargo.toml index 4490bd767..b2a5cc313 100644 --- a/aquadoggo/Cargo.toml +++ b/aquadoggo/Cargo.toml @@ -1,11 +1,12 @@ [package] name = "aquadoggo" -version = "0.2.0" +version = "0.3.0" authors = [ "sophiiistika ", "adz ", "sandreae ", "cafca ", + "pietgeursen ", ] description = "Embeddable p2p network node" license = "AGPL-3.0-or-later" From c60dd2e7de9f8a3289aa4fa68058e87442731d9b Mon Sep 17 00:00:00 2001 From: Sam Andreae Date: Fri, 1 Jul 2022 11:58:44 +0100 Subject: [PATCH 08/13] Update aquadoggo to v0.3.0 in aquadoggo_cli --- Cargo.lock | 2 +- aquadoggo_cli/Cargo.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index e5311cc59..3b800a649 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -99,7 +99,7 @@ checksum = "08f9b8508dccb7687a1d6c4ce66b2b0ecef467c94667de27d8d7fe1f8d2a9cdc" [[package]] name = "aquadoggo" -version = "0.2.0" +version = "0.3.0" dependencies = [ "anyhow", "arrayvec 0.5.2", diff --git a/aquadoggo_cli/Cargo.toml b/aquadoggo_cli/Cargo.toml index 1fefee65d..c9c42ffb9 100644 --- a/aquadoggo_cli/Cargo.toml +++ b/aquadoggo_cli/Cargo.toml @@ -23,5 +23,5 @@ env_logger = "0.9.0" structopt = "0.3.23" [dependencies.aquadoggo] -version = "~0.2.0" +version = "~0.3.0" path = "../aquadoggo" From ec3ff3691c622077a416e2f473b51dee7be952b1 Mon Sep 17 00:00:00 2001 From: Piet Geursen Date: Sat, 2 Jul 2022 01:26:26 +1200 Subject: [PATCH 09/13] GraphQL replication client (#137) * Start adding replication client * Create Client to do the gql query and type conversions * Fix typo in version * Add replication request to service * Wire up config * Successful replication request + response * Update gql api so `get_entries_newer_than_seq` can get all entries * Query uses optional seq num * Store entry and broadcast it on bus * Refactor a little * Remembered we have to verify entries * Fix import order * Make clippy happy * Add replication args to aquadoggo cli * Add verification code, verification fails on seq 4 * Get whole cert pool for verfication * tidy * Clippy happy, pandas happy * Extract methods for clarity * Remove unused config folder * Add replication service PR to changelog * Fix typo * WIP: use publish_entry when replicating and insert operation * Add some comments to nasty code * WIP: Sketch an integration test for replication * Make own struct for test db config * Add log option to test config * Add log param to test_db fixture * Use test config for create_test_db params * Implement Default for TestDatabaseConfig * Seperate database creation and population logic * Rename config struct for populating db * Namespace test data on TestDatabase * Derive default for TestData * Test runner which passes in a DatabaseManager and closes all pools on teardown * Pass in mutable store to populate * Doc strings and comments * Populate the test db * Use valid public_key * Add env_logger to dev-dependencies * Do the replication! * Minor clean-ups, derive Debug for auto-generated GraphQL client structs * Refactor config construction * Tidy and comments * Don't use double macro for exporting mock replication context * Tidy imports * Tidy imports in test_utils * Revert to using double again * Refactor get_entries_newer_than_seq_cursor test to not use mocks * Clean up nasty publish entry code * Make clippy happy * Fixes after merge * Add case names to test * Update p2panda-rs * Minor formatting changes, add docstrings * Use seq_num instead of sequence Co-authored-by: Andreas Dzialocha Co-authored-by: Sam Andreae --- CHANGELOG.md | 1 + Cargo.lock | 343 +++++++++++----- aquadoggo/Cargo.toml | 5 +- aquadoggo/src/config.rs | 8 +- aquadoggo/src/db/stores/document.rs | 32 +- aquadoggo/src/db/stores/entry.rs | 32 +- aquadoggo/src/db/stores/operation.rs | 2 +- aquadoggo/src/db/stores/test_utils.rs | 284 +++++++++---- aquadoggo/src/graphql/client/mutation.rs | 2 +- aquadoggo/src/graphql/mod.rs | 3 +- aquadoggo/src/graphql/replication/author.rs | 11 +- .../src/graphql/replication/client/mod.rs | 117 +++++- .../get_entries_newer_than_seq.graphql | 14 + .../graphql/replication/client/schema.graphql | 4 +- aquadoggo/src/graphql/replication/context.rs | 28 +- .../graphql/replication/entry_and_payload.rs | 16 + aquadoggo/src/graphql/replication/mod.rs | 198 ++++----- .../graphql/replication/sequence_number.rs | 1 + aquadoggo/src/lib.rs | 5 +- aquadoggo/src/manager.rs | 2 +- aquadoggo/src/materializer/service.rs | 16 +- .../src/materializer/tasks/dependency.rs | 18 +- aquadoggo/src/materializer/tasks/reduce.rs | 52 +-- aquadoggo/src/node.rs | 6 +- aquadoggo/src/replication/config.rs | 59 +++ aquadoggo/src/replication/mod.rs | 7 + aquadoggo/src/replication/service.rs | 377 ++++++++++++++++++ aquadoggo/src/test_helpers.rs | 14 + aquadoggo_cli/Cargo.toml | 1 + aquadoggo_cli/src/main.rs | 64 ++- 30 files changed, 1335 insertions(+), 387 deletions(-) create mode 100644 aquadoggo/src/graphql/replication/client/queries/get_entries_newer_than_seq.graphql create mode 100644 aquadoggo/src/replication/config.rs create mode 100644 aquadoggo/src/replication/mod.rs create mode 100644 aquadoggo/src/replication/service.rs diff --git a/CHANGELOG.md b/CHANGELOG.md index 0a3384d43..3cbb57687 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -28,6 +28,7 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - Inform materialization service about new operations [#161](https://github.com/p2panda/aquadoggo/pull/161) - e2e publish entry tests [#167](https://github.com/p2panda/aquadoggo/pull/167) - Reschedule pending tasks on startup [#168](https://github.com/p2panda/aquadoggo/pull/168) +- GraphQL replication service gets and verifies new entries and inserts them into the db [#137](https://github.com/p2panda/aquadoggo/pull/137) - Debug logging in reduce task [#175](https://github.com/p2panda/aquadoggo/pull/175) ### Changed diff --git a/Cargo.lock b/Cargo.lock index 3b800a649..60a0dda64 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -93,9 +93,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.57" +version = "1.0.58" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08f9b8508dccb7687a1d6c4ce66b2b0ecef467c94667de27d8d7fe1f8d2a9cdc" +checksum = "bb07d2053ccdbe10e2af2995a2f116c1330396493dc1269f6a91d0ae82e19704" [[package]] name = "aquadoggo" @@ -110,6 +110,7 @@ dependencies = [ "bamboo-rs-core-ed25519-yasmf", "deadqueue", "directories", + "env_logger", "envy", "futures", "graphql_client", @@ -120,7 +121,6 @@ dependencies = [ "log", "lru", "mockall", - "mockall_double", "once_cell", "openssl-probe", "p2panda-rs", @@ -144,6 +144,7 @@ dependencies = [ name = "aquadoggo_cli" version = "0.1.0" dependencies = [ + "anyhow", "aquadoggo", "env_logger", "structopt", @@ -329,9 +330,9 @@ checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" [[package]] name = "axum" -version = "0.5.7" +version = "0.5.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc47084705629d09d15060d70a8dbfce479c842303d05929ce29c74c995916ae" +checksum = "db018aeacdb84550c2c6c5a1c1f553f2a330294222f45c1dbe24f0b3d2320c01" dependencies = [ "async-trait", "axum-core", @@ -364,9 +365,9 @@ dependencies = [ [[package]] name = "axum-core" -version = "0.2.5" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2efed1c501becea07ce48118786ebcf229531d0d3b28edf224a720020d9e106" +checksum = "cf4d047478b986f14a13edad31a009e2e05cb241f9805d0d75e4cba4e129ad4d" dependencies = [ "async-trait", "bytes", @@ -425,9 +426,9 @@ checksum = "904dfeac50f3cdaba28fc6f57fdcddb75f49ed61346676a78c4ffe55877802fd" [[package]] name = "base64ct" -version = "1.1.1" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6b4d9b1225d28d360ec6a231d65af1fd99a2a095154c8040689617290569c5c" +checksum = "3bdca834647821e0b13d9539a8634eb62d3501b6b6c2cec1722786ee6671b851" [[package]] name = "bitflags" @@ -667,6 +668,22 @@ version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "245097e9a4535ee1e3e3931fcfcd55a796a44c643e8596ff6566d68f09b87bbc" +[[package]] +name = "core-foundation" +version = "0.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "194a7a9e6de53fa55116934067c844d9d749312f75c6f6d0980e8c252f8c2146" +dependencies = [ + "core-foundation-sys", + "libc", +] + +[[package]] +name = "core-foundation-sys" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5827cebf4670468b8772dd191856768aedcb1b0278a04f989f7766351917b9dc" + [[package]] name = "cpufeatures" version = "0.2.2" @@ -693,9 +710,9 @@ checksum = "ccaeedb56da03b09f598226e25e80088cb4cd25f316e6e4df7d695f0feeb1403" [[package]] name = "crossbeam-channel" -version = "0.5.4" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5aaa7bd5fb665c6864b5f963dd9097905c54125909c7aa94c9e18507cdbe6c53" +checksum = "4c02a4d71819009c192cf4872265391563fd6a84c81ff2c0f2a7026ca4c1d85c" dependencies = [ "cfg-if", "crossbeam-utils", @@ -714,15 +731,15 @@ dependencies = [ [[package]] name = "crossbeam-epoch" -version = "0.9.8" +version = "0.9.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1145cf131a2c6ba0615079ab6a638f7e1973ac9c2634fcbeaaad6114246efe8c" +checksum = "07db9d94cbd326813772c968ccd25999e5f8ae22f4f8d1b11effa37ef6ce281d" dependencies = [ "autocfg", "cfg-if", "crossbeam-utils", - "lazy_static", "memoffset", + "once_cell", "scopeguard", ] @@ -738,12 +755,12 @@ dependencies = [ [[package]] name = "crossbeam-utils" -version = "0.8.8" +version = "0.8.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0bf124c720b7686e3c2663cf54062ab0f68a88af2fb6a030e87e30bf721fcb38" +checksum = "7d82ee10ce34d7bc12c2122495e7593a9c41347ecdd64185af4ecf72cb1a7f83" dependencies = [ "cfg-if", - "lazy_static", + "once_cell", ] [[package]] @@ -999,9 +1016,9 @@ dependencies = [ [[package]] name = "either" -version = "1.6.1" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e78d4f1cc4ae33bbfc157ed5d5a5ef3bc29227303d595861deb238fcec4e9457" +checksum = "3f107b87b6afc2a64fd13cac55fe06d6c8859f12d4b14cbcdd2c67d0976781be" [[package]] name = "elliptic-curve" @@ -1054,9 +1071,9 @@ dependencies = [ [[package]] name = "erased-serde" -version = "0.3.20" +version = "0.3.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad132dd8d0d0b546348d7d86cb3191aad14b34e5f979781fc005c80d4ac67ffd" +checksum = "81d013529d5574a60caeda29e179e695125448e5de52e3874f7b4c1d7360e18e" dependencies = [ "serde", ] @@ -1070,6 +1087,12 @@ dependencies = [ "rustversion", ] +[[package]] +name = "event-listener" +version = "2.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77f3309417938f28bf8228fcff79a4a37103981e3e186d2ccd19c74b38f4eb71" + [[package]] name = "failure" version = "0.1.8" @@ -1159,6 +1182,21 @@ version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" +[[package]] +name = "foreign-types" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f6f339eb8adc052cd2ca78910fda869aefa38d22d5cb648e6485e4d3fc06f3b1" +dependencies = [ + "foreign-types-shared", +] + +[[package]] +name = "foreign-types-shared" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" + [[package]] name = "form_urlencoded" version = "1.0.1" @@ -1338,9 +1376,9 @@ dependencies = [ [[package]] name = "ghost" -version = "0.1.4" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76c813ffb63e8fd3df6f1ac3cc1ea392c7612ac2de4d0b44dcbfe03e5c4bf94a" +checksum = "b93490550b1782c589a350f2211fff2e34682e25fed17ef53fc4fa8fe184975e" dependencies = [ "proc-macro2", "quote", @@ -1379,6 +1417,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a9b58571cfc3cc42c3e8ff44fc6cfbb6c0dea17ed22d20f9d8f1efc4e8209a3f" dependencies = [ "graphql_query_derive", + "reqwest", "serde", "serde_json", ] @@ -1391,7 +1430,7 @@ checksum = "b4bf9cd823359d74ad3d3ecf1afd4a975f4ff2f891cdf9a66744606daf52de8c" dependencies = [ "graphql-introspection-query", "graphql-parser", - "heck", + "heck 0.3.3", "lazy_static", "proc-macro2", "quote", @@ -1456,13 +1495,19 @@ dependencies = [ "ahash", ] +[[package]] +name = "hashbrown" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "db0d4cf898abf0081f964436dc980e96670a0f36863e4b83aaacdb65c9d7ccc3" + [[package]] name = "hashlink" version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7249a3129cbc1ffccd74857f81464a323a152173cdb134e0fd81bc803b29facf" dependencies = [ - "hashbrown", + "hashbrown 0.11.2", ] [[package]] @@ -1499,6 +1544,15 @@ dependencies = [ "unicode-segmentation", ] +[[package]] +name = "heck" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2540771e65fc8cb83cd6e8a237f70c319bd5c29f78ed1084ba5d50eeac86f7f9" +dependencies = [ + "unicode-segmentation", +] + [[package]] name = "hermit-abi" version = "0.1.19" @@ -1658,6 +1712,19 @@ dependencies = [ "want", ] +[[package]] +name = "hyper-tls" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6183ddfa99b85da61a140bea0efc93fdf56ceaa041b37d553518030827f9905" +dependencies = [ + "bytes", + "hyper", + "native-tls", + "tokio", + "tokio-native-tls", +] + [[package]] name = "ident_case" version = "1.0.1" @@ -1677,12 +1744,12 @@ dependencies = [ [[package]] name = "indexmap" -version = "1.8.2" +version = "1.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6012d540c5baa3589337a98ce73408de9b5a25ec9fc2c6fd6be8f0d39e0ca5a" +checksum = "10a35a97730320ffe8e2d410b5d3b69279b98d2c14bdb8b70ea89ecf7888d41e" dependencies = [ "autocfg", - "hashbrown", + "hashbrown 0.12.1", "serde", ] @@ -1755,9 +1822,9 @@ checksum = "349d5a591cd28b49e1d1037471617a32ddcda5731b99419008085f72d5a53836" [[package]] name = "libsqlite3-sys" -version = "0.23.2" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2cafc7c74096c336d9d27145f7ebd4f4b6f95ba16aa5a282387267e6925cb58" +checksum = "898745e570c7d0453cc1fbc4a701eb6c662ed54e8fec8b7d14be137ebeeb9d14" dependencies = [ "cc", "pkg-config", @@ -1801,7 +1868,7 @@ version = "0.7.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c84e6fe5655adc6ce00787cf7dcaf8dc4f998a0565d23eafc207a8b08ca3349a" dependencies = [ - "hashbrown", + "hashbrown 0.11.2", ] [[package]] @@ -1824,13 +1891,11 @@ checksum = "73cbba799671b762df5a175adf59ce145165747bb891505c43d09aefbbf38beb" [[package]] name = "md-5" -version = "0.9.1" +version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b5a279bb9607f9f53c22d496eade00d138d1bdcccd07d74650387cf94942a15" +checksum = "658646b21e0b72f7866c7038ab086d3d5e1cd6271f060fd37defb241949d0582" dependencies = [ - "block-buffer 0.9.0", - "digest 0.9.0", - "opaque-debug 0.3.0", + "digest 0.10.3", ] [[package]] @@ -1883,9 +1948,9 @@ dependencies = [ [[package]] name = "mio" -version = "0.8.3" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "713d550d9b44d89174e066b7a6217ae06234c10cb47819a88290d2b353c31799" +checksum = "57ee1c23c7c63b0c9250c339ffdc69255f110b298b901b9f6c82547b7b87caaf" dependencies = [ "libc", "log", @@ -1920,18 +1985,6 @@ dependencies = [ "syn", ] -[[package]] -name = "mockall_double" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae71c7bb287375187c775cf82e2dcf1bef3388aaf58f0789a77f9c7ab28466f6" -dependencies = [ - "cfg-if", - "proc-macro2", - "quote", - "syn", -] - [[package]] name = "multer" version = "2.0.2" @@ -1950,6 +2003,24 @@ dependencies = [ "version_check", ] +[[package]] +name = "native-tls" +version = "0.2.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fd7e2f3618557f980e0b17e8856252eee3c97fa12c54dff0ca290fb6266ca4a9" +dependencies = [ + "lazy_static", + "libc", + "log", + "openssl", + "openssl-probe", + "openssl-sys", + "schannel", + "security-framework", + "security-framework-sys", + "tempfile", +] + [[package]] name = "nom" version = "7.1.1" @@ -2073,12 +2144,51 @@ dependencies = [ "tls_codec", ] +[[package]] +name = "openssl" +version = "0.10.40" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fb81a6430ac911acb25fe5ac8f1d2af1b4ea8a4fdfda0f1ee4292af2e2d8eb0e" +dependencies = [ + "bitflags", + "cfg-if", + "foreign-types", + "libc", + "once_cell", + "openssl-macros", + "openssl-sys", +] + +[[package]] +name = "openssl-macros" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b501e44f11665960c7e7fcf062c7d96a14ade4aa98116c004b2e37b5be7d736c" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + [[package]] name = "openssl-probe" version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" +[[package]] +name = "openssl-sys" +version = "0.9.74" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "835363342df5fba8354c5b453325b110ffd54044e588c539cf2f20a8014e4cb1" +dependencies = [ + "autocfg", + "cc", + "libc", + "pkg-config", + "vcpkg", +] + [[package]] name = "p256" version = "0.10.1" @@ -2385,18 +2495,18 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.39" +version = "1.0.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c54b25569025b7fc9651de43004ae593a75ad88543b17178aa5e1b9c4f15f56f" +checksum = "dd96a1e8ed2596c337f8eae5f24924ec83f5ad5ab21ea8e455d3566c69fbcaf7" dependencies = [ "unicode-ident", ] [[package]] name = "quote" -version = "1.0.18" +version = "1.0.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1feb54ed693b93a84e14094943b84b7c4eae204c512b7ccb95ab0c66d278ad1" +checksum = "3bcdf212e9776fbcb2d23ab029360416bb1706b1aea2d1a5ba002727cbcab804" dependencies = [ "proc-macro2", ] @@ -2557,17 +2667,20 @@ dependencies = [ "http", "http-body", "hyper", + "hyper-tls", "ipnet", "js-sys", "lazy_static", "log", "mime", + "native-tls", "percent-encoding", "pin-project-lite", "serde", "serde_json", "serde_urlencoded", "tokio", + "tokio-native-tls", "tokio-util 0.7.3", "tower-service", "url", @@ -2684,7 +2797,7 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" dependencies = [ - "semver 1.0.10", + "semver 1.0.12", ] [[package]] @@ -2702,9 +2815,9 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.6" +version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2cc38e8fa666e2de3c4aba7edeb5ffc5246c1c2ed0e3d17e560aeeba736b23f" +checksum = "a0a5f7c728f5d284929a1cccb5bc19884422bfe6ef4d6c409da2c41838983fcf" [[package]] name = "ryu" @@ -2712,6 +2825,16 @@ version = "1.0.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f3f6f92acf49d1b98f7a81226834412ada05458b7364277387724a237f062695" +[[package]] +name = "schannel" +version = "0.1.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "88d6731146462ea25d9244b2ed5fd1d716d25c52e4d54aa4fb0f3c4e9854dbe2" +dependencies = [ + "lazy_static", + "windows-sys", +] + [[package]] name = "scopeguard" version = "1.1.0" @@ -2741,6 +2864,29 @@ dependencies = [ "zeroize", ] +[[package]] +name = "security-framework" +version = "2.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2dc14f172faf8a0194a3aded622712b0de276821addc574fa54fc0a1167e10dc" +dependencies = [ + "bitflags", + "core-foundation", + "core-foundation-sys", + "libc", + "security-framework-sys", +] + +[[package]] +name = "security-framework-sys" +version = "2.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0160a13a177a45bfb43ce71c01580998474f556ad854dcbca936dd2841a5c556" +dependencies = [ + "core-foundation-sys", + "libc", +] + [[package]] name = "semver" version = "0.11.0" @@ -2752,9 +2898,9 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.10" +version = "1.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a41d061efea015927ac527063765e73601444cdc344ba855bc7bd44578b25e1c" +checksum = "a2333e6df6d6598f2b1974829f853c2b4c5f4a6e503c10af918081aa6f8564e1" [[package]] name = "semver-parser" @@ -2817,9 +2963,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.81" +version = "1.0.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b7ce2b32a1aed03c558dc61a5cd328f15aff2dbc17daad8fb8af04d2100e15c" +checksum = "82c2c1fdcd807d1098552c5b9a36e425e42e9fbd7c6a37a8425f390f781f7fa7" dependencies = [ "itoa", "ryu", @@ -2861,19 +3007,6 @@ dependencies = [ "opaque-debug 0.2.3", ] -[[package]] -name = "sha-1" -version = "0.9.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99cd6713db3cf16b6c84e06321e049a9b9f699826e16096d23bbcc44d15d51a6" -dependencies = [ - "block-buffer 0.9.0", - "cfg-if", - "cpufeatures", - "digest 0.9.0", - "opaque-debug 0.3.0", -] - [[package]] name = "sha-1" version = "0.10.0" @@ -2936,9 +3069,9 @@ checksum = "eb703cfe953bccee95685111adeedb76fabe4e97549a58d16f03ea7b9367bb32" [[package]] name = "smallvec" -version = "1.8.0" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2dd574626839106c320a323308629dcb1acfc96e32a8cba364ddc61ac23ee83" +checksum = "2fd0db749597d91ff862fd1d55ea87f7855a744a8425a64695b6fca237d1dad1" [[package]] name = "snafu" @@ -3009,9 +3142,9 @@ dependencies = [ [[package]] name = "sqlx" -version = "0.5.11" +version = "0.5.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc15591eb44ffb5816a4a70a7efd5dd87bfd3aa84c4c200401c4396140525826" +checksum = "551873805652ba0d912fec5bbb0f8b4cdd96baf8e2ebf5970e5671092966019b" dependencies = [ "sqlx-core", "sqlx-macros", @@ -3019,9 +3152,9 @@ dependencies = [ [[package]] name = "sqlx-core" -version = "0.5.11" +version = "0.5.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "195183bf6ff8328bb82c0511a83faf60aacf75840103388851db61d7a9854ae3" +checksum = "e48c61941ccf5ddcada342cd59e3e5173b007c509e1e8e990dafc830294d9dc5" dependencies = [ "ahash", "atoi", @@ -3033,6 +3166,7 @@ dependencies = [ "crossbeam-queue", "dirs", "either", + "event-listener", "flume", "futures-channel", "futures-core", @@ -3041,7 +3175,8 @@ dependencies = [ "futures-util", "hashlink", "hex", - "hmac 0.11.0", + "hkdf", + "hmac 0.12.1", "indexmap", "itoa", "libc", @@ -3056,8 +3191,8 @@ dependencies = [ "rustls", "serde", "serde_json", - "sha-1 0.9.8", - "sha2 0.9.9", + "sha-1 0.10.0", + "sha2 0.10.2", "smallvec", "sqlformat", "sqlx-rt", @@ -3072,17 +3207,17 @@ dependencies = [ [[package]] name = "sqlx-macros" -version = "0.5.11" +version = "0.5.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eee35713129561f5e55c554bba1c378e2a7e67f81257b7311183de98c50e6f94" +checksum = "bc0fba2b0cae21fc00fe6046f8baa4c7fcb49e379f0f592b04696607f69ed2e1" dependencies = [ "dotenv", "either", - "heck", + "heck 0.4.0", "once_cell", "proc-macro2", "quote", - "sha2 0.9.9", + "sha2 0.10.2", "sqlx-core", "sqlx-rt", "syn", @@ -3151,7 +3286,7 @@ version = "0.4.18" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dcb5ae327f9cc13b68763b5749770cb9e048a99bd9dfdfa58d0cf05d5f64afe0" dependencies = [ - "heck", + "heck 0.3.3", "proc-macro-error", "proc-macro2", "quote", @@ -3164,7 +3299,7 @@ version = "0.23.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5bb0dc7ee9c15cea6199cde9a127fa16a4c5819af85395457ad72d68edc85a38" dependencies = [ - "heck", + "heck 0.3.3", "proc-macro2", "quote", "rustversion", @@ -3185,9 +3320,9 @@ checksum = "734676eb262c623cec13c3155096e08d1f8f29adce39ba17948b18dad1e54142" [[package]] name = "syn" -version = "1.0.96" +version = "1.0.98" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0748dd251e24453cb8717f0354206b91557e4ec8703673a4b30208f2abaf1ebf" +checksum = "c50aef8a904de4c23c788f104b7dddc7d6f79c647c7c8ce4cc8f73eb0ca773dd" dependencies = [ "proc-macro2", "quote", @@ -3338,6 +3473,16 @@ dependencies = [ "syn", ] +[[package]] +name = "tokio-native-tls" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f7d995660bd2b7f8c1568414c1126076c13fbb725c40112dc0120b78eb9b717b" +dependencies = [ + "native-tls", + "tokio", +] + [[package]] name = "tokio-rustls" version = "0.22.0" @@ -3412,9 +3557,9 @@ dependencies = [ [[package]] name = "tower" -version = "0.4.12" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a89fd63ad6adf737582df5db40d286574513c69a11dac5214dc3b5603d6713e" +checksum = "b8fa9be0de6cf49e536ce1851f987bd21a43b771b09473c3549a6c853db37c1c" dependencies = [ "futures-core", "futures-util", @@ -3471,9 +3616,9 @@ checksum = "343bc9466d3fe6b0f960ef45960509f84480bf4fd96f92901afe7ff3df9d3a62" [[package]] name = "tower-service" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "360dfd1d6d30e05fda32ace2c8c70e9c0a9da713275777f5a4dbb8a1893930c6" +checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" [[package]] name = "tracing" @@ -3489,9 +3634,9 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.27" +version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7709595b8878a4965ce5e87ebf880a7d39c9afc6837721b21a5a816a8117d921" +checksum = "7b7358be39f2f274f322d2aaed611acc57f382e8eb1e5b48cb9ae30933495ce7" dependencies = [ "once_cell", ] @@ -3577,9 +3722,9 @@ checksum = "5bd2fe26506023ed7b5e1e315add59d6f584c621d037f9368fea9cfb988f368c" [[package]] name = "unicode-normalization" -version = "0.1.19" +version = "0.1.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d54590932941a9e9266f0832deed84ebe1bf2e4c9e4a3554d393d18f5e854bf9" +checksum = "81dee68f85cab8cf68dec42158baf3a79a1cdc065a8b103025965d6ccb7f6cbd" dependencies = [ "tinyvec", ] diff --git a/aquadoggo/Cargo.toml b/aquadoggo/Cargo.toml index b2a5cc313..b478dc0f6 100644 --- a/aquadoggo/Cargo.toml +++ b/aquadoggo/Cargo.toml @@ -28,16 +28,16 @@ deadqueue = { version = "0.2.2", default-features = false, features = [ directories = "3.0.2" envy = "0.4.2" futures = "0.3.21" -graphql_client = "0.10" +graphql_client = { version = "0.10", features = ["reqwest"] } hex = "0.4.3" lipmaa-link = "0.2.2" log = "0.4.14" lru = "0.7.5" mockall = "0.11.0" -mockall_double = "0.3.0" openssl-probe = "0.1.4" p2panda-rs = "0.4.0" rand = "0.8.4" +reqwest = { version = "0.11.9", default-features = false, features = ["json"] } serde = { version = "1.0.130", features = ["derive"] } serde_json = "1.0.67" sqlformat = "0.1.7" @@ -62,6 +62,7 @@ tower-http = { version = "0.2.4", default-features = false, features = [ triggered = "0.1.2" [dev-dependencies] +env_logger = "0.9.0" hyper = "0.14.17" http = "0.2.6" once_cell = "1.12.0" diff --git a/aquadoggo/src/config.rs b/aquadoggo/src/config.rs index 810ba3081..9eb0033a7 100644 --- a/aquadoggo/src/config.rs +++ b/aquadoggo/src/config.rs @@ -7,6 +7,8 @@ use anyhow::Result; use directories::ProjectDirs; use serde::Deserialize; +use crate::replication::ReplicationConfiguration; + /// Data directory name. const DATA_DIR_NAME: &str = "aquadoggo"; @@ -21,7 +23,7 @@ const DEFAULT_SQLITE_NAME: &str = "aquadoggo-node.sqlite3"; /// When no custom directory path is set it reads the process environment $XDG_DATA_HOME variable /// to determine the XDG data directory path which is $HOME/.local/share/aquadoggo on Linux by /// default. -#[derive(Deserialize, Debug)] +#[derive(Deserialize, Debug, Clone)] #[serde(default)] pub struct Configuration { /// Path to data directory. @@ -41,6 +43,9 @@ pub struct Configuration { /// Materializer worker pool size. pub worker_pool_size: u32, + + /// Replication configuration. + pub replication: ReplicationConfiguration, } impl Default for Configuration { @@ -52,6 +57,7 @@ impl Default for Configuration { http_port: 2020, ws_port: 2022, worker_pool_size: 16, + replication: ReplicationConfiguration::default(), } } } diff --git a/aquadoggo/src/db/stores/document.rs b/aquadoggo/src/db/stores/document.rs index 238b588d8..532488757 100644 --- a/aquadoggo/src/db/stores/document.rs +++ b/aquadoggo/src/db/stores/document.rs @@ -372,11 +372,12 @@ mod tests { #[rstest] fn inserts_gets_one_document_view( #[from(test_db)] - #[with(1, 1)] + #[with(1, 1, 1)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { - let author = Author::try_from(db.key_pairs[0].public_key().to_owned()).unwrap(); + let author = + Author::try_from(db.test_data.key_pairs[0].public_key().to_owned()).unwrap(); // Get one entry from the pre-polulated db let entry = db @@ -433,7 +434,7 @@ mod tests { fn document_view_does_not_exist( random_document_view_id: DocumentViewId, #[from(test_db)] - #[with(1, 1)] + #[with(1, 1, 1)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { @@ -450,11 +451,12 @@ mod tests { #[rstest] fn inserts_gets_many_document_views( #[from(test_db)] - #[with(10, 1, false, TEST_SCHEMA_ID.parse().unwrap(), vec![("username", OperationValue::Text("panda".into()))], vec![("username", OperationValue::Text("PANDA".into()))])] + #[with(10, 1, 1, false, TEST_SCHEMA_ID.parse().unwrap(), vec![("username", OperationValue::Text("panda".into()))], vec![("username", OperationValue::Text("PANDA".into()))])] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { - let author = Author::try_from(db.key_pairs[0].public_key().to_owned()).unwrap(); + let author = + Author::try_from(db.test_data.key_pairs[0].public_key().to_owned()).unwrap(); let schema_id = SchemaId::from_str(TEST_SCHEMA_ID).unwrap(); let log_id = LogId::default(); @@ -531,11 +533,11 @@ mod tests { #[rstest] fn inserts_gets_documents( #[from(test_db)] - #[with(1, 1)] + #[with(1, 1, 1)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { - let document_id = db.documents[0].clone(); + let document_id = db.test_data.documents[0].clone(); let document_operations = db .store @@ -578,11 +580,11 @@ mod tests { #[rstest] fn gets_document_by_id( #[from(test_db)] - #[with(1, 1)] + #[with(1, 1, 1)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { - let document_id = db.documents[0].clone(); + let document_id = db.test_data.documents[0].clone(); let document_operations = db .store @@ -625,11 +627,11 @@ mod tests { #[rstest] fn no_view_when_document_deleted( #[from(test_db)] - #[with(10, 1, true)] + #[with(10, 1, 1, true)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { - let document_id = db.documents[0].clone(); + let document_id = db.test_data.documents[0].clone(); let document_operations = db .store @@ -652,11 +654,11 @@ mod tests { #[rstest] fn updates_a_document( #[from(test_db)] - #[with(10, 1)] + #[with(10, 1, 1)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { - let document_id = db.documents[0].clone(); + let document_id = db.test_data.documents[0].clone(); let document_operations = db .store @@ -687,13 +689,13 @@ mod tests { #[rstest] fn gets_documents_by_schema( #[from(test_db)] - #[with(10, 2, false, TEST_SCHEMA_ID.parse().unwrap())] + #[with(10, 2, 1, false, TEST_SCHEMA_ID.parse().unwrap())] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { let schema_id = SchemaId::from_str(TEST_SCHEMA_ID).unwrap(); - for document_id in &db.documents { + for document_id in &db.test_data.documents { let document_operations = db .store .get_operations_by_document_id(document_id) diff --git a/aquadoggo/src/db/stores/entry.rs b/aquadoggo/src/db/stores/entry.rs index 80f5d2a3d..9dfed7f8e 100644 --- a/aquadoggo/src/db/stores/entry.rs +++ b/aquadoggo/src/db/stores/entry.rs @@ -448,11 +448,12 @@ mod tests { #[rstest] fn try_insert_non_unique_entry( #[from(test_db)] - #[with(10, 1)] + #[with(10, 1, 1)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { - let author = Author::try_from(db.key_pairs[0].public_key().to_owned()).unwrap(); + let author = + Author::try_from(db.test_data.key_pairs[0].public_key().to_owned()).unwrap(); let log_id = LogId::new(1); let first_entry = db @@ -476,7 +477,7 @@ mod tests { #[rstest] fn latest_entry( #[from(test_db)] - #[with(20, 1)] + #[with(20, 1, 1)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { @@ -490,7 +491,8 @@ mod tests { .unwrap(); assert!(latest_entry.is_none()); - let author_in_db = Author::try_from(db.key_pairs[0].public_key().to_owned()).unwrap(); + let author_in_db = + Author::try_from(db.test_data.key_pairs[0].public_key().to_owned()).unwrap(); let latest_entry = db .store @@ -504,7 +506,7 @@ mod tests { #[rstest] fn entries_by_schema( #[from(test_db)] - #[with(20, 2, false, TEST_SCHEMA_ID.parse().unwrap())] + #[with(20, 1, 2, false, TEST_SCHEMA_ID.parse().unwrap())] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { @@ -534,11 +536,12 @@ mod tests { #[rstest] fn entry_by_seq_number( #[from(test_db)] - #[with(10, 1)] + #[with(10, 1, 1)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { - let author = Author::try_from(db.key_pairs[0].public_key().to_owned()).unwrap(); + let author = + Author::try_from(db.test_data.key_pairs[0].public_key().to_owned()).unwrap(); for seq_num in 1..10 { let seq_num = SeqNum::new(seq_num).unwrap(); @@ -579,11 +582,12 @@ mod tests { #[rstest] fn get_entry_by_hash( #[from(test_db)] - #[with(20, 1)] + #[with(20, 1, 1)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { - let author = Author::try_from(db.key_pairs[0].public_key().to_owned()).unwrap(); + let author = + Author::try_from(db.test_data.key_pairs[0].public_key().to_owned()).unwrap(); for seq_num in [1, 11, 18] { let seq_num = SeqNum::new(seq_num).unwrap(); @@ -617,11 +621,12 @@ mod tests { #[rstest] fn paginated_log_entries( #[from(test_db)] - #[with(30, 1)] + #[with(30, 1, 1)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { - let author = Author::try_from(db.key_pairs[0].public_key().to_owned()).unwrap(); + let author = + Author::try_from(db.test_data.key_pairs[0].public_key().to_owned()).unwrap(); let entries = db .store @@ -653,11 +658,12 @@ mod tests { #[rstest] fn get_lipmaa_link_entries( #[from(test_db)] - #[with(100, 1)] + #[with(100, 1, 1)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { - let author = Author::try_from(db.key_pairs[0].public_key().to_owned()).unwrap(); + let author = + Author::try_from(db.test_data.key_pairs[0].public_key().to_owned()).unwrap(); let entries = db .store diff --git a/aquadoggo/src/db/stores/operation.rs b/aquadoggo/src/db/stores/operation.rs index cac2b5d96..4670a874f 100644 --- a/aquadoggo/src/db/stores/operation.rs +++ b/aquadoggo/src/db/stores/operation.rs @@ -408,7 +408,7 @@ mod tests { fn get_operations_by_document_id( key_pair: KeyPair, #[from(test_db)] - #[with(5, 1)] + #[with(5, 1, 1)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { diff --git a/aquadoggo/src/db/stores/test_utils.rs b/aquadoggo/src/db/stores/test_utils.rs index 5b2514bc2..58fd98a50 100644 --- a/aquadoggo/src/db/stores/test_utils.rs +++ b/aquadoggo/src/db/stores/test_utils.rs @@ -1,6 +1,7 @@ // SPDX-License-Identifier: AGPL-3.0-or-later use std::convert::TryFrom; +use std::sync::Arc; use futures::Future; use p2panda_rs::document::{DocumentBuilder, DocumentId, DocumentViewId}; @@ -19,6 +20,7 @@ use rstest::fixture; use sqlx::migrate::MigrateDatabase; use sqlx::Any; use tokio::runtime::Builder; +use tokio::sync::Mutex; use crate::db::provider::SqlStorage; use crate::db::traits::DocumentStore; @@ -186,24 +188,65 @@ where } } -pub struct TestDatabaseRunner { +#[async_trait::async_trait] +pub trait AsyncTestFnWithManager { + async fn call(self, db: TestDatabaseManager); +} + +#[async_trait::async_trait] +impl AsyncTestFnWithManager for FN +where + FN: FnOnce(TestDatabaseManager) -> F + Sync + Send, + F: Future + Send, +{ + async fn call(self, db: TestDatabaseManager) { + self(db).await + } +} + +pub struct PopulateDatabaseConfig { /// Number of entries per log/document. - no_of_entries: usize, + pub no_of_entries: usize, - /// Number of authors, each with a log populated as defined above. - no_of_authors: usize, + /// Number of logs for each author. + pub no_of_logs: usize, + + /// Number of authors, each with logs populated as defined above. + pub no_of_authors: usize, /// A boolean flag for wether all logs should contain a delete operation. - with_delete: bool, + pub with_delete: bool, /// The schema used for all operations in the db. - schema: SchemaId, + pub schema: SchemaId, /// The fields used for every CREATE operation. - create_operation_fields: Vec<(&'static str, OperationValue)>, + pub create_operation_fields: Vec<(&'static str, OperationValue)>, /// The fields used for every UPDATE operation. - update_operation_fields: Vec<(&'static str, OperationValue)>, + pub update_operation_fields: Vec<(&'static str, OperationValue)>, +} + +impl Default for PopulateDatabaseConfig { + fn default() -> Self { + Self { + no_of_entries: 0, + no_of_logs: 0, + no_of_authors: 0, + with_delete: false, + schema: TEST_SCHEMA_ID.parse().unwrap(), + create_operation_fields: doggo_test_fields(), + update_operation_fields: doggo_test_fields(), + } + } +} + +// @TODO: I'm keeping this here for now as otherwise we would need to refactor _all_ the tests using it. +// +// We may still want to keep this "single database" runner injected through `rstest` but in any case +// probably best to consider that in a different PR. +pub struct TestDatabaseRunner { + config: PopulateDatabaseConfig, } impl TestDatabaseRunner { @@ -222,15 +265,14 @@ impl TestDatabaseRunner { runtime.block_on(async { // Initialise test database - let db = create_test_db( - self.no_of_entries, - self.no_of_authors, - self.with_delete, - self.schema.clone(), - self.create_operation_fields.clone(), - self.update_operation_fields.clone(), - ) - .await; + let pool = initialize_db().await; + let mut db = TestDatabase { + store: SqlStorage::new(pool), + test_data: TestData::default(), + }; + + // Populate the test db + populate_test_db(&mut db, &self.config).await; // Get a handle of the underlying database connection pool let pool = db.store.pool.clone(); @@ -256,6 +298,48 @@ impl TestDatabaseRunner { } } +/// Method which provides a safe way to write tests with the ability to build many databases and +/// have their pool connections closed automatically when the test succeeds or fails. +/// +/// Takes an (async) test function as an argument and passes over the `TestDatabaseManager` +/// instance which can be used to build databases from inside the tests. +pub fn with_db_manager_teardown(test: F) { + let runtime = Builder::new_current_thread() + .worker_threads(1) + .enable_all() + .thread_name("with_db_teardown") + .build() + .expect("Could not build tokio Runtime for test"); + + // Instantiate the database manager + let db_manager = TestDatabaseManager::new(); + + // Get a handle onto it's collection of pools + let pools = db_manager.pools.clone(); + + runtime.block_on(async { + // Spawn the test in a separate task to make sure we have control over the possible + // panics which might happen inside of it + let handle = tokio::task::spawn(async move { + // Execute the actual test + test.call(db_manager).await; + }); + + // Get a handle of the task so we can use it later + let result = handle.await; + + // Unwind the test by closing down the connections to all the database pools. This + // will be reached even when the test panicked + for pool in pools.lock().await.iter() { + pool.close().await; + } + + // Panic here when test failed. The test fails within its own async task and stays + // there, we need to propagate it further to inform the test runtime about the result + result.unwrap(); + }); +} + /// Fixture for constructing a storage provider instance backed by a pre-populated database. /// /// Returns a `TestDatabaseRunner` which allows to bootstrap a safe async test environment @@ -269,7 +353,9 @@ impl TestDatabaseRunner { pub fn test_db( // Number of entries per log/document #[default(0)] no_of_entries: usize, - // Number of authors, each with a log populated as defined above + // Number of logs for each author + #[default(0)] no_of_logs: usize, + // Number of authors, each with logs populated as defined above #[default(0)] no_of_authors: usize, // A boolean flag for wether all logs should contain a delete operation #[default(false)] with_delete: bool, @@ -280,101 +366,89 @@ pub fn test_db( // The fields used for every UPDATE operation #[default(doggo_test_fields())] update_operation_fields: Vec<(&'static str, OperationValue)>, ) -> TestDatabaseRunner { - TestDatabaseRunner { + let config = PopulateDatabaseConfig { no_of_entries, + no_of_logs, no_of_authors, with_delete, schema, create_operation_fields, update_operation_fields, - } + }; + + TestDatabaseRunner { config } } /// Container for `SqlStore` with access to the document ids and key_pairs used in the /// pre-populated database for testing. pub struct TestDatabase { pub store: SqlStorage, + pub test_data: TestData, +} + +/// Data collected when populating a `TestData` base in order to easily check values which +/// would be otherwise hard or impossible to get through the store methods. +#[derive(Default)] +pub struct TestData { pub key_pairs: Vec, pub documents: Vec, } -/// Helper method for constructing a storage provider instance backed by a pre-populated database. +/// Helper method for populating a `TestDatabase` with configurable data. /// /// Passed parameters define what the db should contain. The first entry in each log contains a /// valid CREATE operation following entries contain duplicate UPDATE operations. If the /// with_delete flag is set to true the last entry in all logs contain be a DELETE operation. -/// -/// Returns a `TestDatabase` containing storage provider instance, a vector of key pairs for all -/// authors in the db, and a vector of the ids for all documents. -async fn create_test_db( - no_of_entries: usize, - no_of_authors: usize, - with_delete: bool, - schema_id: SchemaId, - create_operation_fields: Vec<(&'static str, OperationValue)>, - update_operation_fields: Vec<(&'static str, OperationValue)>, -) -> TestDatabase { - let mut documents: Vec = Vec::new(); - let key_pairs = test_key_pairs(no_of_authors); - - let pool = initialize_db().await; - let store = SqlStorage::new(pool); - - // If we don't want any entries in the db return now - if no_of_entries == 0 { - return TestDatabase { - store, - key_pairs, - documents, - }; - } +pub async fn populate_test_db(db: &mut TestDatabase, config: &PopulateDatabaseConfig) { + let key_pairs = test_key_pairs(config.no_of_authors); for key_pair in &key_pairs { - let mut document_id: Option = None; - let mut previous_operation: Option = None; - for index in 0..no_of_entries { - // Create an operation based on the current index and whether this document should contain - // a DELETE operation. - let next_operation_fields = match index { - // First operation is a CREATE. - 0 => Some(operation_fields(create_operation_fields.clone())), - // Last operation is a DELETE if the with_delete flag is set. - seq if seq == (no_of_entries - 1) && with_delete => None, - // All other operations are UPDATE. - _ => Some(operation_fields(update_operation_fields.clone())), - }; - - // Publish the operation encoded on an entry to storage. - let (entry_encoded, publish_entry_response) = send_to_store( - &store, - &operation( - next_operation_fields, - previous_operation, - Some(schema_id.to_owned()), - ), - document_id.as_ref(), - key_pair, - ) - .await; - - // Set the previous_operations based on the backlink. - previous_operation = publish_entry_response.backlink.map(|hash| hash.into()); - - // If this was the first entry in the document, store the doucment id for later. - if index == 0 { - document_id = Some(entry_encoded.hash().into()); - documents.push(document_id.clone().unwrap()); + db.test_data + .key_pairs + .push(KeyPair::from_private_key(key_pair.private_key()).unwrap()); + + for _log_id in 0..config.no_of_logs { + let mut document_id: Option = None; + let mut previous_operation: Option = None; + for index in 0..config.no_of_entries { + // Create an operation based on the current index and whether this document should contain + // a DELETE operation. + let next_operation_fields = match index { + // First operation is a CREATE. + 0 => Some(operation_fields(config.create_operation_fields.clone())), + // Last operation is a DELETE if the with_delete flag is set. + seq if seq == (config.no_of_entries - 1) && config.with_delete => None, + // All other operations are UPDATE. + _ => Some(operation_fields(config.update_operation_fields.clone())), + }; + + // Publish the operation encoded on an entry to storage. + let (entry_encoded, publish_entry_response) = send_to_store( + &db.store, + &operation( + next_operation_fields, + previous_operation, + Some(config.schema.to_owned()), + ), + document_id.as_ref(), + key_pair, + ) + .await; + + // Set the previous_operations based on the backlink. + previous_operation = publish_entry_response.backlink.map(|hash| hash.into()); + + // If this was the first entry in the document, store the doucment id for later. + if index == 0 { + document_id = Some(entry_encoded.hash().into()); + db.test_data.documents.push(document_id.clone().unwrap()); + } } } } - - TestDatabase { - store, - key_pairs, - documents, - } } +/// Helper method for publishing an operation encoded on an entry to a store. pub async fn send_to_store( store: &SqlStorage, operation: &Operation, @@ -449,6 +523,21 @@ async fn initialize_db() -> Pool { pool } +/// Create test database. +async fn initialize_db_with_url(url: &str) -> Pool { + // Reset database first + drop_database().await; + create_database(url).await.unwrap(); + + // Create connection pool and run all migrations + let pool = connection_pool(url, 25).await.unwrap(); + if run_pending_migrations(&pool).await.is_err() { + pool.close().await; + } + + pool +} + // Delete test database async fn drop_database() { if Any::database_exists(&TEST_CONFIG.database_url) @@ -458,3 +547,26 @@ async fn drop_database() { Any::drop_database(&TEST_CONFIG.database_url).await.unwrap(); } } + +/// A manager which can create many databases and retain a handle on their connection pools. +#[derive(Default)] +pub struct TestDatabaseManager { + pools: Arc>>, +} + +impl TestDatabaseManager { + pub fn new() -> Self { + Self::default() + } + + pub async fn create(&self, url: &str) -> TestDatabase { + // Initialise test database + let pool = initialize_db_with_url(url).await; + let test_db = TestDatabase { + store: SqlStorage::new(pool.clone()), + test_data: TestData::default(), + }; + self.pools.lock().await.push(pool); + test_db + } +} diff --git a/aquadoggo/src/graphql/client/mutation.rs b/aquadoggo/src/graphql/client/mutation.rs index ececb276e..b966a04f0 100644 --- a/aquadoggo/src/graphql/client/mutation.rs +++ b/aquadoggo/src/graphql/client/mutation.rs @@ -583,7 +583,7 @@ mod tests { #[rstest] fn duplicate_publishing_of_entries( #[from(test_db)] - #[with(1, 1, false, TEST_SCHEMA_ID.parse().unwrap())] + #[with(1, 1, 1, false, TEST_SCHEMA_ID.parse().unwrap())] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|populated_db: TestDatabase| async move { diff --git a/aquadoggo/src/graphql/mod.rs b/aquadoggo/src/graphql/mod.rs index afafeefe5..0fbf3b530 100644 --- a/aquadoggo/src/graphql/mod.rs +++ b/aquadoggo/src/graphql/mod.rs @@ -1,7 +1,8 @@ // SPDX-License-Identifier: AGPL-3.0-or-later +//! GraphQL types for handling client and replication requests. pub(crate) mod client; pub(crate) mod replication; mod schema; -pub use schema::{build_root_schema, QueryRoot, RootSchema}; +pub use schema::{build_root_schema, MutationRoot, QueryRoot, RootSchema}; diff --git a/aquadoggo/src/graphql/replication/author.rs b/aquadoggo/src/graphql/replication/author.rs index 888533881..12bf53c59 100644 --- a/aquadoggo/src/graphql/replication/author.rs +++ b/aquadoggo/src/graphql/replication/author.rs @@ -8,7 +8,7 @@ use async_graphql::*; use super::public_key::PublicKey; /// Either the `public_key` or the `alias` of that author. -#[derive(Debug, InputObject)] +#[derive(Debug, InputObject, Clone)] pub struct Author { /// The author's public key pub public_key: Option, @@ -16,6 +16,15 @@ pub struct Author { pub alias: Option, } +impl From for Author { + fn from(author: p2panda_rs::identity::Author) -> Self { + Self { + public_key: Some(PublicKey(author)), + alias: None, + } + } +} + #[derive(Debug)] pub enum AuthorOrAlias { PublicKey(PublicKey), diff --git a/aquadoggo/src/graphql/replication/client/mod.rs b/aquadoggo/src/graphql/replication/client/mod.rs index d5832155a..1f2804373 100644 --- a/aquadoggo/src/graphql/replication/client/mod.rs +++ b/aquadoggo/src/graphql/replication/client/mod.rs @@ -1,15 +1,116 @@ // SPDX-License-Identifier: AGPL-3.0-or-later -use graphql_client::GraphQLQuery; +use anyhow::anyhow; +use graphql_client::{reqwest::post_graphql, GraphQLQuery}; +use p2panda_rs::entry::LogId as PandaLogId; +use p2panda_rs::entry::SeqNum as PandaSeqNum; +use p2panda_rs::identity::Author as PandaAuthor; +use reqwest::Client as ReqwestClient; +use reqwest::IntoUrl; -use super::Entry; -use super::EntryHash; +use super::*; -// The paths are relative to the directory where your `Cargo.toml` is located. -// Both json and the GraphQL schema language are supported as sources for the schema -#[derive(GraphQLQuery)] +/// A graphql client for doing replication requests to another aquadoggo node. +#[derive(Debug)] +pub struct ReplicationClient { + reqwest_client: ReqwestClient, +} + +impl ReplicationClient { + /// Create a new client. + pub fn new() -> Self { + let reqwest_client = ReqwestClient::new(); + Self { reqwest_client } + } + + /// Attempts to get entries newer than the given sequence_number for an author + log_id. + /// + /// Currently does not use pagination, you will need to call this multiple times and eventually + /// you we will get up to date. + pub async fn get_entries_newer_than_seq( + &mut self, + url: U, + log_id: &PandaLogId, + author: &PandaAuthor, + sequence_number: Option<&PandaSeqNum>, + ) -> Result> { + let variables = + create_get_entries_newer_than_seq_request_variable(author, sequence_number, log_id); + + let result = + post_graphql::(&self.reqwest_client, url.clone(), variables) + .await?; + result + .data + .and_then(|data| data.get_entries_newer_than_seq.edges) + .map(convert_edges_to_storage_entries) + .ok_or_else(|| anyhow!("data wasn't in the format expected"))? + } +} + +fn convert_edges_to_storage_entries( + edges: Vec< + Option, + >, +) -> Result, Error> { + // Ooof, the auto generated types aren't very ergonomic to deal with. + let entries = edges + .into_iter() + .flatten() + .map(|edge| -> Result { + let entry_and_payload = EntryAndPayload { + entry: edge.node.entry, + payload: edge.node.payload, + }; + let storage_entry = entry_and_payload.try_into()?; + Ok(storage_entry) + }) + .collect::>>()?; + + Ok(entries) +} + +fn create_get_entries_newer_than_seq_request_variable( + author: &PandaAuthor, + sequence_number: Option<&PandaSeqNum>, + log_id: &PandaLogId, +) -> get_entries_newer_than_seq::Variables { + let author: Author = author.clone().into(); + // We have to do this manual type conversion because of this issue: + // https://github.com/graphql-rust/graphql-client/issues/386 + let author = get_entries_newer_than_seq::Author { + publicKey: author.public_key.clone(), + alias: author.alias.map(|id| id.0), + }; + let sequence_number = + sequence_number.map(|sequence_number| SequenceNumber(sequence_number.to_owned())); + let log_id = LogId(log_id.to_owned()); + + get_entries_newer_than_seq::Variables { + log_id, + author, + sequence_number, + first: None, + after: None, + } +} + +// The paths are relative to the directory where your `Cargo.toml` is located. Both json and the +// GraphQL schema language are supported as sources for the schema. +#[derive(GraphQLQuery, Clone, Copy, Debug)] +#[graphql( + schema_path = "src/graphql/replication/client/schema.graphql", + query_path = "src/graphql/replication/client/queries/get_entry_by_hash.graphql", + response_derives = "Debug" +)] +struct GetEntryByHash; + +// The paths are relative to the directory where your `Cargo.toml` is located. Both JSON and the +// GraphQL schema language are supported as sources for the schema. +#[derive(GraphQLQuery, Debug, Copy, Clone)] #[graphql( schema_path = "src/graphql/replication/client/schema.graphql", - query_path = "src/graphql/replication/client/queries/get_entry_by_hash.graphql" + query_path = "src/graphql/replication/client/queries/get_entries_newer_than_seq.graphql", + response_derives = "Debug" )] -pub struct GetEntryByHash; +struct GetEntriesNewerThanSeq; diff --git a/aquadoggo/src/graphql/replication/client/queries/get_entries_newer_than_seq.graphql b/aquadoggo/src/graphql/replication/client/queries/get_entries_newer_than_seq.graphql new file mode 100644 index 000000000..aacc2de4a --- /dev/null +++ b/aquadoggo/src/graphql/replication/client/queries/get_entries_newer_than_seq.graphql @@ -0,0 +1,14 @@ +query GetEntriesNewerThanSeq($logId: LogId!, $author: Author!, $sequenceNumber: SequenceNumber, $first: Int, $after: String) { + getEntriesNewerThanSeq(logId: $logId, author: $author, sequenceNumber: $sequenceNumber, first: $first, after: $after){ + pageInfo { + hasNextPage + } + edges { + cursor, + node { + entry, + payload + } + } + } +} diff --git a/aquadoggo/src/graphql/replication/client/schema.graphql b/aquadoggo/src/graphql/replication/client/schema.graphql index 466641704..d58580804 100644 --- a/aquadoggo/src/graphql/replication/client/schema.graphql +++ b/aquadoggo/src/graphql/replication/client/schema.graphql @@ -127,8 +127,10 @@ type QueryRoot { """ Get any entries that are newer than the provided sequence_number for a given author and log_id + + If you don't provide sequence_number then get all entries starting at the first """ - getEntriesNewerThanSeq(logId: LogId!, author: Author!, sequenceNumber: SequenceNumber!, first: Int, after: String): EntryAndPayloadConnection! + getEntriesNewerThanSeq(logId: LogId!, author: Author!, sequenceNumber: SequenceNumber, first: Int, after: String): EntryAndPayloadConnection! """ Get a single entry by its log_id, sequence_number and author """ diff --git a/aquadoggo/src/graphql/replication/context.rs b/aquadoggo/src/graphql/replication/context.rs index b0a73175c..49042b6fc 100644 --- a/aquadoggo/src/graphql/replication/context.rs +++ b/aquadoggo/src/graphql/replication/context.rs @@ -5,6 +5,7 @@ use async_graphql::ID; use lru::LruCache; use mockall::automock; use p2panda_rs::entry::decode_entry; +use p2panda_rs::entry::SeqNum; use p2panda_rs::storage_provider::traits::EntryStore as EntryStoreTrait; use crate::db::stores::StorageEntry; @@ -64,17 +65,17 @@ impl> ReplicationContext( + pub async fn entry_by_log_id_and_seq_num<'a>( &mut self, log_id: LogId, - sequence_number: SequenceNumber, + seq_num: SequenceNumber, author_alias: AuthorOrAlias, ) -> Result> { let author = self.get_author(author_alias)?; let result = self .entry_store - .get_entry_at_seq_num(&author.0, &log_id.0, &sequence_number.0) + .get_entry_at_seq_num(&author.0, &log_id.0, &seq_num.0) .await? .map(|entry| entry.into()); @@ -108,22 +109,21 @@ impl> ReplicationContext Result> { let author = self.get_author(author)?; + + // `get_paginated_log_entries` is inclusive of seq_num. Whereas our seq_num should not be + // included. So we add 1 to the the seq_num we were passed. + let seq_num = SeqNum::new(seq_num + 1)?; let result = self .entry_store - .get_paginated_log_entries( - &author.0, - &log_id.0, - sequence_number.as_ref(), - max_number_of_entries, - ) + .get_paginated_log_entries(&author.0, &log_id.0, &seq_num, max_number_of_entries) .await? .into_iter() .map(|entry| entry.into()) @@ -160,7 +160,7 @@ mod tests { use super::super::testing::MockEntryStore; use super::ReplicationContext; - // TODO: test author aliases + // @TODO: Test author aliases #[tokio::test] async fn entry_by_log_id_and_sequence() { @@ -170,7 +170,7 @@ mod tests { "7cf4f58a2d89e93313f2de99604a814ecea9800cf217b140e9c3a7ba59a5d982".to_string(); let log_id: GraphQLLogId = expected_log_id.into(); - let sequence_number: SequenceNumber = expected_seq_num.try_into().unwrap(); + let seq_num: SequenceNumber = expected_seq_num.try_into().unwrap(); let author = Author::new(&expected_author_string).unwrap(); let author_id = GraphQLAuthor { alias: None, @@ -191,7 +191,7 @@ mod tests { let mut context = ReplicationContext::new(1, mock_entry_store); let result = context - .entry_by_log_id_and_sequence(log_id, sequence_number, author_id.try_into().unwrap()) + .entry_by_log_id_and_seq_num(log_id, seq_num, author_id.try_into().unwrap()) .await; assert!(result.is_ok()); diff --git a/aquadoggo/src/graphql/replication/entry_and_payload.rs b/aquadoggo/src/graphql/replication/entry_and_payload.rs index 124683a0d..a33d0b1cc 100644 --- a/aquadoggo/src/graphql/replication/entry_and_payload.rs +++ b/aquadoggo/src/graphql/replication/entry_and_payload.rs @@ -1,6 +1,10 @@ // SPDX-License-Identifier: AGPL-3.0-or-later +use std::convert::TryFrom; + +use anyhow::Error; use async_graphql::*; +use p2panda_rs::storage_provider::traits::AsStorageEntry; use crate::db::stores::StorageEntry; @@ -12,6 +16,7 @@ use super::Entry; pub struct EntryAndPayload { /// Get the entry pub entry: Entry, + /// Get the payload pub payload: Option, } @@ -25,3 +30,14 @@ impl From for EntryAndPayload { Self { entry, payload } } } + +impl TryFrom for StorageEntry { + type Error = Error; + + fn try_from(entry_and_payload: EntryAndPayload) -> Result { + let entry_signed = entry_and_payload.entry.0; + let operation_encoded = entry_and_payload.payload.unwrap().0; + let result = StorageEntry::new(&entry_signed, &operation_encoded)?; + Ok(result) + } +} diff --git a/aquadoggo/src/graphql/replication/mod.rs b/aquadoggo/src/graphql/replication/mod.rs index cf83f241d..348b024bc 100644 --- a/aquadoggo/src/graphql/replication/mod.rs +++ b/aquadoggo/src/graphql/replication/mod.rs @@ -9,7 +9,6 @@ use anyhow::Error as AnyhowError; use async_graphql::connection::{query, Connection, CursorType, Edge, EmptyFields}; use async_graphql::Object; use async_graphql::*; -use mockall_double::double; use p2panda_rs::entry::decode_entry; use p2panda_rs::storage_provider::traits::EntryStore as EntryStoreTrait; use tokio::sync::Mutex; @@ -35,7 +34,8 @@ mod testing; pub use aliased_author::AliasedAuthor; pub use author::{Author, AuthorOrAlias}; -#[double] +#[cfg(test)] +pub use context::MockReplicationContext; pub use context::ReplicationContext; pub use entry::Entry; pub use entry_and_payload::EntryAndPayload; @@ -78,12 +78,14 @@ impl + Sync + Send> /// Get any entries that are newer than the provided sequence_number for a given author and /// log_id + /// + /// If you don't provide sequence_number then get all entries starting at the first async fn get_entries_newer_than_seq<'a>( &self, ctx: &Context<'a>, log_id: LogId, author: Author, - sequence_number: SequenceNumber, + sequence_number: Option, first: Option, after: Option, ) -> Result> { @@ -95,9 +97,10 @@ impl + Sync + Send> first, None, |after: Option, _, first, _| async move { + let sequence_number = sequence_number.map(|seq| seq.as_u64()).unwrap_or_else(|| 0); + // Add the sequence_number to the after cursor to get the starting sequence number. - let start: u64 = sequence_number.as_u64() + after.map(|a| a.as_u64()).unwrap_or(0); - let start_sequence = SequenceNumber::new(start)?; + let start: u64 = sequence_number + after.map(|a| a.as_u64()).unwrap_or(0); // Limit the maximum number of entries to 10k, set a default value of 10 let max_number_of_entries = first.map(|n| n.clamp(0, 10000)).unwrap_or(10); @@ -105,12 +108,7 @@ impl + Sync + Send> let edges = ctx .lock() .await - .get_entries_newer_than_seq( - log_id, - author, - start_sequence, - max_number_of_entries, - ) + .get_entries_newer_than_seq_num(log_id, author, start, max_number_of_entries) .await? .into_iter() .map(|entry| { @@ -144,7 +142,7 @@ impl + Sync + Send> let result = ctx .lock() .await - .entry_by_log_id_and_sequence(log_id, sequence_number, author) + .entry_by_log_id_and_seq_num(log_id, sequence_number, author) .await?; Ok(result) @@ -184,84 +182,106 @@ mod tests { use std::convert::TryFrom; use std::sync::Arc; - use async_graphql::{ - connection::CursorType, EmptyMutation, EmptySubscription, Request, Schema, - }; + use async_graphql::{EmptyMutation, EmptySubscription, Request, Schema}; + use p2panda_rs::identity::Author; + use rstest::rstest; use tokio::sync::Mutex; - use super::testing::MockEntryStore; - use super::{AuthorOrAlias, ReplicationContext, ReplicationRoot, SequenceNumber}; - - #[tokio::test] - async fn get_entries_newer_than_seq_cursor_addition_is_ok() { - // Main point of this test is make sure the cursor + sequence_number logic addition is - // correct. - let log_id = 3u64; - let sequence_number = 123u64; - let author_string = - "7cf4f58a2d89e93313f2de99604a814ecea9800cf217b140e9c3a7ba59a5d982".to_string(); - let after = SequenceNumber::try_from(sequence_number).unwrap(); - let first = 5; - let expected_start = sequence_number + after.as_u64(); - - let gql_query = format!( - " - query{{ - getEntriesNewerThanSeq(logId: {}, author: {{publicKey: \"{}\" }}, sequenceNumber:{}, first: {}, after: \"{}\" ){{ - pageInfo {{ - hasNextPage - }} - }} - }}", - log_id, author_string, sequence_number, first, after.encode_cursor() - ); - - let mut replication_context: ReplicationContext = - ReplicationContext::default(); - - // Prepare our main assertions. - // - Checks that get_entries_newer_than_seq is called with the values we expect - // - Checks that get_entries_newer_than_seq is called once - // - Configures get_entries_newer_than_seq to return an empty Vec - replication_context - .expect_get_entries_newer_than_seq() - .withf({ - let author_string = author_string.clone(); - - move |log_id_, author_, sequence_number_, max_number_of_entries_| { - let author_matches = match author_ { - AuthorOrAlias::PublicKey(public_key) => { - public_key.0.as_str() == author_string - } - _ => false, - }; - sequence_number_.as_u64() == expected_start - && log_id_.as_u64() == log_id - && author_matches - && *max_number_of_entries_ == first - } - }) - .returning(|_, _, _, _| Ok(vec![])) - .once(); - - // Build up a schema with our mocks that can handle gql query strings - let replication_root = ReplicationRoot::::new(); - let schema = Schema::build(replication_root, EmptyMutation, EmptySubscription) - .data(Arc::new(Mutex::new(replication_context))) - .finish(); - - // Act - let result = schema.execute(Request::new(gql_query)).await; - - // Assert - - // Check that we get the Ok returned from get_entries_newer_than_seq - assert!(result.is_ok()); - - // There should not be a next page because we returned an empty vec from - // get_entries_newer_than_seq and it's length is not == `first` == 5; - let json_value = result.data.into_json().unwrap(); - let has_next_page = &json_value["getEntriesNewerThanSeq"]["pageInfo"]["hasNextPage"]; - assert!(!has_next_page.as_bool().unwrap()); + use crate::db::stores::test_utils::{ + populate_test_db, with_db_manager_teardown, PopulateDatabaseConfig, TestDatabaseManager, + }; + use crate::SqlStorage; + + use super::{ReplicationContext, ReplicationRoot}; + + #[rstest] + #[case::default_params(20, None, None, None, true, 10)] + #[case::no_edges_or_next_page(10, Some(10), Some(5), None, false, 0)] + #[case::some_edges_no_next_page(14, Some(10), Some(5), None, false, 4)] + #[case::edges_and_next_page(15, Some(10), Some(5), None, true, 5)] + #[case::edges_and_next_page_again(16, Some(10), Some(5), None, true, 5)] + fn get_entries_newer_than_seq_cursor( + #[case] entries_in_log: usize, + #[case] sequence_number: Option, + #[case] first: Option, + #[case] after: Option, + #[case] expected_has_next_page: bool, + #[case] expected_edges: usize, + ) { + with_db_manager_teardown(move |db_manager: TestDatabaseManager| async move { + // Build and populate Billie's db + let mut billie_db = db_manager.create("sqlite::memory:").await; + + populate_test_db( + &mut billie_db, + &PopulateDatabaseConfig { + no_of_entries: entries_in_log, + no_of_logs: 1, + no_of_authors: 1, + ..Default::default() + }, + ) + .await; + + // Construct the replication context, root and graphql schema. + let replication_context: ReplicationContext = + ReplicationContext::new(1, billie_db.store.clone()); + let replication_root = ReplicationRoot::::new(); + let schema = Schema::build(replication_root, EmptyMutation, EmptySubscription) + .data(Arc::new(Mutex::new(replication_context))) + .finish(); + + // Collect args needed for the query. + let public_key = billie_db + .test_data + .key_pairs + .first() + .unwrap() + .public_key() + .to_owned(); + + let author = Author::try_from(public_key).unwrap(); + let author_str: String = author.as_str().into(); + let log_id = 1u64; + // Optional values are encpoded as `null` + let sequence_number = sequence_number + .map(|seq_num| seq_num.to_string()) + .unwrap_or_else(|| "null".to_string()); + let first = first + .map(|seq_num| seq_num.to_string()) + .unwrap_or_else(|| "null".to_string()); + let after = after + .map(|seq_num| format!("\"{}\"", seq_num)) + .unwrap_or_else(|| "null".to_string()); + + // Construct the query. + let gql_query = format!( + " + query{{ + getEntriesNewerThanSeq(logId: {}, author: {{publicKey: \"{}\" }}, sequenceNumber: {}, first: {}, after: {} ){{ + edges {{ + cursor + }} + pageInfo {{ + hasNextPage + }} + }} + }}", + log_id, author_str, sequence_number, first, after + ); + + // Make the query. + let result = schema.execute(Request::new(gql_query)).await; + + // Check that we get the Ok returned from get_entries_newer_than_seq + assert!(result.is_ok()); + + // Assert the returned hasNextPage and number of edges returned is what we expect. + let json_value = result.data.into_json().unwrap(); + let edges = &json_value["getEntriesNewerThanSeq"]["edges"]; + assert_eq!(edges.as_array().unwrap().len(), expected_edges); + let has_next_page = &json_value["getEntriesNewerThanSeq"]["pageInfo"]["hasNextPage"]; + assert_eq!(has_next_page.as_bool().unwrap(), expected_has_next_page); + }) } } diff --git a/aquadoggo/src/graphql/replication/sequence_number.rs b/aquadoggo/src/graphql/replication/sequence_number.rs index 8d6cfc5f8..b240e66f9 100644 --- a/aquadoggo/src/graphql/replication/sequence_number.rs +++ b/aquadoggo/src/graphql/replication/sequence_number.rs @@ -12,6 +12,7 @@ use serde::{Deserialize, Serialize}; pub struct SequenceNumber(pub PandaSeqNum); impl SequenceNumber { + #[allow(dead_code)] pub fn new(seq: u64) -> Result { let panda_seq_num = PandaSeqNum::new(seq)?; Ok(Self(panda_seq_num)) diff --git a/aquadoggo/src/lib.rs b/aquadoggo/src/lib.rs index 6e45ad9f6..514dc99eb 100644 --- a/aquadoggo/src/lib.rs +++ b/aquadoggo/src/lib.rs @@ -18,16 +18,17 @@ mod config; mod context; mod db; mod errors; -/// Aquadoggo graphql types for handling client and replication requests pub mod graphql; mod http; mod manager; mod materializer; mod node; +mod replication; #[cfg(test)] mod test_helpers; -pub use config::Configuration; +pub use crate::config::Configuration; +pub use crate::replication::ReplicationConfiguration; pub use db::{connection_pool, provider::SqlStorage}; pub use node::Node; diff --git a/aquadoggo/src/manager.rs b/aquadoggo/src/manager.rs index 61c11aa84..673980bd6 100644 --- a/aquadoggo/src/manager.rs +++ b/aquadoggo/src/manager.rs @@ -136,7 +136,7 @@ where /// /// Errors returned and panics by the service will send an exit signal which can be subscribed /// to via the `on_exit` method. - pub fn add + Send + Sync + Copy + 'static>( + pub fn add + Send + Sync + 'static>( &mut self, name: &'static str, service: F, diff --git a/aquadoggo/src/materializer/service.rs b/aquadoggo/src/materializer/service.rs index d15d8b6a7..806c4ec34 100644 --- a/aquadoggo/src/materializer/service.rs +++ b/aquadoggo/src/materializer/service.rs @@ -158,13 +158,13 @@ mod tests { #[rstest] fn materialize_document_from_bus( #[from(test_db)] - #[with(1, 1, false, TEST_SCHEMA_ID.parse().unwrap(), vec![("name", OperationValue::Text("panda".into()))])] + #[with(1, 1, 1, false, TEST_SCHEMA_ID.parse().unwrap(), vec![("name", OperationValue::Text("panda".into()))])] runner: TestDatabaseRunner, ) { // Prepare database which inserts data for one document runner.with_db_teardown(|db: TestDatabase| async move { // Identify document and operation which was inserted for testing - let document_id = db.documents.first().unwrap(); + let document_id = db.test_data.documents.first().unwrap(); let verified_operation = db .store .get_operations_by_document_id(document_id) @@ -233,13 +233,13 @@ mod tests { #[rstest] fn materialize_document_from_last_runtime( #[from(test_db)] - #[with(1, 1, false, TEST_SCHEMA_ID.parse().unwrap(), vec![("name", OperationValue::Text("panda".into()))])] + #[with(1, 1, 1, false, TEST_SCHEMA_ID.parse().unwrap(), vec![("name", OperationValue::Text("panda".into()))])] runner: TestDatabaseRunner, ) { // Prepare database which inserts data for one document runner.with_db_teardown(|db: TestDatabase| async move { // Identify document and operation which was inserted for testing - let document_id = db.documents.first().unwrap(); + let document_id = db.test_data.documents.first().unwrap(); // Store a pending "reduce" task from last runtime in the database so it gets picked up by // the materializer service @@ -294,14 +294,14 @@ mod tests { #[rstest] fn materialize_update_document( #[from(test_db)] - #[with(1, 1, false, TEST_SCHEMA_ID.parse().unwrap(), vec![("name", OperationValue::Text("panda".into()))])] + #[with(1, 1, 1, false, TEST_SCHEMA_ID.parse().unwrap(), vec![("name", OperationValue::Text("panda".into()))])] runner: TestDatabaseRunner, ) { // Prepare database which inserts data for one document runner.with_db_teardown(|db: TestDatabase| async move { // Identify key_[air, document and operation which was inserted for testing - let key_pair = db.key_pairs.first().unwrap(); - let document_id = db.documents.first().unwrap(); + let key_pair = db.test_data.key_pairs.first().unwrap(); + let document_id = db.test_data.documents.first().unwrap(); let verified_operation = db .store .get_operations_by_document_id(document_id) @@ -426,7 +426,7 @@ mod tests { )] fn materialize_complex_documents( #[from(test_db)] - #[with(0, 0)] + #[with(0, 0, 0)] runner: TestDatabaseRunner, #[case] operation: Operation, key_pair: KeyPair, diff --git a/aquadoggo/src/materializer/tasks/dependency.rs b/aquadoggo/src/materializer/tasks/dependency.rs index 3d0577b11..654bd3fc9 100644 --- a/aquadoggo/src/materializer/tasks/dependency.rs +++ b/aquadoggo/src/materializer/tasks/dependency.rs @@ -167,6 +167,7 @@ mod tests { #[rstest] #[case( test_db( + 1, 1, 1, false, @@ -178,6 +179,7 @@ mod tests { )] #[case( test_db( + 1, 1, 1, false, @@ -193,6 +195,7 @@ mod tests { )] #[case( test_db( + 1, 1, 1, false, @@ -207,6 +210,7 @@ mod tests { )] #[case( test_db( + 1, 1, 1, false, @@ -222,6 +226,7 @@ mod tests { )] #[case( test_db( + 1, 1, 1, false, @@ -243,6 +248,7 @@ mod tests { test_db( 4, 1, + 1, false, TEST_SCHEMA_ID.parse().unwrap(), vec![ @@ -270,12 +276,12 @@ mod tests { runner.with_db_teardown(move |db: TestDatabase| async move { let context = Context::new(db.store.clone(), Configuration::default()); - for document_id in &db.documents { + for document_id in &db.test_data.documents { let input = TaskInput::new(Some(document_id.clone()), None); reduce_task(context.clone(), input).await.unwrap().unwrap(); } - for document_id in &db.documents { + for document_id in &db.test_data.documents { let document_view = db .store .get_document_by_id(document_id) @@ -300,12 +306,12 @@ mod tests { #[rstest] fn no_reduce_task_for_materialised_document_relations( #[from(test_db)] - #[with(1, 1)] + #[with(1, 1, 1)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { let context = Context::new(db.store.clone(), Configuration::default()); - let document_id = db.documents[0].clone(); + let document_id = db.test_data.documents[0].clone(); let input = TaskInput::new(Some(document_id.clone()), None); reduce_task(context.clone(), input).await.unwrap().unwrap(); @@ -377,6 +383,7 @@ mod tests { test_db( 2, 1, + 1, true, TEST_SCHEMA_ID.parse().unwrap(), vec![ @@ -390,6 +397,7 @@ mod tests { test_db( 2, 1, + 1, true, TEST_SCHEMA_ID.parse().unwrap(), vec![ @@ -406,7 +414,7 @@ mod tests { fn fails_on_deleted_documents(#[case] runner: TestDatabaseRunner) { runner.with_db_teardown(|db: TestDatabase| async move { let context = Context::new(db.store.clone(), Configuration::default()); - let document_id = db.documents[0].clone(); + let document_id = db.test_data.documents[0].clone(); let input = TaskInput::new(Some(document_id.clone()), None); reduce_task(context.clone(), input).await.unwrap(); diff --git a/aquadoggo/src/materializer/tasks/reduce.rs b/aquadoggo/src/materializer/tasks/reduce.rs index baa8ffc10..d7fcbbf82 100644 --- a/aquadoggo/src/materializer/tasks/reduce.rs +++ b/aquadoggo/src/materializer/tasks/reduce.rs @@ -216,7 +216,8 @@ mod tests { #[from(test_db)] #[with( 2, - 5, + 1, + 20, false, TEST_SCHEMA_ID.parse().unwrap(), vec![("username", OperationValue::Text("panda".into()))], @@ -227,12 +228,12 @@ mod tests { runner.with_db_teardown(|db: TestDatabase| async move { let context = Context::new(db.store.clone(), Configuration::default()); - for document_id in &db.documents { + for document_id in &db.test_data.documents { let input = TaskInput::new(Some(document_id.clone()), None); assert!(reduce_task(context.clone(), input).await.is_ok()); } - for document_id in &db.documents { + for document_id in &db.test_data.documents { let document_view = context.store.get_document_by_id(document_id).await.unwrap(); assert_eq!( @@ -246,12 +247,12 @@ mod tests { #[rstest] fn updates_a_document( #[from(test_db)] - #[with(1, 1)] + #[with(1, 1, 1)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { - let document_id = db.documents.first().unwrap(); - let key_pair = db.key_pairs.first().unwrap(); + let document_id = db.test_data.documents.first().unwrap(); + let key_pair = db.test_data.key_pairs.first().unwrap(); let context = Context::new(db.store.clone(), Configuration::default()); let input = TaskInput::new(Some(document_id.clone()), None); @@ -292,20 +293,13 @@ mod tests { #[rstest] fn reduces_document_to_specific_view_id( #[from(test_db)] - #[with( - 2, - 1, - false, - TEST_SCHEMA_ID.parse().unwrap(), - vec![("username", OperationValue::Text("panda".into()))], - vec![("username", OperationValue::Text("PANDA".into()))] - )] + #[with( 2, 1, 1, false, TEST_SCHEMA_ID.parse().unwrap(), vec![("username", OperationValue::Text("panda".into()))], vec![("username", OperationValue::Text("PANDA".into()))])] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { let document_operations = db .store - .get_operations_by_document_id(&db.documents[0]) + .get_operations_by_document_id(&db.test_data.documents[0]) .await .unwrap(); @@ -356,26 +350,26 @@ mod tests { #[rstest] fn deleted_documents_have_no_view( #[from(test_db)] - #[with(3, 20, true)] + #[with(3, 1, 20, true)] runner: TestDatabaseRunner, ) { runner.with_db_teardown(|db: TestDatabase| async move { let context = Context::new(db.store.clone(), Configuration::default()); - for document_id in &db.documents { + for document_id in &db.test_data.documents { let input = TaskInput::new(Some(document_id.clone()), None); let tasks = reduce_task(context.clone(), input).await.unwrap(); assert!(tasks.is_none()); } - for document_id in &db.documents { + for document_id in &db.test_data.documents { let document_view = context.store.get_document_by_id(document_id).await.unwrap(); assert!(document_view.is_none()) } let document_operations = context .store - .get_operations_by_document_id(&db.documents[0]) + .get_operations_by_document_id(&db.test_data.documents[0]) .await .unwrap(); @@ -390,26 +384,12 @@ mod tests { #[rstest] #[case( - test_db( - 3, - 1, - false, - TEST_SCHEMA_ID.parse().unwrap(), - vec![("username", OperationValue::Text("panda".into()))], - vec![("username", OperationValue::Text("PANDA".into()))] - ), + test_db( 3, 1, 1, false, TEST_SCHEMA_ID.parse().unwrap(), vec![("username", OperationValue::Text("panda".into()))], vec![("username", OperationValue::Text("PANDA".into()))]), true )] // This document is deleted, it shouldn't spawn a dependency task. #[case( - test_db( - 3, - 1, - true, - TEST_SCHEMA_ID.parse().unwrap(), - vec![("username", OperationValue::Text("panda".into()))], - vec![("username", OperationValue::Text("PANDA".into()))] - ), + test_db( 3, 1, 1, true, TEST_SCHEMA_ID.parse().unwrap(), vec![("username", OperationValue::Text("panda".into()))], vec![("username", OperationValue::Text("PANDA".into()))]), false )] fn returns_dependency_task_inputs( @@ -418,7 +398,7 @@ mod tests { ) { runner.with_db_teardown(move |db: TestDatabase| async move { let context = Context::new(db.store.clone(), Configuration::default()); - let document_id = db.documents[0].clone(); + let document_id = db.test_data.documents[0].clone(); let input = TaskInput::new(Some(document_id.clone()), None); let next_task_inputs = reduce_task(context.clone(), input).await.unwrap(); diff --git a/aquadoggo/src/node.rs b/aquadoggo/src/node.rs index 8f2723d99..14da42656 100644 --- a/aquadoggo/src/node.rs +++ b/aquadoggo/src/node.rs @@ -10,6 +10,7 @@ use crate::db::{connection_pool, create_database, run_pending_migrations, Pool}; use crate::http::http_service; use crate::manager::ServiceManager; use crate::materializer::materializer_service; +use crate::replication::replication_service; /// Makes sure database is created and migrated before returning connection pool. async fn initialize_db(config: &Configuration) -> Result { @@ -52,12 +53,15 @@ impl Node { let store = SqlStorage::new(pool.clone()); // Create service manager with shared data between services - let context = Context::new(store, config); + let context = Context::new(store, config.clone()); let mut manager = ServiceManager::::new(1024, context); // Start materializer service manager.add("materializer", materializer_service); + // Start replication service + manager.add("replication", replication_service); + // Start HTTP server with GraphQL API manager.add("http", http_service); diff --git a/aquadoggo/src/replication/config.rs b/aquadoggo/src/replication/config.rs new file mode 100644 index 000000000..030fc72b9 --- /dev/null +++ b/aquadoggo/src/replication/config.rs @@ -0,0 +1,59 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +use std::convert::TryFrom; + +use anyhow::{Error, Result}; +use p2panda_rs::entry::LogId; +use p2panda_rs::identity::Author; +use serde::Deserialize; + +/// Configuration for the replication service. +#[derive(Debug, Clone, Deserialize)] +#[serde(default)] +pub struct ReplicationConfiguration { + /// How often to connect to remote nodes for replication. + pub connection_interval_seconds: u64, + + /// The addresses of remote peers to replicate from. + pub remote_peers: Vec, + + /// The authors to replicate and their log ids. + pub authors_to_replicate: Vec, +} + +impl Default for ReplicationConfiguration { + fn default() -> Self { + Self { + connection_interval_seconds: 30, + remote_peers: Vec::new(), + authors_to_replicate: Vec::new(), + } + } +} + +/// Intermediate data type to configure which log ids of what authors should be replicated. +#[derive(Debug, Clone, Deserialize)] +pub struct AuthorToReplicate(Author, Vec); + +impl AuthorToReplicate { + /// Returns author to be replicated. + pub fn author(&self) -> &Author { + &self.0 + } + + /// Returns log ids from author. + pub fn log_ids(&self) -> &Vec { + &self.1 + } +} + +impl TryFrom<(String, Vec)> for AuthorToReplicate { + type Error = Error; + + fn try_from(value: (String, Vec)) -> Result { + let author = Author::new(&value.0)?; + let log_ids = value.1.into_iter().map(LogId::new).collect(); + + Ok(Self(author, log_ids)) + } +} diff --git a/aquadoggo/src/replication/mod.rs b/aquadoggo/src/replication/mod.rs new file mode 100644 index 000000000..f535123dc --- /dev/null +++ b/aquadoggo/src/replication/mod.rs @@ -0,0 +1,7 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +mod config; +mod service; + +pub use config::ReplicationConfiguration; +pub use service::replication_service; diff --git a/aquadoggo/src/replication/service.rs b/aquadoggo/src/replication/service.rs new file mode 100644 index 000000000..9f9969579 --- /dev/null +++ b/aquadoggo/src/replication/service.rs @@ -0,0 +1,377 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +use std::sync::Arc; +use std::time::Duration; + +use anyhow::{anyhow, Result}; +use bamboo_rs_core_ed25519_yasmf::verify::verify_batch; +use futures::TryFutureExt; +use log::{debug, error, trace, warn}; +use p2panda_rs::entry::LogId; +use p2panda_rs::entry::SeqNum; +use p2panda_rs::identity::Author; +use p2panda_rs::operation::{AsVerifiedOperation, VerifiedOperation}; +use p2panda_rs::storage_provider::traits::{ + AsStorageEntry, EntryStore, OperationStore, StorageProvider, +}; +use tokio::task; + +use crate::bus::{ServiceMessage, ServiceSender}; +use crate::context::Context; +use crate::db::stores::StorageEntry; +use crate::graphql::client::PublishEntryRequest; +use crate::graphql::replication::client::ReplicationClient; +use crate::manager::Shutdown; + +/// Replication service polling other nodes frequently to ask them about new entries from a defined +/// set of authors and log ids. +pub async fn replication_service( + context: Context, + shutdown: Shutdown, + tx: ServiceSender, +) -> Result<()> { + // Initialise a GraphQL client which will talk to the other node + let mut client = ReplicationClient::new(); + + // Prepare replication configuration + let config = &context.config.replication; + let connection_interval = Duration::from_secs(config.connection_interval_seconds); + let authors_to_replicate = Arc::new(config.authors_to_replicate.clone()); + let remote_peers = Arc::new(config.remote_peers.clone()); + + // Start replication service + let handle = task::spawn(async move { + loop { + debug!("Starting replication with remote peers"); + + // Ask every remote peer about latest entries of log ids and authors + for remote_peer in remote_peers.clone().iter() { + for author_to_replicate in authors_to_replicate.clone().iter() { + let author = author_to_replicate.author().clone(); + let log_ids = author_to_replicate.log_ids().clone(); + + for log_id in log_ids { + // Get the latest sequence number we have for this log and author + let latest_seq_num = get_latest_seq_num(&context, &log_id, &author).await; + debug!( + "Latest entry sequence number of {} and {}: {:?}", + log_id.as_u64(), + author, + latest_seq_num + ); + + // Make our replication request to the remote peer + let entries = client + .get_entries_newer_than_seq( + remote_peer, + &log_id, + &author, + latest_seq_num.as_ref(), + ) + .await; + + if let Ok(entries) = entries { + debug!( + "Received {} new entries from peer {}", + entries.len(), + remote_peer + ); + + if let Err(err) = verify_entries(&entries, &context).await { + warn!("Couldn't verify entries: {}", err); + continue; + } + + insert_new_entries(&entries, &context, tx.clone()) + .await + .unwrap_or_else(|err| error!("{:?}", err)); + } else { + warn!("Replication request to peer {} failed", remote_peer); + } + } + } + } + + // Wait a couple of seconds before we attempt next replication requests + tokio::time::sleep(connection_interval).await; + } + }); + + tokio::select! { + _ = handle => (), + _ = shutdown => (), + } + + Ok(()) +} + +/// Helper method to verify a batch of entries coming from an untrusted peer. +async fn verify_entries(entries: &[StorageEntry], context: &Context) -> Result<()> { + // Get the first entry (assumes they're sorted by seq_num smallest to largest) + // @TODO: We can not trust that the other peer sorted the entries for us? + let first_entry = entries.get(0).cloned(); + let mut entries_to_verify = entries.to_vec(); + + match first_entry.as_ref() { + // If the entry is the first in the log then we can don't need to attempt to get the + // skiplink and previous + Some(entry) if entry.seq_num() == SeqNum::new(1).unwrap() => { + trace!("First entry had seq_num 1 do no need to get previous entries in db"); + } + Some(entry) => { + trace!("Getting certificate pool for entries"); + add_certpool_to_entries_for_verification(&mut entries_to_verify, entry, context) + .await?; + } + None => (), + } + + let entries_to_verify: Vec<(Vec, Option>)> = entries_to_verify + .iter() + .map(|entry| { + ( + entry.entry_bytes(), + entry + .operation_encoded() + .map(|operation| operation.to_bytes()), + ) + }) + .collect(); + + verify_batch(&entries_to_verify)?; + + Ok(()) +} + +/// Helper method to insert a batch of verified entries into the database. +async fn insert_new_entries( + new_entries: &[StorageEntry], + context: &Context, + tx: ServiceSender, +) -> Result<()> { + for entry in new_entries { + // Parse and validate parameters + let args = PublishEntryRequest { + entry_encoded: entry.entry_signed().clone(), + // We know a storage entry has an operation so we safely unwrap here. + operation_encoded: entry.operation_encoded().unwrap().clone(), + }; + + // This is the method used to publish entries arriving from clients. They all contain a + // payload (operation). + // + // @TODO: This is not a great fit for replication, as it performs validation we either do + // not need or already done in a previous step. We plan to refactor this into a more + // modular set of methods which can definitely be used here more cleanly. For now, we do it + // this way. + context + .0 + .store + .publish_entry(&args) + .await + .map_err(|err| anyhow!(format!("Error inserting new entry into db: {:?}", err)))?; + + // @TODO: We have to publish the operation too, once again, this will be improved with the + // above mentioned refactor. + let document_id = context + .0 + .store + .get_document_by_entry(&entry.hash()) + .await + .map_err(|err| anyhow!(format!("Error retrieving document id from db: {:?}", err)))?; + + match document_id { + Some(document_id) => { + let operation = VerifiedOperation::new_from_entry( + entry.entry_signed(), + entry.operation_encoded().unwrap(), + ) + // Safely unwrap here as the entry and operation were already validated. + .unwrap(); + + context + .0 + .store + .insert_operation(&operation, &document_id) + .map_ok({ + let entry = entry.clone(); + let tx = tx.clone(); + + move |_| { + send_new_entry_service_message(tx.clone(), &entry); + } + }) + .map_err(|err| { + anyhow!(format!("Error inserting new operation into db: {:?}", err)) + }) + .await + } + None => Err(anyhow!( + "No document found for published operation".to_string() + )), + }?; + } + + Ok(()) +} + +/// Helper method to retreive all entries from certificate pool to be able to verify Bamboo log +/// integrity. +async fn add_certpool_to_entries_for_verification( + entries: &mut Vec, + first_entry: &StorageEntry, + context: &Context, +) -> Result<()> { + trace!("Getting certificate pool for entries"); + + let mut certpool = context + .0 + .store + .get_certificate_pool( + &first_entry.author(), + &first_entry.log_id(), + &first_entry.seq_num(), + ) + .await?; + + trace!("Got {} certpool entries", certpool.len()); + entries.append(&mut certpool); + + Ok(()) +} + +/// Helper method to inform other services (like materialization service) about new operations. +fn send_new_entry_service_message(tx: ServiceSender, entry: &StorageEntry) { + let bus_message = ServiceMessage::NewOperation(entry.entry_signed().hash().into()); + + if tx.send(bus_message).is_err() { + // Silently fail here as we don't mind if there are no subscribers + } +} + +/// Helper method to get the latest sequence number of a log and author. +async fn get_latest_seq_num(context: &Context, log_id: &LogId, author: &Author) -> Option { + context + .store + .get_latest_entry(author, log_id) + .await + .ok() + .flatten() + .map(|entry| *entry.entry_decoded().seq_num()) +} + +#[cfg(test)] +mod tests { + use std::convert::{TryFrom, TryInto}; + use std::time::Duration; + + use p2panda_rs::identity::Author; + use p2panda_rs::storage_provider::traits::EntryStore; + use p2panda_rs::test_utils::constants::TEST_SCHEMA_ID; + use rstest::rstest; + use tokio::sync::broadcast; + use tokio::task; + + use crate::context::Context; + use crate::db::stores::test_utils::{ + populate_test_db, with_db_manager_teardown, PopulateDatabaseConfig, TestDatabaseManager, + }; + use crate::http::http_service; + use crate::replication::ReplicationConfiguration; + use crate::test_helpers::shutdown_handle; + use crate::Configuration; + + use super::replication_service; + + // @TODO: This will be replaced with using `ctor` in this PR: + // https://github.com/p2panda/aquadoggo/pull/166 + fn init_env_logger() { + let _ = env_logger::builder().is_test(true).try_init(); + } + + #[rstest] + fn full_replication() { + with_db_manager_teardown(|db_manager: TestDatabaseManager| async move { + init_env_logger(); + + // Build and populate Billie's database + let mut billie_db = db_manager.create("sqlite::memory:").await; + let populate_db_config = PopulateDatabaseConfig { + no_of_entries: 1, + no_of_logs: 1, + no_of_authors: 1, + ..Default::default() + }; + populate_test_db(&mut billie_db, &populate_db_config).await; + + // Launch HTTP service of Billie + let (tx, _rx) = broadcast::channel(16); + let tx_billie = tx.clone(); + let shutdown_billie = shutdown_handle(); + let context_billie = Context::new( + billie_db.store.clone(), + Configuration { + http_port: 3022, + ..Configuration::default() + }, + ); + + let http_server_billie = task::spawn(async { + http_service(context_billie, shutdown_billie, tx_billie) + .await + .unwrap(); + }); + + // Our test database helper already populated the database for us. We retreive the + // public keys here of the authors who created these test data entries + let public_key = billie_db + .test_data + .key_pairs + .first() + .unwrap() + .public_key() + .to_owned(); + + let author = Author::try_from(public_key).unwrap(); + let log_ids: Vec = vec![1]; + let author_str: String = author.as_str().into(); + let endpoint: String = "http://localhost:3022/graphql".into(); + + // Construct database and context for Ada + let config_ada = Configuration { + replication: ReplicationConfiguration { + authors_to_replicate: vec![(author_str, log_ids).try_into().unwrap()], + remote_peers: vec![endpoint], + ..ReplicationConfiguration::default() + }, + ..Configuration::default() + }; + let ada_db = db_manager.create("sqlite::memory:").await; + let context_ada = Context::new(ada_db.store.clone(), config_ada); + let tx_ada = tx.clone(); + let shutdown_ada = shutdown_handle(); + + // Ada starts replication service to get data from Billies GraphQL API + let replication_service_ada = task::spawn(async { + replication_service(context_ada, shutdown_ada, tx_ada) + .await + .unwrap(); + }); + + // Wait a little bit for replication to take place + tokio::time::sleep(Duration::from_millis(500)).await; + + // Make sure the services did not stop + assert!(!http_server_billie.is_finished()); + assert!(!replication_service_ada.is_finished()); + + // Check the entry arrived into Ada's database + let entries = ada_db + .store + .get_entries_by_schema(&TEST_SCHEMA_ID.parse().unwrap()) + .await + .unwrap(); + assert_eq!(entries.len(), 1); + }) + } +} diff --git a/aquadoggo/src/test_helpers.rs b/aquadoggo/src/test_helpers.rs index ebb8e70ab..8cc924da7 100644 --- a/aquadoggo/src/test_helpers.rs +++ b/aquadoggo/src/test_helpers.rs @@ -3,6 +3,7 @@ use std::convert::TryFrom; use std::fmt::Debug; use std::net::{SocketAddr, TcpListener}; +use std::time::Duration; use axum::body::HttpBody; use axum::BoxError; @@ -11,6 +12,7 @@ use http::{Request, StatusCode}; use hyper::{Body, Server}; use once_cell::sync::Lazy; use serde::Deserialize; +use tokio::task::{self, JoinHandle}; use tower::make::Shared; use tower_service::Service; @@ -148,3 +150,15 @@ impl TestResponse { self.response.status() } } + +// Helper method to give us a shutdown future which will never resolve +pub fn shutdown_handle() -> JoinHandle<()> { + let shutdown = task::spawn(async { + loop { + // Do this forever .. + tokio::time::sleep(Duration::from_millis(100)).await; + } + }); + + shutdown +} diff --git a/aquadoggo_cli/Cargo.toml b/aquadoggo_cli/Cargo.toml index c9c42ffb9..b998e44b2 100644 --- a/aquadoggo_cli/Cargo.toml +++ b/aquadoggo_cli/Cargo.toml @@ -18,6 +18,7 @@ path = "src/main.rs" doc = false [dependencies] +anyhow = "1.0.43" tokio = { version = "1.17.0", features = ["full"] } env_logger = "0.9.0" structopt = "0.3.23" diff --git a/aquadoggo_cli/src/main.rs b/aquadoggo_cli/src/main.rs index db8355334..44c13ff92 100644 --- a/aquadoggo_cli/src/main.rs +++ b/aquadoggo_cli/src/main.rs @@ -1,8 +1,32 @@ // SPDX-License-Identifier: AGPL-3.0-or-later +use std::convert::{TryFrom, TryInto}; +use std::error::Error; + +use anyhow::Result; +use aquadoggo::{Configuration, Node, ReplicationConfiguration}; use structopt::StructOpt; -use aquadoggo::{Configuration, Node}; +/// Helper method to parse a single key-value pair. +fn parse_key_val(s: &str) -> Result<(T, Vec), Box> +where + T: std::str::FromStr, + T::Err: Error + 'static, + U: std::str::FromStr, + U::Err: Error + 'static, +{ + let pos = s + .find('=') + .ok_or_else(|| format!("invalid KEY=value: no `=` found in `{}`", s))?; + + let key = s[..pos].parse()?; + let values = s[pos + 1..] + .split(' ') + .map(|elem| elem.parse()) + .collect::, _>>()?; + + Ok((key, values)) +} #[derive(StructOpt, Debug)] #[structopt(name = "aquadoggo Node", about = "Node server for the p2panda network")] @@ -10,6 +34,42 @@ struct Opt { /// Path to data folder, $HOME/.local/share/aquadoggo by default on Linux. #[structopt(short, long, parse(from_os_str))] data_dir: Option, + + /// URLs of remote nodes to replicate with. + #[structopt(short, long)] + remote_node_addresses: Vec, + + /// A collection of authors and their logs to replicate. + /// + /// eg -A 123abc="1 2 345" -A 456def="6 7" + /// Adds the the authors: + /// + /// - "123abc" with log_ids 1, 2, 345 + /// - "456def" with log_ids 6 7 + #[structopt(short = "A", parse(try_from_str = parse_key_val), number_of_values = 1)] + authors_to_replicate: Vec<(String, Vec)>, +} + +impl TryFrom for Configuration { + type Error = anyhow::Error; + + fn try_from(opt: Opt) -> Result { + let mut config = Configuration::new(opt.data_dir)?; + + let authors_to_replicate = opt + .authors_to_replicate + .into_iter() + .map(|elem| elem.try_into()) + .collect::>()?; + + config.replication = ReplicationConfiguration { + remote_peers: opt.remote_node_addresses, + authors_to_replicate, + ..ReplicationConfiguration::default() + }; + + Ok(config) + } } #[tokio::main] @@ -18,7 +78,7 @@ async fn main() { // Parse command line arguments and load configuration let opt = Opt::from_args(); - let config = Configuration::new(opt.data_dir).expect("Could not load configuration"); + let config = opt.try_into().expect("Could not load configuration"); // Start p2panda node in async runtime let node = Node::start(config).await; From 05c23f0bb768a7c6c199f973ca862235f2c37e10 Mon Sep 17 00:00:00 2001 From: Andreas Dzialocha Date: Fri, 1 Jul 2022 15:36:05 +0200 Subject: [PATCH 10/13] Refactor replication API (#184) * Remove all before we put it back in again * Add replication_service dummy * Update params and fields for GraphQL client API * Attempt to write better GraphQL query descriptions * WIP: Introduce scalars module * Add some more scalars * Move u64 string representation into scalars * Remove unnecessary serde attribute * Implement other replication API methods * Rename params and add descriptions * Add test * Add a test for entry_by_hash, return error when not found * Add a test for entry_by_log_id_and_seq_num, return error when not found * Bring back replication client and service * Make clippy happy * Add entry to CHANGELOG.md * Update dependencies * Fix all dependencies * Move structs implementing request traits into db module * Rename to pagination module * Make fmt happy * Add certificate_pool resolver and test for it * Roll back async-graphql to version 3.0.38 * Use scalars for Client API as well * Use rust version 1.61.0 to prevent async-graphql bug * Rename query to be consistent with naming * Access the seq_num directly on StorageEntry * Remove unneeded import * Add a note on fixed Rust version for CI Co-authored-by: Sam Andreae --- .github/workflows/tests.yml | 17 +- CHANGELOG.md | 1 + Cargo.lock | 394 +++-------------- Cargo.toml | 2 - aquadoggo/Cargo.toml | 92 ++-- aquadoggo/src/bin/dump_gql_schema.rs | 16 - aquadoggo/src/db/errors.rs | 3 - aquadoggo/src/db/mod.rs | 1 + aquadoggo/src/db/provider.rs | 9 +- .../src/{graphql/client => db}/request.rs | 34 +- aquadoggo/src/db/stores/test_utils.rs | 63 ++- aquadoggo/src/graphql/client/mod.rs | 5 +- aquadoggo/src/graphql/client/mutation.rs | 106 +++-- aquadoggo/src/graphql/client/query.rs | 66 ++- aquadoggo/src/graphql/client/response.rs | 125 ++---- aquadoggo/src/graphql/client/u64_string.rs | 114 ----- aquadoggo/src/graphql/mod.rs | 7 +- aquadoggo/src/graphql/pagination.rs | 23 + .../src/graphql/replication/aliased_author.rs | 21 - aquadoggo/src/graphql/replication/author.rs | 46 -- aquadoggo/src/graphql/replication/client.rs | 76 ++++ .../src/graphql/replication/client/mod.rs | 116 ----- .../get_entries_newer_than_seq.graphql | 14 - .../client/queries/get_entry_by_hash.graphql | 6 - .../graphql/replication/client/schema.graphql | 167 ------- aquadoggo/src/graphql/replication/context.rs | 199 --------- aquadoggo/src/graphql/replication/entry.rs | 29 -- .../graphql/replication/entry_and_payload.rs | 43 -- .../src/graphql/replication/entry_hash.rs | 23 - aquadoggo/src/graphql/replication/log_id.rs | 30 -- aquadoggo/src/graphql/replication/mod.rs | 288 +----------- aquadoggo/src/graphql/replication/payload.rs | 11 - .../src/graphql/replication/public_key.rs | 11 - aquadoggo/src/graphql/replication/query.rs | 415 ++++++++++++++++++ aquadoggo/src/graphql/replication/response.rs | 156 +++++++ .../graphql/replication/sequence_number.rs | 43 -- .../replication/single_entry_and_payload.rs | 50 --- .../src/graphql/replication/testing/mod.rs | 58 --- aquadoggo/src/graphql/scalars/document_id.rs | 30 ++ .../src/graphql/scalars/encoded_entry.rs | 29 ++ .../src/graphql/scalars/encoded_operation.rs | 23 + aquadoggo/src/graphql/scalars/entry_hash.rs | 50 +++ aquadoggo/src/graphql/scalars/log_id.rs | 79 ++++ aquadoggo/src/graphql/scalars/mod.rs | 17 + aquadoggo/src/graphql/scalars/public_key.rs | 31 ++ aquadoggo/src/graphql/scalars/seq_num.rs | 115 +++++ aquadoggo/src/graphql/schema.rs | 11 +- aquadoggo/src/lib.rs | 3 +- aquadoggo/src/replication/service.rs | 68 +-- aquadoggo_cli/Cargo.toml | 6 +- aquadoggo_cli/src/main.rs | 5 +- 51 files changed, 1358 insertions(+), 1989 deletions(-) delete mode 100644 aquadoggo/src/bin/dump_gql_schema.rs rename aquadoggo/src/{graphql/client => db}/request.rs (74%) delete mode 100644 aquadoggo/src/graphql/client/u64_string.rs create mode 100644 aquadoggo/src/graphql/pagination.rs delete mode 100644 aquadoggo/src/graphql/replication/aliased_author.rs delete mode 100644 aquadoggo/src/graphql/replication/author.rs create mode 100644 aquadoggo/src/graphql/replication/client.rs delete mode 100644 aquadoggo/src/graphql/replication/client/mod.rs delete mode 100644 aquadoggo/src/graphql/replication/client/queries/get_entries_newer_than_seq.graphql delete mode 100644 aquadoggo/src/graphql/replication/client/queries/get_entry_by_hash.graphql delete mode 100644 aquadoggo/src/graphql/replication/client/schema.graphql delete mode 100644 aquadoggo/src/graphql/replication/context.rs delete mode 100644 aquadoggo/src/graphql/replication/entry.rs delete mode 100644 aquadoggo/src/graphql/replication/entry_and_payload.rs delete mode 100644 aquadoggo/src/graphql/replication/entry_hash.rs delete mode 100644 aquadoggo/src/graphql/replication/log_id.rs delete mode 100644 aquadoggo/src/graphql/replication/payload.rs delete mode 100644 aquadoggo/src/graphql/replication/public_key.rs create mode 100644 aquadoggo/src/graphql/replication/query.rs create mode 100644 aquadoggo/src/graphql/replication/response.rs delete mode 100644 aquadoggo/src/graphql/replication/sequence_number.rs delete mode 100644 aquadoggo/src/graphql/replication/single_entry_and_payload.rs delete mode 100644 aquadoggo/src/graphql/replication/testing/mod.rs create mode 100644 aquadoggo/src/graphql/scalars/document_id.rs create mode 100644 aquadoggo/src/graphql/scalars/encoded_entry.rs create mode 100644 aquadoggo/src/graphql/scalars/encoded_operation.rs create mode 100644 aquadoggo/src/graphql/scalars/entry_hash.rs create mode 100644 aquadoggo/src/graphql/scalars/log_id.rs create mode 100644 aquadoggo/src/graphql/scalars/mod.rs create mode 100644 aquadoggo/src/graphql/scalars/public_key.rs create mode 100644 aquadoggo/src/graphql/scalars/seq_num.rs diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index 8da77bd35..ab18749e1 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -8,6 +8,11 @@ env: target ~/.cargo cargo_manifest: aquadoggo/Cargo.toml + # There is an issue with `async-graphql` or Rust which randomly lets our + # tests fail. With this Rust version and async-graphql 3.0.38 it seems to + # work for now though. + # Related issue: https://github.com/p2panda/aquadoggo/issues/185 + rust_version: 1.61.0 jobs: rust-test-sqlite: @@ -21,7 +26,7 @@ jobs: uses: actions-rs/toolchain@v1 with: profile: minimal - toolchain: stable + toolchain: ${{ env.rust_version }} override: true - name: Restore from cargo cache @@ -74,7 +79,7 @@ jobs: uses: actions-rs/toolchain@v1 with: profile: minimal - toolchain: stable + toolchain: ${{ env.rust_version }} override: true - name: Restore from cargo cache @@ -106,7 +111,7 @@ jobs: uses: actions-rs/toolchain@v1 with: profile: minimal - toolchain: stable + toolchain: ${{ env.rust_version }} override: true - name: Restore from cargo cache @@ -132,7 +137,7 @@ jobs: uses: actions-rs/toolchain@v1 with: profile: minimal - toolchain: stable + toolchain: ${{ env.rust_version }} components: rustfmt override: true @@ -159,7 +164,7 @@ jobs: uses: actions-rs/toolchain@v1 with: profile: minimal - toolchain: stable + toolchain: ${{ env.rust_version }} components: clippy override: true @@ -186,7 +191,7 @@ jobs: uses: actions-rs/toolchain@v1 with: profile: minimal - toolchain: stable + toolchain: ${{ env.rust_version }} override: true - name: Restore from cargo cache diff --git a/CHANGELOG.md b/CHANGELOG.md index 3cbb57687..65b668cd0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -45,6 +45,7 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - Use `VerifiedOperation` [#158](https://github.com/p2panda/aquadoggo/pull/158) - Refactor `test_db` helper method [#176](https://github.com/p2panda/aquadoggo/pull/176) - Update `publishEntry` params and `nextEntryArgs` response fields [#181](https://github.com/p2panda/aquadoggo/pull/181) +- Refactor scalars and replication API, replace `graphql-client` with `gql_client` [#184](https://github.com/p2panda/aquadoggo/pull/184) ### Fixed diff --git a/Cargo.lock b/Cargo.lock index 60a0dda64..eb9104d4e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -102,7 +102,6 @@ name = "aquadoggo" version = "0.3.0" dependencies = [ "anyhow", - "arrayvec 0.5.2", "async-graphql", "async-graphql-axum", "async-trait", @@ -113,29 +112,25 @@ dependencies = [ "env_logger", "envy", "futures", - "graphql_client", - "hex", + "gql_client", "http", "hyper", "lipmaa-link 0.2.2", "log", - "lru", - "mockall", "once_cell", "openssl-probe", "p2panda-rs", "rand 0.8.5", "reqwest", - "rstest 0.12.0", - "rstest_reuse 0.1.3", + "rstest", + "rstest_reuse", "serde", "serde_json", - "sqlformat", "sqlx", "thiserror", "tokio", "tower", - "tower-http 0.2.5", + "tower-http", "tower-service", "triggered", ] @@ -169,12 +164,6 @@ version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8da52d66c7071e2e3fa2a1e5c6d088fec47b593032b254f5e980de8ea54454d6" -[[package]] -name = "ascii" -version = "0.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eab1c04a571841102f5345a8fc0f6bb3d31c315dec879b5c6e42e40ce7ffa34e" - [[package]] name = "ascii_utils" version = "0.9.3" @@ -304,9 +293,9 @@ dependencies = [ [[package]] name = "atoi" -version = "0.4.0" +version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "616896e05fc0e2649463a93a15183c6a16bf03413a7af88ef1285ddedfa9cda5" +checksum = "d7c57d12312ff59c811c0643f4d80830505833c9ffaebd193d819392b265be8e" dependencies = [ "num-traits", ] @@ -358,7 +347,7 @@ dependencies = [ "tokio", "tokio-tungstenite", "tower", - "tower-http 0.3.4", + "tower-http", "tower-layer", "tower-service", ] @@ -633,19 +622,6 @@ dependencies = [ "vec_map", ] -[[package]] -name = "combine" -version = "3.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da3da6baa321ec19e1cc41d31bf599f00c783d0517095cdaf0332e3fe8d20680" -dependencies = [ - "ascii", - "byteorder", - "either", - "memchr", - "unreachable", -] - [[package]] name = "console_error_panic_hook" version = "0.1.7" @@ -695,18 +671,18 @@ dependencies = [ [[package]] name = "crc" -version = "2.1.0" +version = "3.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49fc9a695bca7f35f5f4c15cddc84415f66a74ea78eef08e90c5024f2b540e23" +checksum = "53757d12b596c16c78b83458d732a5d1a17ab3f53f2f7412f6fb57cc8a140ab3" dependencies = [ "crc-catalog", ] [[package]] name = "crc-catalog" -version = "1.1.1" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ccaeedb56da03b09f598226e25e80088cb4cd25f316e6e4df7d695f0feeb1403" +checksum = "2d0165d2900ae6778e36e80bbc4da3b5eefccee9ba939761f9c2882a5d9af3ff" [[package]] name = "crossbeam-channel" @@ -894,12 +870,6 @@ dependencies = [ "const-oid", ] -[[package]] -name = "difflib" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6184e33543162437515c2e2b48714794e37845ec9851711914eec9d308f6ebe8" - [[package]] name = "digest" version = "0.8.1" @@ -931,9 +901,9 @@ dependencies = [ [[package]] name = "directories" -version = "3.0.2" +version = "4.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e69600ff1703123957937708eb27f7a564e48885c537782722ed0ba3189ce1d7" +checksum = "f51c5d4ddabd36886dd3e1438cb358cdcb0d7c499cb99cb4ac2e38e18b5cb210" dependencies = [ "dirs-sys", ] @@ -970,12 +940,6 @@ version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "77c90badedccf4105eca100756a0b1289e191f6fcbdadd3cee1d2f614f97da8f" -[[package]] -name = "downcast" -version = "0.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1435fa1053d8b2fbbe9be7e97eca7f33d37b28409959813daefc1446a14247f1" - [[package]] name = "ecdsa" version = "0.13.4" @@ -1093,28 +1057,6 @@ version = "2.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "77f3309417938f28bf8228fcff79a4a37103981e3e186d2ccd19c74b38f4eb71" -[[package]] -name = "failure" -version = "0.1.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d32e9bd16cc02eae7db7ef620b392808b89f6a5e16bb3497d159c6b92a0f4f86" -dependencies = [ - "backtrace", - "failure_derive", -] - -[[package]] -name = "failure_derive" -version = "0.1.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aa4da3c766cd7a0db8242e326e9e4e081edd567072893ed320008189715366a4" -dependencies = [ - "proc-macro2", - "quote", - "syn", - "synstructure", -] - [[package]] name = "fake-simd" version = "0.1.2" @@ -1149,15 +1091,6 @@ dependencies = [ "subtle", ] -[[package]] -name = "float-cmp" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98de4bbd547a563b716d8dfa9aad1cb19bfab00f4fa09a6a4ed21dbcf44ce9c4" -dependencies = [ - "num-traits", -] - [[package]] name = "float-ord" version = "0.3.2" @@ -1207,12 +1140,6 @@ dependencies = [ "percent-encoding", ] -[[package]] -name = "fragile" -version = "1.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85dcb89d2b10c5f6133de2efd8c11959ce9dbb46a2f7a4cab208c4eeda6ce1ab" - [[package]] name = "futures" version = "0.3.21" @@ -1392,64 +1319,17 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "78cc372d058dcf6d5ecd98510e7fbc9e5aec4d21de70f65fea8fecebcd881bd4" [[package]] -name = "graphql-introspection-query" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f2a4732cf5140bd6c082434494f785a19cfb566ab07d1382c3671f5812fed6d" -dependencies = [ - "serde", -] - -[[package]] -name = "graphql-parser" -version = "0.2.3" +name = "gql_client" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5613c31f18676f164112732202124f373bb2103ff017b3b85ca954ea6a66ada" +checksum = "a2ad0b978ded703acd23762f99e0f701450c9d0066912c0d7f9bf65bf017d87d" dependencies = [ - "combine", - "failure", -] - -[[package]] -name = "graphql_client" -version = "0.10.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9b58571cfc3cc42c3e8ff44fc6cfbb6c0dea17ed22d20f9d8f1efc4e8209a3f" -dependencies = [ - "graphql_query_derive", + "log", "reqwest", "serde", "serde_json", ] -[[package]] -name = "graphql_client_codegen" -version = "0.10.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4bf9cd823359d74ad3d3ecf1afd4a975f4ff2f891cdf9a66744606daf52de8c" -dependencies = [ - "graphql-introspection-query", - "graphql-parser", - "heck 0.3.3", - "lazy_static", - "proc-macro2", - "quote", - "serde", - "serde_json", - "syn", -] - -[[package]] -name = "graphql_query_derive" -version = "0.10.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e56b093bfda71de1da99758b036f4cc811fd2511c8a76f75680e9ffbd2bb4251" -dependencies = [ - "graphql_client_codegen", - "proc-macro2", - "syn", -] - [[package]] name = "group" version = "0.11.0" @@ -1488,26 +1368,20 @@ checksum = "eabb4a44450da02c90444cf74558da904edde8fb4e9035a9a6a4e15445af0bd7" [[package]] name = "hashbrown" -version = "0.11.2" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab5ef0d4909ef3724cc8cce6ccc8572c5c817592e9285f5464f8e86f8bd3726e" +checksum = "db0d4cf898abf0081f964436dc980e96670a0f36863e4b83aaacdb65c9d7ccc3" dependencies = [ "ahash", ] -[[package]] -name = "hashbrown" -version = "0.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db0d4cf898abf0081f964436dc980e96670a0f36863e4b83aaacdb65c9d7ccc3" - [[package]] name = "hashlink" -version = "0.7.0" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7249a3129cbc1ffccd74857f81464a323a152173cdb134e0fd81bc803b29facf" +checksum = "d452c155cb93fecdfb02a73dd57b5d8e442c2063bd7aac72f1bc5e4263a43086" dependencies = [ - "hashbrown 0.11.2", + "hashbrown", ] [[package]] @@ -1749,7 +1623,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "10a35a97730320ffe8e2d410b5d3b69279b98d2c14bdb8b70ea89ecf7888d41e" dependencies = [ "autocfg", - "hashbrown 0.12.1", + "hashbrown", "serde", ] @@ -1862,15 +1736,6 @@ dependencies = [ "cfg-if", ] -[[package]] -name = "lru" -version = "0.7.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c84e6fe5655adc6ce00787cf7dcaf8dc4f998a0565d23eafc207a8b08ca3349a" -dependencies = [ - "hashbrown 0.11.2", -] - [[package]] name = "maplit" version = "1.0.2" @@ -1958,33 +1823,6 @@ dependencies = [ "windows-sys", ] -[[package]] -name = "mockall" -version = "0.11.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5641e476bbaf592a3939a7485fa079f427b4db21407d5ebfd5bba4e07a1f6f4c" -dependencies = [ - "cfg-if", - "downcast", - "fragile", - "lazy_static", - "mockall_derive", - "predicates", - "predicates-tree", -] - -[[package]] -name = "mockall_derive" -version = "0.11.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "262d56735932ee0240d515656e5a7667af3af2a5b0af4da558c4cff2b2aeb0c7" -dependencies = [ - "cfg-if", - "proc-macro2", - "quote", - "syn", -] - [[package]] name = "multer" version = "2.0.2" @@ -2031,12 +1869,6 @@ dependencies = [ "minimal-lexical", ] -[[package]] -name = "normalize-line-endings" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61807f77802ff30975e01f4f071c8ba10c022052f98b3294119f3e615d13e5be" - [[package]] name = "num-traits" version = "0.2.15" @@ -2226,8 +2058,8 @@ dependencies = [ "openmls_traits", "rand 0.7.3", "regex", - "rstest 0.15.0", - "rstest_reuse 0.3.0", + "rstest", + "rstest_reuse", "serde", "serde-wasm-bindgen", "serde_repr", @@ -2429,36 +2261,6 @@ version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "eb9f9e6e233e5c4a35559a617bf40a4ec447db2e84c20b55a6f83167b7e57872" -[[package]] -name = "predicates" -version = "2.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5aab5be6e4732b473071984b3164dbbfb7a3674d30ea5ff44410b6bcd960c3c" -dependencies = [ - "difflib", - "float-cmp", - "itertools", - "normalize-line-endings", - "predicates-core", - "regex", -] - -[[package]] -name = "predicates-core" -version = "1.0.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da1c2388b1513e1b605fcec39a95e0a9e8ef088f71443ef37099fa9ae6673fcb" - -[[package]] -name = "predicates-tree" -version = "1.0.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d86de6de25020a36c6d3643a86d9a6a9f552107c0559c60ea03551b5e16c032" -dependencies = [ - "predicates-core", - "termtree", -] - [[package]] name = "proc-macro-crate" version = "1.1.3" @@ -2716,19 +2518,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "rstest" -version = "0.12.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d912f35156a3f99a66ee3e11ac2e0b3f34ac85a07e05263d05a7e2c8810d616f" -dependencies = [ - "cfg-if", - "proc-macro2", - "quote", - "rustc_version 0.4.0", - "syn", -] - [[package]] name = "rstest" version = "0.15.0" @@ -2738,7 +2527,7 @@ dependencies = [ "futures", "futures-timer", "rstest_macros", - "rustc_version 0.4.0", + "rustc_version", ] [[package]] @@ -2750,18 +2539,7 @@ dependencies = [ "cfg-if", "proc-macro2", "quote", - "rustc_version 0.4.0", - "syn", -] - -[[package]] -name = "rstest_reuse" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32c6cfaae58c048728261723a72b80a0aa9f3768e9a7da3b302a24d262525219" -dependencies = [ - "quote", - "rustc_version 0.3.3", + "rustc_version", "syn", ] @@ -2772,7 +2550,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b29d3117bce27ea307d1fb7ce12c64ba11b3fd04311a42d32bc5f0072e6e3d4d" dependencies = [ "quote", - "rustc_version 0.4.0", + "rustc_version", "syn", ] @@ -2782,37 +2560,36 @@ version = "0.1.21" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7ef03e0a2b150c7a90d01faf6254c9c48a41e95fb2a8c2ac1c6f0d2b9aefc342" -[[package]] -name = "rustc_version" -version = "0.3.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0dfe2087c51c460008730de8b57e6a320782fbfb312e1f4d520e6c6fae155ee" -dependencies = [ - "semver 0.11.0", -] - [[package]] name = "rustc_version" version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" dependencies = [ - "semver 1.0.12", + "semver", ] [[package]] name = "rustls" -version = "0.19.1" +version = "0.20.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35edb675feee39aec9c99fa5ff985081995a06d594114ae14cbe797ad7b7a6d7" +checksum = "5aab8ee6c7097ed6057f43c187a62418d0c05a4bd5f18b3571db50ee0f9ce033" dependencies = [ - "base64", "log", "ring", "sct", "webpki", ] +[[package]] +name = "rustls-pemfile" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7522c9de787ff061458fe9a829dc790a3f5b22dc571694fc5883f448b94d9a9" +dependencies = [ + "base64", +] + [[package]] name = "rustversion" version = "1.0.7" @@ -2843,9 +2620,9 @@ checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" [[package]] name = "sct" -version = "0.6.1" +version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b362b83898e0e69f38515b82ee15aa80636befe47c3b6d3d89a911e78fc228ce" +checksum = "d53dcdb7c9f8158937a7981b48accfd39a43af418591a5d008c7b22b5e1b7ca4" dependencies = [ "ring", "untrusted", @@ -2887,30 +2664,12 @@ dependencies = [ "libc", ] -[[package]] -name = "semver" -version = "0.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f301af10236f6df4160f7c3f04eec6dbc70ace82d23326abad5edee88801c6b6" -dependencies = [ - "semver-parser", -] - [[package]] name = "semver" version = "1.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a2333e6df6d6598f2b1974829f853c2b4c5f4a6e503c10af918081aa6f8564e1" -[[package]] -name = "semver-parser" -version = "0.10.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00b0bef5b7f9e0df16536d3961cfb6e84331c065b4066afb39768d0e319411f7" -dependencies = [ - "pest", -] - [[package]] name = "serde" version = "1.0.137" @@ -3142,9 +2901,9 @@ dependencies = [ [[package]] name = "sqlx" -version = "0.5.13" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "551873805652ba0d912fec5bbb0f8b4cdd96baf8e2ebf5970e5671092966019b" +checksum = "1f82cbe94f41641d6c410ded25bbf5097c240cefdf8e3b06d04198d0a96af6a4" dependencies = [ "sqlx-core", "sqlx-macros", @@ -3152,9 +2911,9 @@ dependencies = [ [[package]] name = "sqlx-core" -version = "0.5.13" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e48c61941ccf5ddcada342cd59e3e5173b007c509e1e8e990dafc830294d9dc5" +checksum = "6b69bf218860335ddda60d6ce85ee39f6cf6e5630e300e19757d1de15886a093" dependencies = [ "ahash", "atoi", @@ -3189,6 +2948,7 @@ dependencies = [ "percent-encoding", "rand 0.8.5", "rustls", + "rustls-pemfile", "serde", "serde_json", "sha-1 0.10.0", @@ -3200,16 +2960,15 @@ dependencies = [ "thiserror", "tokio-stream", "url", - "webpki", "webpki-roots", "whoami", ] [[package]] name = "sqlx-macros" -version = "0.5.13" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc0fba2b0cae21fc00fe6046f8baa4c7fcb49e379f0f592b04696607f69ed2e1" +checksum = "f40c63177cf23d356b159b60acd27c54af7423f1736988502e36bae9a712118f" dependencies = [ "dotenv", "either", @@ -3226,9 +2985,9 @@ dependencies = [ [[package]] name = "sqlx-rt" -version = "0.5.13" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4db708cd3e459078f85f39f96a00960bd841f66ee2a669e90bf36907f5a79aae" +checksum = "874e93a365a598dc3dadb197565952cb143ae4aa716f7bcc933a8d836f6bf89f" dependencies = [ "once_cell", "tokio", @@ -3370,12 +3129,6 @@ dependencies = [ "winapi-util", ] -[[package]] -name = "termtree" -version = "0.2.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "507e9898683b6c43a9aa55b64259b721b52ba226e0f3779137e50ad114a4c90b" - [[package]] name = "textwrap" version = "0.11.0" @@ -3485,9 +3238,9 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.22.0" +version = "0.23.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc6844de72e57df1980054b38be3a9f4702aba4858be64dd700181a8a6d0e1b6" +checksum = "c43ee83903113e03984cb9e5cebe6c04a5116269e900e3ddba8f068a62adda59" dependencies = [ "rustls", "tokio", @@ -3571,24 +3324,6 @@ dependencies = [ "tracing", ] -[[package]] -name = "tower-http" -version = "0.2.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aba3f3efabf7fb41fae8534fc20a817013dd1c12cb45441efb6c82e6556b4cd8" -dependencies = [ - "bitflags", - "bytes", - "futures-core", - "futures-util", - "http", - "http-body", - "http-range-header", - "pin-project-lite", - "tower-layer", - "tower-service", -] - [[package]] name = "tower-http" version = "0.3.4" @@ -3763,15 +3498,6 @@ dependencies = [ "subtle", ] -[[package]] -name = "unreachable" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "382810877fe448991dfc7f0dd6e3ae5d58088fd0ea5e35189655f84e6814fa56" -dependencies = [ - "void", -] - [[package]] name = "untrusted" version = "0.7.1" @@ -3838,12 +3564,6 @@ version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" -[[package]] -name = "void" -version = "1.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a02e4885ed3bc0f2de90ea6dd45ebcbb66dacffe03547fadbb0eeae2770887d" - [[package]] name = "want" version = "0.3.0" @@ -3944,9 +3664,9 @@ dependencies = [ [[package]] name = "webpki" -version = "0.21.4" +version = "0.22.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8e38c0608262c46d4a56202ebabdeb094cef7e560ca7a226c6bf055188aa4ea" +checksum = "f095d78192e208183081cc07bc5515ef55216397af48b873e5edcd72637fa1bd" dependencies = [ "ring", "untrusted", @@ -3954,9 +3674,9 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "0.21.1" +version = "0.22.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aabe153544e473b775453675851ecc86863d2a81d786d741f6b76778f2a48940" +checksum = "44d8de8415c823c8abd270ad483c6feeac771fad964890779f9a8cb24fbbc1bf" dependencies = [ "webpki", ] diff --git a/Cargo.toml b/Cargo.toml index 44bb99af1..fc24a6d0e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -3,5 +3,3 @@ members = [ "aquadoggo", "aquadoggo_cli", ] - -default-members = ["aquadoggo_cli"] diff --git a/aquadoggo/Cargo.toml b/aquadoggo/Cargo.toml index b478dc0f6..659361dc3 100644 --- a/aquadoggo/Cargo.toml +++ b/aquadoggo/Cargo.toml @@ -6,7 +6,6 @@ authors = [ "adz ", "sandreae ", "cafca ", - "pietgeursen ", ] description = "Embeddable p2p network node" license = "AGPL-3.0-or-later" @@ -15,63 +14,38 @@ readme = "README.md" edition = "2018" [dependencies] -anyhow = "1.0.43" -arrayvec = "0.5.2" -async-trait = "0.1.53" -async-graphql = "3.0.38" -async-graphql-axum = "3.0.35" -axum = "0.5.1" -bamboo-rs-core-ed25519-yasmf = "0.1.1" -deadqueue = { version = "0.2.2", default-features = false, features = [ - "unlimited", -] } -directories = "3.0.2" -envy = "0.4.2" -futures = "0.3.21" -graphql_client = { version = "0.10", features = ["reqwest"] } -hex = "0.4.3" -lipmaa-link = "0.2.2" -log = "0.4.14" -lru = "0.7.5" -mockall = "0.11.0" -openssl-probe = "0.1.4" -p2panda-rs = "0.4.0" -rand = "0.8.4" -reqwest = { version = "0.11.9", default-features = false, features = ["json"] } -serde = { version = "1.0.130", features = ["derive"] } -serde_json = "1.0.67" -sqlformat = "0.1.7" -sqlx = { version = "0.5.7", features = [ - "any", - "postgres", - "sqlite", - "runtime-tokio-rustls", -] } -thiserror = "1.0.29" -tokio = { version = "1.17.0", features = [ - "macros", - "net", - "rt", - "rt-multi-thread", - "sync", - "time", -] } -tower-http = { version = "0.2.4", default-features = false, features = [ - "cors", -] } -triggered = "0.1.2" +anyhow = "^1.0.58" +async-graphql = "^3.0.38" +async-graphql-axum = "^3.0.38" +async-trait = "^0.1.56" +axum = "^0.5.10" +bamboo-rs-core-ed25519-yasmf = "^0.1.1" +deadqueue = { version = "^0.2.2", default-features = false, features = [ "unlimited" ] } +directories = "^4.0.1" +envy = "^0.4.2" +futures = "^0.3.21" +gql_client = "^1.0.6" +lipmaa-link = "^0.2.2" +log = "^0.4.17" +openssl-probe = "^0.1.5" +p2panda-rs = "^0.4.0" +serde = { version = "^1.0.137", features = ["derive"] } +sqlx = { version = "^0.6.0", features = [ "any", "postgres", "sqlite", "runtime-tokio-rustls" ] } +thiserror = "^1.0.31" +tokio = { version = "^1.19.2", features = [ "macros", "net", "rt", "rt-multi-thread", "sync", "time" ] } +tower-http = { version = "^0.3.4", default-features = false, features = [ "cors" ] } +triggered = "^0.1.2" [dev-dependencies] -env_logger = "0.9.0" -hyper = "0.14.17" -http = "0.2.6" -once_cell = "1.12.0" -reqwest = { version = "0.11.9", default-features = false, features = [ - "json", - "stream", -] } -rstest = "0.12.0" -rstest_reuse = "0.1.3" -tower-service = "0.3.1" -tower = "0.4.12" -p2panda-rs = { version = "0.4.0", features = ["testing"] } +env_logger = "^0.9.0" +http = "^0.2.8" +hyper = "^0.14.19" +once_cell = "^1.12.0" +p2panda-rs = { version = "^0.4.0", features = [ "testing" ] } +rand = "^0.8.5" +reqwest = { version = "^0.11.11", default-features = false, features = [ "json", "stream" ] } +rstest = "^0.15.0" +rstest_reuse = "^0.3.0" +serde_json = "^1.0.82" +tower = "^0.4.13" +tower-service = "^0.3.2" diff --git a/aquadoggo/src/bin/dump_gql_schema.rs b/aquadoggo/src/bin/dump_gql_schema.rs deleted file mode 100644 index cee0f2d7e..000000000 --- a/aquadoggo/src/bin/dump_gql_schema.rs +++ /dev/null @@ -1,16 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later - -use aquadoggo::graphql::build_root_schema; -use aquadoggo::{connection_pool, SqlStorage}; -use tokio::sync::broadcast; - -#[tokio::main] -async fn main() { - let (tx, _) = broadcast::channel(16); - let pool = connection_pool("sqlite::memory:", 1).await.unwrap(); - let store = SqlStorage::new(pool); - let schema = build_root_schema(store, tx); - let sdl = schema.sdl(); - - println!("{sdl}"); -} diff --git a/aquadoggo/src/db/errors.rs b/aquadoggo/src/db/errors.rs index 124d8a221..8bcf975ed 100644 --- a/aquadoggo/src/db/errors.rs +++ b/aquadoggo/src/db/errors.rs @@ -10,9 +10,6 @@ pub enum SqlStorageError { #[error("SQL query failed: {0}")] Transaction(String), - #[error("Insertion of row into table {0} did not show any effect")] - Insertion(String), - #[error("Deletion of row from table {0} did not show any effect")] Deletion(String), } diff --git a/aquadoggo/src/db/mod.rs b/aquadoggo/src/db/mod.rs index aeb6faaaa..46196bc12 100644 --- a/aquadoggo/src/db/mod.rs +++ b/aquadoggo/src/db/mod.rs @@ -8,6 +8,7 @@ use sqlx::migrate::MigrateDatabase; pub mod errors; pub mod models; pub mod provider; +pub mod request; pub mod stores; pub mod traits; pub mod utils; diff --git a/aquadoggo/src/db/provider.rs b/aquadoggo/src/db/provider.rs index 5adfc8734..16af351ec 100644 --- a/aquadoggo/src/db/provider.rs +++ b/aquadoggo/src/db/provider.rs @@ -7,12 +7,11 @@ use p2panda_rs::operation::VerifiedOperation; use p2panda_rs::storage_provider::traits::StorageProvider; use sqlx::query_scalar; +use crate::db::request::{EntryArgsRequest, PublishEntryRequest}; use crate::db::stores::{StorageEntry, StorageLog}; use crate::db::Pool; use crate::errors::StorageProviderResult; -use crate::graphql::client::{ - EntryArgsRequest, EntryArgsResponse, PublishEntryRequest, PublishEntryResponse, -}; +use crate::graphql::client::NextEntryArguments; /// Sql based storage that implements `StorageProvider`. #[derive(Clone, Debug)] @@ -31,9 +30,9 @@ impl SqlStorage { /// databases. #[async_trait] impl StorageProvider for SqlStorage { - type EntryArgsResponse = EntryArgsResponse; + type EntryArgsResponse = NextEntryArguments; type EntryArgsRequest = EntryArgsRequest; - type PublishEntryResponse = PublishEntryResponse; + type PublishEntryResponse = NextEntryArguments; type PublishEntryRequest = PublishEntryRequest; /// Returns the related document for any entry. diff --git a/aquadoggo/src/graphql/client/request.rs b/aquadoggo/src/db/request.rs similarity index 74% rename from aquadoggo/src/graphql/client/request.rs rename to aquadoggo/src/db/request.rs index a7fd21301..21a4db9bb 100644 --- a/aquadoggo/src/graphql/client/request.rs +++ b/aquadoggo/src/db/request.rs @@ -1,7 +1,6 @@ // SPDX-License-Identifier: AGPL-3.0-or-later use p2panda_rs::document::DocumentId; -use p2panda_rs::schema::SchemaId; use p2panda_rs::storage_provider::traits::{AsEntryArgsRequest, AsPublishEntryRequest}; use p2panda_rs::storage_provider::ValidationError; use p2panda_rs::Validate; @@ -11,22 +10,23 @@ use p2panda_rs::entry::{decode_entry, EntrySigned}; use p2panda_rs::identity::Author; use p2panda_rs::operation::OperationEncoded; -/// Request body of `panda_getEntryArguments`. +/// Struct used to validate params and query database to retreive next entry arguments. #[derive(Deserialize, Debug)] pub struct EntryArgsRequest { - /// The entry author - pub author: Author, - /// The entry document - pub document: Option, + /// The entry author. + pub public_key: Author, + + /// The entry document id. + pub document_id: Option, } impl AsEntryArgsRequest for EntryArgsRequest { fn author(&self) -> &Author { - &self.author + &self.public_key } fn document_id(&self) -> &Option { - &self.document + &self.document_id } } @@ -48,23 +48,23 @@ impl Validate for EntryArgsRequest { } } -/// Request body of `panda_publishEntry`. +/// Struct used to validate params and publish new entry in database. #[derive(Deserialize, Debug)] -#[serde(rename_all = "camelCase")] pub struct PublishEntryRequest { /// The encoded entry - pub entry_encoded: EntrySigned, + pub entry: EntrySigned, + /// The encoded operation - pub operation_encoded: OperationEncoded, + pub operation: OperationEncoded, } impl AsPublishEntryRequest for PublishEntryRequest { fn entry_signed(&self) -> &EntrySigned { - &self.entry_encoded + &self.entry } fn operation_encoded(&self) -> &OperationEncoded { - &self.operation_encoded + &self.operation } } @@ -78,9 +78,3 @@ impl Validate for PublishEntryRequest { Ok(()) } } - -#[derive(Deserialize, Debug)] -#[serde(rename_all = "camelCase")] -pub struct QueryEntriesRequest { - pub schema: SchemaId, -} diff --git a/aquadoggo/src/db/stores/test_utils.rs b/aquadoggo/src/db/stores/test_utils.rs index 58fd98a50..7a4447b45 100644 --- a/aquadoggo/src/db/stores/test_utils.rs +++ b/aquadoggo/src/db/stores/test_utils.rs @@ -23,9 +23,10 @@ use tokio::runtime::Builder; use tokio::sync::Mutex; use crate::db::provider::SqlStorage; +use crate::db::request::{EntryArgsRequest, PublishEntryRequest}; use crate::db::traits::DocumentStore; use crate::db::{connection_pool, create_database, run_pending_migrations, Pool}; -use crate::graphql::client::{EntryArgsRequest, PublishEntryRequest, PublishEntryResponse}; +use crate::graphql::client::NextEntryArguments; use crate::test_helpers::TEST_CONFIG; /// The fields used as defaults in the tests. @@ -107,26 +108,23 @@ pub async fn construct_publish_entry_request( ) -> PublishEntryRequest { let author = Author::try_from(key_pair.public_key().to_owned()).unwrap(); let entry_args_request = EntryArgsRequest { - author: author.clone(), - document: document_id.cloned(), + public_key: author.clone(), + document_id: document_id.cloned(), }; let next_entry_args = provider.get_entry_args(&entry_args_request).await.unwrap(); let entry = Entry::new( - &next_entry_args.log_id, + &next_entry_args.log_id.into(), Some(operation), - next_entry_args.skiplink.as_ref(), - next_entry_args.backlink.as_ref(), - &next_entry_args.seq_num, + next_entry_args.skiplink.map(Hash::from).as_ref(), + next_entry_args.backlink.map(Hash::from).as_ref(), + &next_entry_args.seq_num.into(), ) .unwrap(); - let entry_encoded = sign_and_encode(&entry, key_pair).unwrap(); - let operation_encoded = OperationEncoded::try_from(operation).unwrap(); - PublishEntryRequest { - entry_encoded, - operation_encoded, - } + let entry = sign_and_encode(&entry, key_pair).unwrap(); + let operation = OperationEncoded::try_from(operation).unwrap(); + PublishEntryRequest { entry, operation } } /// Helper for inserting an entry, operation and document_view into the database. @@ -142,12 +140,12 @@ pub async fn insert_entry_operation_and_view( let request = construct_publish_entry_request(provider, operation, key_pair, document_id).await; - let operation_id: OperationId = request.entry_encoded.hash().into(); + let operation_id: OperationId = request.entry.hash().into(); let document_id = document_id .cloned() - .unwrap_or_else(|| request.entry_encoded.hash().into()); + .unwrap_or_else(|| request.entry.hash().into()); - let document_view_id: DocumentViewId = request.entry_encoded.hash().into(); + let document_view_id: DocumentViewId = request.entry.hash().into(); let author = Author::try_from(key_pair.public_key().to_owned()).unwrap(); @@ -410,15 +408,16 @@ pub async fn populate_test_db(db: &mut TestDatabase, config: &PopulateDatabaseCo for _log_id in 0..config.no_of_logs { let mut document_id: Option = None; let mut previous_operation: Option = None; + for index in 0..config.no_of_entries { - // Create an operation based on the current index and whether this document should contain - // a DELETE operation. + // Create an operation based on the current index and whether this document should + // contain a DELETE operation let next_operation_fields = match index { - // First operation is a CREATE. + // First operation is CREATE 0 => Some(operation_fields(config.create_operation_fields.clone())), - // Last operation is a DELETE if the with_delete flag is set. + // Last operation is DELETE if the with_delete flag is set seq if seq == (config.no_of_entries - 1) && config.with_delete => None, - // All other operations are UPDATE. + // All other operations are UPDATE _ => Some(operation_fields(config.update_operation_fields.clone())), }; @@ -435,8 +434,8 @@ pub async fn populate_test_db(db: &mut TestDatabase, config: &PopulateDatabaseCo ) .await; - // Set the previous_operations based on the backlink. - previous_operation = publish_entry_response.backlink.map(|hash| hash.into()); + // Set the previous_operations based on the backlink + previous_operation = publish_entry_response.backlink.map(DocumentViewId::from); // If this was the first entry in the document, store the doucment id for later. if index == 0 { @@ -454,26 +453,26 @@ pub async fn send_to_store( operation: &Operation, document_id: Option<&DocumentId>, key_pair: &KeyPair, -) -> (EntrySigned, PublishEntryResponse) { +) -> (EntrySigned, NextEntryArguments) { // Get an Author from the key_pair. let author = Author::try_from(key_pair.public_key().to_owned()).unwrap(); // Get the next entry arguments for this author and the passed document id. let next_entry_args = store .get_entry_args(&EntryArgsRequest { - author: author.clone(), - document: document_id.cloned(), + public_key: author.clone(), + document_id: document_id.cloned(), }) .await .unwrap(); // Construct the next entry. let next_entry = Entry::new( - &next_entry_args.log_id, + &next_entry_args.log_id.into(), Some(operation), - next_entry_args.skiplink.as_ref(), - next_entry_args.backlink.as_ref(), - &next_entry_args.seq_num, + next_entry_args.skiplink.map(Hash::from).as_ref(), + next_entry_args.backlink.map(Hash::from).as_ref(), + &next_entry_args.seq_num.into(), ) .unwrap(); @@ -483,8 +482,8 @@ pub async fn send_to_store( // Publish the entry and get the next entry args. let publish_entry_request = PublishEntryRequest { - entry_encoded: entry_encoded.clone(), - operation_encoded, + entry: entry_encoded.clone(), + operation: operation_encoded, }; let publish_entry_response = store.publish_entry(&publish_entry_request).await.unwrap(); diff --git a/aquadoggo/src/graphql/client/mod.rs b/aquadoggo/src/graphql/client/mod.rs index 3e32c0344..88fa3372c 100644 --- a/aquadoggo/src/graphql/client/mod.rs +++ b/aquadoggo/src/graphql/client/mod.rs @@ -2,11 +2,8 @@ mod mutation; mod query; -mod request; mod response; -mod u64_string; pub use mutation::ClientMutationRoot; pub use query::ClientRoot; -pub use request::{EntryArgsRequest, PublishEntryRequest}; -pub use response::{EntryArgsResponse, PublishEntryResponse}; +pub use response::NextEntryArguments; diff --git a/aquadoggo/src/graphql/client/mutation.rs b/aquadoggo/src/graphql/client/mutation.rs index b966a04f0..a9b90ddaf 100644 --- a/aquadoggo/src/graphql/client/mutation.rs +++ b/aquadoggo/src/graphql/client/mutation.rs @@ -1,15 +1,17 @@ // SPDX-License-Identifier: AGPL-3.0-or-later use async_graphql::{Context, Error, Object, Result}; -use p2panda_rs::entry::EntrySigned; -use p2panda_rs::operation::{AsVerifiedOperation, OperationEncoded, VerifiedOperation}; +use p2panda_rs::operation::{AsVerifiedOperation, VerifiedOperation}; use p2panda_rs::storage_provider::traits::{OperationStore, StorageProvider}; +use p2panda_rs::Validate; use crate::bus::{ServiceMessage, ServiceSender}; use crate::db::provider::SqlStorage; -use crate::graphql::client::{PublishEntryRequest, PublishEntryResponse}; +use crate::db::request::PublishEntryRequest; +use crate::graphql::client::NextEntryArguments; +use crate::graphql::scalars; -/// Mutations for use by p2panda clients. +/// GraphQL queries for the Client API. #[derive(Default, Debug, Copy, Clone)] pub struct ClientMutationRoot; @@ -21,22 +23,23 @@ impl ClientMutationRoot { async fn publish_entry( &self, ctx: &Context<'_>, - #[graphql(name = "entry", desc = "Encoded entry to publish")] entry_encoded_param: String, + #[graphql(name = "entry", desc = "Signed and encoded entry to publish")] + entry: scalars::EncodedEntry, #[graphql( name = "operation", - desc = "Encoded entry payload, which contains a p2panda operation matching the \ - provided encoded entry." + desc = "p2panda operation representing the entry payload." )] - operation_encoded_param: String, - ) -> Result { + operation: scalars::EncodedOperation, + ) -> Result { let store = ctx.data::()?; let tx = ctx.data::()?; // Parse and validate parameters let args = PublishEntryRequest { - entry_encoded: EntrySigned::new(&entry_encoded_param)?, - operation_encoded: OperationEncoded::new(&operation_encoded_param)?, + entry: entry.into(), + operation: operation.into(), }; + args.validate()?; // Validate and store entry in database // @TODO: Check all validation steps here for both entries and operations. Also, there is @@ -45,15 +48,10 @@ impl ClientMutationRoot { // Load related document from database // @TODO: We probably have this instance already inside of "publish_entry"? - match store - .get_document_by_entry(&args.entry_encoded.hash()) - .await? - { + match store.get_document_by_entry(&args.entry.hash()).await? { Some(document_id) => { - let verified_operation = VerifiedOperation::new_from_entry( - &args.entry_encoded, - &args.operation_encoded, - )?; + let verified_operation = + VerifiedOperation::new_from_entry(&args.entry, &args.operation)?; // Store operation in database // @TODO: This is not done by "publish_entry", maybe it needs to move there as @@ -85,9 +83,9 @@ impl ClientMutationRoot { mod tests { use std::convert::TryFrom; - use async_graphql::{from_value, value, Request, Value, Variables}; + use async_graphql::{value, Request, Variables}; use p2panda_rs::document::DocumentId; - use p2panda_rs::entry::{sign_and_encode, Entry, EntrySigned, LogId, SeqNum}; + use p2panda_rs::entry::{sign_and_encode, Entry, EntrySigned}; use p2panda_rs::hash::Hash; use p2panda_rs::identity::{Author, KeyPair}; use p2panda_rs::operation::{Operation, OperationEncoded, OperationValue}; @@ -102,8 +100,8 @@ mod tests { use tokio::sync::broadcast; use crate::bus::ServiceMessage; + use crate::db::request::EntryArgsRequest; use crate::db::stores::test_utils::{test_db, TestDatabase, TestDatabaseRunner}; - use crate::graphql::client::{EntryArgsRequest, PublishEntryResponse}; use crate::http::{build_server, HttpServiceContext}; use crate::test_helpers::TestClient; @@ -153,29 +151,21 @@ mod tests { #[rstest] fn publish_entry(#[from(test_db)] runner: TestDatabaseRunner, publish_entry_request: Request) { runner.with_db_teardown(move |db: TestDatabase| async move { - let (tx, _rx) = broadcast::channel(16); + let (tx, _) = broadcast::channel(16); let context = HttpServiceContext::new(db.store, tx); - let response = context.schema.execute(publish_entry_request).await; - let received: PublishEntryResponse = match response.data { - Value::Object(result_outer) => { - from_value(result_outer.get("publishEntry").unwrap().to_owned()).unwrap() - } - _ => panic!("Expected return value to be an object"), - }; - - // The response should contain args for the next entry in the same log - let expected = PublishEntryResponse { - log_id: LogId::new(1), - seq_num: SeqNum::new(2).unwrap(), - backlink: Some( - "00201c221b573b1e0c67c5e2c624a93419774cdf46b3d62414c44a698df1237b1c16" - .parse() - .unwrap(), - ), - skiplink: None, - }; - assert_eq!(expected, received); + + assert_eq!( + response.data, + value!({ + "publishEntry": { + "logId": "1", + "seqNum": "2", + "backlink": "00201c221b573b1e0c67c5e2c624a93419774cdf46b3d62414c44a698df1237b1c16", + "skiplink": null, + } + }) + ); }); } @@ -248,10 +238,10 @@ mod tests { json!({ "data": { "publishEntry": { - "logId":"1", - "seqNum":"2", - "backlink":"00201c221b573b1e0c67c5e2c624a93419774cdf46b3d62414c44a698df1237b1c16", - "skiplink":null + "logId": "1", + "seqNum": "2", + "backlink": "00201c221b573b1e0c67c5e2c624a93419774cdf46b3d62414c44a698df1237b1c16", + "skiplink": null } } }) @@ -284,7 +274,15 @@ mod tests { )] #[case::operation_does_not_match( ENTRY_ENCODED, - &{operation_encoded(Some(operation_fields(vec![("silly", OperationValue::Text("Sausage".to_string()))])), None, None).as_str().to_owned()}, + &{operation_encoded( + Some( + operation_fields( + vec![("silly", OperationValue::Text("Sausage".to_string()))] + ) + ), + None, + None + ).as_str().to_owned()}, "operation needs to match payload hash of encoded entry" )] #[case::valid_entry_with_extra_hex_char_at_end( @@ -526,8 +524,8 @@ mod tests { let next_entry_args = db .store .get_entry_args(&EntryArgsRequest { - author: author.clone(), - document: document.as_ref().cloned(), + public_key: author.clone(), + document_id: document.as_ref().cloned(), }) .await .unwrap(); @@ -544,11 +542,11 @@ mod tests { }; let entry = Entry::new( - &next_entry_args.log_id, + &next_entry_args.log_id.into(), Some(&operation), - next_entry_args.skiplink.as_ref(), - next_entry_args.backlink.as_ref(), - &next_entry_args.seq_num, + next_entry_args.skiplink.map(Hash::from).as_ref(), + next_entry_args.backlink.map(Hash::from).as_ref(), + &next_entry_args.seq_num.into(), ) .unwrap(); diff --git a/aquadoggo/src/graphql/client/query.rs b/aquadoggo/src/graphql/client/query.rs index 90a20e585..0997c1a17 100644 --- a/aquadoggo/src/graphql/client/query.rs +++ b/aquadoggo/src/graphql/client/query.rs @@ -2,13 +2,15 @@ use async_graphql::{Context, Error, Object, Result}; use p2panda_rs::document::DocumentId; -use p2panda_rs::identity::Author; use p2panda_rs::storage_provider::traits::StorageProvider; +use p2panda_rs::Validate; use crate::db::provider::SqlStorage; -use crate::graphql::client::{EntryArgsRequest, EntryArgsResponse}; +use crate::db::request::EntryArgsRequest; +use crate::graphql::client::response::NextEntryArguments; +use crate::graphql::scalars; -/// The GraphQL root for the client api that p2panda clients can use to connect to a node. +/// GraphQL queries for the Client API. #[derive(Default, Debug, Copy, Clone)] pub struct ClientRoot; @@ -20,24 +22,22 @@ impl ClientRoot { ctx: &Context<'_>, #[graphql( name = "publicKey", - desc = "Public key that will publish using the returned entry arguments" + desc = "Public key of author that will encode and sign the next entry \ + using the returned arguments" )] - public_key_param: String, + public_key: scalars::PublicKey, #[graphql( name = "documentId", - desc = "Document id to which the entry's operation will apply" + desc = "Document the entry's UPDATE or DELETE operation is referring to, \ + can be left empty when it is a CREATE operation" )] - document_id_param: Option, - ) -> Result { - // Parse and validate parameters - let document_id = match document_id_param { - Some(val) => Some(val.parse::()?), - None => None, - }; + document_id: Option, + ) -> Result { let args = EntryArgsRequest { - author: Author::new(&public_key_param)?, - document: document_id, + public_key: public_key.into(), + document_id: document_id.map(DocumentId::from), }; + args.validate()?; // Load and return next entry arguments let store = ctx.data::()?; @@ -47,14 +47,12 @@ impl ClientRoot { #[cfg(test)] mod tests { - use async_graphql::Response; - use p2panda_rs::entry::{LogId, SeqNum}; + use async_graphql::{value, Response}; use rstest::rstest; use serde_json::json; use tokio::sync::broadcast; use crate::db::stores::test_utils::{test_db, TestDatabase, TestDatabaseRunner}; - use crate::graphql::client::EntryArgsResponse; use crate::http::build_server; use crate::http::HttpServiceContext; use crate::test_helpers::TestClient; @@ -66,9 +64,9 @@ mod tests { let context = HttpServiceContext::new(db.store, tx); let client = TestClient::new(build_server(context)); - // Selected fields need to be alphabetically sorted because that's what the `json` macro - // that is used in the assert below produces. - let response = client + // Selected fields need to be alphabetically sorted because that's what the `json` + // macro that is used in the assert below produces. + let received_entry_args = client .post("/graphql") .json(&json!({ "query": r#"{ @@ -87,21 +85,17 @@ mod tests { .json::() .await; - let expected_entry_args = EntryArgsResponse { - log_id: LogId::new(1), - seq_num: SeqNum::new(1).unwrap(), - backlink: None, - skiplink: None, - }; - let received_entry_args: EntryArgsResponse = match response.data { - async_graphql::Value::Object(result_outer) => { - async_graphql::from_value(result_outer.get("nextEntryArgs").unwrap().to_owned()) - .unwrap() - } - _ => panic!("Expected return value to be an object"), - }; - - assert_eq!(received_entry_args, expected_entry_args); + assert_eq!( + received_entry_args.data, + value!({ + "nextEntryArgs": { + "logId": "1", + "seqNum": "1", + "backlink": null, + "skiplink": null, + } + }) + ); }) } diff --git a/aquadoggo/src/graphql/client/response.rs b/aquadoggo/src/graphql/client/response.rs index 3c6dbbd98..17070ded1 100644 --- a/aquadoggo/src/graphql/client/response.rs +++ b/aquadoggo/src/graphql/client/response.rs @@ -1,122 +1,49 @@ // SPDX-License-Identifier: AGPL-3.0-or-later -use async_graphql::Object; -use serde::{Deserialize, Serialize}; - +use async_graphql::SimpleObject; use p2panda_rs::entry::{LogId, SeqNum}; use p2panda_rs::hash::Hash; use p2panda_rs::storage_provider::traits::{AsEntryArgsResponse, AsPublishEntryResponse}; +use serde::{Deserialize, Serialize}; -use crate::db::models::EntryRow; - -/// Response body of `panda_getEntryArguments`. -/// -/// `seq_num` and `log_id` are returned as strings to be able to represent large integers in JSON. -#[derive(Serialize, Deserialize, Debug, Eq, PartialEq)] -#[serde(rename_all = "camelCase")] -pub struct EntryArgsResponse { - /// The log id of the entry - #[serde(with = "super::u64_string::log_id_string_serialisation")] - pub log_id: LogId, - - /// The sequence number of the entry - #[serde(with = "super::u64_string::seq_num_string_serialisation")] - pub seq_num: SeqNum, - - /// The hash of the entry backlink - pub backlink: Option, - - /// The hash of the entry skiplink - pub skiplink: Option, -} +use crate::graphql::scalars; -#[Object] -impl EntryArgsResponse { +/// Arguments required to sign and encode the next entry for an author. +#[derive(SimpleObject, Serialize, Deserialize, Debug, Eq, PartialEq)] +pub struct NextEntryArguments { + /// Log id of the entry. #[graphql(name = "logId")] - async fn log_id(&self) -> String { - self.log_id.clone().as_u64().to_string() - } + pub log_id: scalars::LogId, + /// Sequence number of the entry. #[graphql(name = "seqNum")] - async fn seq_num(&self) -> String { - self.seq_num.clone().as_u64().to_string() - } + pub seq_num: scalars::SeqNum, - async fn backlink(&self) -> Option { - self.backlink.clone().map(|hash| hash.as_str().to_string()) - } + /// Hash of the entry backlink. + pub backlink: Option, - async fn skiplink(&self) -> Option { - self.skiplink.clone().map(|hash| hash.as_str().to_string()) - } + /// Hash of the entry skiplink. + pub skiplink: Option, } -impl AsEntryArgsResponse for EntryArgsResponse { +impl AsEntryArgsResponse for NextEntryArguments { fn new(backlink: Option, skiplink: Option, seq_num: SeqNum, log_id: LogId) -> Self { - EntryArgsResponse { - log_id, - seq_num, - backlink, - skiplink, + Self { + log_id: log_id.into(), + seq_num: seq_num.into(), + backlink: backlink.map(scalars::EntryHash::from), + skiplink: skiplink.map(scalars::EntryHash::from), } } } -/// Response body of `panda_publishEntry`. -/// -/// `seq_num` and `log_id` are returned as strings to be able to represent large integers in JSON. -#[derive(Serialize, Deserialize, Debug, PartialEq)] -#[serde(rename_all = "camelCase")] -pub struct PublishEntryResponse { - /// The log id of the entry - #[serde(with = "super::u64_string::log_id_string_serialisation")] - pub log_id: LogId, - - /// The sequence number of the entry - #[serde(with = "super::u64_string::seq_num_string_serialisation")] - pub seq_num: SeqNum, - - /// The optional hash of the backlink - pub backlink: Option, - - /// The optional hash of the skiplink - pub skiplink: Option, -} - -#[Object] -impl PublishEntryResponse { - #[graphql(name = "logId")] - async fn log_id(&self) -> String { - self.log_id.clone().as_u64().to_string() - } - - #[graphql(name = "seqNum")] - async fn seq_num(&self) -> String { - self.seq_num.clone().as_u64().to_string() - } - - async fn backlink(&self) -> Option { - self.backlink.clone().map(|hash| hash.as_str().to_string()) - } - - async fn skiplink(&self) -> Option { - self.skiplink.clone().map(|hash| hash.as_str().to_string()) - } -} - -impl AsPublishEntryResponse for PublishEntryResponse { +impl AsPublishEntryResponse for NextEntryArguments { fn new(backlink: Option, skiplink: Option, seq_num: SeqNum, log_id: LogId) -> Self { - PublishEntryResponse { - backlink, - skiplink, - seq_num, - log_id, + Self { + log_id: log_id.into(), + seq_num: seq_num.into(), + backlink: backlink.map(scalars::EntryHash::from), + skiplink: skiplink.map(scalars::EntryHash::from), } } } - -#[derive(Serialize, Debug)] -#[serde(rename_all = "camelCase")] -pub struct QueryEntriesResponse { - pub entries: Vec, -} diff --git a/aquadoggo/src/graphql/client/u64_string.rs b/aquadoggo/src/graphql/client/u64_string.rs deleted file mode 100644 index 6d37cb050..000000000 --- a/aquadoggo/src/graphql/client/u64_string.rs +++ /dev/null @@ -1,114 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later - -use core::fmt; - -use serde::de::{self, Unexpected, Visitor}; - -/// Serialise log id as strings. -/// -/// To be used as a parameter for Serde's `with` field attribute. -pub mod log_id_string_serialisation { - use p2panda_rs::entry::LogId; - use serde::{Deserializer, Serializer}; - - use super::U64StringVisitor; - - pub fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, - { - let u64_val = deserializer.deserialize_string(U64StringVisitor)?; - Ok(LogId::new(u64_val)) - } - - pub fn serialize(value: &LogId, serializer: S) -> Result - where - S: Serializer, - { - serializer.serialize_str(&value.as_u64().to_string()) - } -} - -/// Serialise sequence numbers as strings. -/// -/// To be used as a parameter for Serde's `with` field attribute. -pub mod seq_num_string_serialisation { - use p2panda_rs::entry::SeqNum; - use serde::de::Error; - use serde::{Deserializer, Serializer}; - - use super::U64StringVisitor; - - pub fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, - { - let u64_val = deserializer.deserialize_string(U64StringVisitor)?; - SeqNum::new(u64_val).map_err(D::Error::custom) - } - - pub fn serialize(value: &SeqNum, serializer: S) -> Result - where - S: Serializer, - { - serializer.serialize_str(&value.as_u64().to_string()) - } -} - -/// Serde visitor for deserialising string representations of u64 values. -struct U64StringVisitor; - -impl<'de> Visitor<'de> for U64StringVisitor { - type Value = u64; - - fn expecting(&self, formatter: &mut fmt::Formatter) -> fmt::Result { - formatter.write_str("string representation of a u64") - } - - fn visit_str(self, value: &str) -> Result - where - E: de::Error, - { - value.parse::().map_err(|_err| { - E::invalid_value(Unexpected::Str(value), &"string representation of a u64") - }) - } -} - -#[cfg(test)] -mod tests { - use p2panda_rs::entry::{LogId, SeqNum}; - use serde::{Deserialize, Serialize}; - - #[test] - fn log_id() { - #[derive(Serialize, Deserialize, PartialEq, Debug)] - struct Value { - #[serde(with = "super::log_id_string_serialisation")] - log_id: LogId, - } - - let val = Value { - log_id: LogId::new(1), - }; - let serialised = serde_json::to_string(&val).unwrap(); - assert_eq!(serialised, "{\"log_id\":\"1\"}".to_string()); - assert_eq!(val, serde_json::from_str(&serialised).unwrap()); - } - - #[test] - fn seq_num() { - #[derive(Serialize, Deserialize, PartialEq, Debug)] - struct Value { - #[serde(with = "super::seq_num_string_serialisation")] - seq_num: SeqNum, - } - - let val = Value { - seq_num: SeqNum::new(1).unwrap(), - }; - let serialised = serde_json::to_string(&val).unwrap(); - assert_eq!(serialised, "{\"seq_num\":\"1\"}".to_string()); - assert_eq!(val, serde_json::from_str(&serialised).unwrap()); - } -} diff --git a/aquadoggo/src/graphql/mod.rs b/aquadoggo/src/graphql/mod.rs index 0fbf3b530..73f89b55b 100644 --- a/aquadoggo/src/graphql/mod.rs +++ b/aquadoggo/src/graphql/mod.rs @@ -1,8 +1,9 @@ // SPDX-License-Identifier: AGPL-3.0-or-later -//! GraphQL types for handling client and replication requests. -pub(crate) mod client; -pub(crate) mod replication; +pub mod client; +pub mod pagination; +pub mod replication; +pub mod scalars; mod schema; pub use schema::{build_root_schema, MutationRoot, QueryRoot, RootSchema}; diff --git a/aquadoggo/src/graphql/pagination.rs b/aquadoggo/src/graphql/pagination.rs new file mode 100644 index 000000000..f3572ef92 --- /dev/null +++ b/aquadoggo/src/graphql/pagination.rs @@ -0,0 +1,23 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +use serde::{Deserialize, Serialize}; + +/// Generic pagination response of GraphQL connection API. +#[derive(Serialize, Deserialize)] +#[serde(rename_all = "camelCase")] +pub struct Paginated { + pub edges: Vec>, + pub page_info: PaginatedInfo, +} + +#[derive(Serialize, Deserialize)] +pub struct PaginatedEdges { + pub cursor: C, + pub node: T, +} + +#[derive(Serialize, Deserialize)] +#[serde(rename_all = "camelCase")] +pub struct PaginatedInfo { + pub has_next_page: bool, +} diff --git a/aquadoggo/src/graphql/replication/aliased_author.rs b/aquadoggo/src/graphql/replication/aliased_author.rs deleted file mode 100644 index 4dd08ddca..000000000 --- a/aquadoggo/src/graphql/replication/aliased_author.rs +++ /dev/null @@ -1,21 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later - -use async_graphql::*; - -use super::public_key::PublicKey; - -/// AliasedAuthor is one of either the public_key or an alias -/// -/// The intention of this is to reduce bandwidth when making requests by using a short "alias" -/// rather than the full author public_key -/// -/// To get an alias of an author, use the `author_aliases` method which will return this type. -/// -/// When using as an input to a query, exactly one of public_key or alias must be set otherwise it is an error. -#[derive(Debug, InputObject, SimpleObject)] -pub struct AliasedAuthor { - /// The author's public key - pub public_key: PublicKey, - /// The author alias - pub alias: ID, -} diff --git a/aquadoggo/src/graphql/replication/author.rs b/aquadoggo/src/graphql/replication/author.rs deleted file mode 100644 index 12bf53c59..000000000 --- a/aquadoggo/src/graphql/replication/author.rs +++ /dev/null @@ -1,46 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later - -use std::convert::TryFrom; - -use anyhow::{anyhow, Error}; -use async_graphql::*; - -use super::public_key::PublicKey; - -/// Either the `public_key` or the `alias` of that author. -#[derive(Debug, InputObject, Clone)] -pub struct Author { - /// The author's public key - pub public_key: Option, - /// The author alias - pub alias: Option, -} - -impl From for Author { - fn from(author: p2panda_rs::identity::Author) -> Self { - Self { - public_key: Some(PublicKey(author)), - alias: None, - } - } -} - -#[derive(Debug)] -pub enum AuthorOrAlias { - PublicKey(PublicKey), - Alias(ID), -} - -impl TryFrom for AuthorOrAlias { - type Error = Error; - - fn try_from(author: Author) -> Result { - match (author.public_key, author.alias) { - (Some(key), None) => Ok(AuthorOrAlias::PublicKey(key)), - (None, Some(alias)) => Ok(AuthorOrAlias::Alias(alias)), - _ => Err(anyhow!( - "Author must have either publicKey or alias set, but not both" - )), - } - } -} diff --git a/aquadoggo/src/graphql/replication/client.rs b/aquadoggo/src/graphql/replication/client.rs new file mode 100644 index 000000000..6efb2d490 --- /dev/null +++ b/aquadoggo/src/graphql/replication/client.rs @@ -0,0 +1,76 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +use std::convert::TryInto; + +use anyhow::{anyhow, Result}; +use gql_client::Client; +use p2panda_rs::entry::{LogId, SeqNum}; +use p2panda_rs::identity::Author; +use serde::{Deserialize, Serialize}; + +use crate::db::stores::StorageEntry; +use crate::graphql::pagination::Paginated; +use crate::graphql::replication::response::EncodedEntryAndOperation; +use crate::graphql::scalars; + +/// Response type of `entries_newer_than_seq_num` query. +#[derive(Serialize, Deserialize)] +#[serde(rename_all = "camelCase")] +struct Response { + entries_newer_than_seq_num: Paginated, +} + +/// Attempts to get entries newer than the given sequence number for a public key and log id. +pub async fn entries_newer_than_seq_num( + endpoint: &str, + log_id: &LogId, + public_key: &Author, + latest_seq_num: Option<&SeqNum>, +) -> Result> { + // @TODO: Currently this method does not use pagination, you will need to call this multiple + // times and eventually you we will get up to date. + let query = format!( + r#" + query {{ + entriesNewerThanSeqNum( + logId: "{}", + publicKey: "{}", + seqNum: {} + ) {{ + edges {{ + cursor + node {{ + entry + operation + }} + }} + pageInfo {{ + hasNextPage + }} + }} + }} + "#, + log_id.as_u64(), + public_key.as_str(), + latest_seq_num + .map(|num| num.as_u64().to_string()) + .unwrap_or_else(|| "null".into()), + ); + + // Make GraphQL request + let client = Client::new(endpoint); + let response: Response = client + .query_unwrap(&query) + .await + .map_err(|err| anyhow!("Replication query failed with error {}", err))?; + + // Convert results to correct return type + let entries = response + .entries_newer_than_seq_num + .edges + .into_iter() + .map(|edge| edge.node.try_into()) + .collect::>>()?; + + Ok(entries) +} diff --git a/aquadoggo/src/graphql/replication/client/mod.rs b/aquadoggo/src/graphql/replication/client/mod.rs deleted file mode 100644 index 1f2804373..000000000 --- a/aquadoggo/src/graphql/replication/client/mod.rs +++ /dev/null @@ -1,116 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later - -use anyhow::anyhow; -use graphql_client::{reqwest::post_graphql, GraphQLQuery}; -use p2panda_rs::entry::LogId as PandaLogId; -use p2panda_rs::entry::SeqNum as PandaSeqNum; -use p2panda_rs::identity::Author as PandaAuthor; -use reqwest::Client as ReqwestClient; -use reqwest::IntoUrl; - -use super::*; - -/// A graphql client for doing replication requests to another aquadoggo node. -#[derive(Debug)] -pub struct ReplicationClient { - reqwest_client: ReqwestClient, -} - -impl ReplicationClient { - /// Create a new client. - pub fn new() -> Self { - let reqwest_client = ReqwestClient::new(); - Self { reqwest_client } - } - - /// Attempts to get entries newer than the given sequence_number for an author + log_id. - /// - /// Currently does not use pagination, you will need to call this multiple times and eventually - /// you we will get up to date. - pub async fn get_entries_newer_than_seq( - &mut self, - url: U, - log_id: &PandaLogId, - author: &PandaAuthor, - sequence_number: Option<&PandaSeqNum>, - ) -> Result> { - let variables = - create_get_entries_newer_than_seq_request_variable(author, sequence_number, log_id); - - let result = - post_graphql::(&self.reqwest_client, url.clone(), variables) - .await?; - result - .data - .and_then(|data| data.get_entries_newer_than_seq.edges) - .map(convert_edges_to_storage_entries) - .ok_or_else(|| anyhow!("data wasn't in the format expected"))? - } -} - -fn convert_edges_to_storage_entries( - edges: Vec< - Option, - >, -) -> Result, Error> { - // Ooof, the auto generated types aren't very ergonomic to deal with. - let entries = edges - .into_iter() - .flatten() - .map(|edge| -> Result { - let entry_and_payload = EntryAndPayload { - entry: edge.node.entry, - payload: edge.node.payload, - }; - let storage_entry = entry_and_payload.try_into()?; - Ok(storage_entry) - }) - .collect::>>()?; - - Ok(entries) -} - -fn create_get_entries_newer_than_seq_request_variable( - author: &PandaAuthor, - sequence_number: Option<&PandaSeqNum>, - log_id: &PandaLogId, -) -> get_entries_newer_than_seq::Variables { - let author: Author = author.clone().into(); - // We have to do this manual type conversion because of this issue: - // https://github.com/graphql-rust/graphql-client/issues/386 - let author = get_entries_newer_than_seq::Author { - publicKey: author.public_key.clone(), - alias: author.alias.map(|id| id.0), - }; - let sequence_number = - sequence_number.map(|sequence_number| SequenceNumber(sequence_number.to_owned())); - let log_id = LogId(log_id.to_owned()); - - get_entries_newer_than_seq::Variables { - log_id, - author, - sequence_number, - first: None, - after: None, - } -} - -// The paths are relative to the directory where your `Cargo.toml` is located. Both json and the -// GraphQL schema language are supported as sources for the schema. -#[derive(GraphQLQuery, Clone, Copy, Debug)] -#[graphql( - schema_path = "src/graphql/replication/client/schema.graphql", - query_path = "src/graphql/replication/client/queries/get_entry_by_hash.graphql", - response_derives = "Debug" -)] -struct GetEntryByHash; - -// The paths are relative to the directory where your `Cargo.toml` is located. Both JSON and the -// GraphQL schema language are supported as sources for the schema. -#[derive(GraphQLQuery, Debug, Copy, Clone)] -#[graphql( - schema_path = "src/graphql/replication/client/schema.graphql", - query_path = "src/graphql/replication/client/queries/get_entries_newer_than_seq.graphql", - response_derives = "Debug" -)] -struct GetEntriesNewerThanSeq; diff --git a/aquadoggo/src/graphql/replication/client/queries/get_entries_newer_than_seq.graphql b/aquadoggo/src/graphql/replication/client/queries/get_entries_newer_than_seq.graphql deleted file mode 100644 index aacc2de4a..000000000 --- a/aquadoggo/src/graphql/replication/client/queries/get_entries_newer_than_seq.graphql +++ /dev/null @@ -1,14 +0,0 @@ -query GetEntriesNewerThanSeq($logId: LogId!, $author: Author!, $sequenceNumber: SequenceNumber, $first: Int, $after: String) { - getEntriesNewerThanSeq(logId: $logId, author: $author, sequenceNumber: $sequenceNumber, first: $first, after: $after){ - pageInfo { - hasNextPage - } - edges { - cursor, - node { - entry, - payload - } - } - } -} diff --git a/aquadoggo/src/graphql/replication/client/queries/get_entry_by_hash.graphql b/aquadoggo/src/graphql/replication/client/queries/get_entry_by_hash.graphql deleted file mode 100644 index b8e510d2d..000000000 --- a/aquadoggo/src/graphql/replication/client/queries/get_entry_by_hash.graphql +++ /dev/null @@ -1,6 +0,0 @@ -query GetEntryByHash($hash: EntryHash) { - entryByHash(hash: $hash){ - entry - } -} - diff --git a/aquadoggo/src/graphql/replication/client/schema.graphql b/aquadoggo/src/graphql/replication/client/schema.graphql deleted file mode 100644 index d58580804..000000000 --- a/aquadoggo/src/graphql/replication/client/schema.graphql +++ /dev/null @@ -1,167 +0,0 @@ -""" -AliasedAuthor is one of either the public_key or an alias - -The intention of this is to reduce bandwidth when making requests by using a short "alias" -rather than the full author public_key - -To get an alias of an author, use the `author_aliases` method which will return this type. - -When using as an input to a query, exactly one of public_key or alias must be set otherwise it is an error. -""" -type AliasedAuthor { - """ - The author's public key - """ - publicKey: PublicKey! - """ - The author alias - """ - alias: ID! -} -""" -Either the `public_key` or the `alias` of that author. -""" -input Author { - """ - The author's public key - """ - publicKey: PublicKey - """ - The author alias - """ - alias: ID -} -scalar Entry -""" -An entry with an optional payload -""" -type EntryAndPayload { - """ - Get the entry - """ - entry: Entry! - """ - Get the payload - """ - payload: Payload -} -type EntryAndPayloadConnection { - """ - Information to aid in pagination. - """ - pageInfo: PageInfo! - """ - A list of edges. - """ - edges: [EntryAndPayloadEdge] -} -""" -An edge in a connection. -""" -type EntryAndPayloadEdge { - """ - The item at the end of the edge - """ - node: EntryAndPayload! - """ - A cursor for use in pagination - """ - cursor: String! -} -type EntryArgsResponse { - logId: String! - seqNum: String! - backlink: String - skiplink: String -} -scalar EntryHash -scalar LogId -""" -All of the graphql mutation sub modules merged into one top level root -""" -type MutationRoot { - """ - Publish an entry using parameters obtained through `nextEntryArgs` query. - - Returns arguments for publishing the next entry in the same log. - """ - publishEntry(entry: String!, operation: String!): PublishEntryResponse! -} -""" -Information about pagination in a connection -""" -type PageInfo { - """ - When paginating backwards, are there more items? - """ - hasPreviousPage: Boolean! - """ - When paginating forwards, are there more items? - """ - hasNextPage: Boolean! - """ - When paginating backwards, the cursor to continue. - """ - startCursor: String - """ - When paginating forwards, the cursor to continue. - """ - endCursor: String -} -scalar Payload -scalar PublicKey -type PublishEntryResponse { - logId: String! - seqNum: String! - backlink: String - skiplink: String -} -""" -All of the graphql query sub modules merged into one top level root -""" -type QueryRoot { - """ - Get an entry by its hash - """ - entryByHash(hash: EntryHash!): SingleEntryAndPayload - """ - Get any entries that are newer than the provided sequence_number for a given author and - log_id - - If you don't provide sequence_number then get all entries starting at the first - """ - getEntriesNewerThanSeq(logId: LogId!, author: Author!, sequenceNumber: SequenceNumber, first: Int, after: String): EntryAndPayloadConnection! - """ - Get a single entry by its log_id, sequence_number and author - """ - entryByLogIdAndSequence(logId: LogId!, sequenceNumber: SequenceNumber!, author: Author!): SingleEntryAndPayload - """ - Get aliases of the provided `public_keys` that you can use in future requests to save - bandwidth. - """ - authorAliases(publicKeys: [PublicKey!]!): [AliasedAuthor!]! - """ - Return required arguments for publishing the next entry. - """ - nextEntryArgs(publicKey: String!, documentId: String): EntryArgsResponse! -} -scalar SequenceNumber -type SingleEntryAndPayload { - """ - The entry - """ - entry: Entry! - """ - The payload - """ - payload: Payload - """ - Get the certificate pool for this entry that can be used to verify the entry is valid - """ - certificatePool: [Entry!]! -} -schema { - query: QueryRoot - mutation: MutationRoot -} - diff --git a/aquadoggo/src/graphql/replication/context.rs b/aquadoggo/src/graphql/replication/context.rs deleted file mode 100644 index 49042b6fc..000000000 --- a/aquadoggo/src/graphql/replication/context.rs +++ /dev/null @@ -1,199 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later - -use anyhow::{anyhow, Result}; -use async_graphql::ID; -use lru::LruCache; -use mockall::automock; -use p2panda_rs::entry::decode_entry; -use p2panda_rs::entry::SeqNum; -use p2panda_rs::storage_provider::traits::EntryStore as EntryStoreTrait; - -use crate::db::stores::StorageEntry; - -pub use super::aliased_author::AliasedAuthor; -pub use super::public_key::PublicKey; -use super::AuthorOrAlias; -use super::Entry; -use super::EntryAndPayload; -use super::EntryHash; -use super::LogId; -use super::SequenceNumber; -use super::SingleEntryAndPayload; - -#[derive(Debug)] -pub struct ReplicationContext> { - author_aliases: LruCache, - next_alias: usize, - entry_store: EntryStore, -} - -#[automock] -#[allow(dead_code)] -impl> ReplicationContext { - pub fn new(author_aliases_cache_size: usize, entry_store: EntryStore) -> Self { - Self { - author_aliases: LruCache::new(author_aliases_cache_size), - next_alias: Default::default(), - entry_store, - } - } - - pub fn insert_author_aliases(&mut self, public_keys: Vec) -> Vec { - public_keys - .into_iter() - .map(|public_key| { - self.next_alias += 1; - self.author_aliases - .put(ID(self.next_alias.to_string()), public_key.clone()); - AliasedAuthor { - public_key, - alias: self.next_alias.into(), - } - }) - .collect() - } - - pub fn author_aliases_to_public_keys( - &mut self, - ids: Vec, - ) -> Result>> { - ids.into_iter() - .map(|id| { - let result = self.author_aliases.get(&id).cloned(); - Ok(result) - }) - .collect() - } - - pub async fn entry_by_log_id_and_seq_num<'a>( - &mut self, - log_id: LogId, - seq_num: SequenceNumber, - author_alias: AuthorOrAlias, - ) -> Result> { - let author = self.get_author(author_alias)?; - - let result = self - .entry_store - .get_entry_at_seq_num(&author.0, &log_id.0, &seq_num.0) - .await? - .map(|entry| entry.into()); - - Ok(result) - } - - pub async fn entry_by_hash<'a>( - &mut self, - hash: EntryHash, - ) -> Result> { - let result = self - .entry_store - .get_entry_by_hash(&hash.into()) - .await? - .map(|entry_row| entry_row.into()); - - Ok(result) - } - - pub async fn get_skiplinks<'a>(&mut self, entry: &Entry) -> Result> { - let author = entry.as_ref().author(); - let entry = decode_entry(entry.as_ref(), None)?; - let result = self - .entry_store - .get_certificate_pool(&author, entry.log_id(), entry.seq_num()) - .await? - .into_iter() - .map(|entry| entry.entry_signed().clone().into()) - .collect(); - - Ok(result) - } - - pub async fn get_entries_newer_than_seq_num( - &mut self, - log_id: LogId, - author: AuthorOrAlias, - seq_num: u64, - max_number_of_entries: usize, - ) -> Result> { - let author = self.get_author(author)?; - - // `get_paginated_log_entries` is inclusive of seq_num. Whereas our seq_num should not be - // included. So we add 1 to the the seq_num we were passed. - let seq_num = SeqNum::new(seq_num + 1)?; - let result = self - .entry_store - .get_paginated_log_entries(&author.0, &log_id.0, &seq_num, max_number_of_entries) - .await? - .into_iter() - .map(|entry| entry.into()) - .collect(); - - Ok(result) - } - - fn get_author(&mut self, author_alias: AuthorOrAlias) -> Result { - let author = match author_alias { - AuthorOrAlias::Alias(alias) => self - .author_aliases - .get(&alias) - .ok_or_else(|| { - anyhow!("author alias did not exist, you may need to re-alias your authors") - })? - .clone(), - AuthorOrAlias::PublicKey(public_key) => public_key, - }; - Ok(author) - } -} - -#[cfg(test)] -mod tests { - use std::convert::TryInto; - - use p2panda_rs::identity::Author; - - use crate::graphql::replication::{ - Author as GraphQLAuthor, LogId as GraphQLLogId, PublicKey, SequenceNumber, - }; - - use super::super::testing::MockEntryStore; - use super::ReplicationContext; - - // @TODO: Test author aliases - - #[tokio::test] - async fn entry_by_log_id_and_sequence() { - let expected_log_id = 123; - let expected_seq_num = 345u64; - let expected_author_string = - "7cf4f58a2d89e93313f2de99604a814ecea9800cf217b140e9c3a7ba59a5d982".to_string(); - - let log_id: GraphQLLogId = expected_log_id.into(); - let seq_num: SequenceNumber = expected_seq_num.try_into().unwrap(); - let author = Author::new(&expected_author_string).unwrap(); - let author_id = GraphQLAuthor { - alias: None, - public_key: Some(PublicKey(author)), - }; - - let mut mock_entry_store = MockEntryStore::new(); - mock_entry_store - .expect_get_entry_at_seq_num() - .withf(move |author, log_id, seq_num| { - author.as_str() == expected_author_string - && log_id.as_u64() == expected_log_id - && seq_num.as_u64() == expected_seq_num - }) - .times(1) - .returning(|_, _, _| Ok(None)); - - let mut context = ReplicationContext::new(1, mock_entry_store); - - let result = context - .entry_by_log_id_and_seq_num(log_id, seq_num, author_id.try_into().unwrap()) - .await; - - assert!(result.is_ok()); - } -} diff --git a/aquadoggo/src/graphql/replication/entry.rs b/aquadoggo/src/graphql/replication/entry.rs deleted file mode 100644 index 561908124..000000000 --- a/aquadoggo/src/graphql/replication/entry.rs +++ /dev/null @@ -1,29 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later - -use async_graphql::*; -use p2panda_rs::entry::EntrySigned as PandaEntry; -use serde::{Deserialize, Serialize}; - -/// A p2panda_rs entry encoded as a String -#[derive(Serialize, Deserialize, Clone, Debug)] -pub struct Entry(pub PandaEntry); - -impl AsRef for Entry { - fn as_ref(&self) -> &PandaEntry { - &self.0 - } -} - -scalar!(Entry); - -impl From for Value { - fn from(entry: Entry) -> Self { - async_graphql::ScalarType::to_value(&entry) - } -} - -impl From for Entry { - fn from(panda_entry: PandaEntry) -> Self { - Entry(panda_entry) - } -} diff --git a/aquadoggo/src/graphql/replication/entry_and_payload.rs b/aquadoggo/src/graphql/replication/entry_and_payload.rs deleted file mode 100644 index a33d0b1cc..000000000 --- a/aquadoggo/src/graphql/replication/entry_and_payload.rs +++ /dev/null @@ -1,43 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later - -use std::convert::TryFrom; - -use anyhow::Error; -use async_graphql::*; -use p2panda_rs::storage_provider::traits::AsStorageEntry; - -use crate::db::stores::StorageEntry; - -use super::payload::Payload; -use super::Entry; - -/// An entry with an optional payload -#[derive(SimpleObject, Debug)] -pub struct EntryAndPayload { - /// Get the entry - pub entry: Entry, - - /// Get the payload - pub payload: Option, -} - -impl From for EntryAndPayload { - fn from(entry_row: StorageEntry) -> Self { - let entry = Entry(entry_row.entry_signed().to_owned()); - let payload = entry_row - .operation_encoded() - .map(|encoded| Payload(encoded.to_owned())); - Self { entry, payload } - } -} - -impl TryFrom for StorageEntry { - type Error = Error; - - fn try_from(entry_and_payload: EntryAndPayload) -> Result { - let entry_signed = entry_and_payload.entry.0; - let operation_encoded = entry_and_payload.payload.unwrap().0; - let result = StorageEntry::new(&entry_signed, &operation_encoded)?; - Ok(result) - } -} diff --git a/aquadoggo/src/graphql/replication/entry_hash.rs b/aquadoggo/src/graphql/replication/entry_hash.rs deleted file mode 100644 index 2fc1f6dec..000000000 --- a/aquadoggo/src/graphql/replication/entry_hash.rs +++ /dev/null @@ -1,23 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later - -use async_graphql::*; -use p2panda_rs::hash::Hash; -use serde::{Deserialize, Serialize}; - -/// The p2panda_rs hash of an entry -#[derive(Serialize, Deserialize, Clone, Debug)] -pub struct EntryHash(Hash); - -impl From for Hash { - fn from(e: EntryHash) -> Self { - e.0 - } -} - -scalar!(EntryHash); - -impl From for Value { - fn from(entry: EntryHash) -> Self { - async_graphql::ScalarType::to_value(&entry) - } -} diff --git a/aquadoggo/src/graphql/replication/log_id.rs b/aquadoggo/src/graphql/replication/log_id.rs deleted file mode 100644 index df0ea4244..000000000 --- a/aquadoggo/src/graphql/replication/log_id.rs +++ /dev/null @@ -1,30 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later - -use async_graphql::*; -use p2panda_rs::entry::LogId as PandaLogId; -use serde::{Deserialize, Serialize}; - -/// The log id of a bamboo entry -#[derive(Clone, Copy, Debug, Serialize, Deserialize)] -pub struct LogId(pub PandaLogId); - -impl LogId { - #[allow(dead_code)] - pub fn as_u64(&self) -> u64 { - self.0.as_u64() - } -} - -impl From for LogId { - fn from(n: u64) -> Self { - Self(PandaLogId::new(n)) - } -} - -impl AsRef for LogId { - fn as_ref(&self) -> &PandaLogId { - &self.0 - } -} - -scalar!(LogId); diff --git a/aquadoggo/src/graphql/replication/mod.rs b/aquadoggo/src/graphql/replication/mod.rs index 348b024bc..10a51aa26 100644 --- a/aquadoggo/src/graphql/replication/mod.rs +++ b/aquadoggo/src/graphql/replication/mod.rs @@ -1,287 +1,9 @@ // SPDX-License-Identifier: AGPL-3.0-or-later -use std::convert::TryFrom; -use std::convert::TryInto; -use std::marker::PhantomData; -use std::sync::Arc; - -use anyhow::Error as AnyhowError; -use async_graphql::connection::{query, Connection, CursorType, Edge, EmptyFields}; -use async_graphql::Object; -use async_graphql::*; -use p2panda_rs::entry::decode_entry; -use p2panda_rs::storage_provider::traits::EntryStore as EntryStoreTrait; -use tokio::sync::Mutex; - -use crate::db::stores::StorageEntry; - -pub mod aliased_author; -pub mod author; pub mod client; -pub mod context; -pub mod entry; -pub mod entry_and_payload; -pub mod entry_hash; -pub mod log_id; -pub mod payload; -pub mod public_key; -pub mod sequence_number; -pub mod single_entry_and_payload; - -#[cfg(test)] -mod testing; - -pub use aliased_author::AliasedAuthor; -pub use author::{Author, AuthorOrAlias}; - -#[cfg(test)] -pub use context::MockReplicationContext; -pub use context::ReplicationContext; -pub use entry::Entry; -pub use entry_and_payload::EntryAndPayload; -pub use entry_hash::EntryHash; -pub use log_id::LogId; -pub use payload::Payload; -pub use public_key::PublicKey; -pub use sequence_number::SequenceNumber; -pub use single_entry_and_payload::SingleEntryAndPayload; - -#[derive(Debug, Default)] -/// The root graphql object for replication -pub struct ReplicationRoot { - entry_store: PhantomData, -} - -impl ReplicationRoot { - /// Create a new ReplicationRoot - pub fn new() -> Self { - Self { - entry_store: PhantomData::default(), - } - } -} - -#[Object] -impl + Sync + Send> - ReplicationRoot -{ - /// Get an entry by its hash - async fn entry_by_hash<'a>( - &self, - ctx: &Context<'a>, - hash: EntryHash, - ) -> Result> { - let ctx: &Arc>> = ctx.data()?; - let result = ctx.lock().await.entry_by_hash(hash).await?; - Ok(result) - } - - /// Get any entries that are newer than the provided sequence_number for a given author and - /// log_id - /// - /// If you don't provide sequence_number then get all entries starting at the first - async fn get_entries_newer_than_seq<'a>( - &self, - ctx: &Context<'a>, - log_id: LogId, - author: Author, - sequence_number: Option, - first: Option, - after: Option, - ) -> Result> { - let ctx: &Arc>> = ctx.data()?; - let author: AuthorOrAlias = author.try_into()?; - query( - after, - None, - first, - None, - |after: Option, _, first, _| async move { - let sequence_number = sequence_number.map(|seq| seq.as_u64()).unwrap_or_else(|| 0); - - // Add the sequence_number to the after cursor to get the starting sequence number. - let start: u64 = sequence_number + after.map(|a| a.as_u64()).unwrap_or(0); - - // Limit the maximum number of entries to 10k, set a default value of 10 - let max_number_of_entries = first.map(|n| n.clamp(0, 10000)).unwrap_or(10); - - let edges = ctx - .lock() - .await - .get_entries_newer_than_seq_num(log_id, author, start, max_number_of_entries) - .await? - .into_iter() - .map(|entry| { - let decoded = decode_entry(entry.entry.as_ref(), None).unwrap(); - let sequence_number = SequenceNumber(*decoded.seq_num()); - Edge::new(sequence_number, entry) - }) - .collect::>(); - - let has_next_page = edges.len() == max_number_of_entries; - let mut connection = Connection::new(false, has_next_page); - - connection.append(edges); - - Result::<_, Error>::Ok(connection) - }, - ) - .await - } - - /// Get a single entry by its log_id, sequence_number and author - async fn entry_by_log_id_and_sequence<'a>( - &self, - ctx: &Context<'a>, - log_id: LogId, - sequence_number: SequenceNumber, - author: Author, - ) -> Result> { - let ctx: &Arc>> = ctx.data()?; - let author: AuthorOrAlias = author.try_into()?; - let result = ctx - .lock() - .await - .entry_by_log_id_and_seq_num(log_id, sequence_number, author) - .await?; - - Ok(result) - } - - /// Get aliases of the provided `public_keys` that you can use in future requests to save - /// bandwidth. - // Maybe this should be a mutation - async fn author_aliases<'a>( - &self, - ctx: &Context<'a>, - public_keys: Vec, - ) -> Result> { - let ctx: &Arc>> = ctx.data()?; - let result = ctx.lock().await.insert_author_aliases(public_keys); - - Ok(result) - } -} - -impl CursorType for SequenceNumber { - type Error = AnyhowError; - - fn decode_cursor(s: &str) -> Result { - let num: u64 = s.parse()?; - let result = SequenceNumber::try_from(num)?; - Ok(result) - } - - fn encode_cursor(&self) -> String { - self.as_u64().to_string() - } -} - -#[cfg(test)] -mod tests { - use std::convert::TryFrom; - use std::sync::Arc; - - use async_graphql::{EmptyMutation, EmptySubscription, Request, Schema}; - use p2panda_rs::identity::Author; - use rstest::rstest; - use tokio::sync::Mutex; - - use crate::db::stores::test_utils::{ - populate_test_db, with_db_manager_teardown, PopulateDatabaseConfig, TestDatabaseManager, - }; - use crate::SqlStorage; - - use super::{ReplicationContext, ReplicationRoot}; - - #[rstest] - #[case::default_params(20, None, None, None, true, 10)] - #[case::no_edges_or_next_page(10, Some(10), Some(5), None, false, 0)] - #[case::some_edges_no_next_page(14, Some(10), Some(5), None, false, 4)] - #[case::edges_and_next_page(15, Some(10), Some(5), None, true, 5)] - #[case::edges_and_next_page_again(16, Some(10), Some(5), None, true, 5)] - fn get_entries_newer_than_seq_cursor( - #[case] entries_in_log: usize, - #[case] sequence_number: Option, - #[case] first: Option, - #[case] after: Option, - #[case] expected_has_next_page: bool, - #[case] expected_edges: usize, - ) { - with_db_manager_teardown(move |db_manager: TestDatabaseManager| async move { - // Build and populate Billie's db - let mut billie_db = db_manager.create("sqlite::memory:").await; - - populate_test_db( - &mut billie_db, - &PopulateDatabaseConfig { - no_of_entries: entries_in_log, - no_of_logs: 1, - no_of_authors: 1, - ..Default::default() - }, - ) - .await; - - // Construct the replication context, root and graphql schema. - let replication_context: ReplicationContext = - ReplicationContext::new(1, billie_db.store.clone()); - let replication_root = ReplicationRoot::::new(); - let schema = Schema::build(replication_root, EmptyMutation, EmptySubscription) - .data(Arc::new(Mutex::new(replication_context))) - .finish(); - - // Collect args needed for the query. - let public_key = billie_db - .test_data - .key_pairs - .first() - .unwrap() - .public_key() - .to_owned(); - - let author = Author::try_from(public_key).unwrap(); - let author_str: String = author.as_str().into(); - let log_id = 1u64; - // Optional values are encpoded as `null` - let sequence_number = sequence_number - .map(|seq_num| seq_num.to_string()) - .unwrap_or_else(|| "null".to_string()); - let first = first - .map(|seq_num| seq_num.to_string()) - .unwrap_or_else(|| "null".to_string()); - let after = after - .map(|seq_num| format!("\"{}\"", seq_num)) - .unwrap_or_else(|| "null".to_string()); - - // Construct the query. - let gql_query = format!( - " - query{{ - getEntriesNewerThanSeq(logId: {}, author: {{publicKey: \"{}\" }}, sequenceNumber: {}, first: {}, after: {} ){{ - edges {{ - cursor - }} - pageInfo {{ - hasNextPage - }} - }} - }}", - log_id, author_str, sequence_number, first, after - ); - - // Make the query. - let result = schema.execute(Request::new(gql_query)).await; - - // Check that we get the Ok returned from get_entries_newer_than_seq - assert!(result.is_ok()); +mod query; +mod response; - // Assert the returned hasNextPage and number of edges returned is what we expect. - let json_value = result.data.into_json().unwrap(); - let edges = &json_value["getEntriesNewerThanSeq"]["edges"]; - assert_eq!(edges.as_array().unwrap().len(), expected_edges); - let has_next_page = &json_value["getEntriesNewerThanSeq"]["pageInfo"]["hasNextPage"]; - assert_eq!(has_next_page.as_bool().unwrap(), expected_has_next_page); - }) - } -} +pub use client::entries_newer_than_seq_num; +pub use query::ReplicationRoot; +pub use response::EncodedEntryAndOperation; diff --git a/aquadoggo/src/graphql/replication/payload.rs b/aquadoggo/src/graphql/replication/payload.rs deleted file mode 100644 index 0e31f8ca9..000000000 --- a/aquadoggo/src/graphql/replication/payload.rs +++ /dev/null @@ -1,11 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later - -use async_graphql::*; -use p2panda_rs::operation::OperationEncoded; -use serde::{Deserialize, Serialize}; - -/// The payload of an entry -#[derive(Clone, Debug, Serialize, Deserialize)] -pub struct Payload(pub OperationEncoded); - -scalar!(Payload); diff --git a/aquadoggo/src/graphql/replication/public_key.rs b/aquadoggo/src/graphql/replication/public_key.rs deleted file mode 100644 index 78aabf77b..000000000 --- a/aquadoggo/src/graphql/replication/public_key.rs +++ /dev/null @@ -1,11 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later - -use async_graphql::*; -use p2panda_rs::identity::Author; -use serde::{Deserialize, Serialize}; - -/// The public key of an entry -#[derive(Clone, Debug, Serialize, Deserialize)] -pub struct PublicKey(pub Author); - -scalar!(PublicKey); diff --git a/aquadoggo/src/graphql/replication/query.rs b/aquadoggo/src/graphql/replication/query.rs new file mode 100644 index 000000000..a5cfd3d7f --- /dev/null +++ b/aquadoggo/src/graphql/replication/query.rs @@ -0,0 +1,415 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +use anyhow::Error; +use async_graphql::connection::{query, Connection, CursorType, Edge, EmptyFields}; +use async_graphql::{Context, Object, Result}; +use p2panda_rs::entry::SeqNum; +use p2panda_rs::storage_provider::traits::{AsStorageEntry, EntryStore}; + +use crate::db::provider::SqlStorage; +use crate::graphql::replication::response::EncodedEntryAndOperation; +use crate::graphql::scalars; + +/// Maximum number of items per paginated query. +const MAX_PAGINATION_SIZE: usize = 10_000; + +/// Default number of items per paginated query. +const DEFAULT_PAGINATION_SIZE: usize = 10; + +/// Response type for paginated queries. +type ConnectionResult = + Connection; + +/// GraphQL queries for the Replication API. +#[derive(Default, Debug, Copy, Clone)] +pub struct ReplicationRoot; + +#[Object] +impl ReplicationRoot { + /// Get a single entry by its hash. + async fn entry_by_hash<'a>( + &self, + ctx: &Context<'a>, + hash: scalars::EntryHash, + ) -> Result { + let store = ctx.data::()?; + let result = store.get_entry_by_hash(&hash.clone().into()).await?; + + match result { + Some(inner) => Ok(EncodedEntryAndOperation::from(inner)), + None => Err(async_graphql::Error::new(format!( + "Entry with hash {} could not be found", + hash + ))), + } + } + + /// Get a single entry by its log id, sequence number and public key. + async fn entry_by_log_id_and_seq_num<'a>( + &self, + ctx: &Context<'a>, + #[graphql(name = "logId", desc = "Log id of entry")] log_id: scalars::LogId, + #[graphql(name = "seqNum", desc = "Sequence number of entry")] seq_num: scalars::SeqNum, + #[graphql(name = "publicKey", desc = "Public key of the entry author")] + public_key: scalars::PublicKey, + ) -> Result { + let store = ctx.data::()?; + + let result = store + .get_entry_at_seq_num(&public_key.clone().into(), &log_id.into(), &seq_num.into()) + .await?; + + match result { + Some(inner) => Ok(EncodedEntryAndOperation::from(inner)), + None => Err(async_graphql::Error::new(format!( + "Entry with log id {}, sequence number {} and public key {} could not be found", + log_id, seq_num, public_key, + ))), + } + } + + /// Get any entries that are newer than the provided sequence number for a given public key and + /// log id. + async fn entries_newer_than_seq_num<'a>( + &self, + ctx: &Context<'a>, + #[graphql(name = "logId", desc = "Log id of entries")] log_id: scalars::LogId, + #[graphql(name = "publicKey", desc = "Public key of the author")] + public_key: scalars::PublicKey, + #[graphql( + name = "seqNum", + desc = "Query entries starting from this sequence number" + )] + seq_num: Option, + first: Option, + after: Option, + ) -> Result { + let store = ctx.data::()?; + + query( + after, + None, + first, + None, + |after: Option, _, first, _| async move { + // Add `seq_num` to the `after` cursor to get starting sequence number + let seq_num = seq_num.map(|seq| seq.as_u64()).unwrap_or_else(|| 0); + let start: u64 = seq_num + after.map(|a| a.as_u64()).unwrap_or_else(|| 0); + + // `get_paginated_log_entries` is inclusive of seq_num. Whereas our seq_num should + // not be included. So we add 1 to the the seq_num we were passed + let start_seq_num = SeqNum::new(start + 1)?; + + // Limit the maximum number of entries and set a default + let max_number_of_entries = first + .map(|n| n.clamp(0, MAX_PAGINATION_SIZE)) + .unwrap_or(DEFAULT_PAGINATION_SIZE); + + let edges = store + .get_paginated_log_entries( + &public_key.into(), + &log_id.into(), + &start_seq_num, + max_number_of_entries, + ) + .await? + .into_iter() + .map(|entry_and_operation| { + // Every pagination edge represents an entry and operation with the + // sequence number as the pagination cursor + Edge::new( + entry_and_operation.seq_num().to_owned().into(), + entry_and_operation.into(), + ) + }) + .collect::>(); + + // @TODO: This returns true even when there is nothing on the next page, exactly + // when the last page has the maximum number of elements + let has_next_page = edges.len() == max_number_of_entries; + let has_previous_page = start > 0; + + let mut connection = Connection::new(has_previous_page, has_next_page); + connection.append(edges); + + Result::<_, Error>::Ok(connection) + }, + ) + .await + } +} + +/// Use sequence numbers as cursor to paginate entry queries. +impl CursorType for scalars::SeqNum { + type Error = Error; + + fn decode_cursor(str: &str) -> Result { + Ok(str.parse()?) + } + + fn encode_cursor(&self) -> String { + self.to_string() + } +} + +#[cfg(test)] +mod tests { + use std::convert::{TryFrom, TryInto}; + + use async_graphql::{EmptyMutation, EmptySubscription, Request, Schema}; + use p2panda_rs::hash::Hash; + use p2panda_rs::identity::Author; + use p2panda_rs::test_utils::fixtures::random_hash; + use rstest::rstest; + + use crate::db::stores::test_utils::{ + populate_test_db, test_db, with_db_manager_teardown, PopulateDatabaseConfig, TestDatabase, + TestDatabaseManager, TestDatabaseRunner, + }; + + use super::ReplicationRoot; + + #[rstest] + fn entry_by_hash( + #[from(test_db)] + #[with(1, 1, 1)] + runner: TestDatabaseRunner, + ) { + runner.with_db_teardown(|db: TestDatabase| async move { + let replication_root = ReplicationRoot::default(); + let schema = Schema::build(replication_root, EmptyMutation, EmptySubscription) + .data(db.store) + .finish(); + + // The test runner creates a test entry for us. The entry hash is automatically the + // document id as it contains the CREATE operation + let entry_hash_str = db.test_data.documents.first().unwrap().as_str(); + + // Construct the query + let gql_query = format!( + r#" + query {{ + entryByHash(hash: "{}") {{ + entry + operation + }} + }} + "#, + entry_hash_str + ); + + // Make the query + let result = schema.execute(Request::new(gql_query.clone())).await; + + // Check that query was successful + assert!( + result.is_ok(), + "Query: {} \nResult: {:?}", + gql_query, + result + ); + }); + } + + #[rstest] + fn entry_by_hash_not_found( + #[from(test_db)] runner: TestDatabaseRunner, + #[from(random_hash)] random_hash: Hash, + ) { + runner.with_db_teardown(|db: TestDatabase| async move { + let replication_root = ReplicationRoot::default(); + let schema = Schema::build(replication_root, EmptyMutation, EmptySubscription) + .data(db.store) + .finish(); + + let gql_query = format!( + r#"query {{ entryByHash(hash: "{}") {{ entry operation }} }}"#, + random_hash.as_str() + ); + + // Make sure that query returns an error as entry was not found + let result = schema.execute(Request::new(gql_query.clone())).await; + assert!(result.is_err(), "{:?}", result); + }); + } + + #[rstest] + fn entry_by_log_id_and_seq_num( + #[from(test_db)] + #[with(1, 1, 1)] + runner: TestDatabaseRunner, + ) { + runner.with_db_teardown(|db: TestDatabase| async move { + let replication_root = ReplicationRoot::default(); + let schema = Schema::build(replication_root, EmptyMutation, EmptySubscription) + .data(db.store) + .finish(); + + // The test runner creates a test entry for us, we can retreive the public key from the + // author + let public_key: Author = db + .test_data + .key_pairs + .first() + .unwrap() + .public_key() + .to_owned() + .try_into() + .unwrap(); + + // Construct the query + let gql_query = format!( + r#" + query {{ + entryByLogIdAndSeqNum(logId: "{}", seqNum: "{}", publicKey: "{}") {{ + entry + operation + }} + }} + "#, + 1, + 1, + public_key.as_str() + ); + + // Make the query + let result = schema.execute(Request::new(gql_query.clone())).await; + + // Check that query was successful + assert!( + result.is_ok(), + "Query: {} \nResult: {:?}", + gql_query, + result + ); + }); + } + + #[rstest] + fn entry_by_log_id_and_seq_num_not_found(#[from(test_db)] runner: TestDatabaseRunner) { + runner.with_db_teardown(|db: TestDatabase| async move { + let replication_root = ReplicationRoot::default(); + let schema = Schema::build(replication_root, EmptyMutation, EmptySubscription) + .data(db.store) + .finish(); + + // Make a request with garbage data which will not exist in our test database + let gql_query = format!( + r#"query {{ + entryByLogIdAndSeqNum(logId: "{}", seqNum: "{}", publicKey: "{}") {{ + entry + operation + }} + }}"#, + 2, 12, "64977654a6274f6157f2c3efe27ed89037c344f3af5c499e410946f50e25b6d7" + ); + + // Make sure that query returns an error as entry was not found + let result = schema.execute(Request::new(gql_query.clone())).await; + assert!(result.is_err(), "{:?}", result); + }); + } + + #[rstest] + #[case::default_params(20, None, None, None, true, 10)] + #[case::no_edges_or_next_page(10, Some(10), Some(5), None, false, 0)] + #[case::some_edges_no_next_page(14, Some(10), Some(5), None, false, 4)] + #[case::edges_and_next_page(15, Some(10), Some(5), None, true, 5)] + #[case::edges_and_next_page_again(16, Some(10), Some(5), None, true, 5)] + fn entries_newer_than_seq_num_cursor( + #[case] entries_in_log: usize, + #[case] sequence_number: Option, + #[case] first: Option, + #[case] after: Option, + #[case] expected_has_next_page: bool, + #[case] expected_edges: usize, + ) { + with_db_manager_teardown(move |db_manager: TestDatabaseManager| async move { + // Build and populate Billie's database + let mut billie_db = db_manager.create("sqlite::memory:").await; + + populate_test_db( + &mut billie_db, + &PopulateDatabaseConfig { + no_of_entries: entries_in_log, + no_of_logs: 1, + no_of_authors: 1, + ..Default::default() + }, + ) + .await; + + // Construct the replication context, root and graphql schema + let replication_root = ReplicationRoot::default(); + let schema = Schema::build(replication_root, EmptyMutation, EmptySubscription) + .data(billie_db.store) + .finish(); + + // Get public key from author of generated test data + let public_key: String = { + let key_from_db = billie_db + .test_data + .key_pairs + .first() + .unwrap() + .public_key() + .to_owned(); + + let author = Author::try_from(key_from_db).unwrap(); + author.as_str().into() + }; + + // Test data has been written to first log + let log_id = 1u64; + + // Turn parameters into strings by wrapping them around quotation marks when existing, + // otherwise give them "null" value + let seq_num = sequence_number + .map(|num| format!("\"{}\"", num)) + .unwrap_or_else(|| "null".to_string()); + let first = first + .map(|num| num.to_string()) + .unwrap_or_else(|| "null".to_string()); + let after = after + .map(|seq_num| format!("\"{}\"", seq_num)) + .unwrap_or_else(|| "null".to_string()); + + // Construct the query + let gql_query = format!( + r#" + query {{ + entriesNewerThanSeqNum( + logId: "{}", + publicKey: "{}", + seqNum: {}, + first: {}, + after: {} + ) {{ + edges {{ + cursor + }} + pageInfo {{ + hasNextPage + }} + }} + }} + "#, + log_id, public_key, seq_num, first, after + ); + + // Make the query + let result = schema.execute(Request::new(gql_query.clone())).await; + + // Check that we get the Ok returned from entries_newer_than_seq + assert!(result.is_ok(), "{:?}", result); + + // Assert the returned hasNextPage and number of edges returned is what we expect + let json_value = result.data.into_json().unwrap(); + let edges = &json_value["entriesNewerThanSeqNum"]["edges"]; + assert_eq!(edges.as_array().unwrap().len(), expected_edges); + + let has_next_page = &json_value["entriesNewerThanSeqNum"]["pageInfo"]["hasNextPage"]; + assert_eq!(has_next_page.as_bool().unwrap(), expected_has_next_page); + }) + } +} diff --git a/aquadoggo/src/graphql/replication/response.rs b/aquadoggo/src/graphql/replication/response.rs new file mode 100644 index 000000000..3a9eb0684 --- /dev/null +++ b/aquadoggo/src/graphql/replication/response.rs @@ -0,0 +1,156 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +use std::convert::TryFrom; + +use anyhow::{anyhow, Error}; +use async_graphql::{ComplexObject, Context, SimpleObject}; +use p2panda_rs::entry::{decode_entry, EntrySigned}; +use p2panda_rs::storage_provider::traits::{AsStorageEntry, EntryStore}; +use serde::{Deserialize, Serialize}; + +use crate::db::provider::SqlStorage; +use crate::db::stores::StorageEntry; +use crate::graphql::scalars; + +/// Encoded and signed entry with its regarding encoded operation payload. +#[derive(SimpleObject, Serialize, Deserialize, Debug, Eq, PartialEq)] +#[graphql(complex)] +pub struct EncodedEntryAndOperation { + /// Signed and encoded bamboo entry. + pub entry: scalars::EncodedEntry, + + /// p2panda operation, CBOR bytes encoded as hexadecimal string. + pub operation: Option, +} + +#[ComplexObject] +impl EncodedEntryAndOperation { + /// Get the certificate pool for this entry that can be used to verify the entry is valid. + async fn certificate_pool<'a>( + &self, + ctx: &Context<'a>, + ) -> async_graphql::Result> { + let store = ctx.data::()?; + + // Decode entry + let entry_encoded: EntrySigned = self.entry.clone().into(); + let entry = decode_entry(&entry_encoded, None)?; + + // Load certificate pool from database + let result = store + .get_certificate_pool(&entry_encoded.author(), entry.log_id(), entry.seq_num()) + .await?; + + let entries = result + .into_iter() + .map(|entry| entry.entry_signed().clone().into()) + .collect(); + + Ok(entries) + } +} + +impl From for EncodedEntryAndOperation { + fn from(entry_row: StorageEntry) -> Self { + let entry = entry_row.entry_signed().to_owned().into(); + let operation = entry_row.operation_encoded().map(|op| op.to_owned().into()); + Self { entry, operation } + } +} + +impl TryFrom for StorageEntry { + type Error = Error; + + fn try_from(encoded: EncodedEntryAndOperation) -> anyhow::Result { + let operation = encoded + .operation + .ok_or_else(|| anyhow!("Storage entry requires operation to be given"))?; + + Ok(StorageEntry::new(&encoded.entry.into(), &operation.into())?) + } +} + +#[cfg(test)] +mod tests { + use std::convert::TryInto; + + use async_graphql::{EmptyMutation, EmptySubscription, Request, Schema}; + use bamboo_rs_core_ed25519_yasmf::verify_batch; + use p2panda_rs::entry::{EntrySigned, LogId}; + use p2panda_rs::identity::Author; + use p2panda_rs::storage_provider::traits::{AsStorageEntry, EntryStore}; + use rstest::rstest; + + use crate::db::stores::test_utils::{test_db, TestDatabase, TestDatabaseRunner}; + use crate::graphql::replication::ReplicationRoot; + + #[rstest] + fn validate_with_certificate_pool( + #[from(test_db)] + #[with(100, 1, 1)] + runner: TestDatabaseRunner, + ) { + runner.with_db_teardown(|db: TestDatabase| async move { + let replication_root = ReplicationRoot::default(); + let schema = Schema::build(replication_root, EmptyMutation, EmptySubscription) + .data(db.store.clone()) + .finish(); + + // Retreive last entry of author from test database + let author: Author = db + .test_data + .key_pairs + .first() + .unwrap() + .public_key() + .to_owned() + .try_into() + .unwrap(); + + let latest_entry_hash = db + .store + .get_latest_entry(&author, &LogId::new(1)) + .await + .unwrap() + .unwrap() + .hash(); + + // Make GraphQL query + let gql_query = format!( + r#" + query {{ + entryByHash(hash: "{}") {{ + certificatePool + }} + }} + "#, + latest_entry_hash.as_str() + ); + + let result = schema.execute(Request::new(gql_query.clone())).await; + assert!(result.is_ok(), "{:?}", result); + + // Extract data from result + let json = result.data.into_json().unwrap(); + let entries = json + .get("entryByHash") + .unwrap() + .get("certificatePool") + .unwrap() + .as_array() + .unwrap(); + + // Prepare entries and batch-validate them + let entries_to_verify: Vec<(Vec, Option>)> = entries + .iter() + .map(|entry| { + let entry = EntrySigned::new(entry.as_str().unwrap()).unwrap(); + (entry.to_bytes(), None) + }) + .collect(); + + // Make sure we can validate single entry based on the certificate pool + assert!(verify_batch(&entries_to_verify).is_ok()); + }); + } +} diff --git a/aquadoggo/src/graphql/replication/sequence_number.rs b/aquadoggo/src/graphql/replication/sequence_number.rs deleted file mode 100644 index b240e66f9..000000000 --- a/aquadoggo/src/graphql/replication/sequence_number.rs +++ /dev/null @@ -1,43 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later - -use std::convert::TryFrom; - -use anyhow::Result; -use async_graphql::*; -use p2panda_rs::entry::{SeqNum as PandaSeqNum, SeqNumError}; -use serde::{Deserialize, Serialize}; - -/// The sequence number of an entry -#[derive(Clone, Copy, Debug, Serialize, Deserialize)] -pub struct SequenceNumber(pub PandaSeqNum); - -impl SequenceNumber { - #[allow(dead_code)] - pub fn new(seq: u64) -> Result { - let panda_seq_num = PandaSeqNum::new(seq)?; - Ok(Self(panda_seq_num)) - } -} - -impl SequenceNumber { - pub fn as_u64(self) -> u64 { - self.0.as_u64() - } -} - -impl AsRef for SequenceNumber { - fn as_ref(&self) -> &PandaSeqNum { - &self.0 - } -} - -impl TryFrom for SequenceNumber { - type Error = SeqNumError; - - fn try_from(value: u64) -> Result { - let seq_num = PandaSeqNum::new(value)?; - Ok(Self(seq_num)) - } -} - -scalar!(SequenceNumber); diff --git a/aquadoggo/src/graphql/replication/single_entry_and_payload.rs b/aquadoggo/src/graphql/replication/single_entry_and_payload.rs deleted file mode 100644 index f55882b2e..000000000 --- a/aquadoggo/src/graphql/replication/single_entry_and_payload.rs +++ /dev/null @@ -1,50 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-or-later - -use std::sync::Arc; - -use async_graphql::*; -use tokio::sync::Mutex; - -use crate::db::provider::SqlStorage; -use crate::db::stores::StorageEntry; -use crate::graphql::replication::context::ReplicationContext; - -use super::payload::Payload; -use super::Entry; - -/// A p2panda entry with optional payload and the certificate pool required to verify it. -#[derive(Debug)] -pub struct SingleEntryAndPayload { - pub entry: Entry, - pub payload: Option, -} - -#[Object] -impl SingleEntryAndPayload { - /// The entry - async fn entry(&self) -> &Entry { - &self.entry - } - - /// The payload - async fn payload(&self) -> Option<&Payload> { - self.payload.as_ref() - } - - /// Get the certificate pool for this entry that can be used to verify the entry is valid. - async fn certificate_pool<'a>(&self, ctx: &Context<'a>) -> Result> { - let ctx: &Arc>> = ctx.data()?; - let result = ctx.lock().await.get_skiplinks(&self.entry).await?; - Ok(result) - } -} - -impl From for SingleEntryAndPayload { - fn from(entry_and_payload: StorageEntry) -> Self { - let entry = Entry(entry_and_payload.entry_signed().to_owned()); - let payload = entry_and_payload - .operation_encoded() - .map(|encoded| Payload(encoded.to_owned())); - Self { entry, payload } - } -} diff --git a/aquadoggo/src/graphql/replication/testing/mod.rs b/aquadoggo/src/graphql/replication/testing/mod.rs deleted file mode 100644 index 08ea673a8..000000000 --- a/aquadoggo/src/graphql/replication/testing/mod.rs +++ /dev/null @@ -1,58 +0,0 @@ -use async_trait::async_trait; -use mockall::mock; - -use p2panda_rs::entry::{LogId, SeqNum}; -use p2panda_rs::hash::Hash; -use p2panda_rs::identity::Author; -use p2panda_rs::schema::SchemaId; -use p2panda_rs::storage_provider::errors::EntryStorageError; -use p2panda_rs::storage_provider::traits::EntryStore; - -use crate::db::stores::StorageEntry; - -mock! { - pub EntryStore {} - #[async_trait] - impl EntryStore for EntryStore { - async fn insert_entry(&self, value: StorageEntry) -> Result<(), EntryStorageError>; - - async fn get_entry_at_seq_num( - &self, - author: &Author, - log_id: &LogId, - seq_num: &SeqNum, - ) -> Result, EntryStorageError>; - - async fn get_entry_by_hash( - &self, - hash: &Hash, - ) -> Result, EntryStorageError>; - - async fn get_latest_entry( - &self, - author: &Author, - log_id: &LogId, - ) -> Result, EntryStorageError>; - - async fn get_entries_by_schema( - &self, - schema: &SchemaId, - ) -> Result, EntryStorageError>; - - async fn get_paginated_log_entries( - &self, - author: &Author, - log_id: &LogId, - seq_num: &SeqNum, - max_number_of_entries: usize, - ) -> Result, EntryStorageError>; - - async fn get_certificate_pool( - &self, - author_id: &Author, - log_id: &LogId, - seq_num: &SeqNum, - ) -> Result, EntryStorageError>; - - } -} diff --git a/aquadoggo/src/graphql/scalars/document_id.rs b/aquadoggo/src/graphql/scalars/document_id.rs new file mode 100644 index 000000000..58544c652 --- /dev/null +++ b/aquadoggo/src/graphql/scalars/document_id.rs @@ -0,0 +1,30 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +use std::fmt::Display; + +use async_graphql::scalar; +use serde::{Deserialize, Serialize}; + +/// Id of a p2panda document. +#[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize)] +pub struct DocumentId(p2panda_rs::document::DocumentId); + +impl From for DocumentId { + fn from(document_id: p2panda_rs::document::DocumentId) -> Self { + Self(document_id) + } +} + +impl From for p2panda_rs::document::DocumentId { + fn from(document_id: DocumentId) -> p2panda_rs::document::DocumentId { + document_id.0 + } +} + +impl Display for DocumentId { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0) + } +} + +scalar!(DocumentId); diff --git a/aquadoggo/src/graphql/scalars/encoded_entry.rs b/aquadoggo/src/graphql/scalars/encoded_entry.rs new file mode 100644 index 000000000..2647be347 --- /dev/null +++ b/aquadoggo/src/graphql/scalars/encoded_entry.rs @@ -0,0 +1,29 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +use async_graphql::{scalar, Value}; +use p2panda_rs::entry::EntrySigned; +use serde::{Deserialize, Serialize}; + +/// Signed bamboo entry, encoded as a hexadecimal string. +#[derive(Serialize, Deserialize, Eq, PartialEq, Clone, Debug)] +pub struct EncodedEntry(EntrySigned); + +impl From for EncodedEntry { + fn from(entry: EntrySigned) -> Self { + Self(entry) + } +} + +impl From for EntrySigned { + fn from(entry: EncodedEntry) -> EntrySigned { + entry.0 + } +} + +impl From for Value { + fn from(entry: EncodedEntry) -> Self { + async_graphql::ScalarType::to_value(&entry) + } +} + +scalar!(EncodedEntry); diff --git a/aquadoggo/src/graphql/scalars/encoded_operation.rs b/aquadoggo/src/graphql/scalars/encoded_operation.rs new file mode 100644 index 000000000..299c40cfb --- /dev/null +++ b/aquadoggo/src/graphql/scalars/encoded_operation.rs @@ -0,0 +1,23 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +use async_graphql::scalar; +use p2panda_rs::operation::OperationEncoded; +use serde::{Deserialize, Serialize}; + +/// Entry payload and p2panda operation, CBOR bytes encoded as a hexadecimal string. +#[derive(Serialize, Deserialize, Eq, PartialEq, Clone, Debug)] +pub struct EncodedOperation(OperationEncoded); + +impl From for EncodedOperation { + fn from(operation: OperationEncoded) -> Self { + Self(operation) + } +} + +impl From for OperationEncoded { + fn from(operation: EncodedOperation) -> OperationEncoded { + operation.0 + } +} + +scalar!(EncodedOperation); diff --git a/aquadoggo/src/graphql/scalars/entry_hash.rs b/aquadoggo/src/graphql/scalars/entry_hash.rs new file mode 100644 index 000000000..171816690 --- /dev/null +++ b/aquadoggo/src/graphql/scalars/entry_hash.rs @@ -0,0 +1,50 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +use std::fmt::Display; + +use async_graphql::{scalar, Value}; +use p2panda_rs::hash::Hash; +use serde::{Deserialize, Serialize}; + +/// Hash of a signed bamboo entry. +#[derive(Serialize, Deserialize, Eq, PartialEq, Clone, Debug)] +pub struct EntryHash(Hash); + +impl From for Hash { + fn from(hash: EntryHash) -> Self { + hash.0 + } +} + +impl From for EntryHash { + fn from(hash: Hash) -> Self { + Self(hash) + } +} + +impl From for Value { + fn from(entry: EntryHash) -> Self { + async_graphql::ScalarType::to_value(&entry) + } +} + +impl Display for EntryHash { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0) + } +} + +scalar!(EntryHash); + +#[cfg(test)] +mod tests { + use p2panda_rs::document::DocumentViewId; + + use super::EntryHash; + + impl From for DocumentViewId { + fn from(hash: EntryHash) -> Self { + hash.0.into() + } + } +} diff --git a/aquadoggo/src/graphql/scalars/log_id.rs b/aquadoggo/src/graphql/scalars/log_id.rs new file mode 100644 index 000000000..e41e5f932 --- /dev/null +++ b/aquadoggo/src/graphql/scalars/log_id.rs @@ -0,0 +1,79 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +use std::fmt::Display; + +use async_graphql::scalar; +use serde::{Deserialize, Serialize}; + +/// Log id of a bamboo entry. +#[derive(Clone, Copy, Eq, PartialEq, Debug)] +pub struct LogId(p2panda_rs::entry::LogId); + +impl From for LogId { + fn from(log_id: p2panda_rs::entry::LogId) -> Self { + Self(log_id) + } +} + +impl From for p2panda_rs::entry::LogId { + fn from(log_id: LogId) -> p2panda_rs::entry::LogId { + log_id.0 + } +} + +impl Serialize for LogId { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + // Represent u64 log id as string to be able to encode large numbers in GraphQL JSON + // response. + serializer.serialize_str(&self.0.as_u64().to_string()) + } +} + +impl<'de> Deserialize<'de> for LogId { + fn deserialize(deserializer: D) -> Result + where + D: serde::Deserializer<'de>, + { + let str: String = Deserialize::deserialize(deserializer)?; + + let log_id: p2panda_rs::entry::LogId = str + .parse() + .map_err(|_| serde::de::Error::custom("Could not parse log_id string as u64"))?; + + Ok(LogId(log_id)) + } +} + +impl Display for LogId { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0.as_u64()) + } +} + +scalar!(LogId); + +#[cfg(test)] +mod tests { + use serde::{Deserialize, Serialize}; + + use super::LogId; + + #[test] + fn serde_log_id_as_string() { + #[derive(Serialize, Deserialize, PartialEq, Debug)] + struct Value { + log_id: LogId, + } + + let val = Value { + log_id: p2panda_rs::entry::LogId::new(1).into(), + }; + + let serialised = serde_json::to_string(&val).unwrap(); + assert_eq!(serialised, "{\"log_id\":\"1\"}".to_string()); + assert_eq!(val, serde_json::from_str(&serialised).unwrap()); + } +} diff --git a/aquadoggo/src/graphql/scalars/mod.rs b/aquadoggo/src/graphql/scalars/mod.rs new file mode 100644 index 000000000..2924fc5bc --- /dev/null +++ b/aquadoggo/src/graphql/scalars/mod.rs @@ -0,0 +1,17 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +mod document_id; +mod encoded_entry; +mod encoded_operation; +mod entry_hash; +mod log_id; +mod public_key; +mod seq_num; + +pub use document_id::DocumentId; +pub use encoded_entry::EncodedEntry; +pub use encoded_operation::EncodedOperation; +pub use entry_hash::EntryHash; +pub use log_id::LogId; +pub use public_key::PublicKey; +pub use seq_num::SeqNum; diff --git a/aquadoggo/src/graphql/scalars/public_key.rs b/aquadoggo/src/graphql/scalars/public_key.rs new file mode 100644 index 000000000..23b447994 --- /dev/null +++ b/aquadoggo/src/graphql/scalars/public_key.rs @@ -0,0 +1,31 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +use std::fmt::Display; + +use async_graphql::scalar; +use p2panda_rs::identity::Author; +use serde::{Deserialize, Serialize}; + +/// Public key of the entry author. +#[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize)] +pub struct PublicKey(Author); + +impl From for PublicKey { + fn from(author: Author) -> Self { + Self(author) + } +} + +impl From for Author { + fn from(public_key: PublicKey) -> Author { + public_key.0 + } +} + +impl Display for PublicKey { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0) + } +} + +scalar!(PublicKey); diff --git a/aquadoggo/src/graphql/scalars/seq_num.rs b/aquadoggo/src/graphql/scalars/seq_num.rs new file mode 100644 index 000000000..a945b23ea --- /dev/null +++ b/aquadoggo/src/graphql/scalars/seq_num.rs @@ -0,0 +1,115 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + +use std::convert::{TryFrom, TryInto}; +use std::fmt::Display; +use std::str::FromStr; + +use anyhow::Result; +use async_graphql::scalar; +use p2panda_rs::entry::SeqNumError; +use serde::{Deserialize, Serialize}; + +/// Sequence number of an entry. +#[derive(Clone, Copy, Debug, Eq, PartialEq)] +pub struct SeqNum(p2panda_rs::entry::SeqNum); + +impl SeqNum { + /// Return sequence number as u64. + pub fn as_u64(&self) -> u64 { + self.0.as_u64() + } + + /// Convert sequence number to string. + #[allow(clippy::inherent_to_string_shadow_display)] + pub fn to_string(self) -> String { + self.as_u64().to_string() + } +} + +/// Convert from p2panda types to GraphQL scalars and back. +impl From for SeqNum { + fn from(seq_num: p2panda_rs::entry::SeqNum) -> Self { + Self(seq_num) + } +} + +impl From for p2panda_rs::entry::SeqNum { + fn from(seq_num: SeqNum) -> p2panda_rs::entry::SeqNum { + seq_num.0 + } +} + +/// Convert from strings to sequence number. +impl FromStr for SeqNum { + type Err = SeqNumError; + + fn from_str(str: &str) -> Result { + let num = u64::from_str(str).map_err(|_| SeqNumError::InvalidU64String)?; + Ok(Self(p2panda_rs::entry::SeqNum::new(num)?)) + } +} + +impl TryFrom for SeqNum { + type Error = SeqNumError; + + fn try_from(str: String) -> Result { + SeqNum::from_str(&str) + } +} + +/// Represent u64 sequence number as string to be able to encode large numbers in GraphQL JSON +/// response. +impl Serialize for SeqNum { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + serializer.serialize_str(&self.to_string()) + } +} + +impl<'de> Deserialize<'de> for SeqNum { + fn deserialize(deserializer: D) -> Result + where + D: serde::Deserializer<'de>, + { + let str: String = Deserialize::deserialize(deserializer)?; + + let seq_num: SeqNum = str + .try_into() + .map_err(|_| serde::de::Error::custom("Could not parse seq_num string as u64"))?; + + Ok(seq_num) + } +} + +impl Display for SeqNum { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0.as_u64()) + } +} + +scalar!(SeqNum); + +#[cfg(test)] +mod tests { + use serde::{Deserialize, Serialize}; + + use super::SeqNum; + + #[test] + fn serde_seq_num_as_string() { + #[derive(Serialize, Deserialize, PartialEq, Debug)] + struct Value { + seq_num: SeqNum, + } + + let val = Value { + seq_num: p2panda_rs::entry::SeqNum::new(1).unwrap().into(), + }; + + let serialised = serde_json::to_string(&val).unwrap(); + assert_eq!(serialised, "{\"seq_num\":\"1\"}".to_string()); + assert_eq!(val, serde_json::from_str(&serialised).unwrap()); + } +} diff --git a/aquadoggo/src/graphql/schema.rs b/aquadoggo/src/graphql/schema.rs index 7fb3a9089..f867ed397 100644 --- a/aquadoggo/src/graphql/schema.rs +++ b/aquadoggo/src/graphql/schema.rs @@ -1,19 +1,15 @@ // SPDX-License-Identifier: AGPL-3.0-or-later -use std::sync::Arc; - use async_graphql::{EmptySubscription, MergedObject, Schema}; -use tokio::sync::Mutex; use crate::bus::ServiceSender; use crate::db::provider::SqlStorage; use crate::graphql::client::{ClientMutationRoot, ClientRoot}; -use crate::graphql::replication::context::ReplicationContext; use crate::graphql::replication::ReplicationRoot; /// All of the graphql query sub modules merged into one top level root. #[derive(MergedObject, Debug)] -pub struct QueryRoot(pub ReplicationRoot, pub ClientRoot); +pub struct QueryRoot(pub ReplicationRoot, pub ClientRoot); /// All of the graphql mutation sub modules merged into one top level root. #[derive(MergedObject, Debug, Copy, Clone, Default)] @@ -24,9 +20,7 @@ pub type RootSchema = Schema; /// Build the root graphql schema that can handle graphql requests. pub fn build_root_schema(store: SqlStorage, tx: ServiceSender) -> RootSchema { - let replication_context = Arc::new(Mutex::new(ReplicationContext::new(1000, store.clone()))); - - let replication_root = ReplicationRoot::::new(); + let replication_root = ReplicationRoot::default(); let client_query_root = ClientRoot::default(); let query_root = QueryRoot(replication_root, client_query_root); @@ -34,7 +28,6 @@ pub fn build_root_schema(store: SqlStorage, tx: ServiceSender) -> RootSchema { let mutation_root = MutationRoot(client_mutation_root); Schema::build(query_root, mutation_root, EmptySubscription) - .data(replication_context) .data(store) .data(tx) .finish() diff --git a/aquadoggo/src/lib.rs b/aquadoggo/src/lib.rs index 514dc99eb..e96823e6a 100644 --- a/aquadoggo/src/lib.rs +++ b/aquadoggo/src/lib.rs @@ -18,7 +18,7 @@ mod config; mod context; mod db; mod errors; -pub mod graphql; +mod graphql; mod http; mod manager; mod materializer; @@ -30,5 +30,4 @@ mod test_helpers; pub use crate::config::Configuration; pub use crate::replication::ReplicationConfiguration; -pub use db::{connection_pool, provider::SqlStorage}; pub use node::Node; diff --git a/aquadoggo/src/replication/service.rs b/aquadoggo/src/replication/service.rs index 9f9969579..5fbd63f81 100644 --- a/aquadoggo/src/replication/service.rs +++ b/aquadoggo/src/replication/service.rs @@ -18,9 +18,9 @@ use tokio::task; use crate::bus::{ServiceMessage, ServiceSender}; use crate::context::Context; +use crate::db::request::PublishEntryRequest; use crate::db::stores::StorageEntry; -use crate::graphql::client::PublishEntryRequest; -use crate::graphql::replication::client::ReplicationClient; +use crate::graphql::replication::client; use crate::manager::Shutdown; /// Replication service polling other nodes frequently to ask them about new entries from a defined @@ -30,9 +30,6 @@ pub async fn replication_service( shutdown: Shutdown, tx: ServiceSender, ) -> Result<()> { - // Initialise a GraphQL client which will talk to the other node - let mut client = ReplicationClient::new(); - // Prepare replication configuration let config = &context.config.replication; let connection_interval = Duration::from_secs(config.connection_interval_seconds); @@ -61,32 +58,37 @@ pub async fn replication_service( ); // Make our replication request to the remote peer - let entries = client - .get_entries_newer_than_seq( - remote_peer, - &log_id, - &author, - latest_seq_num.as_ref(), - ) - .await; - - if let Ok(entries) = entries { - debug!( - "Received {} new entries from peer {}", - entries.len(), - remote_peer - ); - - if let Err(err) = verify_entries(&entries, &context).await { - warn!("Couldn't verify entries: {}", err); - continue; + let response = client::entries_newer_than_seq_num( + remote_peer, + &log_id, + &author, + latest_seq_num.as_ref(), + ) + .await; + + match response { + Ok(entries) => { + debug!( + "Received {} new entries from peer {}", + entries.len(), + remote_peer + ); + + if let Err(err) = verify_entries(&entries, &context).await { + warn!("Couldn't verify entries: {}", err); + continue; + } + + insert_new_entries(&entries, &context, tx.clone()) + .await + .unwrap_or_else(|err| error!("{:?}", err)); + } + Err(err) => { + warn!( + "Replication request to peer {} failed: {}", + remote_peer, err + ); } - - insert_new_entries(&entries, &context, tx.clone()) - .await - .unwrap_or_else(|err| error!("{:?}", err)); - } else { - warn!("Replication request to peer {} failed", remote_peer); } } } @@ -152,9 +154,9 @@ async fn insert_new_entries( for entry in new_entries { // Parse and validate parameters let args = PublishEntryRequest { - entry_encoded: entry.entry_signed().clone(), + entry: entry.entry_signed().clone(), // We know a storage entry has an operation so we safely unwrap here. - operation_encoded: entry.operation_encoded().unwrap().clone(), + operation: entry.operation_encoded().unwrap().clone(), }; // This is the method used to publish entries arriving from clients. They all contain a @@ -224,6 +226,8 @@ async fn add_certpool_to_entries_for_verification( ) -> Result<()> { trace!("Getting certificate pool for entries"); + // @TODO: This gets the certificate pool from the database, but what if we need to get it from + // the other peer? let mut certpool = context .0 .store diff --git a/aquadoggo_cli/Cargo.toml b/aquadoggo_cli/Cargo.toml index b998e44b2..57612cd18 100644 --- a/aquadoggo_cli/Cargo.toml +++ b/aquadoggo_cli/Cargo.toml @@ -18,10 +18,10 @@ path = "src/main.rs" doc = false [dependencies] -anyhow = "1.0.43" -tokio = { version = "1.17.0", features = ["full"] } +anyhow = "1.0.58" +tokio = { version = "1.19.2", features = ["full"] } env_logger = "0.9.0" -structopt = "0.3.23" +structopt = "0.3.26" [dependencies.aquadoggo] version = "~0.3.0" diff --git a/aquadoggo_cli/src/main.rs b/aquadoggo_cli/src/main.rs index 44c13ff92..286ff834e 100644 --- a/aquadoggo_cli/src/main.rs +++ b/aquadoggo_cli/src/main.rs @@ -41,9 +41,8 @@ struct Opt { /// A collection of authors and their logs to replicate. /// - /// eg -A 123abc="1 2 345" -A 456def="6 7" - /// Adds the the authors: - /// + /// eg. -A 123abc="1 2 345" -A 456def="6 7" + /// .. adds the authors: /// - "123abc" with log_ids 1, 2, 345 /// - "456def" with log_ids 6 7 #[structopt(short = "A", parse(try_from_str = parse_key_val), number_of_values = 1)] From acd446ffb54428aa477072c2a13c1e99978f202e Mon Sep 17 00:00:00 2001 From: Andreas Dzialocha Date: Fri, 1 Jul 2022 15:37:42 +0200 Subject: [PATCH 11/13] Update CHANGELOG.md --- CHANGELOG.md | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 65b668cd0..556435ceb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,8 +7,18 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 ## [Unreleased] +### Added + +- GraphQL replication service gets and verifies new entries and inserts them into the db [#137](https://github.com/p2panda/aquadoggo/pull/137) + +### Changed + +- Refactor scalars and replication API, replace `graphql-client` with `gql_client` [#184](https://github.com/p2panda/aquadoggo/pull/184) + ## [0.3.0] +Released on 2022-07-01: :package: [`crate`](https://crates.io/crates/aquadoggo/0.3.0) + ### Added - Introduce GraphQL endpoint [#81](https://github.com/p2panda/aquadoggo/pull/81) @@ -28,7 +38,6 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - Inform materialization service about new operations [#161](https://github.com/p2panda/aquadoggo/pull/161) - e2e publish entry tests [#167](https://github.com/p2panda/aquadoggo/pull/167) - Reschedule pending tasks on startup [#168](https://github.com/p2panda/aquadoggo/pull/168) -- GraphQL replication service gets and verifies new entries and inserts them into the db [#137](https://github.com/p2panda/aquadoggo/pull/137) - Debug logging in reduce task [#175](https://github.com/p2panda/aquadoggo/pull/175) ### Changed @@ -45,7 +54,6 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - Use `VerifiedOperation` [#158](https://github.com/p2panda/aquadoggo/pull/158) - Refactor `test_db` helper method [#176](https://github.com/p2panda/aquadoggo/pull/176) - Update `publishEntry` params and `nextEntryArgs` response fields [#181](https://github.com/p2panda/aquadoggo/pull/181) -- Refactor scalars and replication API, replace `graphql-client` with `gql_client` [#184](https://github.com/p2panda/aquadoggo/pull/184) ### Fixed From 37b9a68864ca07a3c387798765ce6441b1dc0fd4 Mon Sep 17 00:00:00 2001 From: Andreas Dzialocha Date: Fri, 1 Jul 2022 19:01:52 +0200 Subject: [PATCH 12/13] Move to unreleased section in CHANGELOG --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3a27805f9..8bf10f164 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,6 +10,7 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 ### Added - GraphQL replication service gets and verifies new entries and inserts them into the db [#137](https://github.com/p2panda/aquadoggo/pull/137) +- Add schema task and schema provider that update when new schema views are materialised [#166](https://github.com/p2panda/aquadoggo/pull/166/files) ### Changed @@ -38,7 +39,6 @@ Released on 2022-07-01: :package: [`crate`](https://crates.io/crates/aquadoggo/0 - Inform materialization service about new operations [#161](https://github.com/p2panda/aquadoggo/pull/161) - e2e publish entry tests [#167](https://github.com/p2panda/aquadoggo/pull/167) - Reschedule pending tasks on startup [#168](https://github.com/p2panda/aquadoggo/pull/168) -- Add schema task and schema provider that update when new schema views are materialised [#166](https://github.com/p2panda/aquadoggo/pull/166/files) - Debug logging in reduce task [#175](https://github.com/p2panda/aquadoggo/pull/175) ### Changed From fe2a9146af1c00926ab1ace9e07aff60af222709 Mon Sep 17 00:00:00 2001 From: Andreas Dzialocha Date: Fri, 1 Jul 2022 19:03:53 +0200 Subject: [PATCH 13/13] Minor clean ups --- CHANGELOG.md | 2 +- aquadoggo/src/materializer/tasks/schema.rs | 8 ++++---- aquadoggo/src/schema/mod.rs | 2 ++ 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8bf10f164..458a021f2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,7 +10,7 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 ### Added - GraphQL replication service gets and verifies new entries and inserts them into the db [#137](https://github.com/p2panda/aquadoggo/pull/137) -- Add schema task and schema provider that update when new schema views are materialised [#166](https://github.com/p2panda/aquadoggo/pull/166/files) +- Add schema task and schema provider that update when new schema views are materialised [#166](https://github.com/p2panda/aquadoggo/pull/166) ### Changed diff --git a/aquadoggo/src/materializer/tasks/schema.rs b/aquadoggo/src/materializer/tasks/schema.rs index 8f6e9c47b..e8798c2d9 100644 --- a/aquadoggo/src/materializer/tasks/schema.rs +++ b/aquadoggo/src/materializer/tasks/schema.rs @@ -1,7 +1,6 @@ // SPDX-License-Identifier: AGPL-3.0-or-later use log::debug; - use p2panda_rs::document::DocumentViewId; use p2panda_rs::operation::{AsOperation, AsVerifiedOperation, OperationValue}; use p2panda_rs::schema::SchemaId; @@ -18,7 +17,8 @@ use crate::materializer::TaskInput; /// has all its immediate dependencies available in the store. It collects all required views for /// the schema, instantiates it and adds it to the schema provider. pub async fn schema_task(context: Context, input: TaskInput) -> TaskResult { - debug!("Working on task {}", input); + debug!("Working on schema task {}", input); + let input_view_id = match (input.document_id, input.document_view_id) { (None, Some(view_id)) => Ok(view_id), // The task input must contain only a view id. @@ -116,12 +116,12 @@ async fn get_related_schema_definitions( #[cfg(test)] mod tests { - use log::debug; - use p2panda_rs::hash::Hash; use std::convert::TryFrom; + use log::debug; use p2panda_rs::document::DocumentViewId; use p2panda_rs::entry::{sign_and_encode, Entry}; + use p2panda_rs::hash::Hash; use p2panda_rs::identity::{Author, KeyPair}; use p2panda_rs::operation::{ AsVerifiedOperation, Operation, OperationEncoded, OperationFields, OperationValue, diff --git a/aquadoggo/src/schema/mod.rs b/aquadoggo/src/schema/mod.rs index 4426cabec..d6ed73e77 100644 --- a/aquadoggo/src/schema/mod.rs +++ b/aquadoggo/src/schema/mod.rs @@ -1,3 +1,5 @@ +// SPDX-License-Identifier: AGPL-3.0-or-later + mod provider; pub use provider::SchemaProvider;