diff --git a/aggregator/src/aggregator.rs b/aggregator/src/aggregator.rs index e2a0ef675..e00c1662a 100644 --- a/aggregator/src/aggregator.rs +++ b/aggregator/src/aggregator.rs @@ -42,14 +42,12 @@ use janus_core::{ time::{Clock, DurationExt, IntervalExt, TimeExt}, }; use janus_messages::{ - problem_type::DapProblemType, - query_type::{FixedSize, TimeInterval}, - AggregateShare, AggregateShareAad, AggregateShareReq, AggregationJobContinueReq, - AggregationJobId, AggregationJobInitializeReq, AggregationJobResp, AggregationJobRound, - BatchSelector, Collection, CollectionJobId, CollectionReq, Duration, HpkeCiphertext, - HpkeConfigList, InputShareAad, Interval, PartialBatchSelector, PlaintextInputShare, - PrepareStep, PrepareStepResult, Report, ReportIdChecksum, ReportShare, ReportShareError, Role, - TaskId, + problem_type::DapProblemType, query_type::FixedSize, AggregateShare, AggregateShareAad, + AggregateShareReq, AggregationJobContinueReq, AggregationJobId, AggregationJobInitializeReq, + AggregationJobResp, AggregationJobRound, BatchSelector, Collection, CollectionJobId, + CollectionReq, Duration, HpkeCiphertext, HpkeConfigList, InputShareAad, Interval, + PartialBatchSelector, PlaintextInputShare, PrepareStep, PrepareStepResult, Report, + ReportIdChecksum, ReportShare, ReportShareError, Role, TaskId, }; use opentelemetry::{ metrics::{Counter, Histogram, Meter}, @@ -836,20 +834,6 @@ impl VdafOps { report: Report, ) -> Result<(), Arc> { match task.query_type() { - task::QueryType::TimeInterval => { - vdaf_ops_dispatch!(self, (vdaf, _, VdafType, VERIFY_KEY_LENGTH) => { - Self::handle_upload_generic::( - Arc::clone(vdaf), - clock, - upload_decrypt_failure_counter, - upload_decode_failure_counter, - task, - report_writer, - report, - ) - .await - }) - } task::QueryType::FixedSize { .. } => { vdaf_ops_dispatch!(self, (vdaf, _, VdafType, VERIFY_KEY_LENGTH) => { Self::handle_upload_generic::( @@ -883,20 +867,6 @@ impl VdafOps { req_bytes: &[u8], ) -> Result { match task.query_type() { - task::QueryType::TimeInterval => { - vdaf_ops_dispatch!(self, (vdaf, verify_key, VdafType, VERIFY_KEY_LENGTH) => { - Self::handle_aggregate_init_generic::( - datastore, - vdaf, - aggregate_step_failure_counter, - task, - aggregation_job_id, - verify_key, - req_bytes, - ) - .await - }) - } task::QueryType::FixedSize { .. } => { vdaf_ops_dispatch!(self, (vdaf, verify_key, VdafType, VERIFY_KEY_LENGTH) => { Self::handle_aggregate_init_generic::( @@ -930,21 +900,6 @@ impl VdafOps { request_hash: [u8; 32], ) -> Result { match task.query_type() { - task::QueryType::TimeInterval => { - vdaf_ops_dispatch!(self, (vdaf, _, VdafType, VERIFY_KEY_LENGTH) => { - Self::handle_aggregate_continue_generic::( - datastore, - Arc::clone(vdaf), - aggregate_step_failure_counter, - task, - batch_aggregation_shard_count, - aggregation_job_id, - req, - request_hash, - ) - .await - }) - } task::QueryType::FixedSize { .. } => { vdaf_ops_dispatch!(self, (vdaf, _, VdafType, VERIFY_KEY_LENGTH) => { Self::handle_aggregate_continue_generic::( @@ -1714,17 +1669,6 @@ impl VdafOps { collection_req_bytes: &[u8], ) -> Result<(), Error> { match task.query_type() { - task::QueryType::TimeInterval => { - vdaf_ops_dispatch!(self, (vdaf, _, VdafType, VERIFY_KEY_LENGTH) => { - Self::handle_create_collection_job_generic::< - VERIFY_KEY_LENGTH, - TimeInterval, - VdafType, - _, - >(datastore, task, Arc::clone(vdaf), collection_job_id, collection_req_bytes) - .await - }) - } task::QueryType::FixedSize { .. } => { vdaf_ops_dispatch!(self, (vdaf, _, VdafType, VERIFY_KEY_LENGTH) => { Self::handle_create_collection_job_generic::< @@ -2015,17 +1959,6 @@ impl VdafOps { collection_job_id: &CollectionJobId, ) -> Result>, Error> { match task.query_type() { - task::QueryType::TimeInterval => { - vdaf_ops_dispatch!(self, (vdaf, _, VdafType, VERIFY_KEY_LENGTH) => { - Self::handle_get_collection_job_generic::< - VERIFY_KEY_LENGTH, - TimeInterval, - VdafType, - _, - >(datastore, task, Arc::clone(vdaf), collection_job_id) - .await - }) - } task::QueryType::FixedSize { .. } => { vdaf_ops_dispatch!(self, (vdaf, _, VdafType, VERIFY_KEY_LENGTH) => { Self::handle_get_collection_job_generic::< @@ -2189,17 +2122,6 @@ impl VdafOps { collection_job_id: &CollectionJobId, ) -> Result<(), Error> { match task.query_type() { - task::QueryType::TimeInterval => { - vdaf_ops_dispatch!(self, (vdaf, _, VdafType, VERIFY_KEY_LENGTH) => { - Self::handle_delete_collection_job_generic::< - VERIFY_KEY_LENGTH, - TimeInterval, - VdafType, - _, - >(datastore, task, Arc::clone(vdaf), collection_job_id) - .await - }) - } task::QueryType::FixedSize { .. } => { vdaf_ops_dispatch!(self, (vdaf, _, VdafType, VERIFY_KEY_LENGTH) => { Self::handle_delete_collection_job_generic::< @@ -2272,17 +2194,6 @@ impl VdafOps { req_bytes: &[u8], ) -> Result { match task.query_type() { - task::QueryType::TimeInterval => { - vdaf_ops_dispatch!(self, (vdaf, _, VdafType, VERIFY_KEY_LENGTH) => { - Self::handle_aggregate_share_generic::< - VERIFY_KEY_LENGTH, - TimeInterval, - VdafType, - _, - >(datastore, clock, task, Arc::clone(vdaf), req_bytes, batch_aggregation_shard_count) - .await - }) - } task::QueryType::FixedSize { .. } => { vdaf_ops_dispatch!(self, (vdaf, _, VdafType, VERIFY_KEY_LENGTH) => { Self::handle_aggregate_share_generic::< @@ -2636,7 +2547,6 @@ mod tests { use futures::future::try_join_all; use janus_aggregator_core::{ datastore::{ - models::{CollectionJob, CollectionJobState}, test_util::{ephemeral_datastore, EphemeralDatastore}, Datastore, }, @@ -2645,14 +2555,13 @@ mod tests { }; use janus_core::{ hpke::{self, HpkeApplicationInfo, Label}, - task::{VdafInstance, PRIO3_VERIFY_KEY_LENGTH}, + task::VdafInstance, test_util::{dummy_vdaf, install_test_trace_subscriber}, time::{Clock, MockClock, TimeExt}, }; use janus_messages::{ - query_type::TimeInterval, Duration, Extension, HpkeCiphertext, HpkeConfig, HpkeConfigId, - InputShareAad, Interval, PlaintextInputShare, Report, ReportId, ReportMetadata, - ReportShare, Role, TaskId, Time, + Duration, Extension, HpkeCiphertext, HpkeConfig, HpkeConfigId, InputShareAad, + PlaintextInputShare, Report, ReportId, ReportMetadata, ReportShare, Role, TaskId, Time, }; use prio::{ codec::Encode, @@ -2733,7 +2642,7 @@ mod tests { let clock = MockClock::default(); let vdaf = Prio3Count::new_count(2).unwrap(); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -2959,53 +2868,6 @@ mod tests { }); } - #[tokio::test] - async fn upload_report_for_collected_batch() { - install_test_trace_subscriber(); - - let (_, aggregator, clock, task, datastore, _ephemeral_datastore) = - setup_upload_test(default_aggregator_config()).await; - let report = create_report(&task, clock.now()); - - // Insert a collection job for the batch interval including our report. - let batch_interval = Interval::new( - report - .metadata() - .time() - .to_batch_interval_start(task.time_precision()) - .unwrap(), - *task.time_precision(), - ) - .unwrap(); - datastore - .run_tx(|tx| { - let task = task.clone(); - Box::pin(async move { - tx.put_collection_job(&CollectionJob::< - PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, - Prio3Count, - >::new( - *task.id(), - random(), - batch_interval, - (), - CollectionJobState::Start, - )) - .await - }) - }) - .await - .unwrap(); - - // Try to upload the report, verify that we get the expected error. - assert_matches!(aggregator.handle_upload(task.id(), &report.get_encoded()).await.unwrap_err().as_ref(), Error::ReportRejected(err_task_id, err_report_id, err_time) => { - assert_eq!(task.id(), err_task_id); - assert_eq!(report.metadata().id(), err_report_id); - assert_eq!(report.metadata().time(), err_time); - }); - } - pub(crate) fn generate_helper_report_share>( task_id: TaskId, report_metadata: ReportMetadata, diff --git a/aggregator/src/aggregator/aggregate_init_tests.rs b/aggregator/src/aggregator/aggregate_init_tests.rs index e518fc171..435081596 100644 --- a/aggregator/src/aggregator/aggregate_init_tests.rs +++ b/aggregator/src/aggregator/aggregate_init_tests.rs @@ -15,8 +15,8 @@ use janus_core::{ time::{Clock, MockClock, TimeExt as _}, }; use janus_messages::{ - query_type::TimeInterval, AggregationJobId, AggregationJobInitializeReq, PartialBatchSelector, - ReportMetadata, ReportShare, Role, + query_type::FixedSize, AggregationJobId, AggregationJobInitializeReq, BatchId, + PartialBatchSelector, ReportMetadata, ReportShare, Role, }; use prio::codec::Encode; use rand::random; @@ -90,7 +90,8 @@ pub(super) struct AggregationJobInitTestCase { pub(super) report_share_generator: ReportShareGenerator, pub(super) report_shares: Vec, pub(super) aggregation_job_id: AggregationJobId, - aggregation_job_init_req: AggregationJobInitializeReq, + aggregation_job_init_req: AggregationJobInitializeReq, + pub(super) batch_id: BatchId, pub(super) aggregation_param: dummy_vdaf::AggregationParam, pub(super) handler: Box, pub(super) datastore: Arc>, @@ -115,7 +116,12 @@ pub(super) async fn setup_aggregate_init_test() -> AggregationJobInitTestCase { async fn setup_aggregate_init_test_without_sending_request() -> AggregationJobInitTestCase { install_test_trace_subscriber(); - let task = TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Helper).build(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Helper, + ) + .build(); let clock = MockClock::default(); let ephemeral_datastore = ephemeral_datastore().await; let datastore = Arc::new(ephemeral_datastore.datastore(clock.clone()).await); @@ -141,9 +147,10 @@ async fn setup_aggregate_init_test_without_sending_request() -> AggregationJobIn ]); let aggregation_job_id = random(); + let batch_id = random(); let aggregation_job_init_req = AggregationJobInitializeReq::new( aggregation_param.get_encoded(), - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(batch_id), report_shares.clone(), ); @@ -154,6 +161,7 @@ async fn setup_aggregate_init_test_without_sending_request() -> AggregationJobIn report_share_generator, aggregation_job_id, aggregation_job_init_req, + batch_id, aggregation_param, handler: Box::new(handler), datastore, @@ -164,7 +172,7 @@ async fn setup_aggregate_init_test_without_sending_request() -> AggregationJobIn pub(crate) async fn put_aggregation_job( task: &Task, aggregation_job_id: &AggregationJobId, - aggregation_job: &AggregationJobInitializeReq, + aggregation_job: &AggregationJobInitializeReq, handler: &impl Handler, ) -> TestConn { put(task.aggregation_job_uri(aggregation_job_id).unwrap().path()) @@ -174,7 +182,7 @@ pub(crate) async fn put_aggregation_job( ) .with_request_header( KnownHeaderName::ContentType, - AggregationJobInitializeReq::::MEDIA_TYPE, + AggregationJobInitializeReq::::MEDIA_TYPE, ) .with_request_body(aggregation_job.get_encoded()) .run_async(handler) @@ -201,7 +209,7 @@ async fn aggregation_job_init_authorization_dap_auth_token() { .with_request_header(auth_header, auth_value) .with_request_header( KnownHeaderName::ContentType, - AggregationJobInitializeReq::::MEDIA_TYPE, + AggregationJobInitializeReq::::MEDIA_TYPE, ) .with_request_body(test_case.aggregation_job_init_req.get_encoded()) .run_async(&test_case.handler) @@ -236,7 +244,7 @@ async fn aggregation_job_init_malformed_authorization_header(#[case] header_valu ) .with_request_header( KnownHeaderName::ContentType, - AggregationJobInitializeReq::::MEDIA_TYPE, + AggregationJobInitializeReq::::MEDIA_TYPE, ) .with_request_body(test_case.aggregation_job_init_req.get_encoded()) .run_async(&test_case.handler) @@ -252,7 +260,7 @@ async fn aggregation_job_mutation_aggregation_job() { // Put the aggregation job again, but with a different aggregation parameter. let mutated_aggregation_job_init_req = AggregationJobInitializeReq::new( dummy_vdaf::AggregationParam(1).get_encoded(), - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(test_case.batch_id), test_case.report_shares, ); @@ -292,7 +300,7 @@ async fn aggregation_job_mutation_report_shares() { ] { let mutated_aggregation_job_init_req = AggregationJobInitializeReq::new( test_case.aggregation_param.get_encoded(), - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(test_case.batch_id), mutated_report_shares, ); let response = put_aggregation_job( @@ -328,7 +336,7 @@ async fn aggregation_job_mutation_report_aggregations() { let mutated_aggregation_job_init_req = AggregationJobInitializeReq::new( test_case.aggregation_param.get_encoded(), - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(test_case.batch_id), mutated_report_shares, ); let response = put_aggregation_job( diff --git a/aggregator/src/aggregator/aggregation_job_continue.rs b/aggregator/src/aggregator/aggregation_job_continue.rs index 4aa63cd14..c53fb4cdb 100644 --- a/aggregator/src/aggregator/aggregation_job_continue.rs +++ b/aggregator/src/aggregator/aggregation_job_continue.rs @@ -82,27 +82,6 @@ impl VdafOps { break report_agg; }; - // Make sure this report isn't in an interval that has already started collection. - let conflicting_aggregate_share_jobs = tx - .get_aggregate_share_jobs_including_time::( - &vdaf, - task.id(), - report_aggregation.time(), - ) - .await?; - if !conflicting_aggregate_share_jobs.is_empty() { - *report_aggregation = report_aggregation - .clone() - .with_state(ReportAggregationState::Failed( - ReportShareError::BatchCollected, - )) - .with_last_prep_step(Some(PrepareStep::new( - *prep_step.report_id(), - PrepareStepResult::Failed(ReportShareError::BatchCollected), - ))); - continue; - } - let prep_state = match report_aggregation.state() { ReportAggregationState::Waiting(prep_state, _) => prep_state, _ => { @@ -406,7 +385,7 @@ mod tests { time::{IntervalExt, MockClock}, }; use janus_messages::{ - query_type::TimeInterval, AggregationJobContinueReq, AggregationJobId, AggregationJobResp, + query_type::FixedSize, AggregationJobContinueReq, AggregationJobId, AggregationJobResp, AggregationJobRound, Interval, PrepareStep, PrepareStepResult, Role, }; use prio::codec::Encode; @@ -432,8 +411,12 @@ mod tests { install_test_trace_subscriber(); let aggregation_job_id = random(); - let task = - TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Helper).build(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Helper, + ) + .build(); let clock = MockClock::default(); let ephemeral_datastore = ephemeral_datastore().await; let meter = noop_meter(); @@ -455,17 +438,15 @@ mod tests { tx.put_task(&task).await.unwrap(); tx.put_report_share(task.id(), &report.0).await.unwrap(); - tx.put_aggregation_job( - &AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - aggregation_job_id, - dummy_vdaf::AggregationParam::default(), - (), - Interval::from_time(report.0.metadata().time()).unwrap(), - AggregationJobState::InProgress, - AggregationJobRound::from(0), - ), - ) + tx.put_aggregation_job(&AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( + *task.id(), + aggregation_job_id, + dummy_vdaf::AggregationParam::default(), + random(), + Interval::from_time(report.0.metadata().time()).unwrap(), + AggregationJobState::InProgress, + AggregationJobRound::from(0), + )) .await .unwrap(); @@ -611,7 +592,7 @@ mod tests { .unwrap(); let aggregation_job = tx - .get_aggregation_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_aggregation_job::<0, FixedSize, dummy_vdaf::Vdaf>( &task_id, &aggregation_job_id, ) @@ -661,7 +642,7 @@ mod tests { (*test_case.task.id(), test_case.aggregation_job_id); Box::pin(async move { let aggregation_job = tx - .get_aggregation_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_aggregation_job::<0, FixedSize, dummy_vdaf::Vdaf>( &task_id, &aggregation_job_id, ) @@ -703,7 +684,7 @@ mod tests { // round mismatch error instead of tripping the check for a request to continue // to round 0. let aggregation_job = tx - .get_aggregation_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_aggregation_job::<0, FixedSize, dummy_vdaf::Vdaf>( &task_id, &aggregation_job_id, ) diff --git a/aggregator/src/aggregator/aggregation_job_creator.rs b/aggregator/src/aggregator/aggregation_job_creator.rs index f297d49ff..9c0b6910a 100644 --- a/aggregator/src/aggregator/aggregation_job_creator.rs +++ b/aggregator/src/aggregator/aggregation_job_creator.rs @@ -17,8 +17,7 @@ use janus_core::{ time::{Clock, DurationExt as _, TimeExt as _}, }; use janus_messages::{ - query_type::{FixedSize, TimeInterval}, - AggregationJobRound, Duration as DurationMsg, Interval, Role, TaskId, + query_type::FixedSize, AggregationJobRound, Duration as DurationMsg, Interval, Role, TaskId, }; use opentelemetry::{ metrics::{Histogram, Meter, Unit}, @@ -270,77 +269,6 @@ impl AggregationJobCreator { task: Arc, ) -> anyhow::Result { match (task.query_type(), task.vdaf()) { - (task::QueryType::TimeInterval, VdafInstance::Prio3Count) => { - let vdaf = Arc::new(Prio3::new_count(2)?); - self.create_aggregation_jobs_for_time_interval_task_no_param::(task, vdaf) - .await - } - - (task::QueryType::TimeInterval, VdafInstance::Prio3CountVec { length }) => { - let vdaf = Arc::new(Prio3::new_sum_vec_multithreaded(2, 1, *length)?); - self.create_aggregation_jobs_for_time_interval_task_no_param::< - PRIO3_VERIFY_KEY_LENGTH, - Prio3SumVecMultithreaded - >(task, vdaf).await - } - - (task::QueryType::TimeInterval, VdafInstance::Prio3Sum { bits }) => { - let vdaf = Arc::new(Prio3::new_sum(2, *bits)?); - self.create_aggregation_jobs_for_time_interval_task_no_param::(task, vdaf) - .await - } - - (task::QueryType::TimeInterval, VdafInstance::Prio3SumVec { bits, length }) => { - let vdaf = Arc::new(Prio3::new_sum_vec_multithreaded(2, *bits, *length)?); - self.create_aggregation_jobs_for_time_interval_task_no_param::(task, vdaf) - .await - } - - (task::QueryType::TimeInterval, VdafInstance::Prio3Histogram { buckets }) => { - let vdaf = Arc::new(Prio3::new_histogram(2, buckets)?); - self.create_aggregation_jobs_for_time_interval_task_no_param::(task, vdaf) - .await - } - - #[cfg(feature = "fpvec_bounded_l2")] - ( - task::QueryType::TimeInterval, - VdafInstance::Prio3FixedPoint16BitBoundedL2VecSum { length }, - ) => { - let vdaf: Arc>> = - Arc::new(Prio3::new_fixedpoint_boundedl2_vec_sum_multithreaded( - 2, *length, - )?); - self.create_aggregation_jobs_for_time_interval_task_no_param::>>(task, vdaf) - .await - } - - #[cfg(feature = "fpvec_bounded_l2")] - ( - task::QueryType::TimeInterval, - VdafInstance::Prio3FixedPoint32BitBoundedL2VecSum { length }, - ) => { - let vdaf: Arc>> = - Arc::new(Prio3::new_fixedpoint_boundedl2_vec_sum_multithreaded( - 2, *length, - )?); - self.create_aggregation_jobs_for_time_interval_task_no_param::>>(task, vdaf) - .await - } - - #[cfg(feature = "fpvec_bounded_l2")] - ( - task::QueryType::TimeInterval, - VdafInstance::Prio3FixedPoint64BitBoundedL2VecSum { length }, - ) => { - let vdaf: Arc>> = - Arc::new(Prio3::new_fixedpoint_boundedl2_vec_sum_multithreaded( - 2, *length, - )?); - self.create_aggregation_jobs_for_time_interval_task_no_param::>>(task, vdaf) - .await - } - (task::QueryType::FixedSize { max_batch_size }, VdafInstance::Prio3Count) => { let vdaf = Arc::new(Prio3::new_count(2)?); let max_batch_size = *max_batch_size; @@ -438,107 +366,6 @@ impl AggregationJobCreator { } } - async fn create_aggregation_jobs_for_time_interval_task_no_param< - const SEED_SIZE: usize, - A: vdaf::Aggregator, - >( - self: Arc, - task: Arc, - vdaf: Arc, - ) -> anyhow::Result - where - A: Send + Sync + 'static, - A::AggregateShare: Send + Sync, - A::PrepareMessage: Send + Sync, - A::PrepareShare: Send + Sync, - A::PrepareState: Send + Sync + Encode, - A::OutputShare: Send + Sync, - { - Ok(self - .datastore - .run_tx_with_name("aggregation_job_creator_time_no_param", |tx| { - let this = Arc::clone(&self); - let task = Arc::clone(&task); - let vdaf = Arc::clone(&vdaf); - - Box::pin(async move { - // Find some unaggregated client reports. - let report_ids_and_times = tx - .get_unaggregated_client_report_ids_for_task(task.id()) - .await?; - - // Generate aggregation jobs & report aggregations based on the reports we read. - let mut aggregation_job_writer = AggregationJobWriter::new(Arc::clone(&task)); - for agg_job_reports in - report_ids_and_times.chunks(this.max_aggregation_job_size) - { - if agg_job_reports.len() < this.min_aggregation_job_size { - if !agg_job_reports.is_empty() { - let report_ids: Vec<_> = agg_job_reports - .iter() - .map(|(report_id, _)| *report_id) - .collect(); - tx.mark_reports_unaggregated(task.id(), &report_ids).await?; - } - continue; - } - - let aggregation_job_id = random(); - debug!( - task_id = %task.id(), - %aggregation_job_id, - report_count = %agg_job_reports.len(), - "Creating aggregation job" - ); - - let min_client_timestamp = - agg_job_reports.iter().map(|(_, time)| time).min().unwrap(); // unwrap safety: agg_job_reports is non-empty - let max_client_timestamp = - agg_job_reports.iter().map(|(_, time)| time).max().unwrap(); // unwrap safety: agg_job_reports is non-empty - let client_timestamp_interval = Interval::new( - *min_client_timestamp, - max_client_timestamp - .difference(min_client_timestamp)? - .add(&DurationMsg::from_seconds(1))?, - )?; - - let aggregation_job = AggregationJob::::new( - *task.id(), - aggregation_job_id, - (), - (), - client_timestamp_interval, - AggregationJobState::InProgress, - AggregationJobRound::from(0), - ); - - let report_aggregations = agg_job_reports - .iter() - .enumerate() - .map(|(ord, (report_id, time))| { - Ok(ReportAggregation::::new( - *task.id(), - aggregation_job_id, - *report_id, - *time, - ord.try_into()?, - None, - ReportAggregationState::Start, - )) - }) - .collect::>()?; - - aggregation_job_writer.put(aggregation_job, report_aggregations)?; - } - - // Write the aggregation jobs & report aggregations we created. - aggregation_job_writer.write(tx, vdaf).await?; - Ok(!aggregation_job_writer.is_empty()) - }) - }) - .await?) - } - async fn create_aggregation_jobs_for_fixed_size_task_no_param< const SEED_SIZE: usize, A: vdaf::Aggregator, @@ -741,7 +568,7 @@ mod tests { use futures::{future::try_join_all, TryFutureExt}; use janus_aggregator_core::{ datastore::{ - models::{AggregationJob, AggregationJobState, Batch, BatchState, LeaderStoredReport}, + models::{AggregationJob, Batch, BatchState, LeaderStoredReport}, test_util::ephemeral_datastore, Transaction, }, @@ -758,8 +585,7 @@ mod tests { time::{Clock, IntervalExt, MockClock}, }; use janus_messages::{ - query_type::{FixedSize, TimeInterval}, - AggregationJobRound, Interval, ReportId, Role, TaskId, Time, + query_type::FixedSize, AggregationJobRound, Interval, ReportId, Role, TaskId, Time, }; use prio::vdaf::{self, prio3::Prio3Count}; use std::{collections::HashSet, iter, sync::Arc, time::Duration}; @@ -786,17 +612,15 @@ mod tests { let report_time = Time::from_seconds_since_epoch(0); let leader_task = TaskBuilder::new( - TaskQueryType::TimeInterval, + TaskQueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) .build(); - let batch_identifier = - TimeInterval::to_batch_identifier(&leader_task, &(), &report_time).unwrap(); let leader_report = LeaderStoredReport::new_dummy(*leader_task.id(), report_time); let helper_task = TaskBuilder::new( - TaskQueryType::TimeInterval, + TaskQueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Helper, ) @@ -826,7 +650,7 @@ mod tests { noop_meter(), Duration::from_secs(3600), AGGREGATION_JOB_CREATION_INTERVAL, - 0, + 1, 100, )); let stopper = Stopper::new(); @@ -846,7 +670,7 @@ mod tests { let (leader_aggregations, leader_batches) = read_aggregate_info_for_task::< PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, Prio3Count, _, >(tx, leader_task.id()) @@ -854,7 +678,7 @@ mod tests { let (helper_aggregations, helper_batches) = read_aggregate_info_for_task::< PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, Prio3Count, _, >(tx, helper_task.id()) @@ -872,18 +696,18 @@ mod tests { assert_eq!(leader_aggregations.len(), 1); let leader_aggregation = leader_aggregations.into_iter().next().unwrap(); - assert_eq!(leader_aggregation.0.partial_batch_identifier(), &()); assert_eq!(leader_aggregation.0.round(), AggregationJobRound::from(0)); assert_eq!( leader_aggregation.1, Vec::from([*leader_report.metadata().id()]) ); + let batch_id = *leader_aggregation.0.batch_id(); assert_eq!( leader_batches, Vec::from([Batch::new( *leader_task.id(), - batch_identifier, + batch_id, (), BatchState::Open, 1, @@ -895,367 +719,6 @@ mod tests { assert!(helper_batches.is_empty()); } - #[tokio::test] - async fn create_aggregation_jobs_for_time_interval_task() { - // Setup. - install_test_trace_subscriber(); - let clock = MockClock::default(); - let ephemeral_datastore = ephemeral_datastore().await; - let ds = ephemeral_datastore.datastore(clock.clone()).await; - const MIN_AGGREGATION_JOB_SIZE: usize = 50; - const MAX_AGGREGATION_JOB_SIZE: usize = 60; - - let task = Arc::new( - TaskBuilder::new( - TaskQueryType::TimeInterval, - VdafInstance::Prio3Count, - Role::Leader, - ) - .build(), - ); - - // Create 2 max-size batches, a min-size batch, one extra report (which will be added to the - // min-size batch). - let report_time = clock.now(); - let batch_identifier = TimeInterval::to_batch_identifier(&task, &(), &report_time).unwrap(); - let reports: Vec<_> = - iter::repeat_with(|| LeaderStoredReport::new_dummy(*task.id(), report_time)) - .take(2 * MAX_AGGREGATION_JOB_SIZE + MIN_AGGREGATION_JOB_SIZE + 1) - .collect(); - let all_report_ids: HashSet = reports - .iter() - .map(|report| *report.metadata().id()) - .collect(); - - ds.run_tx(|tx| { - let (task, reports) = (Arc::clone(&task), reports.clone()); - Box::pin(async move { - tx.put_task(&task).await?; - for report in reports.iter() { - tx.put_client_report(&dummy_vdaf::Vdaf::new(), report) - .await?; - } - Ok(()) - }) - }) - .await - .unwrap(); - - // Run. - let job_creator = Arc::new(AggregationJobCreator::new( - ds, - noop_meter(), - Duration::from_secs(3600), - Duration::from_secs(1), - MIN_AGGREGATION_JOB_SIZE, - MAX_AGGREGATION_JOB_SIZE, - )); - Arc::clone(&job_creator) - .create_aggregation_jobs_for_task(Arc::clone(&task)) - .await - .unwrap(); - - // Verify. - let (agg_jobs, batches) = job_creator - .datastore - .run_tx(|tx| { - let task = task.clone(); - Box::pin(async move { - Ok(read_aggregate_info_for_task::< - PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, - Prio3Count, - _, - >(tx, task.id()) - .await) - }) - }) - .await - .unwrap(); - let mut seen_report_ids = HashSet::new(); - for (agg_job, report_ids) in &agg_jobs { - // Jobs are created in round 0 - assert_eq!(agg_job.round(), AggregationJobRound::from(0)); - - // The batch is at most MAX_AGGREGATION_JOB_SIZE in size. - assert!(report_ids.len() <= MAX_AGGREGATION_JOB_SIZE); - - // The batch is at least MIN_AGGREGATION_JOB_SIZE in size. - assert!(report_ids.len() >= MIN_AGGREGATION_JOB_SIZE); - - // Report IDs are non-repeated across or inside aggregation jobs. - for report_id in report_ids { - assert!(!seen_report_ids.contains(report_id)); - seen_report_ids.insert(*report_id); - } - } - - // Every client report was added to some aggregation job. - assert_eq!(all_report_ids, seen_report_ids); - - // Batches are created appropriately. - assert_eq!( - batches, - Vec::from([Batch::new( - *task.id(), - batch_identifier, - (), - BatchState::Open, - agg_jobs.len().try_into().unwrap(), - Interval::from_time(&report_time).unwrap(), - )]) - ); - } - - #[tokio::test] - async fn create_aggregation_jobs_for_time_interval_task_not_enough_reports() { - // Setup. - install_test_trace_subscriber(); - let clock = MockClock::default(); - let ephemeral_datastore = ephemeral_datastore().await; - let ds = ephemeral_datastore.datastore(clock.clone()).await; - let task = Arc::new( - TaskBuilder::new( - TaskQueryType::TimeInterval, - VdafInstance::Prio3Count, - Role::Leader, - ) - .build(), - ); - let report_time = clock.now(); - let batch_identifier = TimeInterval::to_batch_identifier(&task, &(), &report_time).unwrap(); - let first_report = LeaderStoredReport::new_dummy(*task.id(), report_time); - let second_report = LeaderStoredReport::new_dummy(*task.id(), report_time); - - ds.run_tx(|tx| { - let (task, first_report) = (Arc::clone(&task), first_report.clone()); - Box::pin(async move { - tx.put_task(&task).await?; - tx.put_client_report(&dummy_vdaf::Vdaf::new(), &first_report) - .await - }) - }) - .await - .unwrap(); - - // Run. - let job_creator = Arc::new(AggregationJobCreator::new( - ds, - noop_meter(), - Duration::from_secs(3600), - Duration::from_secs(1), - 2, - 100, - )); - Arc::clone(&job_creator) - .create_aggregation_jobs_for_task(Arc::clone(&task)) - .await - .unwrap(); - - // Verify -- we haven't received enough reports yet, so we don't create anything. - let (agg_jobs, batches) = job_creator - .datastore - .run_tx(|tx| { - let task = Arc::clone(&task); - Box::pin(async move { - Ok(read_aggregate_info_for_task::< - PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, - Prio3Count, - _, - >(tx, task.id()) - .await) - }) - }) - .await - .unwrap(); - assert!(agg_jobs.is_empty()); - assert!(batches.is_empty()); - - // Setup again -- add another report. - job_creator - .datastore - .run_tx(|tx| { - let second_report = second_report.clone(); - Box::pin(async move { - tx.put_client_report(&dummy_vdaf::Vdaf::new(), &second_report) - .await - }) - }) - .await - .unwrap(); - - // Run. - Arc::clone(&job_creator) - .create_aggregation_jobs_for_task(Arc::clone(&task)) - .await - .unwrap(); - - // Verify -- the additional report we wrote allows an aggregation job to be created. - let (agg_jobs, batches) = job_creator - .datastore - .run_tx(|tx| { - let task = Arc::clone(&task); - Box::pin(async move { - Ok(read_aggregate_info_for_task::< - PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, - Prio3Count, - _, - >(tx, task.id()) - .await) - }) - }) - .await - .unwrap(); - assert_eq!(agg_jobs.len(), 1); - let report_ids: HashSet<_> = agg_jobs.into_iter().next().unwrap().1.into_iter().collect(); - assert_eq!( - report_ids, - HashSet::from([ - *first_report.metadata().id(), - *second_report.metadata().id() - ]) - ); - - assert_eq!( - batches, - Vec::from([Batch::new( - *task.id(), - batch_identifier, - (), - BatchState::Open, - 1, - Interval::from_time(&report_time).unwrap(), - )]) - ); - } - - #[tokio::test] - async fn create_aggregation_jobs_for_time_interval_task_batch_closed() { - // Setup. - install_test_trace_subscriber(); - let clock = MockClock::default(); - let ephemeral_datastore = ephemeral_datastore().await; - let ds = ephemeral_datastore.datastore(clock.clone()).await; - const MIN_AGGREGATION_JOB_SIZE: usize = 50; - const MAX_AGGREGATION_JOB_SIZE: usize = 60; - - let task = Arc::new( - TaskBuilder::new( - TaskQueryType::TimeInterval, - VdafInstance::Prio3Count, - Role::Leader, - ) - .build(), - ); - - // Create a min-size batch. - let report_time = clock.now(); - let batch_identifier = TimeInterval::to_batch_identifier(&task, &(), &report_time).unwrap(); - let reports: Vec<_> = - iter::repeat_with(|| LeaderStoredReport::new_dummy(*task.id(), report_time)) - .take(2 * MAX_AGGREGATION_JOB_SIZE + MIN_AGGREGATION_JOB_SIZE + 1) - .collect(); - let all_report_ids: HashSet = reports - .iter() - .map(|report| *report.metadata().id()) - .collect(); - - ds.run_tx(|tx| { - let (task, reports) = (Arc::clone(&task), reports.clone()); - Box::pin(async move { - tx.put_task(&task).await?; - for report in reports.iter() { - tx.put_client_report(&dummy_vdaf::Vdaf::new(), report) - .await?; - } - tx.put_batch( - &Batch::::new( - *task.id(), - batch_identifier, - (), - BatchState::Closed, - 0, - Interval::from_time(&report_time).unwrap(), - ), - ) - .await?; - Ok(()) - }) - }) - .await - .unwrap(); - - // Run. - let job_creator = Arc::new(AggregationJobCreator::new( - ds, - noop_meter(), - Duration::from_secs(3600), - Duration::from_secs(1), - MIN_AGGREGATION_JOB_SIZE, - MAX_AGGREGATION_JOB_SIZE, - )); - Arc::clone(&job_creator) - .create_aggregation_jobs_for_task(Arc::clone(&task)) - .await - .unwrap(); - - // Verify. - let (agg_jobs, batches) = job_creator - .datastore - .run_tx(|tx| { - let task = task.clone(); - Box::pin(async move { - Ok(read_aggregate_info_for_task::< - PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, - Prio3Count, - _, - >(tx, task.id()) - .await) - }) - }) - .await - .unwrap(); - let mut seen_report_ids = HashSet::new(); - for (agg_job, report_ids) in &agg_jobs { - // Job immediately finished since all reports are in a closed batch. - assert_eq!(agg_job.state(), &AggregationJobState::Finished); - - // Jobs are created in round 0. - assert_eq!(agg_job.round(), AggregationJobRound::from(0)); - - // The batch is at most MAX_AGGREGATION_JOB_SIZE in size. - assert!(report_ids.len() <= MAX_AGGREGATION_JOB_SIZE); - - // The batch is at least MIN_AGGREGATION_JOB_SIZE in size. - assert!(report_ids.len() >= MIN_AGGREGATION_JOB_SIZE); - - // Report IDs are non-repeated across or inside aggregation jobs. - for report_id in report_ids { - assert!(!seen_report_ids.contains(report_id)); - seen_report_ids.insert(*report_id); - } - } - - // Every client report was added to some aggregation job. - assert_eq!(all_report_ids, seen_report_ids); - - // Batches are created appropriately. - assert_eq!( - batches, - Vec::from([Batch::new( - *task.id(), - batch_identifier, - (), - BatchState::Closed, - 0, - Interval::from_time(&report_time).unwrap(), - )]) - ); - } - #[tokio::test] async fn create_aggregation_jobs_for_fixed_size_task() { // Setup. diff --git a/aggregator/src/aggregator/aggregation_job_driver.rs b/aggregator/src/aggregator/aggregation_job_driver.rs index 8b6d9a20f..8b4c58778 100644 --- a/aggregator/src/aggregator/aggregation_job_driver.rs +++ b/aggregator/src/aggregator/aggregation_job_driver.rs @@ -19,10 +19,9 @@ use janus_aggregator_core::{ }; use janus_core::{time::Clock, vdaf_dispatch}; use janus_messages::{ - query_type::{FixedSize, TimeInterval}, - AggregationJobContinueReq, AggregationJobInitializeReq, AggregationJobResp, - PartialBatchSelector, PrepareStep, PrepareStepResult, ReportId, ReportShare, ReportShareError, - Role, + query_type::FixedSize, AggregationJobContinueReq, AggregationJobInitializeReq, + AggregationJobResp, PartialBatchSelector, PrepareStep, PrepareStepResult, ReportId, + ReportShare, ReportShareError, Role, }; use opentelemetry::{ metrics::{Counter, Histogram, Meter, Unit}, @@ -101,11 +100,6 @@ impl AggregationJobDriver { lease: Arc>, ) -> Result<()> { match lease.leased().query_type() { - task::QueryType::TimeInterval => { - vdaf_dispatch!(lease.leased().vdaf(), (vdaf, VdafType, VERIFY_KEY_LENGTH) => { - self.step_aggregation_job_generic::(datastore, Arc::new(vdaf), lease).await - }) - } task::QueryType::FixedSize { .. } => { vdaf_dispatch!(lease.leased().vdaf(), (vdaf, VdafType, VERIFY_KEY_LENGTH) => { self.step_aggregation_job_generic::(datastore, Arc::new(vdaf), lease).await @@ -726,17 +720,6 @@ impl AggregationJobDriver { lease: Lease, ) -> Result<()> { match lease.leased().query_type() { - task::QueryType::TimeInterval => { - vdaf_dispatch!(lease.leased().vdaf(), (vdaf, VdafType, VERIFY_KEY_LENGTH) => { - self.cancel_aggregation_job_generic::< - VERIFY_KEY_LENGTH, - C, - TimeInterval, - VdafType, - >(vdaf, datastore, lease) - .await - }) - } task::QueryType::FixedSize { .. } => { vdaf_dispatch!(lease.leased().vdaf(), (vdaf, VdafType, VERIFY_KEY_LENGTH) => { self.cancel_aggregation_job_generic::< @@ -910,7 +893,7 @@ mod tests { }, test_util::ephemeral_datastore, }, - query_type::{AccumulableQueryType, CollectableQueryType}, + query_type::CollectableQueryType, task::{test_util::TaskBuilder, QueryType, VerifyKey}, test_util::noop_meter, }; @@ -925,11 +908,10 @@ mod tests { Runtime, }; use janus_messages::{ - query_type::{FixedSize, TimeInterval}, - AggregationJobContinueReq, AggregationJobInitializeReq, AggregationJobResp, - AggregationJobRound, Duration, Extension, ExtensionType, HpkeConfig, InputShareAad, + query_type::FixedSize, AggregationJobContinueReq, AggregationJobInitializeReq, + AggregationJobResp, AggregationJobRound, Duration, Extension, HpkeConfig, InputShareAad, Interval, PartialBatchSelector, PlaintextInputShare, PrepareStep, PrepareStepResult, - ReportIdChecksum, ReportMetadata, ReportShare, ReportShareError, Role, TaskId, Time, + ReportIdChecksum, ReportMetadata, ReportShare, Role, TaskId, Time, }; use prio::{ codec::Encode, @@ -961,7 +943,7 @@ mod tests { let ds = Arc::new(ephemeral_datastore.datastore(clock.clone()).await); let vdaf = Arc::new(Prio3::new_count(2).unwrap()); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -975,7 +957,7 @@ mod tests { .now() .to_batch_interval_start(task.time_precision()) .unwrap(); - let batch_identifier = TimeInterval::to_batch_identifier(&task, &(), &time).unwrap(); + let batch_id = random(); let report_metadata = ReportMetadata::new(random(), time); let verify_key: VerifyKey = task.primary_vdaf_verify_key().unwrap(); @@ -1012,13 +994,13 @@ mod tests { tx.put_aggregation_job(&AggregationJob::< PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, Prio3Count, >::new( *task.id(), aggregation_job_id, (), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -1040,9 +1022,9 @@ mod tests { .await?; tx.put_batch( - &Batch::::new( + &Batch::::new( *task.id(), - batch_identifier, + batch_id, (), BatchState::Closing, 1, @@ -1052,10 +1034,10 @@ mod tests { .await?; let collection_job = - CollectionJob::::new( + CollectionJob::::new( *task.id(), random(), - batch_identifier, + batch_id, (), CollectionJobState::Start, ); @@ -1072,7 +1054,7 @@ mod tests { let helper_responses = Vec::from([ ( "PUT", - AggregationJobInitializeReq::::MEDIA_TYPE, + AggregationJobInitializeReq::::MEDIA_TYPE, AggregationJobResp::MEDIA_TYPE, AggregationJobResp::new(Vec::from([PrepareStep::new( *report.metadata().id(), @@ -1157,11 +1139,11 @@ mod tests { } let want_aggregation_job = - AggregationJob::::new( + AggregationJob::::new( *task.id(), aggregation_job_id, (), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::Finished, @@ -1176,9 +1158,9 @@ mod tests { None, ReportAggregationState::Finished, ); - let want_batch = Batch::::new( + let want_batch = Batch::::new( *task.id(), - batch_identifier, + batch_id, (), BatchState::Closed, 0, @@ -1195,7 +1177,7 @@ mod tests { Box::pin(async move { let aggregation_job = tx - .get_aggregation_job::( + .get_aggregation_job::( task.id(), &aggregation_job_id, ) @@ -1211,10 +1193,7 @@ mod tests { ) .await? .unwrap(); - let batch = tx - .get_batch(task.id(), &batch_identifier, &()) - .await? - .unwrap(); + let batch = tx.get_batch(task.id(), &batch_id, &()).await?.unwrap(); let collection_job = tx .get_collection_job(vdaf.as_ref(), &collection_job_id) .await? @@ -1232,7 +1211,7 @@ mod tests { } #[tokio::test] - async fn step_time_interval_aggregation_job_init() { + async fn step_fixed_size_aggregation_job_init() { // Setup: insert a client report and add it to a new aggregation job. install_test_trace_subscriber(); let mut server = mockito::Server::new_async().await; @@ -1242,7 +1221,7 @@ mod tests { let vdaf = Arc::new(Prio3::new_count(2).unwrap()); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -1252,12 +1231,13 @@ mod tests { ])) .build(); - let time = clock - .now() - .to_batch_interval_start(task.time_precision()) - .unwrap(); - let batch_identifier = TimeInterval::to_batch_identifier(&task, &(), &time).unwrap(); - let report_metadata = ReportMetadata::new(random(), time); + let report_metadata = ReportMetadata::new( + random(), + clock + .now() + .to_batch_interval_start(task.time_precision()) + .unwrap(), + ); let verify_key: VerifyKey = task.primary_vdaf_verify_key().unwrap(); @@ -1279,43 +1259,25 @@ mod tests { Vec::new(), transcript.input_shares.clone(), ); - let repeated_extension_report = generate_report::( - *task.id(), - ReportMetadata::new(random(), time), - helper_hpke_keypair.config(), - transcript.public_share.clone(), - Vec::from([ - Extension::new(ExtensionType::Tbd, Vec::new()), - Extension::new(ExtensionType::Tbd, Vec::new()), - ]), - transcript.input_shares.clone(), - ); - let missing_report_id = random(); + let batch_id = random(); let aggregation_job_id = random(); let lease = ds .run_tx(|tx| { - let (vdaf, task, report, repeated_extension_report) = ( - vdaf.clone(), - task.clone(), - report.clone(), - repeated_extension_report.clone(), - ); + let (vdaf, task, report) = (vdaf.clone(), task.clone(), report.clone()); Box::pin(async move { tx.put_task(&task).await?; tx.put_client_report(vdaf.borrow(), &report).await?; - tx.put_client_report(vdaf.borrow(), &repeated_extension_report) - .await?; tx.put_aggregation_job(&AggregationJob::< PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, Prio3Count, >::new( *task.id(), aggregation_job_id, (), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -1335,41 +1297,15 @@ mod tests { ReportAggregationState::Start, )) .await?; - tx.put_report_aggregation(&ReportAggregation::< - PRIO3_VERIFY_KEY_LENGTH, - Prio3Count, - >::new( - *task.id(), - aggregation_job_id, - *repeated_extension_report.metadata().id(), - *repeated_extension_report.metadata().time(), - 1, - None, - ReportAggregationState::Start, - )) - .await?; - tx.put_report_aggregation(&ReportAggregation::< - PRIO3_VERIFY_KEY_LENGTH, - Prio3Count, - >::new( - *task.id(), - aggregation_job_id, - missing_report_id, - time, - 2, - None, - ReportAggregationState::Start, - )) - .await?; tx.put_batch( - &Batch::::new( + &Batch::::new( *task.id(), - batch_identifier, + batch_id, (), - BatchState::Closing, + BatchState::Open, 1, - Interval::from_time(&time).unwrap(), + Interval::from_time(report.metadata().time()).unwrap(), ), ) .await?; @@ -1391,7 +1327,7 @@ mod tests { // verification -- but mockito does not expose this functionality at time of writing.) let leader_request = AggregationJobInitializeReq::new( ().get_encoded(), - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(batch_id), Vec::from([ReportShare::new( report.metadata().clone(), report.public_share().get_encoded(), @@ -1428,7 +1364,7 @@ mod tests { ) .match_header( CONTENT_TYPE.as_str(), - AggregationJobInitializeReq::::MEDIA_TYPE, + AggregationJobInitializeReq::::MEDIA_TYPE, ) .match_body(leader_request.get_encoded()) .with_status(200) @@ -1464,18 +1400,16 @@ mod tests { mocked_aggregate_success.assert_async().await; let want_aggregation_job = - AggregationJob::::new( + AggregationJob::::new( *task.id(), aggregation_job_id, (), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, AggregationJobRound::from(1), ); - let leader_prep_state = transcript.leader_prep_state(0).clone(); - let prep_msg = transcript.prepare_messages[0].clone(); let want_report_aggregation = ReportAggregation::::new( *task.id(), aggregation_job_id, @@ -1483,54 +1417,27 @@ mod tests { *report.metadata().time(), 0, None, - ReportAggregationState::Waiting(leader_prep_state, Some(prep_msg)), + ReportAggregationState::Waiting( + transcript.leader_prep_state(0).clone(), + Some(transcript.prepare_messages[0].clone()), + ), ); - let want_repeated_extension_report_aggregation = - ReportAggregation::::new( - *task.id(), - aggregation_job_id, - *repeated_extension_report.metadata().id(), - *repeated_extension_report.metadata().time(), - 1, - None, - ReportAggregationState::Failed(ReportShareError::UnrecognizedMessage), - ); - let want_missing_report_report_aggregation = - ReportAggregation::::new( - *task.id(), - aggregation_job_id, - missing_report_id, - time, - 2, - None, - ReportAggregationState::Failed(ReportShareError::ReportDropped), - ); - let want_batch = Batch::::new( + let want_batch = Batch::::new( *task.id(), - batch_identifier, + batch_id, (), - BatchState::Closing, + BatchState::Open, 1, - Interval::from_time(&time).unwrap(), + Interval::from_time(report.metadata().time()).unwrap(), ); - let ( - got_aggregation_job, - got_report_aggregation, - got_repeated_extension_report_aggregation, - got_missing_report_report_aggregation, - got_batch, - ) = ds + let (got_aggregation_job, got_report_aggregation, got_batch) = ds .run_tx(|tx| { - let (vdaf, task, report_id, repeated_extension_report_id) = ( - Arc::clone(&vdaf), - task.clone(), - *report.metadata().id(), - *repeated_extension_report.metadata().id(), - ); + let (vdaf, task, report_id) = + (Arc::clone(&vdaf), task.clone(), *report.metadata().id()); Box::pin(async move { let aggregation_job = tx - .get_aggregation_job::( + .get_aggregation_job::( task.id(), &aggregation_job_id, ) @@ -1546,37 +1453,8 @@ mod tests { ) .await? .unwrap(); - let repeated_extension_report_aggregation = tx - .get_report_aggregation( - vdaf.as_ref(), - &Role::Leader, - task.id(), - &aggregation_job_id, - &repeated_extension_report_id, - ) - .await? - .unwrap(); - let missing_report_report_aggregation = tx - .get_report_aggregation( - vdaf.as_ref(), - &Role::Leader, - task.id(), - &aggregation_job_id, - &missing_report_id, - ) - .await? - .unwrap(); - let batch = tx - .get_batch(task.id(), &batch_identifier, &()) - .await? - .unwrap(); - Ok(( - aggregation_job, - report_aggregation, - repeated_extension_report_aggregation, - missing_report_report_aggregation, - batch, - )) + let batch = tx.get_batch(task.id(), &batch_id, &()).await?.unwrap(); + Ok((aggregation_job, report_aggregation, batch)) }) }) .await @@ -1584,20 +1462,13 @@ mod tests { assert_eq!(want_aggregation_job, got_aggregation_job); assert_eq!(want_report_aggregation, got_report_aggregation); - assert_eq!( - want_repeated_extension_report_aggregation, - got_repeated_extension_report_aggregation - ); - assert_eq!( - want_missing_report_report_aggregation, - got_missing_report_report_aggregation - ); assert_eq!(want_batch, got_batch); } #[tokio::test] - async fn step_fixed_size_aggregation_job_init() { - // Setup: insert a client report and add it to a new aggregation job. + async fn step_fixed_size_aggregation_job_continue() { + // Setup: insert a client report and add it to an aggregation job whose state has already + // been stepped once. install_test_trace_subscriber(); let mut server = mockito::Server::new_async().await; let clock = MockClock::default(); @@ -1615,7 +1486,6 @@ mod tests { Url::parse(&server.url()).unwrap(), ])) .build(); - let report_metadata = ReportMetadata::new( random(), clock @@ -1646,10 +1516,21 @@ mod tests { ); let batch_id = random(); let aggregation_job_id = random(); + let leader_prep_state = transcript.leader_prep_state(0); + let leader_aggregate_share = vdaf + .aggregate(&(), [transcript.output_share(Role::Leader).clone()]) + .unwrap(); + let prep_msg = &transcript.prepare_messages[0]; - let lease = ds + let (lease, collection_job) = ds .run_tx(|tx| { - let (vdaf, task, report) = (vdaf.clone(), task.clone(), report.clone()); + let (vdaf, task, report, leader_prep_state, prep_msg) = ( + vdaf.clone(), + task.clone(), + report.clone(), + leader_prep_state.clone(), + prep_msg.clone(), + ); Box::pin(async move { tx.put_task(&task).await?; tx.put_client_report(vdaf.borrow(), &report).await?; @@ -1666,7 +1547,7 @@ mod tests { Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, - AggregationJobRound::from(0), + AggregationJobRound::from(1), )) .await?; tx.put_report_aggregation(&ReportAggregation::< @@ -1679,7 +1560,7 @@ mod tests { *report.metadata().time(), 0, None, - ReportAggregationState::Start, + ReportAggregationState::Waiting(leader_prep_state, Some(prep_msg)), )) .await?; @@ -1688,17 +1569,29 @@ mod tests { *task.id(), batch_id, (), - BatchState::Open, + BatchState::Closing, 1, Interval::from_time(report.metadata().time()).unwrap(), ), ) .await?; - Ok(tx + let collection_job = + CollectionJob::::new( + *task.id(), + random(), + batch_id, + (), + CollectionJobState::Start, + ); + tx.put_collection_job(&collection_job).await?; + + let lease = tx .acquire_incomplete_aggregation_jobs(&StdDuration::from_secs(60), 1) .await? - .remove(0)) + .remove(0); + + Ok((lease, collection_job)) }) }) .await @@ -1706,39 +1599,36 @@ mod tests { assert_eq!(lease.leased().task_id(), task.id()); assert_eq!(lease.leased().aggregation_job_id(), &aggregation_job_id); - // Setup: prepare mocked HTTP response. (first an error response, then a success) + // Setup: prepare mocked HTTP responses. (first an error response, then a success) // (This is fragile in that it expects the leader request to be deterministically encoded. // It would be nicer to retrieve the request bytes from the mock, then do our own parsing & // verification -- but mockito does not expose this functionality at time of writing.) - let leader_request = AggregationJobInitializeReq::new( - ().get_encoded(), - PartialBatchSelector::new_fixed_size(batch_id), - Vec::from([ReportShare::new( - report.metadata().clone(), - report.public_share().get_encoded(), - report.helper_encrypted_input_share().clone(), + let leader_request = AggregationJobContinueReq::new( + AggregationJobRound::from(1), + Vec::from([PrepareStep::new( + *report.metadata().id(), + PrepareStepResult::Continued(prep_msg.get_encoded()), )]), ); - let (_, helper_vdaf_msg) = transcript.helper_prep_state(0); let helper_response = AggregationJobResp::new(Vec::from([PrepareStep::new( *report.metadata().id(), - PrepareStepResult::Continued(helper_vdaf_msg.get_encoded()), + PrepareStepResult::Finished, )])); let mocked_aggregate_failure = server .mock( - "PUT", + "POST", task.aggregation_job_uri(&aggregation_job_id) .unwrap() .path(), ) .with_status(500) .with_header("Content-Type", "application/problem+json") - .with_body("{\"type\": \"urn:ietf:params:ppm:dap:error:unauthorizedRequest\"}") + .with_body("{\"type\": \"urn:ietf:params:ppm:dap:error:unrecognizedTask\"}") .create_async() .await; let mocked_aggregate_success = server .mock( - "PUT", + "POST", task.aggregation_job_uri(&aggregation_job_id) .unwrap() .path(), @@ -1747,10 +1637,7 @@ mod tests { "DAP-Auth-Token", str::from_utf8(agg_auth_token.as_ref()).unwrap(), ) - .match_header( - CONTENT_TYPE.as_str(), - AggregationJobInitializeReq::::MEDIA_TYPE, - ) + .match_header(CONTENT_TYPE.as_str(), AggregationJobContinueReq::MEDIA_TYPE) .match_body(leader_request.get_encoded()) .with_status(200) .with_header(CONTENT_TYPE.as_str(), AggregationJobResp::MEDIA_TYPE) @@ -1772,7 +1659,7 @@ mod tests { error.downcast().unwrap(), Error::Http { problem_details, dap_problem_type } => { assert_eq!(problem_details.status.unwrap(), StatusCode::INTERNAL_SERVER_ERROR); - assert_eq!(dap_problem_type, Some(DapProblemType::UnauthorizedRequest)); + assert_eq!(dap_problem_type, Some(DapProblemType::UnrecognizedTask)); } ); aggregation_job_driver @@ -1792,8 +1679,8 @@ mod tests { batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), - AggregationJobState::InProgress, - AggregationJobRound::from(1), + AggregationJobState::Finished, + AggregationJobRound::from(2), ); let want_report_aggregation = ReportAggregation::::new( *task.id(), @@ -1802,694 +1689,31 @@ mod tests { *report.metadata().time(), 0, None, - ReportAggregationState::Waiting( - transcript.leader_prep_state(0).clone(), - Some(transcript.prepare_messages[0].clone()), - ), + ReportAggregationState::Finished, ); - let want_batch = Batch::::new( + let want_batch_aggregations = Vec::from([BatchAggregation::< + PRIO3_VERIFY_KEY_LENGTH, + FixedSize, + Prio3Count, + >::new( *task.id(), batch_id, (), - BatchState::Open, + 0, + BatchAggregationState::Aggregating, + Some(leader_aggregate_share), 1, + ReportIdChecksum::for_report_id(report.metadata().id()), + )]); + let want_batch = Batch::::new( + *task.id(), + batch_id, + (), + BatchState::Closed, + 0, Interval::from_time(report.metadata().time()).unwrap(), ); - - let (got_aggregation_job, got_report_aggregation, got_batch) = ds - .run_tx(|tx| { - let (vdaf, task, report_id) = - (Arc::clone(&vdaf), task.clone(), *report.metadata().id()); - Box::pin(async move { - let aggregation_job = tx - .get_aggregation_job::( - task.id(), - &aggregation_job_id, - ) - .await? - .unwrap(); - let report_aggregation = tx - .get_report_aggregation( - vdaf.as_ref(), - &Role::Leader, - task.id(), - &aggregation_job_id, - &report_id, - ) - .await? - .unwrap(); - let batch = tx.get_batch(task.id(), &batch_id, &()).await?.unwrap(); - Ok((aggregation_job, report_aggregation, batch)) - }) - }) - .await - .unwrap(); - - assert_eq!(want_aggregation_job, got_aggregation_job); - assert_eq!(want_report_aggregation, got_report_aggregation); - assert_eq!(want_batch, got_batch); - } - - #[tokio::test] - async fn step_time_interval_aggregation_job_continue() { - // Setup: insert a client report and add it to an aggregation job whose state has already - // been stepped once. - install_test_trace_subscriber(); - let mut server = mockito::Server::new_async().await; - let clock = MockClock::default(); - let ephemeral_datastore = ephemeral_datastore().await; - let ds = Arc::new(ephemeral_datastore.datastore(clock.clone()).await); - let vdaf = Arc::new(Prio3::new_count(2).unwrap()); - - let task = TaskBuilder::new( - QueryType::TimeInterval, - VdafInstance::Prio3Count, - Role::Leader, - ) - .with_aggregator_endpoints(Vec::from([ - Url::parse("http://irrelevant").unwrap(), // leader URL doesn't matter - Url::parse(&server.url()).unwrap(), - ])) - .build(); - let time = clock - .now() - .to_batch_interval_start(task.time_precision()) - .unwrap(); - let active_batch_identifier = TimeInterval::to_batch_identifier(&task, &(), &time).unwrap(); - let other_batch_identifier = Interval::new( - active_batch_identifier - .start() - .add(task.time_precision()) - .unwrap(), - *task.time_precision(), - ) - .unwrap(); - let collection_identifier = Interval::new( - *active_batch_identifier.start(), - Duration::from_seconds(2 * task.time_precision().as_seconds()), - ) - .unwrap(); - let report_metadata = ReportMetadata::new(random(), time); - let verify_key: VerifyKey = - task.primary_vdaf_verify_key().unwrap(); - - let transcript = run_vdaf( - vdaf.as_ref(), - verify_key.as_bytes(), - &(), - report_metadata.id(), - &0, - ); - - let agg_auth_token = task.primary_aggregator_auth_token(); - let helper_hpke_keypair = generate_test_hpke_config_and_private_key(); - let report = generate_report::( - *task.id(), - report_metadata, - helper_hpke_keypair.config(), - transcript.public_share.clone(), - Vec::new(), - transcript.input_shares.clone(), - ); - let aggregation_job_id = random(); - - let leader_prep_state = transcript.leader_prep_state(0); - let leader_aggregate_share = vdaf - .aggregate(&(), [transcript.output_share(Role::Leader).clone()]) - .unwrap(); - let prep_msg = &transcript.prepare_messages[0]; - - let (lease, want_collection_job) = ds - .run_tx(|tx| { - let (vdaf, task, report, leader_prep_state, prep_msg) = ( - vdaf.clone(), - task.clone(), - report.clone(), - leader_prep_state.clone(), - prep_msg.clone(), - ); - Box::pin(async move { - tx.put_task(&task).await?; - tx.put_client_report(vdaf.borrow(), &report).await?; - tx.mark_report_aggregated(task.id(), report.metadata().id()) - .await?; - - tx.put_aggregation_job(&AggregationJob::< - PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, - Prio3Count, - >::new( - *task.id(), - aggregation_job_id, - (), - (), - Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) - .unwrap(), - AggregationJobState::InProgress, - AggregationJobRound::from(1), - )) - .await?; - tx.put_report_aggregation(&ReportAggregation::< - PRIO3_VERIFY_KEY_LENGTH, - Prio3Count, - >::new( - *task.id(), - aggregation_job_id, - *report.metadata().id(), - *report.metadata().time(), - 0, - None, - ReportAggregationState::Waiting(leader_prep_state, Some(prep_msg)), - )) - .await?; - - tx.put_batch( - &Batch::::new( - *task.id(), - active_batch_identifier, - (), - BatchState::Closing, - 1, - Interval::from_time(report.metadata().time()).unwrap(), - ), - ) - .await?; - tx.put_batch( - &Batch::::new( - *task.id(), - other_batch_identifier, - (), - BatchState::Closing, - 1, - Interval::EMPTY, - ), - ) - .await?; - - let collection_job = - CollectionJob::::new( - *task.id(), - random(), - collection_identifier, - (), - CollectionJobState::Start, - ); - tx.put_collection_job(&collection_job).await?; - - let lease = tx - .acquire_incomplete_aggregation_jobs(&StdDuration::from_secs(60), 1) - .await? - .remove(0); - - Ok((lease, collection_job)) - }) - }) - .await - .unwrap(); - assert_eq!(lease.leased().task_id(), task.id()); - assert_eq!(lease.leased().aggregation_job_id(), &aggregation_job_id); - - // Setup: prepare mocked HTTP responses. (first an error response, then a success) - // (This is fragile in that it expects the leader request to be deterministically encoded. - // It would be nicer to retrieve the request bytes from the mock, then do our own parsing & - // verification -- but mockito does not expose this functionality at time of writing.) - let leader_request = AggregationJobContinueReq::new( - AggregationJobRound::from(1), - Vec::from([PrepareStep::new( - *report.metadata().id(), - PrepareStepResult::Continued(prep_msg.get_encoded()), - )]), - ); - let helper_response = AggregationJobResp::new(Vec::from([PrepareStep::new( - *report.metadata().id(), - PrepareStepResult::Finished, - )])); - let mocked_aggregate_failure = server - .mock( - "POST", - task.aggregation_job_uri(&aggregation_job_id) - .unwrap() - .path(), - ) - .with_status(500) - .with_header("Content-Type", "application/problem+json") - .with_body("{\"type\": \"urn:ietf:params:ppm:dap:error:unrecognizedTask\"}") - .create_async() - .await; - let mocked_aggregate_success = server - .mock( - "POST", - task.aggregation_job_uri(&aggregation_job_id) - .unwrap() - .path(), - ) - .match_header( - "DAP-Auth-Token", - str::from_utf8(agg_auth_token.as_ref()).unwrap(), - ) - .match_header(CONTENT_TYPE.as_str(), AggregationJobContinueReq::MEDIA_TYPE) - .match_body(leader_request.get_encoded()) - .with_status(200) - .with_header(CONTENT_TYPE.as_str(), AggregationJobResp::MEDIA_TYPE) - .with_body(helper_response.get_encoded()) - .create_async() - .await; - - // Run: create an aggregation job driver & try to step the aggregation we've created twice. - let aggregation_job_driver = AggregationJobDriver::new( - reqwest::Client::builder().build().unwrap(), - &noop_meter(), - 32, - ); - let error = aggregation_job_driver - .step_aggregation_job(ds.clone(), Arc::new(lease.clone())) - .await - .unwrap_err(); - assert_matches!( - error.downcast().unwrap(), - Error::Http { problem_details, dap_problem_type } => { - assert_eq!(problem_details.status.unwrap(), StatusCode::INTERNAL_SERVER_ERROR); - assert_eq!(dap_problem_type, Some(DapProblemType::UnrecognizedTask)); - } - ); - aggregation_job_driver - .step_aggregation_job(ds.clone(), Arc::new(lease)) - .await - .unwrap(); - - // Verify. - mocked_aggregate_failure.assert_async().await; - mocked_aggregate_success.assert_async().await; - - let want_aggregation_job = - AggregationJob::::new( - *task.id(), - aggregation_job_id, - (), - (), - Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) - .unwrap(), - AggregationJobState::Finished, - AggregationJobRound::from(2), - ); - let want_report_aggregation = ReportAggregation::::new( - *task.id(), - aggregation_job_id, - *report.metadata().id(), - *report.metadata().time(), - 0, - None, - ReportAggregationState::Finished, - ); - let batch_interval_start = report - .metadata() - .time() - .to_batch_interval_start(task.time_precision()) - .unwrap(); - let want_batch_aggregations = Vec::from([BatchAggregation::< - PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, - Prio3Count, - >::new( - *task.id(), - Interval::new(batch_interval_start, *task.time_precision()).unwrap(), - (), - 0, - BatchAggregationState::Aggregating, - Some(leader_aggregate_share), - 1, - ReportIdChecksum::for_report_id(report.metadata().id()), - )]); - let want_active_batch = Batch::::new( - *task.id(), - active_batch_identifier, - (), - BatchState::Closed, - 0, - Interval::from_time(report.metadata().time()).unwrap(), - ); - let want_other_batch = Batch::::new( - *task.id(), - other_batch_identifier, - (), - BatchState::Closing, - 1, - Interval::EMPTY, - ); - - let ( - got_aggregation_job, - got_report_aggregation, - got_batch_aggregations, - got_active_batch, - got_other_batch, - got_collection_job, - ) = ds - .run_tx(|tx| { - let vdaf = Arc::clone(&vdaf); - let task = task.clone(); - let report_metadata = report.metadata().clone(); - let collection_job_id = *want_collection_job.id(); - - Box::pin(async move { - let aggregation_job = tx - .get_aggregation_job::( - task.id(), - &aggregation_job_id, - ) - .await? - .unwrap(); - let report_aggregation = tx - .get_report_aggregation( - vdaf.as_ref(), - &Role::Leader, - task.id(), - &aggregation_job_id, - report_metadata.id(), - ) - .await? - .unwrap(); - let batch_aggregations = - TimeInterval::get_batch_aggregations_for_collection_identifier::< - PRIO3_VERIFY_KEY_LENGTH, - Prio3Count, - _, - >( - tx, - &task, - &vdaf, - &Interval::new( - report_metadata - .time() - .to_batch_interval_start(task.time_precision()) - .unwrap(), - *task.time_precision(), - ) - .unwrap(), - &(), - ) - .await - .unwrap(); - let got_active_batch = tx - .get_batch(task.id(), &active_batch_identifier, &()) - .await? - .unwrap(); - let got_other_batch = tx - .get_batch(task.id(), &other_batch_identifier, &()) - .await? - .unwrap(); - let got_collection_job = tx - .get_collection_job(vdaf.as_ref(), &collection_job_id) - .await? - .unwrap(); - - Ok(( - aggregation_job, - report_aggregation, - batch_aggregations, - got_active_batch, - got_other_batch, - got_collection_job, - )) - }) - }) - .await - .unwrap(); - - // Map the batch aggregation ordinal value to 0, as it may vary due to sharding. - let got_batch_aggregations: Vec<_> = got_batch_aggregations - .into_iter() - .map(|agg| { - BatchAggregation::new( - *agg.task_id(), - *agg.batch_identifier(), - (), - 0, - *agg.state(), - agg.aggregate_share().cloned(), - agg.report_count(), - *agg.checksum(), - ) - }) - .collect(); - - assert_eq!(want_aggregation_job, got_aggregation_job); - assert_eq!(want_report_aggregation, got_report_aggregation); - assert_eq!(want_batch_aggregations, got_batch_aggregations); - assert_eq!(want_active_batch, got_active_batch); - assert_eq!(want_other_batch, got_other_batch); - assert_eq!(want_collection_job, got_collection_job); - } - - #[tokio::test] - async fn step_fixed_size_aggregation_job_continue() { - // Setup: insert a client report and add it to an aggregation job whose state has already - // been stepped once. - install_test_trace_subscriber(); - let mut server = mockito::Server::new_async().await; - let clock = MockClock::default(); - let ephemeral_datastore = ephemeral_datastore().await; - let ds = Arc::new(ephemeral_datastore.datastore(clock.clone()).await); - let vdaf = Arc::new(Prio3::new_count(2).unwrap()); - - let task = TaskBuilder::new( - QueryType::FixedSize { max_batch_size: 10 }, - VdafInstance::Prio3Count, - Role::Leader, - ) - .with_aggregator_endpoints(Vec::from([ - Url::parse("http://irrelevant").unwrap(), // leader URL doesn't matter - Url::parse(&server.url()).unwrap(), - ])) - .build(); - let report_metadata = ReportMetadata::new( - random(), - clock - .now() - .to_batch_interval_start(task.time_precision()) - .unwrap(), - ); - let verify_key: VerifyKey = - task.primary_vdaf_verify_key().unwrap(); - - let transcript = run_vdaf( - vdaf.as_ref(), - verify_key.as_bytes(), - &(), - report_metadata.id(), - &0, - ); - - let agg_auth_token = task.primary_aggregator_auth_token(); - let helper_hpke_keypair = generate_test_hpke_config_and_private_key(); - let report = generate_report::( - *task.id(), - report_metadata, - helper_hpke_keypair.config(), - transcript.public_share.clone(), - Vec::new(), - transcript.input_shares.clone(), - ); - let batch_id = random(); - let aggregation_job_id = random(); - let leader_prep_state = transcript.leader_prep_state(0); - let leader_aggregate_share = vdaf - .aggregate(&(), [transcript.output_share(Role::Leader).clone()]) - .unwrap(); - let prep_msg = &transcript.prepare_messages[0]; - - let (lease, collection_job) = ds - .run_tx(|tx| { - let (vdaf, task, report, leader_prep_state, prep_msg) = ( - vdaf.clone(), - task.clone(), - report.clone(), - leader_prep_state.clone(), - prep_msg.clone(), - ); - Box::pin(async move { - tx.put_task(&task).await?; - tx.put_client_report(vdaf.borrow(), &report).await?; - - tx.put_aggregation_job(&AggregationJob::< - PRIO3_VERIFY_KEY_LENGTH, - FixedSize, - Prio3Count, - >::new( - *task.id(), - aggregation_job_id, - (), - batch_id, - Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) - .unwrap(), - AggregationJobState::InProgress, - AggregationJobRound::from(1), - )) - .await?; - tx.put_report_aggregation(&ReportAggregation::< - PRIO3_VERIFY_KEY_LENGTH, - Prio3Count, - >::new( - *task.id(), - aggregation_job_id, - *report.metadata().id(), - *report.metadata().time(), - 0, - None, - ReportAggregationState::Waiting(leader_prep_state, Some(prep_msg)), - )) - .await?; - - tx.put_batch( - &Batch::::new( - *task.id(), - batch_id, - (), - BatchState::Closing, - 1, - Interval::from_time(report.metadata().time()).unwrap(), - ), - ) - .await?; - - let collection_job = - CollectionJob::::new( - *task.id(), - random(), - batch_id, - (), - CollectionJobState::Start, - ); - tx.put_collection_job(&collection_job).await?; - - let lease = tx - .acquire_incomplete_aggregation_jobs(&StdDuration::from_secs(60), 1) - .await? - .remove(0); - - Ok((lease, collection_job)) - }) - }) - .await - .unwrap(); - assert_eq!(lease.leased().task_id(), task.id()); - assert_eq!(lease.leased().aggregation_job_id(), &aggregation_job_id); - - // Setup: prepare mocked HTTP responses. (first an error response, then a success) - // (This is fragile in that it expects the leader request to be deterministically encoded. - // It would be nicer to retrieve the request bytes from the mock, then do our own parsing & - // verification -- but mockito does not expose this functionality at time of writing.) - let leader_request = AggregationJobContinueReq::new( - AggregationJobRound::from(1), - Vec::from([PrepareStep::new( - *report.metadata().id(), - PrepareStepResult::Continued(prep_msg.get_encoded()), - )]), - ); - let helper_response = AggregationJobResp::new(Vec::from([PrepareStep::new( - *report.metadata().id(), - PrepareStepResult::Finished, - )])); - let mocked_aggregate_failure = server - .mock( - "POST", - task.aggregation_job_uri(&aggregation_job_id) - .unwrap() - .path(), - ) - .with_status(500) - .with_header("Content-Type", "application/problem+json") - .with_body("{\"type\": \"urn:ietf:params:ppm:dap:error:unrecognizedTask\"}") - .create_async() - .await; - let mocked_aggregate_success = server - .mock( - "POST", - task.aggregation_job_uri(&aggregation_job_id) - .unwrap() - .path(), - ) - .match_header( - "DAP-Auth-Token", - str::from_utf8(agg_auth_token.as_ref()).unwrap(), - ) - .match_header(CONTENT_TYPE.as_str(), AggregationJobContinueReq::MEDIA_TYPE) - .match_body(leader_request.get_encoded()) - .with_status(200) - .with_header(CONTENT_TYPE.as_str(), AggregationJobResp::MEDIA_TYPE) - .with_body(helper_response.get_encoded()) - .create_async() - .await; - - // Run: create an aggregation job driver & try to step the aggregation we've created twice. - let aggregation_job_driver = AggregationJobDriver::new( - reqwest::Client::builder().build().unwrap(), - &noop_meter(), - 32, - ); - let error = aggregation_job_driver - .step_aggregation_job(ds.clone(), Arc::new(lease.clone())) - .await - .unwrap_err(); - assert_matches!( - error.downcast().unwrap(), - Error::Http { problem_details, dap_problem_type } => { - assert_eq!(problem_details.status.unwrap(), StatusCode::INTERNAL_SERVER_ERROR); - assert_eq!(dap_problem_type, Some(DapProblemType::UnrecognizedTask)); - } - ); - aggregation_job_driver - .step_aggregation_job(ds.clone(), Arc::new(lease)) - .await - .unwrap(); - - // Verify. - mocked_aggregate_failure.assert_async().await; - mocked_aggregate_success.assert_async().await; - - let want_aggregation_job = - AggregationJob::::new( - *task.id(), - aggregation_job_id, - (), - batch_id, - Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) - .unwrap(), - AggregationJobState::Finished, - AggregationJobRound::from(2), - ); - let want_report_aggregation = ReportAggregation::::new( - *task.id(), - aggregation_job_id, - *report.metadata().id(), - *report.metadata().time(), - 0, - None, - ReportAggregationState::Finished, - ); - let want_batch_aggregations = Vec::from([BatchAggregation::< - PRIO3_VERIFY_KEY_LENGTH, - FixedSize, - Prio3Count, - >::new( - *task.id(), - batch_id, - (), - 0, - BatchAggregationState::Aggregating, - Some(leader_aggregate_share), - 1, - ReportIdChecksum::for_report_id(report.metadata().id()), - )]); - let want_batch = Batch::::new( - *task.id(), - batch_id, - (), - BatchState::Closed, - 0, - Interval::from_time(report.metadata().time()).unwrap(), - ); - let want_collection_job = collection_job.with_state(CollectionJobState::Collectable); + let want_collection_job = collection_job.with_state(CollectionJobState::Collectable); let ( got_aggregation_job, @@ -2580,7 +1804,7 @@ mod tests { let vdaf = Arc::new(Prio3::new_count(2).unwrap()); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -2589,7 +1813,7 @@ mod tests { .now() .to_batch_interval_start(task.time_precision()) .unwrap(); - let batch_identifier = TimeInterval::to_batch_identifier(&task, &(), &time).unwrap(); + let batch_id = random(); let report_metadata = ReportMetadata::new(random(), time); let verify_key: VerifyKey = task.primary_vdaf_verify_key().unwrap(); @@ -2613,17 +1837,15 @@ mod tests { ); let aggregation_job_id = random(); - let aggregation_job = - AggregationJob::::new( - *task.id(), - aggregation_job_id, - (), - (), - Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) - .unwrap(), - AggregationJobState::InProgress, - AggregationJobRound::from(0), - ); + let aggregation_job = AggregationJob::::new( + *task.id(), + aggregation_job_id, + (), + batch_id, + Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)).unwrap(), + AggregationJobState::InProgress, + AggregationJobRound::from(0), + ); let report_aggregation = ReportAggregation::::new( *task.id(), aggregation_job_id, @@ -2650,9 +1872,9 @@ mod tests { tx.put_report_aggregation(&report_aggregation).await?; tx.put_batch( - &Batch::::new( + &Batch::::new( *task.id(), - batch_identifier, + batch_id, (), BatchState::Open, 1, @@ -2688,9 +1910,9 @@ mod tests { // longer be acquired. let want_aggregation_job = aggregation_job.with_state(AggregationJobState::Abandoned); let want_report_aggregation = report_aggregation; - let want_batch = Batch::::new( + let want_batch = Batch::::new( *task.id(), - batch_identifier, + batch_id, (), BatchState::Open, 0, @@ -2703,7 +1925,7 @@ mod tests { (Arc::clone(&vdaf), task.clone(), *report.metadata().id()); Box::pin(async move { let aggregation_job = tx - .get_aggregation_job::( + .get_aggregation_job::( task.id(), &aggregation_job_id, ) @@ -2719,10 +1941,7 @@ mod tests { ) .await? .unwrap(); - let batch = tx - .get_batch(task.id(), &batch_identifier, &()) - .await? - .unwrap(); + let batch = tx.get_batch(task.id(), &batch_id, &()).await?.unwrap(); let leases = tx .acquire_incomplete_aggregation_jobs(&StdDuration::from_secs(60), 1) .await?; @@ -2794,7 +2013,7 @@ mod tests { let stopper = Stopper::new(); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -2815,7 +2034,7 @@ mod tests { .now() .to_batch_interval_start(task.time_precision()) .unwrap(); - let batch_identifier = TimeInterval::to_batch_identifier(&task, &(), &time).unwrap(); + let batch_id = random(); let report_metadata = ReportMetadata::new(random(), time); let transcript = run_vdaf(&vdaf, verify_key.as_bytes(), &(), report_metadata.id(), &0); let report = generate_report::( @@ -2841,13 +2060,13 @@ mod tests { tx.put_aggregation_job(&AggregationJob::< PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, Prio3Count, >::new( *task.id(), aggregation_job_id, (), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -2869,9 +2088,9 @@ mod tests { .await?; tx.put_batch( - &Batch::::new( + &Batch::::new( *task.id(), - batch_identifier, + batch_id, (), BatchState::Open, 1, @@ -2926,7 +2145,7 @@ mod tests { ) .match_header( CONTENT_TYPE.as_str(), - AggregationJobInitializeReq::::MEDIA_TYPE, + AggregationJobInitializeReq::::MEDIA_TYPE, ) .with_status(500) .expect(3) @@ -2948,7 +2167,7 @@ mod tests { ) .match_header( CONTENT_TYPE.as_str(), - AggregationJobInitializeReq::::MEDIA_TYPE, + AggregationJobInitializeReq::::MEDIA_TYPE, ) .with_status(500) .expect(1) @@ -2984,10 +2203,7 @@ mod tests { .get_aggregation_job(task.id(), &aggregation_job_id) .await? .unwrap(); - let got_batch = tx - .get_batch(task.id(), &batch_identifier, &()) - .await? - .unwrap(); + let got_batch = tx.get_batch(task.id(), &batch_id, &()).await?.unwrap(); Ok((got_aggregation_job, got_batch)) }) }) @@ -2995,11 +2211,11 @@ mod tests { .unwrap(); assert_eq!( got_aggregation_job, - AggregationJob::::new( + AggregationJob::::new( *task.id(), aggregation_job_id, (), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::Abandoned, @@ -3008,9 +2224,9 @@ mod tests { ); assert_eq!( got_batch, - Batch::::new( + Batch::::new( *task.id(), - batch_identifier, + batch_id, (), BatchState::Open, 0, diff --git a/aggregator/src/aggregator/collection_job_driver.rs b/aggregator/src/aggregator/collection_job_driver.rs index 4c2454628..02c3a99d1 100644 --- a/aggregator/src/aggregator/collection_job_driver.rs +++ b/aggregator/src/aggregator/collection_job_driver.rs @@ -18,7 +18,7 @@ use janus_aggregator_core::{ }; use janus_core::{time::Clock, vdaf_dispatch}; use janus_messages::{ - query_type::{FixedSize, QueryType, TimeInterval}, + query_type::{FixedSize, QueryType}, AggregateShare, AggregateShareReq, BatchSelector, }; use opentelemetry::{ @@ -80,17 +80,6 @@ impl CollectionJobDriver { lease: Arc>, ) -> Result<(), Error> { match lease.leased().query_type() { - task::QueryType::TimeInterval => { - vdaf_dispatch!(lease.leased().vdaf(), (vdaf, VdafType, VERIFY_KEY_LENGTH) => { - self.step_collection_job_generic::< - VERIFY_KEY_LENGTH, - C, - TimeInterval, - VdafType - >(datastore, Arc::new(vdaf), lease) - .await - }) - } task::QueryType::FixedSize { .. } => { vdaf_dispatch!(lease.leased().vdaf(), (vdaf, VdafType, VERIFY_KEY_LENGTH) => { self.step_collection_job_generic::< @@ -227,7 +216,7 @@ impl CollectionJobDriver { Method::POST, task.aggregate_shares_uri()?, AGGREGATE_SHARES_ROUTE, - AggregateShareReq::::MEDIA_TYPE, + AggregateShareReq::::MEDIA_TYPE, req, task.primary_aggregator_auth_token(), &self.metrics.http_request_duration_histogram, @@ -312,16 +301,6 @@ impl CollectionJobDriver { lease: Lease, ) -> Result<(), Error> { match lease.leased().query_type() { - task::QueryType::TimeInterval => { - vdaf_dispatch!(lease.leased().vdaf(), (vdaf, VdafType, VERIFY_KEY_LENGTH) => { - self.abandon_collection_job_generic::( - datastore, - Arc::new(vdaf), - lease, - ) - .await - }) - } task::QueryType::FixedSize { .. } => { vdaf_dispatch!(lease.leased().vdaf(), (vdaf, VdafType, VERIFY_KEY_LENGTH) => { self.abandon_collection_job_generic::( @@ -550,7 +529,7 @@ mod tests { Runtime, }; use janus_messages::{ - query_type::TimeInterval, AggregateShare, AggregateShareReq, AggregationJobRound, + query_type::FixedSize, AggregateShare, AggregateShareReq, AggregationJobRound, BatchSelector, Duration, HpkeCiphertext, HpkeConfigId, Interval, ReportIdChecksum, Role, }; use prio::codec::{Decode, Encode}; @@ -567,24 +546,28 @@ mod tests { ) -> ( Task, Option>, - CollectionJob<0, TimeInterval, dummy_vdaf::Vdaf>, + CollectionJob<0, FixedSize, dummy_vdaf::Vdaf>, ) { let time_precision = Duration::from_seconds(500); - let task = TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Leader) - .with_aggregator_endpoints(Vec::from([ - Url::parse("http://irrelevant").unwrap(), // leader URL doesn't matter - Url::parse(&server.url()).unwrap(), - ])) - .with_time_precision(time_precision) - .with_min_batch_size(10) - .build(); - let batch_interval = Interval::new(clock.now(), Duration::from_seconds(2000)).unwrap(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Leader, + ) + .with_aggregator_endpoints(Vec::from([ + Url::parse("http://irrelevant").unwrap(), // leader URL doesn't matter + Url::parse(&server.url()).unwrap(), + ])) + .with_time_precision(time_precision) + .with_min_batch_size(10) + .build(); + let batch_id = random(); let aggregation_param = AggregationParam(0); - let collection_job = CollectionJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + let collection_job = CollectionJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( *task.id(), random(), - batch_interval, + batch_id, aggregation_param, CollectionJobState::Collectable, ); @@ -596,7 +579,7 @@ mod tests { Box::pin(async move { tx.put_task(&task).await?; - tx.put_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>(&collection_job) + tx.put_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>(&collection_job) .await?; let aggregation_job_id = random(); @@ -604,17 +587,15 @@ mod tests { .now() .to_batch_interval_start(task.time_precision()) .unwrap(); - tx.put_aggregation_job( - &AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - aggregation_job_id, - aggregation_param, - (), - Interval::from_time(&report_timestamp).unwrap(), - AggregationJobState::Finished, - AggregationJobRound::from(1), - ), - ) + tx.put_aggregation_job(&AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( + *task.id(), + aggregation_job_id, + aggregation_param, + batch_id, + Interval::from_time(&report_timestamp).unwrap(), + AggregationJobState::Finished, + AggregationJobRound::from(1), + )) .await?; let report = LeaderStoredReport::new_dummy(*task.id(), report_timestamp); @@ -634,36 +615,18 @@ mod tests { .await?; tx.put_batch_aggregation( - &BatchAggregation::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + &BatchAggregation::<0, FixedSize, dummy_vdaf::Vdaf>::new( *task.id(), - Interval::new(clock.now(), time_precision).unwrap(), + batch_id, aggregation_param, 0, BatchAggregationState::Aggregating, Some(dummy_vdaf::AggregateShare(0)), - 5, + 10, ReportIdChecksum::get_decoded(&[3; 32]).unwrap(), ), ) .await?; - tx.put_batch_aggregation( - &BatchAggregation::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - Interval::new( - clock.now().add(&Duration::from_seconds(1000)).unwrap(), - time_precision, - ) - .unwrap(), - aggregation_param, - 0, - BatchAggregationState::Aggregating, - Some(dummy_vdaf::AggregateShare(0)), - 5, - ReportIdChecksum::get_decoded(&[2; 32]).unwrap(), - ), - ) - .await?; - if acquire_lease { let lease = tx .acquire_incomplete_collection_jobs(&StdDuration::from_secs(100), 1) @@ -692,16 +655,20 @@ mod tests { let ds = Arc::new(ephemeral_datastore.datastore(clock.clone()).await); let time_precision = Duration::from_seconds(500); - let task = TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Leader) - .with_aggregator_endpoints(Vec::from([ - Url::parse("http://irrelevant").unwrap(), // leader URL doesn't matter - Url::parse(&server.url()).unwrap(), - ])) - .with_time_precision(time_precision) - .with_min_batch_size(10) - .build(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Leader, + ) + .with_aggregator_endpoints(Vec::from([ + Url::parse("http://irrelevant").unwrap(), // leader URL doesn't matter + Url::parse(&server.url()).unwrap(), + ])) + .with_time_precision(time_precision) + .with_min_batch_size(10) + .build(); let agg_auth_token = task.primary_aggregator_auth_token(); - let batch_interval = Interval::new(clock.now(), Duration::from_seconds(2000)).unwrap(); + let batch_id = random(); let aggregation_param = AggregationParam(0); let report_timestamp = clock .now() @@ -715,29 +682,25 @@ mod tests { tx.put_task(&task).await?; let collection_job_id = random(); - tx.put_collection_job( - &CollectionJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - collection_job_id, - batch_interval, - aggregation_param, - CollectionJobState::Collectable, - ), - ) + tx.put_collection_job(&CollectionJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( + *task.id(), + collection_job_id, + batch_id, + aggregation_param, + CollectionJobState::Collectable, + )) .await?; let aggregation_job_id = random(); - tx.put_aggregation_job( - &AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - aggregation_job_id, - aggregation_param, - (), - Interval::from_time(&report_timestamp).unwrap(), - AggregationJobState::Finished, - AggregationJobRound::from(1), - ), - ) + tx.put_aggregation_job(&AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( + *task.id(), + aggregation_job_id, + aggregation_param, + batch_id, + Interval::from_time(&report_timestamp).unwrap(), + AggregationJobState::Finished, + AggregationJobRound::from(1), + )) .await?; let report = LeaderStoredReport::new_dummy(*task.id(), report_timestamp); @@ -787,40 +750,22 @@ mod tests { // Put some batch aggregations in the DB. ds.run_tx(|tx| { - let (clock, task) = (clock.clone(), task.clone()); + let task = task.clone(); Box::pin(async move { tx.update_batch_aggregation( - &BatchAggregation::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + &BatchAggregation::<0, FixedSize, dummy_vdaf::Vdaf>::new( *task.id(), - Interval::new(clock.now(), time_precision).unwrap(), + batch_id, aggregation_param, 0, BatchAggregationState::Aggregating, Some(dummy_vdaf::AggregateShare(0)), - 5, + 10, ReportIdChecksum::get_decoded(&[3; 32]).unwrap(), ), ) .await?; - tx.update_batch_aggregation( - &BatchAggregation::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - Interval::new( - clock.now().add(&Duration::from_seconds(1000)).unwrap(), - time_precision, - ) - .unwrap(), - aggregation_param, - 0, - BatchAggregationState::Aggregating, - Some(dummy_vdaf::AggregateShare(0)), - 5, - ReportIdChecksum::get_decoded(&[2; 32]).unwrap(), - ), - ) - .await?; - Ok(()) }) }) @@ -828,10 +773,10 @@ mod tests { .unwrap(); let leader_request = AggregateShareReq::new( - BatchSelector::new_time_interval(batch_interval), + BatchSelector::new_fixed_size(batch_id), aggregation_param.get_encoded(), 10, - ReportIdChecksum::get_decoded(&[3 ^ 2; 32]).unwrap(), + ReportIdChecksum::get_decoded(&[3; 32]).unwrap(), ); // Simulate helper failing to service the aggregate share request. @@ -843,7 +788,7 @@ mod tests { ) .match_header( CONTENT_TYPE.as_str(), - AggregateShareReq::::MEDIA_TYPE, + AggregateShareReq::::MEDIA_TYPE, ) .match_body(leader_request.get_encoded()) .with_status(500) @@ -872,7 +817,7 @@ mod tests { ds.run_tx(|tx| { Box::pin(async move { let collection_job = tx - .get_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>( &dummy_vdaf::Vdaf::new(), &collection_job_id, ) @@ -901,7 +846,7 @@ mod tests { ) .match_header( CONTENT_TYPE.as_str(), - AggregateShareReq::::MEDIA_TYPE, + AggregateShareReq::::MEDIA_TYPE, ) .match_body(leader_request.get_encoded()) .with_status(200) @@ -922,7 +867,7 @@ mod tests { let helper_aggregate_share = helper_response.encrypted_aggregate_share().clone(); Box::pin(async move { let collection_job = tx - .get_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>( &dummy_vdaf::Vdaf::new(), &collection_job_id, ) @@ -977,7 +922,7 @@ mod tests { let collection_job = collection_job.clone(); Box::pin(async move { let abandoned_collection_job = tx - .get_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>( &dummy_vdaf::Vdaf::new(), collection_job.id(), ) @@ -1083,7 +1028,7 @@ mod tests { .run_tx(|tx| { let collection_job = collection_job.clone(); Box::pin(async move { - tx.get_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + tx.get_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>( &dummy_vdaf::Vdaf::new(), collection_job.id(), ) @@ -1117,7 +1062,7 @@ mod tests { ds.run_tx(|tx| { let collection_job = collection_job.clone(); Box::pin(async move { - tx.update_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>(&collection_job) + tx.update_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>(&collection_job) .await }) }) @@ -1156,7 +1101,7 @@ mod tests { let collection_job = collection_job.clone(); Box::pin(async move { let collection_job = tx - .get_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>( &dummy_vdaf::Vdaf::new(), collection_job.id(), ) diff --git a/aggregator/src/aggregator/collection_job_tests.rs b/aggregator/src/aggregator/collection_job_tests.rs index 40f1a9d8f..09e7a19ff 100644 --- a/aggregator/src/aggregator/collection_job_tests.rs +++ b/aggregator/src/aggregator/collection_job_tests.rs @@ -23,9 +23,9 @@ use janus_core::{ time::{Clock, IntervalExt, MockClock}, }; use janus_messages::{ - query_type::{FixedSize, QueryType as QueryTypeTrait, TimeInterval}, + query_type::{FixedSize, QueryType as QueryTypeTrait}, AggregateShareAad, AggregationJobRound, BatchId, BatchSelector, Collection, CollectionJobId, - CollectionReq, Duration, FixedSizeQuery, Interval, Query, ReportIdChecksum, Role, Time, + CollectionReq, Duration, FixedSizeQuery, Interval, Query, ReportIdChecksum, Role, }; use prio::codec::{Decode, Encode}; use rand::random; @@ -66,7 +66,7 @@ impl CollectionJobTestCase { test_conn .with_request_header( KnownHeaderName::ContentType, - CollectionReq::::MEDIA_TYPE, + CollectionReq::::MEDIA_TYPE, ) .with_request_body(request.get_encoded()) .run_async(&self.handler) @@ -432,128 +432,6 @@ async fn collection_job_success_fixed_size() { ); } -#[tokio::test] -async fn collection_job_put_idempotence_time_interval() { - let test_case = setup_collection_job_test_case(Role::Leader, QueryType::TimeInterval).await; - - let collection_job_id = random(); - let request = CollectionReq::new( - Query::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(0), - *test_case.task.time_precision(), - ) - .unwrap(), - ), - dummy_vdaf::AggregationParam::default().get_encoded(), - ); - - for _ in 0..2 { - let response = test_case - .put_collection_job(&collection_job_id, &request) - .await; - assert_eq!(response.status(), Some(Status::Created)); - } - - // There should only be a single collection job despite two successful PUTs - test_case - .datastore - .run_tx(|tx| { - let task_id = *test_case.task.id(); - let vdaf = dummy_vdaf::Vdaf::new(); - Box::pin(async move { - let collection_jobs = tx - .get_collection_jobs_for_task::<0, TimeInterval, dummy_vdaf::Vdaf>( - &vdaf, &task_id, - ) - .await - .unwrap(); - assert_eq!(collection_jobs.len(), 1); - assert_eq!(collection_jobs[0].id(), &collection_job_id); - - Ok(()) - }) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn collection_job_put_idempotence_time_interval_mutate_time_interval() { - let test_case = setup_collection_job_test_case(Role::Leader, QueryType::TimeInterval).await; - - let collection_job_id = random(); - let request = CollectionReq::new( - Query::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(0), - *test_case.task.time_precision(), - ) - .unwrap(), - ), - dummy_vdaf::AggregationParam::default().get_encoded(), - ); - - let response = test_case - .put_collection_job(&collection_job_id, &request) - .await; - assert_eq!(response.status(), Some(Status::Created)); - - let mutated_request = CollectionReq::new( - Query::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(test_case.task.time_precision().as_seconds()), - *test_case.task.time_precision(), - ) - .unwrap(), - ), - dummy_vdaf::AggregationParam::default().get_encoded(), - ); - - let response = test_case - .put_collection_job(&collection_job_id, &mutated_request) - .await; - assert_eq!(response.status(), Some(Status::Conflict)); -} - -#[tokio::test] -async fn collection_job_put_idempotence_time_interval_mutate_aggregation_param() { - let test_case = setup_collection_job_test_case(Role::Leader, QueryType::TimeInterval).await; - - let collection_job_id = random(); - let request = CollectionReq::new( - Query::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(0), - *test_case.task.time_precision(), - ) - .unwrap(), - ), - dummy_vdaf::AggregationParam(0).get_encoded(), - ); - - let response = test_case - .put_collection_job(&collection_job_id, &request) - .await; - assert_eq!(response.status(), Some(Status::Created)); - - let mutated_request = CollectionReq::new( - Query::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(0), - *test_case.task.time_precision(), - ) - .unwrap(), - ), - dummy_vdaf::AggregationParam(1).get_encoded(), - ); - - let response = test_case - .put_collection_job(&collection_job_id, &mutated_request) - .await; - assert_eq!(response.status(), Some(Status::Conflict)); -} - #[tokio::test] async fn collection_job_put_idempotence_fixed_size_current_batch() { let (test_case, batch_id_1, batch_id_2, _) = diff --git a/aggregator/src/aggregator/garbage_collector.rs b/aggregator/src/aggregator/garbage_collector.rs index 5ee05ddc0..a3b7a6c89 100644 --- a/aggregator/src/aggregator/garbage_collector.rs +++ b/aggregator/src/aggregator/garbage_collector.rs @@ -94,228 +94,14 @@ mod tests { time::{Clock, MockClock, TimeExt}, }; use janus_messages::{ - query_type::{FixedSize, TimeInterval}, - AggregationJobRound, Duration, HpkeCiphertext, HpkeConfigId, Interval, ReportMetadata, - ReportShare, Role, + query_type::FixedSize, AggregationJobRound, Duration, HpkeCiphertext, HpkeConfigId, + Interval, ReportMetadata, ReportShare, Role, }; use rand::random; use std::sync::Arc; // TODO(#1467): restore check that collection artifacts are properly GC'ed once collection GC is updated - #[tokio::test] - async fn gc_task_leader_time_interval() { - install_test_trace_subscriber(); - - let clock = MockClock::default(); - let ephemeral_datastore = ephemeral_datastore().await; - let ds = Arc::new(ephemeral_datastore.datastore(clock.clone()).await); - let vdaf = dummy_vdaf::Vdaf::new(); - - // Setup. - let task = ds - .run_tx(|tx| { - let (clock, vdaf) = (clock.clone(), vdaf.clone()); - Box::pin(async move { - const REPORT_EXPIRY_AGE: Duration = Duration::from_seconds(3600); - let task = TaskBuilder::new( - task::QueryType::TimeInterval, - VdafInstance::Fake, - Role::Leader, - ) - .with_report_expiry_age(Some(REPORT_EXPIRY_AGE)) - .build(); - tx.put_task(&task).await?; - - let client_timestamp = clock - .now() - .sub(&REPORT_EXPIRY_AGE) - .unwrap() - .sub(&Duration::from_seconds(2)) - .unwrap(); - let report = LeaderStoredReport::new_dummy(*task.id(), client_timestamp); - tx.put_client_report(&vdaf, &report).await.unwrap(); - - let batch_identifier = - Interval::new(client_timestamp, Duration::from_seconds(1)).unwrap(); - let aggregation_job = AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - random(), - AggregationParam(0), - (), - batch_identifier, - AggregationJobState::InProgress, - AggregationJobRound::from(0), - ); - tx.put_aggregation_job(&aggregation_job).await.unwrap(); - - let report_aggregation = ReportAggregation::<0, dummy_vdaf::Vdaf>::new( - *task.id(), - *aggregation_job.id(), - *report.metadata().id(), - client_timestamp, - 0, - None, - ReportAggregationState::Start, - ); - tx.put_report_aggregation(&report_aggregation) - .await - .unwrap(); - - Ok(task) - }) - }) - .await - .unwrap(); - - // Run. - let task = Arc::new(task); - GarbageCollector::new(Arc::clone(&ds), clock.clone()) - .gc_task(Arc::clone(&task)) - .await - .unwrap(); - - // Verify. - let (client_reports, aggregation_jobs, report_aggregations) = ds - .run_tx(|tx| { - let (vdaf, task) = (vdaf.clone(), Arc::clone(&task)); - Box::pin(async move { - let client_reports = tx - .get_client_reports_for_task::<0, dummy_vdaf::Vdaf>(&vdaf, task.id()) - .await?; - let aggregation_jobs = tx - .get_aggregation_jobs_for_task::<0, TimeInterval, dummy_vdaf::Vdaf>( - task.id(), - ) - .await?; - let report_aggregations = tx - .get_report_aggregations_for_task::<0, dummy_vdaf::Vdaf>( - &vdaf, - &Role::Leader, - task.id(), - ) - .await?; - Ok((client_reports, aggregation_jobs, report_aggregations)) - }) - }) - .await - .unwrap(); - assert!(client_reports.is_empty()); - assert!(aggregation_jobs.is_empty()); - assert!(report_aggregations.is_empty()); - } - - #[tokio::test] - async fn gc_task_helper_time_interval() { - install_test_trace_subscriber(); - - let clock = MockClock::default(); - let ephemeral_datastore = ephemeral_datastore().await; - let ds = Arc::new(ephemeral_datastore.datastore(clock.clone()).await); - let vdaf = dummy_vdaf::Vdaf::new(); - - // Setup. - let task = ds - .run_tx(|tx| { - let clock = clock.clone(); - Box::pin(async move { - const REPORT_EXPIRY_AGE: Duration = Duration::from_seconds(3600); - let task = TaskBuilder::new( - task::QueryType::TimeInterval, - VdafInstance::Fake, - Role::Helper, - ) - .with_report_expiry_age(Some(REPORT_EXPIRY_AGE)) - .build(); - tx.put_task(&task).await?; - - let client_timestamp = clock - .now() - .sub(&REPORT_EXPIRY_AGE) - .unwrap() - .sub(&Duration::from_seconds(2)) - .unwrap(); - let report_share = ReportShare::new( - ReportMetadata::new(random(), client_timestamp), - Vec::new(), - HpkeCiphertext::new( - HpkeConfigId::from(13), - Vec::from("encapsulated_context_0"), - Vec::from("payload_0"), - ), - ); - tx.put_report_share(task.id(), &report_share).await.unwrap(); - - let batch_identifier = - Interval::new(client_timestamp, Duration::from_seconds(1)).unwrap(); - let aggregation_job = AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - random(), - AggregationParam(0), - (), - batch_identifier, - AggregationJobState::InProgress, - AggregationJobRound::from(0), - ); - tx.put_aggregation_job(&aggregation_job).await.unwrap(); - - let report_aggregation = ReportAggregation::<0, dummy_vdaf::Vdaf>::new( - *task.id(), - *aggregation_job.id(), - *report_share.metadata().id(), - client_timestamp, - 0, - None, - ReportAggregationState::Start, - ); - tx.put_report_aggregation(&report_aggregation) - .await - .unwrap(); - - Ok(task) - }) - }) - .await - .unwrap(); - - // Run. - let task = Arc::new(task); - GarbageCollector::new(Arc::clone(&ds), clock.clone()) - .gc_task(Arc::clone(&task)) - .await - .unwrap(); - - // Verify. - let (client_reports, aggregation_jobs, report_aggregations) = ds - .run_tx(|tx| { - let (vdaf, task) = (vdaf.clone(), Arc::clone(&task)); - Box::pin(async move { - let client_reports = tx - .get_client_reports_for_task::<0, dummy_vdaf::Vdaf>(&vdaf, task.id()) - .await?; - let aggregation_jobs = tx - .get_aggregation_jobs_for_task::<0, TimeInterval, dummy_vdaf::Vdaf>( - task.id(), - ) - .await?; - let report_aggregations = tx - .get_report_aggregations_for_task::<0, dummy_vdaf::Vdaf>( - &vdaf, - &Role::Leader, - task.id(), - ) - .await?; - - Ok((client_reports, aggregation_jobs, report_aggregations)) - }) - }) - .await - .unwrap(); - assert!(client_reports.is_empty()); - assert!(aggregation_jobs.is_empty()); - assert!(report_aggregations.is_empty()); - } - #[tokio::test] async fn gc_task_leader_fixed_size() { install_test_trace_subscriber(); diff --git a/aggregator/src/aggregator/http_handlers.rs b/aggregator/src/aggregator/http_handlers.rs index 6ee58f870..35023d21c 100644 --- a/aggregator/src/aggregator/http_handlers.rs +++ b/aggregator/src/aggregator/http_handlers.rs @@ -8,7 +8,7 @@ use janus_core::{ time::Clock, }; use janus_messages::{ - problem_type::DapProblemType, query_type::TimeInterval, AggregateShare, AggregateShareReq, + problem_type::DapProblemType, query_type::FixedSize, AggregateShare, AggregateShareReq, AggregationJobContinueReq, AggregationJobId, AggregationJobInitializeReq, AggregationJobResp, Collection, CollectionJobId, CollectionReq, HpkeConfigList, Report, TaskId, }; @@ -356,10 +356,7 @@ async fn aggregation_jobs_put( Vec, ), ) -> Result, Error> { - validate_content_type( - conn, - AggregationJobInitializeReq::::MEDIA_TYPE, - )?; + validate_content_type(conn, AggregationJobInitializeReq::::MEDIA_TYPE)?; let task_id = parse_task_id(&captures)?; let aggregation_job_id = parse_aggregation_job_id(&captures)?; @@ -401,7 +398,7 @@ async fn collection_jobs_put( Vec, ), ) -> Result { - validate_content_type(conn, CollectionReq::::MEDIA_TYPE)?; + validate_content_type(conn, CollectionReq::::MEDIA_TYPE)?; let task_id = parse_task_id(&captures)?; let collection_job_id = parse_collection_job_id(&captures)?; @@ -431,7 +428,7 @@ async fn collection_jobs_post( Some(response_bytes) => { conn.headers_mut().insert( KnownHeaderName::ContentType, - Collection::::MEDIA_TYPE, + Collection::::MEDIA_TYPE, ); conn.set_status(Status::Ok); conn.set_body(response_bytes); @@ -467,7 +464,7 @@ async fn aggregate_shares( Vec, ), ) -> Result, Error> { - validate_content_type(conn, AggregateShareReq::::MEDIA_TYPE)?; + validate_content_type(conn, AggregateShareReq::::MEDIA_TYPE)?; let task_id = parse_task_id(&captures)?; let auth_token = parse_auth_token(&task_id, conn)?; @@ -572,7 +569,7 @@ mod tests { }, test_util::ephemeral_datastore, }, - query_type::{AccumulableQueryType, CollectableQueryType}, + query_type::CollectableQueryType, task::{test_util::TaskBuilder, QueryType, VerifyKey}, test_util::noop_meter, }; @@ -586,11 +583,11 @@ mod tests { time::{Clock, DurationExt, MockClock, TimeExt}, }; use janus_messages::{ - query_type::TimeInterval, AggregateShare as AggregateShareMessage, AggregateShareAad, + query_type::FixedSize, AggregateShare as AggregateShareMessage, AggregateShareAad, AggregateShareReq, AggregationJobContinueReq, AggregationJobId, AggregationJobInitializeReq, AggregationJobResp, AggregationJobRound, BatchSelector, Collection, CollectionJobId, CollectionReq, Duration, Extension, ExtensionType, - HpkeCiphertext, HpkeConfigId, HpkeConfigList, InputShareAad, Interval, + FixedSizeQuery, HpkeCiphertext, HpkeConfigId, HpkeConfigList, InputShareAad, Interval, PartialBatchSelector, PrepareStep, PrepareStepResult, Query, Report, ReportId, ReportIdChecksum, ReportMetadata, ReportShare, ReportShareError, Role, TaskId, Time, }; @@ -617,7 +614,7 @@ mod tests { install_test_trace_subscriber(); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -715,7 +712,7 @@ mod tests { install_test_trace_subscriber(); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -789,7 +786,7 @@ mod tests { const REPORT_EXPIRY_AGE: u64 = 1_000_000; let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -948,7 +945,7 @@ mod tests { // Reports with timestamps past the task's expiration should be rejected. let task_expire_soon = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -1030,7 +1027,7 @@ mod tests { let datastore = ephemeral_datastore.datastore(clock.clone()).await; let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Helper, ) @@ -1080,7 +1077,7 @@ mod tests { install_test_trace_subscriber(); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -1093,7 +1090,7 @@ mod tests { let request = AggregationJobInitializeReq::new( Vec::new(), - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(random()), Vec::new(), ); @@ -1167,7 +1164,7 @@ mod tests { install_test_trace_subscriber(); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Helper, ) @@ -1180,7 +1177,7 @@ mod tests { let request = AggregationJobInitializeReq::new( Vec::new(), - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(random()), Vec::new(), ); @@ -1211,7 +1208,7 @@ mod tests { .path()) .with_request_header( KnownHeaderName::ContentType, - AggregationJobInitializeReq::::MEDIA_TYPE, + AggregationJobInitializeReq::::MEDIA_TYPE, ) .with_request_body(request.get_encoded()); @@ -1244,8 +1241,12 @@ mod tests { // Prepare datastore & request. install_test_trace_subscriber(); - let task = - TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Helper).build(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Helper, + ) + .build(); let clock = MockClock::default(); let ephemeral_datastore = ephemeral_datastore().await; let datastore = Arc::new(ephemeral_datastore.datastore(clock.clone()).await); @@ -1397,13 +1398,11 @@ mod tests { &transcript.input_shares[1], ); - // report_share_5 falls into a batch that has already been collected. - let past_clock = MockClock::new(Time::from_seconds_since_epoch( - task.time_precision().as_seconds() / 2, - )); + // report_share_5 fails decoding due to an issue with the public share. + let public_share_5 = Vec::from([0]); let report_metadata_5 = ReportMetadata::new( random(), - past_clock + clock .now() .to_batch_interval_start(task.time_precision()) .unwrap(), @@ -1415,17 +1414,15 @@ mod tests { report_metadata_5.id(), &(), ); - let report_share_5 = generate_helper_report_share::( - *task.id(), - report_metadata_5, + let report_share_5 = generate_helper_report_share_for_plaintext( + report_metadata_5.clone(), hpke_key.config(), - &transcript.public_share, - Vec::new(), - &transcript.input_shares[1], + public_share_5.clone(), + &transcript.input_shares[1].get_encoded(), + &InputShareAad::new(*task.id(), report_metadata_5, public_share_5).get_encoded(), ); - // report_share_6 fails decoding due to an issue with the public share. - let public_share_6 = Vec::from([0]); + // report_share_6 fails due to having repeated extensions. let report_metadata_6 = ReportMetadata::new( random(), clock @@ -1440,32 +1437,9 @@ mod tests { report_metadata_6.id(), &(), ); - let report_share_6 = generate_helper_report_share_for_plaintext( - report_metadata_6.clone(), - hpke_key.config(), - public_share_6.clone(), - &transcript.input_shares[1].get_encoded(), - &InputShareAad::new(*task.id(), report_metadata_6, public_share_6).get_encoded(), - ); - - // report_share_7 fails due to having repeated extensions. - let report_metadata_7 = ReportMetadata::new( - random(), - clock - .now() - .to_batch_interval_start(task.time_precision()) - .unwrap(), - ); - let transcript = run_vdaf( - &vdaf, - verify_key.as_bytes(), - &dummy_vdaf::AggregationParam(0), - report_metadata_7.id(), - &(), - ); - let report_share_7 = generate_helper_report_share::( + let report_share_6 = generate_helper_report_share::( *task.id(), - report_metadata_7, + report_metadata_6, hpke_key.config(), &transcript.public_share, Vec::from([ @@ -1475,9 +1449,9 @@ mod tests { &transcript.input_shares[0], ); - // report_share_8 has already been aggregated in another aggregation job, with a different + // report_share_7 has already been aggregated in another aggregation job, with a different // aggregation parameter. - let report_metadata_8 = ReportMetadata::new( + let report_metadata_7 = ReportMetadata::new( random(), clock .now() @@ -1488,12 +1462,12 @@ mod tests { &vdaf, verify_key.as_bytes(), &dummy_vdaf::AggregationParam(1), - report_metadata_8.id(), + report_metadata_7.id(), &(), ); - let report_share_8 = generate_helper_report_share::( + let report_share_7 = generate_helper_report_share::( *task.id(), - report_metadata_8, + report_metadata_7, hpke_key.config(), &transcript.public_share, Vec::new(), @@ -1502,15 +1476,15 @@ mod tests { let (conflicting_aggregation_job, non_conflicting_aggregation_job) = datastore .run_tx(|tx| { - let (task, report_share_4, report_share_8) = - (task.clone(), report_share_4.clone(), report_share_8.clone()); + let (task, report_share_4, report_share_7) = + (task.clone(), report_share_4.clone(), report_share_7.clone()); Box::pin(async move { tx.put_task(&task).await?; - // report_share_4 and report_share_8 are already in the datastore as they were + // report_share_4 and report_share_7 are already in the datastore as they were // referenced by existing aggregation jobs. tx.put_report_share(task.id(), &report_share_4).await?; - tx.put_report_share(task.id(), &report_share_8).await?; + tx.put_report_share(task.id(), &report_share_7).await?; // Put in an aggregation job and report aggregation for report_share_4. It uses // the same aggregation parameter as the aggregation job this test will later @@ -1519,13 +1493,13 @@ mod tests { *task.id(), random(), dummy_vdaf::AggregationParam(0), - (), + random(), Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, AggregationJobRound::from(0), ); - tx.put_aggregation_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + tx.put_aggregation_job::<0, FixedSize, dummy_vdaf::Vdaf>( &conflicting_aggregation_job, ) .await @@ -1542,21 +1516,21 @@ mod tests { .await .unwrap(); - // Put in an aggregation job and report aggregation for report_share_8, using a + // Put in an aggregation job and report aggregation for report_share_7, using a // a different aggregation parameter. As the aggregation parameter differs, - // report_share_8 should prepare successfully in the aggregation job we'll PUT + // report_share_7 should prepare successfully in the aggregation job we'll PUT // later. let non_conflicting_aggregation_job = AggregationJob::new( *task.id(), random(), dummy_vdaf::AggregationParam(1), - (), + random(), Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, AggregationJobRound::from(0), ); - tx.put_aggregation_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + tx.put_aggregation_job::<0, FixedSize, dummy_vdaf::Vdaf>( &non_conflicting_aggregation_job, ) .await @@ -1564,8 +1538,8 @@ mod tests { tx.put_report_aggregation::<0, dummy_vdaf::Vdaf>(&ReportAggregation::new( *task.id(), *non_conflicting_aggregation_job.id(), - *report_share_8.metadata().id(), - *report_share_8.metadata().time(), + *report_share_7.metadata().id(), + *report_share_7.metadata().time(), 0, None, ReportAggregationState::Start, @@ -1573,33 +1547,16 @@ mod tests { .await .unwrap(); - // Put in an aggregate share job for the interval that report_share_5 falls into - // which should cause it to later fail to prepare. - tx.put_aggregate_share_job::<0, TimeInterval, dummy_vdaf::Vdaf>( - &AggregateShareJob::new( - *task.id(), - Interval::new( - Time::from_seconds_since_epoch(0), - *task.time_precision(), - ) - .unwrap(), - dummy_vdaf::AggregationParam(0), - dummy_vdaf::AggregateShare(0), - 0, - ReportIdChecksum::default(), - ), - ) - .await?; - Ok((conflicting_aggregation_job, non_conflicting_aggregation_job)) }) }) .await .unwrap(); + let batch_id = random(); let request = AggregationJobInitializeReq::new( dummy_vdaf::AggregationParam(0).get_encoded(), - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(batch_id), Vec::from([ report_share_0.clone(), report_share_1.clone(), @@ -1609,7 +1566,6 @@ mod tests { report_share_5.clone(), report_share_6.clone(), report_share_7.clone(), - report_share_8.clone(), ]), ); @@ -1636,7 +1592,7 @@ mod tests { let aggregate_resp = AggregationJobResp::get_decoded(&body_bytes).unwrap(); // Validate response. - assert_eq!(aggregate_resp.prepare_steps().len(), 9); + assert_eq!(aggregate_resp.prepare_steps().len(), 8); let prepare_step_0 = aggregate_resp.prepare_steps().get(0).unwrap(); assert_eq!(prepare_step_0.report_id(), report_share_0.metadata().id()); @@ -1674,7 +1630,7 @@ mod tests { assert_eq!(prepare_step_5.report_id(), report_share_5.metadata().id()); assert_eq!( prepare_step_5.result(), - &PrepareStepResult::Failed(ReportShareError::BatchCollected) + &PrepareStepResult::Failed(ReportShareError::UnrecognizedMessage), ); let prepare_step_6 = aggregate_resp.prepare_steps().get(6).unwrap(); @@ -1686,21 +1642,14 @@ mod tests { let prepare_step_7 = aggregate_resp.prepare_steps().get(7).unwrap(); assert_eq!(prepare_step_7.report_id(), report_share_7.metadata().id()); - assert_eq!( - prepare_step_7.result(), - &PrepareStepResult::Failed(ReportShareError::UnrecognizedMessage), - ); - - let prepare_step_8 = aggregate_resp.prepare_steps().get(8).unwrap(); - assert_eq!(prepare_step_8.report_id(), report_share_8.metadata().id()); - assert_matches!(prepare_step_8.result(), &PrepareStepResult::Continued(..)); + assert_matches!(prepare_step_7.result(), &PrepareStepResult::Continued(..)); // Check aggregation job in datastore. let aggregation_jobs = datastore .run_tx(|tx| { let task = task.clone(); Box::pin(async move { - tx.get_aggregation_jobs_for_task::<0, TimeInterval, dummy_vdaf::Vdaf>( + tx.get_aggregation_jobs_for_task::<0, FixedSize, dummy_vdaf::Vdaf>( task.id(), ) .await @@ -1721,7 +1670,7 @@ mod tests { saw_non_conflicting_aggregation_job = true; } else if aggregation_job.task_id().eq(task.id()) && aggregation_job.id().eq(&aggregation_job_id) - && aggregation_job.partial_batch_identifier().eq(&()) + && aggregation_job.partial_batch_identifier().eq(&batch_id) && aggregation_job.state().eq(&AggregationJobState::InProgress) { saw_new_aggregation_job = true; @@ -1760,7 +1709,7 @@ mod tests { // should be flagged as a replay. let request = AggregationJobInitializeReq::new( other_aggregation_parameter.get_encoded(), - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(random()), Vec::from([mutated_timestamp_report_share.clone()]), ); @@ -1807,7 +1756,7 @@ mod tests { install_test_trace_subscriber(); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::FakeFailsPrepInit, Role::Helper, ) @@ -1835,7 +1784,7 @@ mod tests { ); let request = AggregationJobInitializeReq::new( dummy_vdaf::AggregationParam(0).get_encoded(), - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(random()), Vec::from([report_share.clone()]), ); @@ -1876,7 +1825,7 @@ mod tests { install_test_trace_subscriber(); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::FakeFailsPrepInit, Role::Helper, ) @@ -1904,7 +1853,7 @@ mod tests { ); let request = AggregationJobInitializeReq::new( dummy_vdaf::AggregationParam(0).get_encoded(), - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(random()), Vec::from([report_share.clone()]), ); @@ -1944,7 +1893,7 @@ mod tests { install_test_trace_subscriber(); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::FakeFailsPrepInit, Role::Helper, ) @@ -1971,7 +1920,7 @@ mod tests { let request = AggregationJobInitializeReq::new( dummy_vdaf::AggregationParam(0).get_encoded(), - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(random()), Vec::from([report_share.clone(), report_share]), ); @@ -2007,7 +1956,7 @@ mod tests { let aggregation_job_id = random(); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Helper, ) @@ -2073,70 +2022,31 @@ mod tests { &transcript_1.input_shares[1], ); - // report_share_2 falls into a batch that has already been collected. - let past_clock = MockClock::new(Time::from_seconds_since_epoch( - task.time_precision().as_seconds() / 2, - )); - let report_metadata_2 = ReportMetadata::new( - random(), - past_clock - .now() - .to_batch_interval_start(task.time_precision()) - .unwrap(), - ); - let transcript_2 = run_vdaf( - vdaf.as_ref(), - verify_key.as_bytes(), - &(), - report_metadata_2.id(), - &0, - ); - let (prep_state_2, _) = transcript_2.helper_prep_state(0); - let prep_msg_2 = transcript_2.prepare_messages[0].clone(); - let report_share_2 = generate_helper_report_share::( - *task.id(), - report_metadata_2.clone(), - hpke_key.config(), - &transcript_2.public_share, - Vec::new(), - &transcript_2.input_shares[1], - ); - + let batch_id = random(); datastore .run_tx(|tx| { let task = task.clone(); - let (report_share_0, report_share_1, report_share_2) = ( - report_share_0.clone(), - report_share_1.clone(), - report_share_2.clone(), - ); - let (prep_state_0, prep_state_1, prep_state_2) = ( - prep_state_0.clone(), - prep_state_1.clone(), - prep_state_2.clone(), - ); - let (report_metadata_0, report_metadata_1, report_metadata_2) = ( - report_metadata_0.clone(), - report_metadata_1.clone(), - report_metadata_2.clone(), - ); + let (report_share_0, report_share_1) = + (report_share_0.clone(), report_share_1.clone()); + let (prep_state_0, prep_state_1) = (prep_state_0.clone(), prep_state_1.clone()); + let (report_metadata_0, report_metadata_1) = + (report_metadata_0.clone(), report_metadata_1.clone()); Box::pin(async move { tx.put_task(&task).await?; tx.put_report_share(task.id(), &report_share_0).await?; tx.put_report_share(task.id(), &report_share_1).await?; - tx.put_report_share(task.id(), &report_share_2).await?; tx.put_aggregation_job(&AggregationJob::< PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, Prio3Count, >::new( *task.id(), aggregation_job_id, (), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -2168,27 +2078,11 @@ mod tests { ), ) .await?; - tx.put_report_aggregation::( - &ReportAggregation::new( - *task.id(), - aggregation_job_id, - *report_metadata_2.id(), - *report_metadata_2.time(), - 2, - None, - ReportAggregationState::Waiting(prep_state_2, None), - ), - ) - .await?; - tx.put_aggregate_share_job::( + tx.put_aggregate_share_job::( &AggregateShareJob::new( *task.id(), - Interval::new( - Time::from_seconds_since_epoch(0), - *task.time_precision(), - ) - .unwrap(), + batch_id, (), AggregateShare::from(OutputShare::from(Vec::from([Field64::from(7)]))), 0, @@ -2203,16 +2097,10 @@ mod tests { let request = AggregationJobContinueReq::new( AggregationJobRound::from(1), - Vec::from([ - PrepareStep::new( - *report_metadata_0.id(), - PrepareStepResult::Continued(prep_msg_0.get_encoded()), - ), - PrepareStep::new( - *report_metadata_2.id(), - PrepareStepResult::Continued(prep_msg_2.get_encoded()), - ), - ]), + Vec::from([PrepareStep::new( + *report_metadata_0.id(), + PrepareStepResult::Continued(prep_msg_0.get_encoded()), + )]), ); // Create aggregator handler, send request, and parse response. @@ -2230,41 +2118,39 @@ mod tests { // Validate response. assert_eq!( aggregate_resp, - AggregationJobResp::new(Vec::from([ - PrepareStep::new(*report_metadata_0.id(), PrepareStepResult::Finished), - PrepareStep::new( - *report_metadata_2.id(), - PrepareStepResult::Failed(ReportShareError::BatchCollected), - ) - ])) + AggregationJobResp::new(Vec::from([PrepareStep::new( + *report_metadata_0.id(), + PrepareStepResult::Finished + ),])) ); // Validate datastore. - let (aggregation_job, report_aggregations) = - datastore - .run_tx(|tx| { - let (vdaf, task) = (Arc::clone(&vdaf), task.clone()); - Box::pin(async move { - let aggregation_job = tx - .get_aggregation_job::( + let (aggregation_job, report_aggregations) = datastore + .run_tx(|tx| { + let (vdaf, task) = (Arc::clone(&vdaf), task.clone()); + Box::pin(async move { + let aggregation_job = tx + .get_aggregation_job::( task.id(), &aggregation_job_id, ) - .await.unwrap().unwrap(); - let report_aggregations = tx - .get_report_aggregations_for_aggregation_job( - vdaf.as_ref(), - &Role::Helper, - task.id(), - &aggregation_job_id, - ) - .await - .unwrap(); - Ok((aggregation_job, report_aggregations)) - }) + .await + .unwrap() + .unwrap(); + let report_aggregations = tx + .get_report_aggregations_for_aggregation_job( + vdaf.as_ref(), + &Role::Helper, + task.id(), + &aggregation_job_id, + ) + .await + .unwrap(); + Ok((aggregation_job, report_aggregations)) }) - .await - .unwrap(); + }) + .await + .unwrap(); assert_eq!( aggregation_job, @@ -2272,7 +2158,7 @@ mod tests { *task.id(), aggregation_job_id, (), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::Finished, @@ -2304,18 +2190,6 @@ mod tests { None, ReportAggregationState::Failed(ReportShareError::ReportDropped), ), - ReportAggregation::new( - *task.id(), - aggregation_job_id, - *report_metadata_2.id(), - *report_metadata_2.time(), - 2, - Some(PrepareStep::new( - *report_metadata_2.id(), - PrepareStepResult::Failed(ReportShareError::BatchCollected) - )), - ReportAggregationState::Failed(ReportShareError::BatchCollected), - ) ]) ); } @@ -2325,7 +2199,7 @@ mod tests { install_test_trace_subscriber(); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Helper, ) @@ -2419,6 +2293,7 @@ mod tests { &0, ); let (prep_state_2, _) = transcript_2.helper_prep_state(0); + let out_share_2 = transcript_2.output_share(Role::Helper); let prep_msg_2 = transcript_2.prepare_messages[0].clone(); let report_share_2 = generate_helper_report_share::( *task.id(), @@ -2429,18 +2304,13 @@ mod tests { &transcript_2.input_shares[1], ); + let first_batch_id = random(); + let second_batch_id = random(); let second_batch_want_batch_aggregations = - empty_batch_aggregations::( + empty_batch_aggregations::( &task, BATCH_AGGREGATION_SHARD_COUNT, - &Interval::new( - report_metadata_2 - .time() - .to_batch_interval_start(task.time_precision()) - .unwrap(), - *task.time_precision(), - ) - .unwrap(), + &second_batch_id, &(), &[], ); @@ -2475,13 +2345,13 @@ mod tests { tx.put_aggregation_job(&AggregationJob::< PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, Prio3Count, >::new( *task.id(), aggregation_job_id_0, (), - (), + first_batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -2575,27 +2445,13 @@ mod tests { // Map the batch aggregation ordinal value to 0, as it may vary due to sharding. let first_batch_got_batch_aggregations: Vec<_> = datastore .run_tx(|tx| { - let (task, vdaf, report_metadata_0) = - (task.clone(), vdaf.clone(), report_metadata_0.clone()); + let (task, vdaf) = (task.clone(), vdaf.clone()); Box::pin(async move { - TimeInterval::get_batch_aggregations_for_collection_identifier::< + FixedSize::get_batch_aggregations_for_collection_identifier::< PRIO3_VERIFY_KEY_LENGTH, Prio3Count, _, - >( - tx, - &task, - &vdaf, - &Interval::new( - report_metadata_0 - .time() - .to_batch_interval_start(task.time_precision()) - .unwrap(), - Duration::from_seconds(task.time_precision().as_seconds()), - ) - .unwrap(), - &(), - ) + >(tx, &task, &vdaf, &first_batch_id, &()) .await }) }) @@ -2603,7 +2459,7 @@ mod tests { .unwrap() .into_iter() .map(|agg| { - BatchAggregation::::new( + BatchAggregation::::new( *agg.task_id(), *agg.batch_identifier(), (), @@ -2617,55 +2473,42 @@ mod tests { .collect(); let aggregate_share = vdaf - .aggregate(&(), [out_share_0.clone(), out_share_1.clone()]) + .aggregate( + &(), + [ + out_share_0.clone(), + out_share_1.clone(), + out_share_2.clone(), + ], + ) .unwrap(); let checksum = ReportIdChecksum::for_report_id(report_metadata_0.id()) - .updated_with(report_metadata_1.id()); + .updated_with(report_metadata_1.id()) + .updated_with(report_metadata_2.id()); assert_eq!( first_batch_got_batch_aggregations, Vec::from([BatchAggregation::new( *task.id(), - Interval::new( - report_metadata_0 - .time() - .to_batch_interval_start(task.time_precision()) - .unwrap(), - *task.time_precision() - ) - .unwrap(), + first_batch_id, (), 0, BatchAggregationState::Aggregating, Some(aggregate_share), - 2, + 3, checksum, ),]) ); let second_batch_got_batch_aggregations = datastore .run_tx(|tx| { - let (task, vdaf, report_metadata_2) = - (task.clone(), vdaf.clone(), report_metadata_2.clone()); + let (task, vdaf) = (task.clone(), vdaf.clone()); Box::pin(async move { - TimeInterval::get_batch_aggregations_for_collection_identifier::< + FixedSize::get_batch_aggregations_for_collection_identifier::< PRIO3_VERIFY_KEY_LENGTH, Prio3Count, _, - >( - tx, - &task, - &vdaf, - &Interval::new( - report_metadata_2 - .time() - .to_batch_interval_start(task.time_precision()) - .unwrap(), - Duration::from_seconds(task.time_precision().as_seconds()), - ) - .unwrap(), - &(), - ) + >(tx, &task, &vdaf, &second_batch_id, &()) .await }) }) @@ -2722,6 +2565,7 @@ mod tests { &0, ); let (prep_state_4, _) = transcript_4.helper_prep_state(0); + let out_share_4 = transcript_4.output_share(Role::Helper); let prep_msg_4 = transcript_4.prepare_messages[0].clone(); let report_share_4 = generate_helper_report_share::( *task.id(), @@ -2749,6 +2593,7 @@ mod tests { &0, ); let (prep_state_5, _) = transcript_5.helper_prep_state(0); + let out_share_5 = transcript_5.output_share(Role::Helper); let prep_msg_5 = transcript_5.prepare_messages[0].clone(); let report_share_5 = generate_helper_report_share::( *task.id(), @@ -2785,13 +2630,13 @@ mod tests { tx.put_aggregation_job(&AggregationJob::< PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, Prio3Count, >::new( *task.id(), aggregation_job_id_1, (), - (), + first_batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -2880,27 +2725,13 @@ mod tests { // be the same) let merged_first_batch_aggregation = datastore .run_tx(|tx| { - let (task, vdaf, report_metadata_0) = - (task.clone(), vdaf.clone(), report_metadata_0.clone()); + let (task, vdaf) = (task.clone(), vdaf.clone()); Box::pin(async move { - TimeInterval::get_batch_aggregations_for_collection_identifier::< + FixedSize::get_batch_aggregations_for_collection_identifier::< PRIO3_VERIFY_KEY_LENGTH, Prio3Count, _, - >( - tx, - &task, - &vdaf, - &Interval::new( - report_metadata_0 - .time() - .to_batch_interval_start(task.time_precision()) - .unwrap(), - Duration::from_seconds(task.time_precision().as_seconds()), - ) - .unwrap(), - &(), - ) + >(tx, &task, &vdaf, &first_batch_id, &()) .await }) }) @@ -2908,7 +2739,7 @@ mod tests { .unwrap() .into_iter() .map(|agg| { - BatchAggregation::::new( + BatchAggregation::::new( *agg.task_id(), *agg.batch_identifier(), (), @@ -2925,57 +2756,48 @@ mod tests { let first_aggregate_share = vdaf .aggregate( &(), - [out_share_0, out_share_1, out_share_3].into_iter().cloned(), + [ + out_share_0, + out_share_1, + out_share_2, + out_share_3, + out_share_4, + out_share_5, + ] + .into_iter() + .cloned(), ) .unwrap(); let first_checksum = ReportIdChecksum::for_report_id(report_metadata_0.id()) .updated_with(report_metadata_1.id()) - .updated_with(report_metadata_3.id()); + .updated_with(report_metadata_2.id()) + .updated_with(report_metadata_3.id()) + .updated_with(report_metadata_4.id()) + .updated_with(report_metadata_5.id()); assert_eq!( merged_first_batch_aggregation, BatchAggregation::new( *task.id(), - Interval::new( - report_metadata_0 - .time() - .to_batch_interval_start(task.time_precision()) - .unwrap(), - *task.time_precision() - ) - .unwrap(), + first_batch_id, (), 0, BatchAggregationState::Aggregating, Some(first_aggregate_share), - 3, + 6, first_checksum, ), ); let second_batch_got_batch_aggregations = datastore .run_tx(|tx| { - let (task, vdaf, report_metadata_2) = - (task.clone(), vdaf.clone(), report_metadata_2.clone()); + let (task, vdaf) = (task.clone(), vdaf.clone()); Box::pin(async move { - TimeInterval::get_batch_aggregations_for_collection_identifier::< + FixedSize::get_batch_aggregations_for_collection_identifier::< PRIO3_VERIFY_KEY_LENGTH, Prio3Count, _, - >( - tx, - &task, - &vdaf, - &Interval::new( - report_metadata_2 - .time() - .to_batch_interval_start(task.time_precision()) - .unwrap(), - Duration::from_seconds(task.time_precision().as_seconds()), - ) - .unwrap(), - &(), - ) + >(tx, &task, &vdaf, &second_batch_id, &()) .await }) }) @@ -2993,8 +2815,12 @@ mod tests { install_test_trace_subscriber(); // Prepare parameters. - let task = - TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Helper).build(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Helper, + ) + .build(); let aggregation_job_id = random(); let report_metadata = ReportMetadata::new( ReportId::from([1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16]), @@ -3026,13 +2852,13 @@ mod tests { tx.put_aggregation_job(&AggregationJob::< DUMMY_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, dummy_vdaf::Vdaf, >::new( *task.id(), aggregation_job_id, dummy_vdaf::AggregationParam(0), - (), + random(), Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -3093,7 +2919,7 @@ mod tests { // Prepare parameters. let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::FakeFailsPrepStep, Role::Helper, ) @@ -3108,6 +2934,7 @@ mod tests { let datastore = Arc::new(ephemeral_datastore.datastore(clock.clone()).await); // Setup datastore. + let batch_id = random(); datastore .run_tx(|tx| { let (task, report_metadata) = (task.clone(), report_metadata.clone()); @@ -3129,13 +2956,13 @@ mod tests { .await?; tx.put_aggregation_job(&AggregationJob::< DUMMY_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, dummy_vdaf::Vdaf, >::new( *task.id(), aggregation_job_id, dummy_vdaf::AggregationParam(0), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -3193,7 +3020,7 @@ mod tests { let (task, report_metadata) = (task.clone(), report_metadata.clone()); Box::pin(async move { let aggregation_job = tx - .get_aggregation_job::( + .get_aggregation_job::( task.id(), &aggregation_job_id, ) @@ -3219,7 +3046,7 @@ mod tests { *task.id(), aggregation_job_id, dummy_vdaf::AggregationParam(0), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::Finished, @@ -3250,8 +3077,12 @@ mod tests { install_test_trace_subscriber(); // Prepare parameters. - let task = - TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Helper).build(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Helper, + ) + .build(); let aggregation_job_id = random(); let report_metadata = ReportMetadata::new( ReportId::from([1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16]), @@ -3283,13 +3114,13 @@ mod tests { .await?; tx.put_aggregation_job(&AggregationJob::< DUMMY_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, dummy_vdaf::Vdaf, >::new( *task.id(), aggregation_job_id, dummy_vdaf::AggregationParam(0), - (), + random(), Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -3351,8 +3182,12 @@ mod tests { install_test_trace_subscriber(); // Prepare parameters. - let task = - TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Helper).build(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Helper, + ) + .build(); let aggregation_job_id = random(); let report_metadata_0 = ReportMetadata::new( ReportId::from([1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16]), @@ -3408,13 +3243,13 @@ mod tests { tx.put_aggregation_job(&AggregationJob::< DUMMY_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, dummy_vdaf::Vdaf, >::new( *task.id(), aggregation_job_id, dummy_vdaf::AggregationParam(0), - (), + random(), Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -3495,8 +3330,12 @@ mod tests { install_test_trace_subscriber(); // Prepare parameters. - let task = - TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Helper).build(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Helper, + ) + .build(); let aggregation_job_id = random(); let report_metadata = ReportMetadata::new( ReportId::from([1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16]), @@ -3528,13 +3367,13 @@ mod tests { .await?; tx.put_aggregation_job(&AggregationJob::< DUMMY_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, dummy_vdaf::Vdaf, >::new( *task.id(), aggregation_job_id, dummy_vdaf::AggregationParam(0), - (), + random(), Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -3590,17 +3429,15 @@ mod tests { #[tokio::test] async fn collection_job_put_request_to_helper() { - let test_case = setup_collection_job_test_case(Role::Helper, QueryType::TimeInterval).await; + let test_case = setup_collection_job_test_case( + Role::Helper, + QueryType::FixedSize { max_batch_size: 10 }, + ) + .await; let collection_job_id: CollectionJobId = random(); let request = CollectionReq::new( - Query::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(0), - *test_case.task.time_precision(), - ) - .unwrap(), - ), + Query::new_fixed_size(FixedSizeQuery::CurrentBatch), dummy_vdaf::AggregationParam::default().get_encoded(), ); @@ -3620,52 +3457,17 @@ mod tests { ); } - #[tokio::test] - async fn collection_job_put_request_invalid_batch_interval() { - let test_case = setup_collection_job_test_case(Role::Leader, QueryType::TimeInterval).await; - - let collection_job_id: CollectionJobId = random(); - let request = CollectionReq::new( - Query::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(0), - // Collect request will be rejected because batch interval is too small - Duration::from_seconds(test_case.task.time_precision().as_seconds() - 1), - ) - .unwrap(), - ), - dummy_vdaf::AggregationParam::default().get_encoded(), - ); - - let mut test_conn = test_case - .put_collection_job(&collection_job_id, &request) - .await; - - assert_eq!(test_conn.status(), Some(Status::BadRequest)); - assert_eq!( - take_problem_details(&mut test_conn).await, - json!({ - "status": Status::BadRequest as u16, - "type": "urn:ietf:params:ppm:dap:error:batchInvalid", - "title": "The batch implied by the query is invalid.", - "taskid": format!("{}", test_case.task.id()), - }) - ); - } - #[tokio::test] async fn collection_job_put_request_invalid_aggregation_parameter() { - let test_case = setup_collection_job_test_case(Role::Leader, QueryType::TimeInterval).await; + let test_case = setup_collection_job_test_case( + Role::Leader, + QueryType::FixedSize { max_batch_size: 10 }, + ) + .await; let collection_job_id: CollectionJobId = random(); let request = CollectionReq::new( - Query::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(test_case.task.time_precision().as_seconds()), - ) - .unwrap(), - ), + Query::new_fixed_size(FixedSizeQuery::CurrentBatch), // dummy_vdaf::AggregationParam is a tuple struct wrapping a u8, so this is not a valid // encoding of an aggregation parameter. Vec::from([0u8, 0u8]), @@ -3692,9 +3494,13 @@ mod tests { install_test_trace_subscriber(); // Prepare parameters. - let task = TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Leader) - .with_min_batch_size(1) - .build(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Leader, + ) + .with_min_batch_size(1) + .build(); let clock = MockClock::default(); let ephemeral_datastore = ephemeral_datastore().await; let datastore = ephemeral_datastore.datastore(clock.clone()).await; @@ -3711,13 +3517,7 @@ mod tests { let collection_job_id: CollectionJobId = random(); let request = CollectionReq::new( - Query::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(task.time_precision().as_seconds()), - ) - .unwrap(), - ), + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id: random() }), dummy_vdaf::AggregationParam::default().get_encoded(), ); @@ -3728,7 +3528,7 @@ mod tests { ) .with_request_header( KnownHeaderName::ContentType, - CollectionReq::::MEDIA_TYPE, + CollectionReq::::MEDIA_TYPE, ) .with_request_body(request.get_encoded()) .run_async(&handler) @@ -3749,16 +3549,15 @@ mod tests { #[tokio::test] async fn collection_job_put_request_unauthenticated() { - let test_case = setup_collection_job_test_case(Role::Leader, QueryType::TimeInterval).await; - - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(0), - *test_case.task.time_precision(), + let test_case = setup_collection_job_test_case( + Role::Leader, + QueryType::FixedSize { max_batch_size: 10 }, ) - .unwrap(); + .await; + let collection_job_id: CollectionJobId = random(); let req = CollectionReq::new( - Query::new_time_interval(batch_interval), + Query::new_fixed_size(FixedSizeQuery::CurrentBatch), dummy_vdaf::AggregationParam::default().get_encoded(), ); @@ -3820,17 +3619,15 @@ mod tests { #[tokio::test] async fn collection_job_post_request_unauthenticated_collection_jobs() { - let test_case = setup_collection_job_test_case(Role::Leader, QueryType::TimeInterval).await; - - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(0), - *test_case.task.time_precision(), + let test_case = setup_collection_job_test_case( + Role::Leader, + QueryType::FixedSize { max_batch_size: 10 }, ) - .unwrap(); + .await; let collection_job_id: CollectionJobId = random(); let request = CollectionReq::new( - Query::new_time_interval(batch_interval), + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id: random() }), dummy_vdaf::AggregationParam::default().get_encoded(), ); @@ -3896,23 +3693,26 @@ mod tests { } #[tokio::test] - async fn collection_job_success_time_interval() { - let test_case = setup_collection_job_test_case(Role::Leader, QueryType::TimeInterval).await; - - let batch_interval = TimeInterval::to_batch_identifier( - &test_case.task, - &(), - &Time::from_seconds_since_epoch(0), + async fn collection_job_success() { + let test_case = setup_collection_job_test_case( + Role::Leader, + QueryType::FixedSize { max_batch_size: 10 }, ) - .unwrap(); + .await; + let batch_id = random(); + let client_timestamp_interval = Interval::new( + Time::from_seconds_since_epoch(0), + *test_case.task.time_precision(), + ) + .unwrap(); let aggregation_param = dummy_vdaf::AggregationParam::default(); let leader_aggregate_share = dummy_vdaf::AggregateShare(0); let helper_aggregate_share = dummy_vdaf::AggregateShare(1); let collection_job_id: CollectionJobId = random(); let request = CollectionReq::new( - Query::new_time_interval(batch_interval), + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id }), aggregation_param.get_encoded(), ); @@ -3922,13 +3722,13 @@ mod tests { let task_id = *test_case.task.id(); Box::pin(async move { - tx.put_batch(&Batch::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + tx.put_batch(&Batch::<0, FixedSize, dummy_vdaf::Vdaf>::new( task_id, - batch_interval, + batch_id, aggregation_param, BatchState::Open, 1, - batch_interval, + client_timestamp_interval, )) .await?; Ok(()) @@ -3941,20 +3741,20 @@ mod tests { .put_collection_job(&collection_job_id, &request) .await; - let want_collection_job = CollectionJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + let want_collection_job = CollectionJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( *test_case.task.id(), collection_job_id, - batch_interval, + batch_id, aggregation_param, CollectionJobState::Start, ); - let want_batches = Vec::from([Batch::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + let want_batches = Vec::from([Batch::<0, FixedSize, dummy_vdaf::Vdaf>::new( *test_case.task.id(), - batch_interval, + batch_id, aggregation_param, BatchState::Closing, 1, - batch_interval, + client_timestamp_interval, )]); let (got_collection_job, got_batches) = test_case @@ -4000,14 +3800,14 @@ mod tests { &helper_aggregate_share_bytes, &AggregateShareAad::new( *task.id(), - BatchSelector::new_time_interval(batch_interval), + BatchSelector::new_fixed_size(batch_id), ) .get_encoded(), ) .unwrap(); let collection_job = tx - .get_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>( &dummy_vdaf::Vdaf::new(), &collection_job_id, ) @@ -4020,7 +3820,7 @@ mod tests { leader_aggregate_share, }); - tx.update_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>(&collection_job) + tx.update_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>(&collection_job) .await .unwrap(); Ok(()) @@ -4034,13 +3834,13 @@ mod tests { assert_eq!(test_conn.status(), Some(Status::Ok)); assert_headers!( &test_conn, - "content-type" => (Collection::::MEDIA_TYPE) + "content-type" => (Collection::::MEDIA_TYPE) ); let body_bytes = take_response_body(&mut test_conn).await; - let collect_resp = Collection::::get_decoded(body_bytes.as_ref()).unwrap(); + let collect_resp = Collection::::get_decoded(body_bytes.as_ref()).unwrap(); assert_eq!(collect_resp.report_count(), 12); - assert_eq!(collect_resp.interval(), &batch_interval); + assert_eq!(collect_resp.interval(), &client_timestamp_interval); assert_eq!(collect_resp.encrypted_aggregate_shares().len(), 2); let decrypted_leader_aggregate_share = hpke::open( @@ -4050,7 +3850,7 @@ mod tests { &collect_resp.encrypted_aggregate_shares()[0], &AggregateShareAad::new( *test_case.task.id(), - BatchSelector::new_time_interval(batch_interval), + BatchSelector::new_fixed_size(batch_id), ) .get_encoded(), ) @@ -4068,7 +3868,7 @@ mod tests { &collect_resp.encrypted_aggregate_shares()[1], &AggregateShareAad::new( *test_case.task.id(), - BatchSelector::new_time_interval(batch_interval), + BatchSelector::new_fixed_size(batch_id), ) .get_encoded(), ) @@ -4082,7 +3882,11 @@ mod tests { #[tokio::test] async fn collection_job_post_request_no_such_collection_job() { - let test_case = setup_collection_job_test_case(Role::Leader, QueryType::TimeInterval).await; + let test_case = setup_collection_job_test_case( + Role::Leader, + QueryType::FixedSize { max_batch_size: 10 }, + ) + .await; let no_such_collection_job_id: CollectionJobId = random(); @@ -4105,12 +3909,12 @@ mod tests { #[tokio::test] async fn collection_job_put_request_batch_queried_too_many_times() { - let test_case = setup_collection_job_test_case(Role::Leader, QueryType::TimeInterval).await; - let interval = Interval::new( - Time::from_seconds_since_epoch(0), - *test_case.task.time_precision(), + let test_case = setup_collection_job_test_case( + Role::Leader, + QueryType::FixedSize { max_batch_size: 10 }, ) - .unwrap(); + .await; + let batch_id = random(); test_case .datastore @@ -4119,12 +3923,11 @@ mod tests { Box::pin(async move { tx.put_batch_aggregation(&BatchAggregation::< DUMMY_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, dummy_vdaf::Vdaf, >::new( *task.id(), - Interval::new(Time::from_seconds_since_epoch(0), *task.time_precision()) - .unwrap(), + batch_id, dummy_vdaf::AggregationParam(0), 0, BatchAggregationState::Aggregating, @@ -4140,7 +3943,7 @@ mod tests { // Sending this request will consume a query for [0, time_precision). let request = CollectionReq::new( - Query::new_time_interval(interval), + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id }), dummy_vdaf::AggregationParam(0).get_encoded(), ); @@ -4150,7 +3953,7 @@ mod tests { // This request will not be allowed due to the query count already being consumed. let invalid_request = CollectionReq::new( - Query::new_time_interval(interval), + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id }), dummy_vdaf::AggregationParam(1).get_encoded(), ); @@ -4169,87 +3972,14 @@ mod tests { ); } - #[tokio::test] - async fn collection_job_put_request_batch_overlap() { - let test_case = setup_collection_job_test_case(Role::Leader, QueryType::TimeInterval).await; - let interval = Interval::new( - Time::from_seconds_since_epoch(0), - *test_case.task.time_precision(), - ) - .unwrap(); - - test_case - .datastore - .run_tx(|tx| { - let task = test_case.task.clone(); - Box::pin(async move { - tx.put_batch_aggregation(&BatchAggregation::< - DUMMY_VERIFY_KEY_LENGTH, - TimeInterval, - dummy_vdaf::Vdaf, - >::new( - *task.id(), - interval, - dummy_vdaf::AggregationParam(0), - 0, - BatchAggregationState::Aggregating, - Some(dummy_vdaf::AggregateShare(0)), - 10, - ReportIdChecksum::get_decoded(&[2; 32]).unwrap(), - )) - .await - }) - }) - .await - .unwrap(); - - // Sending this request will consume a query for [0, 2 * time_precision). - let request = CollectionReq::new( - Query::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_microseconds( - 2 * test_case.task.time_precision().as_microseconds().unwrap(), - ), - ) - .unwrap(), - ), - dummy_vdaf::AggregationParam(0).get_encoded(), - ); - - let test_conn = test_case.put_collection_job(&random(), &request).await; - - assert_eq!(test_conn.status(), Some(Status::Created)); - - // This request will not be allowed due to overlapping with the previous request. - let invalid_request = CollectionReq::new( - Query::new_time_interval(interval), - dummy_vdaf::AggregationParam(1).get_encoded(), - ); - - let mut test_conn = test_case - .put_collection_job(&random(), &invalid_request) - .await; - assert_eq!(test_conn.status(), Some(Status::BadRequest)); - assert_eq!( - take_problem_details(&mut test_conn).await, - json!({ - "status": Status::BadRequest as u16, - "type": "urn:ietf:params:ppm:dap:error:batchOverlap", - "title": "The queried batch overlaps with a previously queried batch.", - "taskid": format!("{}", test_case.task.id()), - }) - ); - } - #[tokio::test] async fn delete_collection_job() { - let test_case = setup_collection_job_test_case(Role::Leader, QueryType::TimeInterval).await; - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(0), - *test_case.task.time_precision(), + let test_case = setup_collection_job_test_case( + Role::Leader, + QueryType::FixedSize { max_batch_size: 10 }, ) - .unwrap(); + .await; + let batch_id = random(); let collection_job_id: CollectionJobId = random(); @@ -4275,7 +4005,7 @@ mod tests { // Create a collection job let request = CollectionReq::new( - Query::new_time_interval(batch_interval), + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id }), dummy_vdaf::AggregationParam::default().get_encoded(), ); @@ -4315,8 +4045,12 @@ mod tests { install_test_trace_subscriber(); // Prepare parameters. - let task = - TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Leader).build(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Leader, + ) + .build(); let clock = MockClock::default(); let ephemeral_datastore = ephemeral_datastore().await; let datastore = ephemeral_datastore.datastore(clock.clone()).await; @@ -4332,9 +4066,7 @@ mod tests { .unwrap(); let request = AggregateShareReq::new( - BatchSelector::new_time_interval( - Interval::new(Time::from_seconds_since_epoch(0), *task.time_precision()).unwrap(), - ), + BatchSelector::new_fixed_size(random()), Vec::new(), 0, ReportIdChecksum::default(), @@ -4347,7 +4079,7 @@ mod tests { ) .with_request_header( KnownHeaderName::ContentType, - AggregateShareReq::::MEDIA_TYPE, + AggregateShareReq::::MEDIA_TYPE, ) .with_request_body(request.get_encoded()) .run_async(&handler) @@ -4365,108 +4097,21 @@ mod tests { ); } - #[tokio::test] - async fn aggregate_share_request_invalid_batch_interval() { - install_test_trace_subscriber(); - - // Prepare parameters. - const REPORT_EXPIRY_AGE: Duration = Duration::from_seconds(3600); - let task = TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Helper) - .with_report_expiry_age(Some(REPORT_EXPIRY_AGE)) - .build(); - let clock = MockClock::default(); - let ephemeral_datastore = ephemeral_datastore().await; - let datastore = ephemeral_datastore.datastore(clock.clone()).await; - - datastore.put_task(&task).await.unwrap(); - - let handler = aggregator_handler( - Arc::new(datastore), - clock.clone(), - &noop_meter(), - default_aggregator_config(), - ) - .unwrap(); - - let request = AggregateShareReq::new( - BatchSelector::new_time_interval( - Interval::new( - clock.now(), - // Collect request will be rejected because batch interval is too small - Duration::from_seconds(task.time_precision().as_seconds() - 1), - ) - .unwrap(), - ), - Vec::new(), - 0, - ReportIdChecksum::default(), - ); - - // Test that a request for an invalid batch fails. (Specifically, the batch interval is too - // small.) - let mut test_conn = post(task.aggregate_shares_uri().unwrap().path()) - .with_request_header( - "DAP-Auth-Token", - task.primary_aggregator_auth_token().as_ref().to_owned(), - ) - .with_request_header( - KnownHeaderName::ContentType, - AggregateShareReq::::MEDIA_TYPE, - ) - .with_request_body(request.get_encoded()) - .run_async(&handler) - .await; - - assert_eq!(test_conn.status(), Some(Status::BadRequest)); - assert_eq!( - take_problem_details(&mut test_conn).await, - json!({ - "status": Status::BadRequest as u16, - "type": "urn:ietf:params:ppm:dap:error:batchInvalid", - "title": "The batch implied by the query is invalid.", - "taskid": format!("{}", task.id()), - }) - ); - - // Test that a request for a too-old batch fails. - let test_conn = post(task.aggregate_shares_uri().unwrap().path()) - .with_request_header( - "DAP-Auth-Token", - task.primary_aggregator_auth_token().as_ref().to_owned(), - ) - .with_request_header( - KnownHeaderName::ContentType, - AggregateShareReq::::MEDIA_TYPE, - ) - .with_request_body( - AggregateShareReq::new( - BatchSelector::new_time_interval( - Interval::new(Time::from_seconds_since_epoch(0), *task.time_precision()) - .unwrap(), - ), - Vec::new(), - 0, - ReportIdChecksum::default(), - ) - .get_encoded(), - ) - .run_async(&handler) - .await; - - assert_eq!(test_conn.status(), Some(Status::BadRequest)); - } - #[tokio::test] async fn aggregate_share_request() { install_test_trace_subscriber(); let collector_hpke_keypair = generate_test_hpke_config_and_private_key(); - let task = TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Helper) - .with_max_batch_query_count(1) - .with_time_precision(Duration::from_seconds(500)) - .with_min_batch_size(10) - .with_collector_hpke_config(collector_hpke_keypair.config().clone()) - .build(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Helper, + ) + .with_max_batch_query_count(1) + .with_time_precision(Duration::from_seconds(500)) + .with_min_batch_size(10) + .with_collector_hpke_config(collector_hpke_keypair.config().clone()) + .build(); let clock = MockClock::default(); let ephemeral_datastore = ephemeral_datastore().await; @@ -4484,9 +4129,7 @@ mod tests { // There are no batch aggregations in the datastore yet let request = AggregateShareReq::new( - BatchSelector::new_time_interval( - Interval::new(Time::from_seconds_since_epoch(0), *task.time_precision()).unwrap(), - ), + BatchSelector::new_fixed_size(random()), dummy_vdaf::AggregationParam(0).get_encoded(), 0, ReportIdChecksum::default(), @@ -4499,7 +4142,7 @@ mod tests { ) .with_request_header( KnownHeaderName::ContentType, - AggregateShareReq::::MEDIA_TYPE, + AggregateShareReq::::MEDIA_TYPE, ) .with_request_body(request.get_encoded()) .run_async(&handler) @@ -4516,6 +4159,9 @@ mod tests { }) ); + let batch_id_1 = random(); + let batch_id_2 = random(); + // Put some batch aggregations in the DB. datastore .run_tx(|tx| { @@ -4525,18 +4171,13 @@ mod tests { dummy_vdaf::AggregationParam(0), dummy_vdaf::AggregationParam(1), ] { - let interval_1 = Interval::new( - Time::from_seconds_since_epoch(500), - *task.time_precision(), - ) - .unwrap(); tx.put_batch_aggregation(&BatchAggregation::< DUMMY_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, dummy_vdaf::Vdaf, >::new( *task.id(), - interval_1, + batch_id_1, aggregation_param, 0, BatchAggregationState::Aggregating, @@ -4546,68 +4187,21 @@ mod tests { )) .await?; - let interval_2 = Interval::new( - Time::from_seconds_since_epoch(1500), - *task.time_precision(), - ) - .unwrap(); tx.put_batch_aggregation(&BatchAggregation::< DUMMY_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, dummy_vdaf::Vdaf, >::new( *task.id(), - interval_2, + batch_id_2, aggregation_param, 0, BatchAggregationState::Aggregating, Some(dummy_vdaf::AggregateShare(128)), - 5, + 10, ReportIdChecksum::get_decoded(&[2; 32]).unwrap(), )) .await?; - - let interval_3 = Interval::new( - Time::from_seconds_since_epoch(2000), - *task.time_precision(), - ) - .unwrap(); - tx.put_batch_aggregation(&BatchAggregation::< - DUMMY_VERIFY_KEY_LENGTH, - TimeInterval, - dummy_vdaf::Vdaf, - >::new( - *task.id(), - interval_3, - aggregation_param, - 0, - BatchAggregationState::Aggregating, - Some(dummy_vdaf::AggregateShare(256)), - 5, - ReportIdChecksum::get_decoded(&[4; 32]).unwrap(), - )) - .await?; - - let interval_4 = Interval::new( - Time::from_seconds_since_epoch(2500), - *task.time_precision(), - ) - .unwrap(); - tx.put_batch_aggregation(&BatchAggregation::< - DUMMY_VERIFY_KEY_LENGTH, - TimeInterval, - dummy_vdaf::Vdaf, - >::new( - *task.id(), - interval_4, - aggregation_param, - 0, - BatchAggregationState::Aggregating, - Some(dummy_vdaf::AggregateShare(512)), - 5, - ReportIdChecksum::get_decoded(&[8; 32]).unwrap(), - )) - .await?; } Ok(()) @@ -4616,15 +4210,9 @@ mod tests { .await .unwrap(); - // Specified interval includes too few reports. + // Specified batch includes too few reports. let request = AggregateShareReq::new( - BatchSelector::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(1000), - ) - .unwrap(), - ), + BatchSelector::new_fixed_size(batch_id_1), dummy_vdaf::AggregationParam(0).get_encoded(), 5, ReportIdChecksum::default(), @@ -4636,7 +4224,7 @@ mod tests { ) .with_request_header( KnownHeaderName::ContentType, - AggregateShareReq::::MEDIA_TYPE, + AggregateShareReq::::MEDIA_TYPE, ) .with_request_body(request.get_encoded()) .run_async(&handler) @@ -4655,31 +4243,19 @@ mod tests { // Make requests that will fail because the checksum or report counts don't match. for misaligned_request in [ - // Interval is big enough, but checksum doesn't match. + // Checksum doesn't match. AggregateShareReq::new( - BatchSelector::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(2000), - ) - .unwrap(), - ), + BatchSelector::new_fixed_size(batch_id_2), dummy_vdaf::AggregationParam(0).get_encoded(), 10, ReportIdChecksum::get_decoded(&[3; 32]).unwrap(), ), - // Interval is big enough, but report count doesn't match. + // Report count doesn't match. AggregateShareReq::new( - BatchSelector::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(2000), - Duration::from_seconds(2000), - ) - .unwrap(), - ), + BatchSelector::new_fixed_size(batch_id_2), dummy_vdaf::AggregationParam(0).get_encoded(), 20, - ReportIdChecksum::get_decoded(&[4 ^ 8; 32]).unwrap(), + ReportIdChecksum::get_decoded(&[2; 32]).unwrap(), ), ] { let mut test_conn = post(task.aggregate_shares_uri().unwrap().path()) @@ -4689,7 +4265,7 @@ mod tests { ) .with_request_header( KnownHeaderName::ContentType, - AggregateShareReq::::MEDIA_TYPE, + AggregateShareReq::::MEDIA_TYPE, ) .with_request_body(misaligned_request.get_encoded()) .run_async(&handler) @@ -4707,108 +4283,68 @@ mod tests { ); } - // Valid requests: intervals are big enough, do not overlap, checksum and report count are - // good. - for (label, request, expected_result) in [ - ( - "first and second batchess", - AggregateShareReq::new( - BatchSelector::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(2000), - ) - .unwrap(), - ), - dummy_vdaf::AggregationParam(0).get_encoded(), - 10, - ReportIdChecksum::get_decoded(&[3 ^ 2; 32]).unwrap(), - ), - dummy_vdaf::AggregateShare(64 + 128), - ), - ( - "third and fourth batches", - AggregateShareReq::new( - BatchSelector::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(2000), - Duration::from_seconds(2000), - ) - .unwrap(), - ), - dummy_vdaf::AggregationParam(0).get_encoded(), - 10, - ReportIdChecksum::get_decoded(&[8 ^ 4; 32]).unwrap(), - ), - // Should get sum over the third and fourth batches - dummy_vdaf::AggregateShare(256 + 512), - ), - ] { - // Request the aggregate share multiple times. If the request parameters don't change, - // then there is no query count violation and all requests should succeed. - for iteration in 0..3 { - let mut test_conn = post(task.aggregate_shares_uri().unwrap().path()) - .with_request_header( - "DAP-Auth-Token", - task.primary_aggregator_auth_token().as_ref().to_owned(), - ) - .with_request_header( - KnownHeaderName::ContentType, - AggregateShareReq::::MEDIA_TYPE, - ) - .with_request_body(request.get_encoded()) - .run_async(&handler) - .await; - - assert_eq!( - test_conn.status(), - Some(Status::Ok), - "test case: {label:?}, iteration: {iteration}" - ); - assert_headers!( - &test_conn, - "content-type" => (AggregateShareMessage::MEDIA_TYPE) - ); - let body_bytes = take_response_body(&mut test_conn).await; - let aggregate_share_resp = AggregateShareMessage::get_decoded(&body_bytes).unwrap(); - - let aggregate_share = hpke::open( - collector_hpke_keypair.config(), - collector_hpke_keypair.private_key(), - &HpkeApplicationInfo::new( - &Label::AggregateShare, - &Role::Helper, - &Role::Collector, - ), - aggregate_share_resp.encrypted_aggregate_share(), - &AggregateShareAad::new(*task.id(), request.batch_selector().clone()) - .get_encoded(), + // Valid request: checksum and report count are good. + // Request the aggregate share multiple times. If the request parameters don't change, + // then there is no query count violation and all requests should succeed. + for iteration in 0..3 { + let request = AggregateShareReq::new( + BatchSelector::new_fixed_size(batch_id_2), + dummy_vdaf::AggregationParam(0).get_encoded(), + 10, + ReportIdChecksum::get_decoded(&[2; 32]).unwrap(), + ); + let expected_result = dummy_vdaf::AggregateShare(128); + + let mut test_conn = post(task.aggregate_shares_uri().unwrap().path()) + .with_request_header( + "DAP-Auth-Token", + task.primary_aggregator_auth_token().as_ref().to_owned(), ) - .unwrap(); + .with_request_header( + KnownHeaderName::ContentType, + AggregateShareReq::::MEDIA_TYPE, + ) + .with_request_body(request.get_encoded()) + .run_async(&handler) + .await; - // Should get the sum over the first and second aggregate shares - let decoded_aggregate_share = - dummy_vdaf::AggregateShare::get_decoded(aggregate_share.as_ref()).unwrap(); - assert_eq!( - decoded_aggregate_share, expected_result, - "test case: {label:?}, iteration: {iteration}" - ); - } + assert_eq!( + test_conn.status(), + Some(Status::Ok), + "iteration: {iteration}" + ); + assert_headers!( + &test_conn, + "content-type" => (AggregateShareMessage::MEDIA_TYPE) + ); + let body_bytes = take_response_body(&mut test_conn).await; + let aggregate_share_resp = AggregateShareMessage::get_decoded(&body_bytes).unwrap(); + + let aggregate_share = hpke::open( + collector_hpke_keypair.config(), + collector_hpke_keypair.private_key(), + &HpkeApplicationInfo::new(&Label::AggregateShare, &Role::Helper, &Role::Collector), + aggregate_share_resp.encrypted_aggregate_share(), + &AggregateShareAad::new(*task.id(), request.batch_selector().clone()).get_encoded(), + ) + .unwrap(); + + // Should get the sum over the first and second aggregate shares + let decoded_aggregate_share = + dummy_vdaf::AggregateShare::get_decoded(aggregate_share.as_ref()).unwrap(); + assert_eq!( + decoded_aggregate_share, expected_result, + "iteration: {iteration}" + ); } - // Requests for collection intervals that overlap with but are not identical to previous - // collection intervals fail. - let all_batch_request = AggregateShareReq::new( - BatchSelector::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(4000), - ) - .unwrap(), - ), - dummy_vdaf::AggregationParam(0).get_encoded(), - 20, - ReportIdChecksum::get_decoded(&[8 ^ 4 ^ 3 ^ 2; 32]).unwrap(), + // Previous sequence of aggregate share requests should have consumed the available queries + // for all the batches. Further requests for any batches will cause query count violations. + let query_count_violation_request = AggregateShareReq::new( + BatchSelector::new_fixed_size(batch_id_2), + dummy_vdaf::AggregationParam(1).get_encoded(), + 10, + ReportIdChecksum::get_decoded(&[2; 32]).unwrap(), ); let mut test_conn = post(task.aggregate_shares_uri().unwrap().path()) .with_request_header( @@ -4817,9 +4353,9 @@ mod tests { ) .with_request_header( KnownHeaderName::ContentType, - AggregateShareReq::::MEDIA_TYPE, + AggregateShareReq::::MEDIA_TYPE, ) - .with_request_body(all_batch_request.get_encoded()) + .with_request_body(query_count_violation_request.get_encoded()) .run_async(&handler) .await; assert_eq!(test_conn.status(), Some(Status::BadRequest)); @@ -4827,63 +4363,11 @@ mod tests { take_problem_details(&mut test_conn).await, json!({ "status": Status::BadRequest as u16, - "type": "urn:ietf:params:ppm:dap:error:batchOverlap", - "title": "The queried batch overlaps with a previously queried batch.", + "type": "urn:ietf:params:ppm:dap:error:batchQueriedTooManyTimes", + "title": "The batch described by the query has been queried too many times.", "taskid": format!("{}", task.id()), - }), + }) ); - - // Previous sequence of aggregate share requests should have consumed the available queries - // for all the batches. Further requests for any batches will cause query count violations. - for query_count_violation_request in [ - AggregateShareReq::new( - BatchSelector::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(2000), - ) - .unwrap(), - ), - dummy_vdaf::AggregationParam(1).get_encoded(), - 10, - ReportIdChecksum::get_decoded(&[3 ^ 2; 32]).unwrap(), - ), - AggregateShareReq::new( - BatchSelector::new_time_interval( - Interval::new( - Time::from_seconds_since_epoch(2000), - Duration::from_seconds(2000), - ) - .unwrap(), - ), - dummy_vdaf::AggregationParam(1).get_encoded(), - 10, - ReportIdChecksum::get_decoded(&[4 ^ 8; 32]).unwrap(), - ), - ] { - let mut test_conn = post(task.aggregate_shares_uri().unwrap().path()) - .with_request_header( - "DAP-Auth-Token", - task.primary_aggregator_auth_token().as_ref().to_owned(), - ) - .with_request_header( - KnownHeaderName::ContentType, - AggregateShareReq::::MEDIA_TYPE, - ) - .with_request_body(query_count_violation_request.get_encoded()) - .run_async(&handler) - .await; - assert_eq!(test_conn.status(), Some(Status::BadRequest)); - assert_eq!( - take_problem_details(&mut test_conn).await, - json!({ - "status": Status::BadRequest as u16, - "type": "urn:ietf:params:ppm:dap:error:batchQueriedTooManyTimes", - "title": "The batch described by the query has been queried too many times.", - "taskid": format!("{}", task.id()), - }) - ); - } } async fn take_response_body(test_conn: &mut TestConn) -> Cow<'_, [u8]> { diff --git a/aggregator/src/aggregator/query_type.rs b/aggregator/src/aggregator/query_type.rs index f944ca3d7..d58fa90e8 100644 --- a/aggregator/src/aggregator/query_type.rs +++ b/aggregator/src/aggregator/query_type.rs @@ -7,7 +7,7 @@ use janus_aggregator_core::{ }; use janus_core::time::Clock; use janus_messages::{ - query_type::{FixedSize, QueryType, TimeInterval}, + query_type::{FixedSize, QueryType}, Role, }; use prio::vdaf; @@ -28,45 +28,6 @@ pub trait UploadableQueryType: QueryType { A::PublicShare: Send + Sync; } -#[async_trait] -impl UploadableQueryType for TimeInterval { - async fn validate_uploaded_report< - const SEED_SIZE: usize, - C: Clock, - A: vdaf::Aggregator + Send + Sync, - >( - tx: &Transaction<'_, C>, - vdaf: &A, - report: &LeaderStoredReport, - ) -> Result<(), datastore::Error> - where - A::InputShare: Send + Sync, - A::PublicShare: Send + Sync, - { - // Reject reports whose timestamps fall into a batch interval that has already been - // collected. - // https://datatracker.ietf.org/doc/html/draft-ietf-ppm-dap-03#section-4.3.2-17 - let conflicting_collect_jobs = tx - .get_collection_jobs_including_time::( - vdaf, - report.task_id(), - report.metadata().time(), - ) - .await?; - if !conflicting_collect_jobs.is_empty() { - return Err(datastore::Error::User( - Error::ReportRejected( - *report.task_id(), - *report.metadata().id(), - *report.metadata().time(), - ) - .into(), - )); - } - Ok(()) - } -} - #[async_trait] impl UploadableQueryType for FixedSize { async fn validate_uploaded_report< @@ -103,71 +64,6 @@ pub trait CollectableQueryType: CoreCollectableQueryType + AccumulableQueryType ) -> Result<(), datastore::Error>; } -#[async_trait] -impl CollectableQueryType for TimeInterval { - async fn validate_query_count< - const SEED_SIZE: usize, - C: Clock, - A: vdaf::Aggregator + Send + Sync, - >( - tx: &Transaction<'_, C>, - vdaf: &A, - task: &Task, - collect_interval: &Self::BatchIdentifier, - ) -> Result<(), datastore::Error> { - // Check how many rows in the relevant table have an intersecting batch interval. - // Each such row consumes one unit of query count. - // https://www.ietf.org/archive/id/draft-ietf-ppm-dap-02.html#section-4.5.6 - let intersecting_intervals: Vec<_> = match task.role() { - Role::Leader => tx - .get_collection_jobs_intersecting_interval::( - vdaf, - task.id(), - collect_interval, - ) - .await? - .into_iter() - .map(|job| *job.batch_interval()) - .collect(), - - Role::Helper => tx - .get_aggregate_share_jobs_intersecting_interval::( - vdaf, - task.id(), - collect_interval, - ) - .await? - .into_iter() - .map(|job| *job.batch_interval()) - .collect(), - - _ => panic!("Unexpected task role {:?}", task.role()), - }; - - // Check that all intersecting collect intervals are equal to this collect interval. - // https://www.ietf.org/archive/id/draft-ietf-ppm-dap-02.html#section-4.5.6-5 - if intersecting_intervals - .iter() - .any(|interval| interval != collect_interval) - { - return Err(datastore::Error::User( - Error::BatchOverlap(*task.id(), *collect_interval).into(), - )); - } - - // Check that the batch query count is being consumed appropriately. - // https://www.ietf.org/archive/id/draft-ietf-ppm-dap-02.html#section-4.5.6 - let max_batch_query_count: usize = task.max_batch_query_count().try_into()?; - if intersecting_intervals.len() >= max_batch_query_count { - return Err(datastore::Error::User( - Error::BatchQueriedTooManyTimes(*task.id(), intersecting_intervals.len() as u64) - .into(), - )); - } - Ok(()) - } -} - #[async_trait] impl CollectableQueryType for FixedSize { async fn validate_query_count< diff --git a/aggregator/src/bin/janus_cli.rs b/aggregator/src/bin/janus_cli.rs index 9af194e18..3ac670ff9 100644 --- a/aggregator/src/bin/janus_cli.rs +++ b/aggregator/src/bin/janus_cli.rs @@ -589,13 +589,13 @@ mod tests { let tasks = Vec::from([ TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) .build(), TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Sum { bits: 64 }, Role::Helper, ) @@ -622,7 +622,7 @@ mod tests { let ds = ephemeral_datastore.datastore(RealClock::default()).await; let tasks = Vec::from([TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -645,13 +645,13 @@ mod tests { async fn replace_task() { let tasks = Vec::from([ TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) .build(), TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Sum { bits: 64 }, Role::Helper, ) @@ -719,7 +719,8 @@ mod tests { - aggregator_endpoints: - https://leader - https://helper - query_type: TimeInterval + query_type: !FixedSize + max_batch_size: 100 vdaf: !Prio3Sum bits: 2 role: Leader @@ -742,7 +743,8 @@ mod tests { - aggregator_endpoints: - https://leader - https://helper - query_type: TimeInterval + query_type: !FixedSize + max_batch_size: 100 vdaf: !Prio3Sum bits: 2 role: Helper diff --git a/aggregator/tests/graceful_shutdown.rs b/aggregator/tests/graceful_shutdown.rs index a6a389cd1..a6ef51ad5 100644 --- a/aggregator/tests/graceful_shutdown.rs +++ b/aggregator/tests/graceful_shutdown.rs @@ -125,7 +125,7 @@ async fn graceful_shutdown(binary: &Path, mut config: Mapping) { ); let task = TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) diff --git a/aggregator_api/src/lib.rs b/aggregator_api/src/lib.rs index 8db66239b..a61675002 100644 --- a/aggregator_api/src/lib.rs +++ b/aggregator_api/src/lib.rs @@ -621,8 +621,8 @@ mod tests { time::MockClock, }; use janus_messages::{ - query_type::TimeInterval, AggregationJobRound, Duration, HpkeAeadId, HpkeConfig, - HpkeConfigId, HpkeKdfId, HpkeKemId, HpkePublicKey, Interval, Role, TaskId, Time, + query_type::FixedSize, AggregationJobRound, Duration, HpkeAeadId, HpkeConfig, HpkeConfigId, + HpkeKdfId, HpkeKemId, HpkePublicKey, Interval, Role, TaskId, Time, }; use rand::{distributions::Standard, random, thread_rng, Rng}; use serde_test::{assert_ser_tokens, assert_tokens, Token}; @@ -658,8 +658,12 @@ mod tests { .run_tx(|tx| { Box::pin(async move { let tasks: Vec<_> = iter::repeat_with(|| { - TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Leader) - .build() + TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Leader, + ) + .build() }) .take(10) .collect(); @@ -765,7 +769,7 @@ mod tests { let req = PostTaskReq { peer_aggregator_endpoint: "http://aggregator.endpoint".try_into().unwrap(), - query_type: QueryType::TimeInterval, + query_type: QueryType::FixedSize { max_batch_size: 10 }, vdaf: VdafInstance::Prio3Count, role: Role::Collector, vdaf_verify_key: URL_SAFE_NO_PAD.encode(&vdaf_verify_key), @@ -809,7 +813,7 @@ mod tests { let req = PostTaskReq { peer_aggregator_endpoint: "http://aggregator.endpoint".try_into().unwrap(), - query_type: QueryType::TimeInterval, + query_type: QueryType::FixedSize { max_batch_size: 10 }, vdaf: VdafInstance::Prio3Count, role: Role::Helper, vdaf_verify_key: URL_SAFE_NO_PAD.encode(&vdaf_verify_key), @@ -851,7 +855,9 @@ mod tests { // Verify: posting a task creates a new task which matches the request. let req = PostTaskReq { peer_aggregator_endpoint: "http://aggregator.endpoint".try_into().unwrap(), - query_type: QueryType::TimeInterval, + query_type: QueryType::FixedSize { + max_batch_size: 224, + }, vdaf: VdafInstance::Prio3Count, role: Role::Helper, vdaf_verify_key: URL_SAFE_NO_PAD.encode(&vdaf_verify_key), @@ -931,7 +937,7 @@ mod tests { // Verify: posting a task with role = helper and an aggregator auth token fails let req = PostTaskReq { peer_aggregator_endpoint: "http://aggregator.endpoint".try_into().unwrap(), - query_type: QueryType::TimeInterval, + query_type: QueryType::FixedSize { max_batch_size: 10 }, vdaf: VdafInstance::Prio3Count, role: Role::Helper, vdaf_verify_key: URL_SAFE_NO_PAD.encode(&vdaf_verify_key), @@ -977,7 +983,9 @@ mod tests { // Verify: posting a task creates a new task which matches the request. let req = PostTaskReq { peer_aggregator_endpoint: "http://aggregator.endpoint".try_into().unwrap(), - query_type: QueryType::TimeInterval, + query_type: QueryType::FixedSize { + max_batch_size: 224, + }, vdaf: VdafInstance::Prio3Count, role: Role::Leader, vdaf_verify_key: URL_SAFE_NO_PAD.encode(&vdaf_verify_key), @@ -1067,7 +1075,7 @@ mod tests { // Verify: posting a task with role = Leader and no aggregator auth token fails let req = PostTaskReq { peer_aggregator_endpoint: "http://aggregator.endpoint".try_into().unwrap(), - query_type: QueryType::TimeInterval, + query_type: QueryType::FixedSize { max_batch_size: 10 }, vdaf: VdafInstance::Prio3Count, role: Role::Leader, vdaf_verify_key: URL_SAFE_NO_PAD.encode(&vdaf_verify_key), @@ -1106,10 +1114,14 @@ mod tests { // Setup: write a task to the datastore. let (handler, _ephemeral_datastore, ds) = setup_api_test().await; - let task = TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Leader) - .with_aggregator_auth_tokens(Vec::from([random()])) - .with_collector_auth_tokens(Vec::from([random()])) - .build(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Leader, + ) + .with_aggregator_auth_tokens(Vec::from([random()])) + .with_collector_auth_tokens(Vec::from([random()])) + .build(); ds.run_tx(|tx| { let task = task.clone(); @@ -1176,9 +1188,12 @@ mod tests { let task_id = ds .run_tx(|tx| { Box::pin(async move { - let task = - TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Leader) - .build(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Leader, + ) + .build(); tx.put_task(&task).await?; @@ -1260,9 +1275,12 @@ mod tests { let task_id = ds .run_tx(|tx| { Box::pin(async move { - let task = - TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Leader) - .build(); + let task = TaskBuilder::new( + QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Leader, + ) + .build(); let task_id = *task.id(); tx.put_task(&task).await?; @@ -1277,21 +1295,16 @@ mod tests { .await?; let aggregation_job_id = random(); - tx.put_aggregation_job( - &AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - task_id, - aggregation_job_id, - AggregationParam(0), - (), - Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(1), - ) + tx.put_aggregation_job(&AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( + task_id, + aggregation_job_id, + AggregationParam(0), + random(), + Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), - AggregationJobState::InProgress, - AggregationJobRound::from(0), - ), - ) + AggregationJobState::InProgress, + AggregationJobRound::from(0), + )) .await?; try_join_all( diff --git a/aggregator_core/src/datastore.rs b/aggregator_core/src/datastore.rs index ca69d233e..2fe7d20f4 100644 --- a/aggregator_core/src/datastore.rs +++ b/aggregator_core/src/datastore.rs @@ -21,10 +21,9 @@ use janus_core::{ time::{Clock, TimeExt}, }; use janus_messages::{ - query_type::{QueryType, TimeInterval}, - AggregationJobId, BatchId, CollectionJobId, Duration, Extension, HpkeCiphertext, HpkeConfig, - Interval, PrepareStep, ReportId, ReportIdChecksum, ReportMetadata, ReportShare, Role, TaskId, - Time, + query_type::QueryType, AggregationJobId, BatchId, CollectionJobId, Duration, Extension, + HpkeCiphertext, HpkeConfig, Interval, PrepareStep, ReportId, ReportIdChecksum, ReportMetadata, + ReportShare, Role, TaskId, Time, }; use opentelemetry::{ metrics::{Counter, Histogram, Meter}, @@ -2442,102 +2441,6 @@ impl Transaction<'_, C> { .transpose() } - /// Returns all collection jobs for the given task which include the given timestamp. Applies only - /// to time-interval tasks. - #[tracing::instrument(skip(self), err)] - pub async fn get_collection_jobs_including_time< - const SEED_SIZE: usize, - A: vdaf::Aggregator, - >( - &self, - vdaf: &A, - task_id: &TaskId, - timestamp: &Time, - ) -> Result>, Error> { - let stmt = self - .prepare_cached( - "SELECT - collection_jobs.collection_job_id, - collection_jobs.batch_identifier, - collection_jobs.aggregation_param, - collection_jobs.state, - collection_jobs.report_count, - collection_jobs.helper_aggregate_share, - collection_jobs.leader_aggregate_share - FROM collection_jobs JOIN tasks ON tasks.id = collection_jobs.task_id - WHERE tasks.task_id = $1 - AND collection_jobs.batch_interval @> $2::TIMESTAMP", - ) - .await?; - self.query( - &stmt, - &[ - /* task_id */ task_id.as_ref(), - /* timestamp */ ×tamp.as_naive_date_time()?, - ], - ) - .await? - .into_iter() - .map(|row| { - let batch_identifier = Interval::get_decoded(row.get("batch_identifier"))?; - let collection_job_id = - row.get_bytea_and_convert::("collection_job_id")?; - Self::collection_job_from_row(vdaf, *task_id, batch_identifier, collection_job_id, &row) - }) - .collect() - } - - /// Returns all collection jobs for the given task whose collect intervals intersect with the given - /// interval. Applies only to time-interval tasks. - #[tracing::instrument(skip(self), err)] - pub async fn get_collection_jobs_intersecting_interval< - const SEED_SIZE: usize, - A: vdaf::Aggregator, - >( - &self, - vdaf: &A, - task_id: &TaskId, - batch_interval: &Interval, - ) -> Result>, Error> { - let stmt = self - .prepare_cached( - "SELECT - collection_jobs.collection_job_id, - collection_jobs.batch_identifier, - collection_jobs.aggregation_param, - collection_jobs.state, - collection_jobs.report_count, - collection_jobs.helper_aggregate_share, - collection_jobs.leader_aggregate_share - FROM collection_jobs JOIN tasks ON tasks.id = collection_jobs.task_id - WHERE tasks.task_id = $1 - AND collection_jobs.batch_interval && $2", - ) - .await?; - self.query( - &stmt, - &[ - /* task_id */ task_id.as_ref(), - /* batch_interval */ &SqlInterval::from(batch_interval), - ], - ) - .await? - .into_iter() - .map(|row| { - let batch_identifier = Interval::get_decoded(row.get("batch_identifier"))?; - let collection_job_id = - row.get_bytea_and_convert::("collection_job_id")?; - Self::collection_job_from_row::( - vdaf, - *task_id, - batch_identifier, - collection_job_id, - &row, - ) - }) - .collect() - } - /// Retrieves all collection jobs for the given batch identifier. Multiple collection jobs may be /// returned with distinct aggregation parameters. #[tracing::instrument(skip(self), err)] @@ -3222,102 +3125,6 @@ impl Transaction<'_, C> { .transpose() } - /// Returns all aggregate share jobs for the given task which include the given timestamp. - /// Applies only to time-interval tasks. - #[tracing::instrument(skip(self), err)] - pub async fn get_aggregate_share_jobs_including_time< - const SEED_SIZE: usize, - A: vdaf::Aggregator, - >( - &self, - vdaf: &A, - task_id: &TaskId, - timestamp: &Time, - ) -> Result>, Error> { - let stmt = self - .prepare_cached( - "SELECT - aggregate_share_jobs.batch_identifier, - aggregate_share_jobs.aggregation_param, - aggregate_share_jobs.helper_aggregate_share, - aggregate_share_jobs.report_count, - aggregate_share_jobs.checksum - FROM aggregate_share_jobs JOIN tasks ON tasks.id = aggregate_share_jobs.task_id - WHERE tasks.task_id = $1 - AND aggregate_share_jobs.batch_interval @> $2::TIMESTAMP", - ) - .await?; - self.query( - &stmt, - &[ - /* task_id */ &task_id.as_ref(), - /* timestamp */ ×tamp.as_naive_date_time()?, - ], - ) - .await? - .into_iter() - .map(|row| { - let batch_identifier = Interval::get_decoded(row.get("batch_identifier"))?; - let aggregation_param = A::AggregationParam::get_decoded(row.get("aggregation_param"))?; - Self::aggregate_share_job_from_row( - vdaf, - task_id, - batch_identifier, - aggregation_param, - &row, - ) - }) - .collect() - } - - /// Returns all aggregate share jobs for the given task whose collect intervals intersect with - /// the given interval. Applies only to time-interval tasks. - #[tracing::instrument(skip(self), err)] - pub async fn get_aggregate_share_jobs_intersecting_interval< - const SEED_SIZE: usize, - A: vdaf::Aggregator, - >( - &self, - vdaf: &A, - task_id: &TaskId, - interval: &Interval, - ) -> Result>, Error> { - let stmt = self - .prepare_cached( - "SELECT - aggregate_share_jobs.batch_identifier, - aggregate_share_jobs.aggregation_param, - aggregate_share_jobs.helper_aggregate_share, - aggregate_share_jobs.report_count, - aggregate_share_jobs.checksum - FROM aggregate_share_jobs JOIN tasks ON tasks.id = aggregate_share_jobs.task_id - WHERE tasks.task_id = $1 - AND aggregate_share_jobs.batch_interval && $2", - ) - .await?; - self.query( - &stmt, - &[ - /* task_id */ &task_id.as_ref(), - /* interval */ &SqlInterval::from(interval), - ], - ) - .await? - .into_iter() - .map(|row| { - let batch_identifier = Interval::get_decoded(row.get("batch_identifier"))?; - let aggregation_param = A::AggregationParam::get_decoded(row.get("aggregation_param"))?; - Self::aggregate_share_job_from_row( - vdaf, - task_id, - batch_identifier, - aggregation_param, - &row, - ) - }) - .collect() - } - /// Returns all aggregate share jobs for the given task with the given batch identifier. /// Multiple aggregate share jobs may be returned with distinct aggregation parameters. #[tracing::instrument(skip(self), err)] @@ -4245,7 +4052,7 @@ pub mod models { time::{DurationExt, IntervalExt, TimeExt}, }; use janus_messages::{ - query_type::{FixedSize, QueryType, TimeInterval}, + query_type::{FixedSize, QueryType}, AggregationJobId, AggregationJobRound, BatchId, CollectionJobId, Duration, Extension, HpkeCiphertext, Interval, PrepareStep, ReportId, ReportIdChecksum, ReportMetadata, ReportShareError, Role, TaskId, Time, @@ -5114,8 +4921,7 @@ pub mod models { /// Gets the batch identifier included in this batch aggregation. /// /// This method would typically be used for code which is generic over the query type. - /// Query-type specific code will typically call one of [`Self::batch_interval`] or - /// [`Self::batch_id`]. + /// Query-type specific code will typically call one of [`Self::batch_id`]. pub fn batch_identifier(&self) -> &Q::BatchIdentifier { &self.batch_identifier } @@ -5187,15 +4993,6 @@ pub mod models { } } - impl> - BatchAggregation - { - /// Gets the batch interval associated with this batch aggregation. - pub fn batch_interval(&self) -> &Interval { - self.batch_identifier() - } - } - impl> BatchAggregation { @@ -5298,8 +5095,7 @@ pub mod models { /// Gets the batch identifier associated with this collection job. /// /// This method would typically be used for code which is generic over the query type. - /// Query-type specific code will typically call one of [`Self::batch_interval`] or - /// [`Self::batch_id`]. + /// Query-type specific code will typically call one of [`Self::batch_id`]. pub fn batch_identifier(&self) -> &Q::BatchIdentifier { &self.batch_identifier } @@ -5321,15 +5117,6 @@ pub mod models { } } - impl> - CollectionJob - { - /// Gets the batch interval associated with this collection job. - pub fn batch_interval(&self) -> &Interval { - self.batch_identifier() - } - } - impl> CollectionJob { @@ -5544,15 +5331,6 @@ pub mod models { } } - impl> - AggregateShareJob - { - /// Gets the batch interval associated with this aggregate share job. - pub fn batch_interval(&self) -> &Interval { - self.batch_identifier() - } - } - impl> AggregateShareJob { @@ -5946,7 +5724,7 @@ mod tests { time::{Clock, DurationExt, MockClock, TimeExt}, }; use janus_messages::{ - query_type::{FixedSize, QueryType, TimeInterval}, + query_type::{FixedSize, QueryType}, AggregateShareAad, AggregationJobId, AggregationJobRound, BatchId, BatchSelector, CollectionJobId, Duration, Extension, ExtensionType, HpkeCiphertext, HpkeConfigId, Interval, PrepareStep, PrepareStepResult, ReportId, ReportIdChecksum, ReportMetadata, @@ -6030,9 +5808,13 @@ mod tests { (VdafInstance::Poplar1 { bits: 8 }, Role::Helper), (VdafInstance::Poplar1 { bits: 64 }, Role::Helper), ] { - let task = TaskBuilder::new(task::QueryType::TimeInterval, vdaf, role) - .with_report_expiry_age(Some(Duration::from_seconds(3600))) - .build(); + let task = TaskBuilder::new( + task::QueryType::FixedSize { max_batch_size: 10 }, + vdaf, + role, + ) + .with_report_expiry_age(Some(Duration::from_seconds(3600))) + .build(); want_tasks.insert(*task.id(), task.clone()); let err = ds @@ -6131,14 +5913,14 @@ mod tests { .run_tx(|tx| { Box::pin(async move { let task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Leader, ) .with_report_expiry_age(Some(REPORT_EXPIRY_AGE)) .build(); let other_task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Leader, ) @@ -6168,11 +5950,11 @@ mod tests { .take(22) .collect(); - let aggregation_job = AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + let aggregation_job = AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( *task.id(), random(), AggregationParam(0), - (), + random(), Interval::new( OLDEST_ALLOWED_REPORT_TIMESTAMP .sub(&Duration::from_seconds(1)) @@ -6184,11 +5966,11 @@ mod tests { AggregationJobRound::from(0), ); let expired_aggregation_job = - AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( *task.id(), random(), AggregationParam(0), - (), + random(), Interval::new( OLDEST_ALLOWED_REPORT_TIMESTAMP .sub(&Duration::from_seconds(2)) @@ -6199,12 +5981,12 @@ mod tests { AggregationJobState::InProgress, AggregationJobRound::from(0), ); - let other_aggregation_job = - AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + let other_aggregation_job: AggregationJob<0, _, dummy_vdaf::Vdaf> = + AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( *other_task.id(), random(), AggregationParam(0), - (), + random(), Interval::new( OLDEST_ALLOWED_REPORT_TIMESTAMP .sub(&Duration::from_seconds(1)) @@ -6332,7 +6114,7 @@ mod tests { const TOTAL_TASK_ID_COUNT: usize = 20; let tasks: Vec<_> = iter::repeat_with(|| { TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Leader, ) @@ -6374,7 +6156,7 @@ mod tests { .unwrap(); let task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Leader, ) @@ -6522,14 +6304,14 @@ mod tests { ) .unwrap(); let task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) .with_report_expiry_age(Some(REPORT_EXPIRY_AGE)) .build(); let unrelated_task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -6696,20 +6478,20 @@ mod tests { let ds = ephemeral_datastore.datastore(clock.clone()).await; let task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Leader, ) .with_report_expiry_age(Some(REPORT_EXPIRY_AGE)) .build(); let unrelated_task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Leader, ) .build(); let no_reports_task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Leader, ) @@ -7004,7 +6786,7 @@ mod tests { let ds = ephemeral_datastore.datastore(MockClock::default()).await; let task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -7323,7 +7105,7 @@ mod tests { const AGGREGATION_JOB_COUNT: usize = 10; let task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -7346,13 +7128,13 @@ mod tests { async move { tx.put_aggregation_job(&AggregationJob::< PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, Prio3Count, >::new( task_id, aggregation_job_id, (), - (), + random(), Interval::new( OLDEST_ALLOWED_REPORT_TIMESTAMP .add(&Duration::from_seconds(LEASE_DURATION.as_secs())) @@ -7373,13 +7155,13 @@ mod tests { // Write an aggregation job that is finished. We don't want to retrieve this one. tx.put_aggregation_job(&AggregationJob::< PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, Prio3Count, >::new( *task.id(), random(), (), - (), + random(), Interval::new(OLDEST_ALLOWED_REPORT_TIMESTAMP, Duration::from_seconds(1)) .unwrap(), AggregationJobState::Finished, @@ -7390,13 +7172,13 @@ mod tests { // Write an expired aggregation job. We don't want to retrieve this one, either. tx.put_aggregation_job(&AggregationJob::< PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, Prio3Count, >::new( *task.id(), random(), (), - (), + random(), Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -7407,7 +7189,7 @@ mod tests { // Write an aggregation job for a task that we are taking on the helper role for. // We don't want to retrieve this one, either. let helper_task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Helper, ) @@ -7415,13 +7197,13 @@ mod tests { tx.put_task(&helper_task).await?; tx.put_aggregation_job(&AggregationJob::< PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, Prio3Count, >::new( *helper_task.id(), random(), (), - (), + random(), Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -7465,7 +7247,7 @@ mod tests { AcquiredAggregationJob::new( *task.id(), agg_job_id, - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, ), want_expiry_time, @@ -7587,7 +7369,7 @@ mod tests { AcquiredAggregationJob::new( *task.id(), job_id, - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, ), want_expiry_time, @@ -7665,7 +7447,7 @@ mod tests { let rslt = ds .run_tx(|tx| { Box::pin(async move { - tx.get_aggregation_job::( + tx.get_aggregation_job::( &random(), &random(), ) @@ -7679,12 +7461,12 @@ mod tests { let rslt = ds .run_tx(|tx| { Box::pin(async move { - tx.update_aggregation_job::( + tx.update_aggregation_job::( &AggregationJob::new( random(), random(), (), - (), + random(), Interval::new( Time::from_seconds_since_epoch(0), Duration::from_seconds(1), @@ -7833,7 +7615,7 @@ mod tests { let ds = ephemeral_datastore.datastore(clock.clone()).await; let task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -7849,13 +7631,13 @@ mod tests { tx.put_task(&task).await?; tx.put_aggregation_job(&AggregationJob::< PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, Prio3Count, >::new( *task.id(), aggregation_job_id, (), - (), + random(), Interval::new( OLDEST_ALLOWED_REPORT_TIMESTAMP, Duration::from_seconds(1), @@ -7995,7 +7777,7 @@ mod tests { let ds = ephemeral_datastore.datastore(clock.clone()).await; let task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Helper, ) @@ -8010,11 +7792,11 @@ mod tests { ds.run_tx(|tx| { let task_id = *task.id(); Box::pin(async move { - tx.put_aggregation_job(&AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + tx.put_aggregation_job(&AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( task_id, aggregation_job_id, dummy_vdaf::AggregationParam(0), - (), + random(), Interval::new(OLDEST_ALLOWED_REPORT_TIMESTAMP, Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -8201,7 +7983,7 @@ mod tests { let vdaf_transcript = run_vdaf(vdaf.as_ref(), &verify_key, &(), &report_id, &0); let task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -8220,13 +8002,13 @@ mod tests { tx.put_task(&task).await?; tx.put_aggregation_job(&AggregationJob::< PRIO3_VERIFY_KEY_LENGTH, - TimeInterval, + FixedSize, Prio3Count, >::new( *task.id(), aggregation_job_id, (), - (), + random(), Interval::new(OLDEST_ALLOWED_REPORT_TIMESTAMP, Duration::from_seconds(1)) .unwrap(), AggregationJobState::InProgress, @@ -8364,21 +8146,13 @@ mod tests { install_test_trace_subscriber(); let task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Leader, ) .build(); - let first_batch_interval = Interval::new( - Time::from_seconds_since_epoch(100), - Duration::from_seconds(100), - ) - .unwrap(); - let second_batch_interval = Interval::new( - Time::from_seconds_since_epoch(200), - Duration::from_seconds(200), - ) - .unwrap(); + let first_batch_id = random(); + let second_batch_id = random(); let aggregation_param = AggregationParam(13); let ds = ephemeral_datastore.datastore(MockClock::default()).await; @@ -8389,26 +8163,26 @@ mod tests { tx.put_task(&task).await.unwrap(); let vdaf = dummy_vdaf::Vdaf::new(); - let first_collection_job = CollectionJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + let first_collection_job = CollectionJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( *task.id(), random(), - first_batch_interval, + first_batch_id, aggregation_param, CollectionJobState::Start, ); tx.put_collection_job(&first_collection_job).await.unwrap(); - let second_collection_job = CollectionJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + let second_collection_job = CollectionJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( *task.id(), random(), - second_batch_interval, + second_batch_id, aggregation_param, CollectionJobState::Start, ); tx.put_collection_job(&second_collection_job).await.unwrap(); let first_collection_job_again = tx - .get_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>( &vdaf, first_collection_job.id(), ) @@ -8418,7 +8192,7 @@ mod tests { assert_eq!(first_collection_job, first_collection_job_again); let second_collection_job_again = tx - .get_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>( &vdaf, second_collection_job.id(), ) @@ -8437,7 +8211,7 @@ mod tests { &[0, 1, 2, 3, 4, 5], &AggregateShareAad::new( *task.id(), - BatchSelector::new_time_interval(first_batch_interval), + BatchSelector::new_fixed_size(first_batch_id), ) .get_encoded(), ) @@ -8450,14 +8224,12 @@ mod tests { leader_aggregate_share: AggregateShare(41), }); - tx.update_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( - &first_collection_job, - ) - .await - .unwrap(); + tx.update_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>(&first_collection_job) + .await + .unwrap(); let updated_first_collection_job = tx - .get_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>( &vdaf, first_collection_job.id(), ) @@ -8482,21 +8254,13 @@ mod tests { let ds = ephemeral_datastore.datastore(MockClock::default()).await; let task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Leader, ) .build(); - let abandoned_batch_interval = Interval::new( - Time::from_seconds_since_epoch(100), - Duration::from_seconds(100), - ) - .unwrap(); - let deleted_batch_interval = Interval::new( - Time::from_seconds_since_epoch(200), - Duration::from_seconds(100), - ) - .unwrap(); + let abandoned_batch_id = random(); + let deleted_batch_id = random(); ds.run_tx(|tx| { let task = task.clone(); @@ -8505,28 +8269,27 @@ mod tests { let vdaf = dummy_vdaf::Vdaf::new(); let aggregation_param = AggregationParam(10); - let abandoned_collection_job = - CollectionJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + let abandoned_collection_job: CollectionJob<0, _, dummy_vdaf::Vdaf> = + CollectionJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( *task.id(), random(), - abandoned_batch_interval, + abandoned_batch_id, aggregation_param, CollectionJobState::Start, ); tx.put_collection_job(&abandoned_collection_job).await?; - let deleted_collection_job = - CollectionJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - random(), - deleted_batch_interval, - aggregation_param, - CollectionJobState::Start, - ); + let deleted_collection_job = CollectionJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( + *task.id(), + random(), + deleted_batch_id, + aggregation_param, + CollectionJobState::Start, + ); tx.put_collection_job(&deleted_collection_job).await?; let abandoned_collection_job_again = tx - .get_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>( &vdaf, abandoned_collection_job.id(), ) @@ -8542,17 +8305,15 @@ mod tests { let deleted_collection_job = deleted_collection_job.with_state(CollectionJobState::Deleted); - tx.update_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + tx.update_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>( &abandoned_collection_job, ) .await?; - tx.update_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( - &deleted_collection_job, - ) - .await?; + tx.update_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>(&deleted_collection_job) + .await?; let abandoned_collection_job_again = tx - .get_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>( &vdaf, abandoned_collection_job.id(), ) @@ -8560,7 +8321,7 @@ mod tests { .unwrap(); let deleted_collection_job_again = tx - .get_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>( &vdaf, deleted_collection_job.id(), ) @@ -8576,7 +8337,7 @@ mod tests { abandoned_collection_job.with_state(CollectionJobState::Start); // Verify: Update should fail - tx.update_collection_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + tx.update_collection_job::<0, FixedSize, dummy_vdaf::Vdaf>( &abandoned_collection_job, ) .await @@ -8744,23 +8505,17 @@ mod tests { task_id, Time::from_seconds_since_epoch(0), )]); - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(100), - ) - .unwrap(); + let batch_id = random(); let aggregation_job_id = random(); - let aggregation_jobs = - Vec::from([AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - task_id, - aggregation_job_id, - AggregationParam(0), - (), - Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) - .unwrap(), - AggregationJobState::Finished, - AggregationJobRound::from(1), - )]); + let aggregation_jobs = Vec::from([AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( + task_id, + aggregation_job_id, + AggregationParam(0), + batch_id, + Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)).unwrap(), + AggregationJobState::Finished, + AggregationJobRound::from(1), + )]); let report_aggregations = Vec::from([ReportAggregation::<0, dummy_vdaf::Vdaf>::new( task_id, aggregation_job_id, @@ -8771,10 +8526,10 @@ mod tests { ReportAggregationState::Start, // Doesn't matter what state the report aggregation is in )]); - let collection_job_test_cases = Vec::from([CollectionJobTestCase:: { + let collection_job_test_cases = Vec::from([CollectionJobTestCase:: { should_be_acquired: true, task_id, - batch_identifier: batch_interval, + batch_identifier: batch_id, agg_param: AggregationParam(0), collection_job_id: None, state: CollectionJobTestCaseState::Collectable, @@ -8784,7 +8539,7 @@ mod tests { &ds, CollectionJobAcquireTestCase { task_ids: Vec::from([task_id]), - query_type: task::QueryType::TimeInterval, + query_type: task::QueryType::FixedSize { max_batch_size: 10 }, reports, aggregation_jobs, report_aggregations, @@ -8994,28 +8749,22 @@ mod tests { let task_id = random(); let other_task_id = random(); - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(100), - ) - .unwrap(); - let aggregation_jobs = - Vec::from([AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - // Aggregation job task ID does not match collection job task ID - other_task_id, - random(), - AggregationParam(0), - (), - Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) - .unwrap(), - AggregationJobState::Finished, - AggregationJobRound::from(1), - )]); + let batch_id = random(); + let aggregation_jobs = Vec::from([AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( + // Aggregation job task ID does not match collection job task ID + other_task_id, + random(), + AggregationParam(0), + batch_id, + Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)).unwrap(), + AggregationJobState::Finished, + AggregationJobRound::from(1), + )]); - let collection_job_test_cases = Vec::from([CollectionJobTestCase:: { + let collection_job_test_cases = Vec::from([CollectionJobTestCase:: { should_be_acquired: false, task_id, - batch_identifier: batch_interval, + batch_identifier: batch_id, agg_param: AggregationParam(0), collection_job_id: None, state: CollectionJobTestCaseState::Start, @@ -9025,7 +8774,7 @@ mod tests { &ds, CollectionJobAcquireTestCase { task_ids: Vec::from([task_id, other_task_id]), - query_type: task::QueryType::TimeInterval, + query_type: task::QueryType::FixedSize { max_batch_size: 10 }, reports: Vec::new(), aggregation_jobs, report_aggregations: Vec::new(), @@ -9050,28 +8799,22 @@ mod tests { Time::from_seconds_since_epoch(0), )]); - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(100), - ) - .unwrap(); - let aggregation_jobs = - Vec::from([AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - task_id, - random(), - // Aggregation job agg param does not match collection job agg param - AggregationParam(1), - (), - Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) - .unwrap(), - AggregationJobState::Finished, - AggregationJobRound::from(1), - )]); + let batch_id = random(); + let aggregation_jobs = Vec::from([AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( + task_id, + random(), + // Aggregation job agg param does not match collection job agg param + AggregationParam(1), + batch_id, + Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)).unwrap(), + AggregationJobState::Finished, + AggregationJobRound::from(1), + )]); - let collection_job_test_cases = Vec::from([CollectionJobTestCase:: { + let collection_job_test_cases = Vec::from([CollectionJobTestCase:: { should_be_acquired: false, task_id, - batch_identifier: batch_interval, + batch_identifier: batch_id, agg_param: AggregationParam(0), collection_job_id: None, state: CollectionJobTestCaseState::Start, @@ -9081,7 +8824,7 @@ mod tests { &ds, CollectionJobAcquireTestCase { task_ids: Vec::from([task_id]), - query_type: task::QueryType::TimeInterval, + query_type: task::QueryType::FixedSize { max_batch_size: 10 }, reports, aggregation_jobs, report_aggregations: Vec::new(), @@ -9101,6 +8844,7 @@ mod tests { let ds = ephemeral_datastore.datastore(clock.clone()).await; let task_id = random(); + let batch_id = random(); let reports = Vec::from([LeaderStoredReport::new_dummy( task_id, // Report associated with the aggregation job is outside the collection job's batch @@ -9108,20 +8852,19 @@ mod tests { Time::from_seconds_since_epoch(200), )]); let aggregation_job_id = random(); - let aggregation_jobs = - Vec::from([AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - task_id, - aggregation_job_id, - AggregationParam(0), - (), - Interval::new( - Time::from_seconds_since_epoch(200), - Duration::from_seconds(1), - ) - .unwrap(), - AggregationJobState::Finished, - AggregationJobRound::from(1), - )]); + let aggregation_jobs = Vec::from([AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( + task_id, + aggregation_job_id, + AggregationParam(0), + batch_id, + Interval::new( + Time::from_seconds_since_epoch(200), + Duration::from_seconds(1), + ) + .unwrap(), + AggregationJobState::Finished, + AggregationJobRound::from(1), + )]); let report_aggregations = Vec::from([ReportAggregation::<0, dummy_vdaf::Vdaf>::new( task_id, aggregation_job_id, @@ -9134,20 +8877,16 @@ mod tests { run_collection_job_acquire_test_case( &ds, - CollectionJobAcquireTestCase:: { + CollectionJobAcquireTestCase:: { task_ids: Vec::from([task_id]), - query_type: task::QueryType::TimeInterval, + query_type: task::QueryType::FixedSize { max_batch_size: 10 }, reports, aggregation_jobs, report_aggregations, - collection_job_test_cases: Vec::from([CollectionJobTestCase:: { + collection_job_test_cases: Vec::from([CollectionJobTestCase:: { should_be_acquired: false, task_id, - batch_identifier: Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(100), - ) - .unwrap(), + batch_identifier: batch_id, agg_param: AggregationParam(0), collection_job_id: None, state: CollectionJobTestCaseState::Start, @@ -9170,22 +8909,16 @@ mod tests { Time::from_seconds_since_epoch(0), )]); let aggregation_job_id = random(); - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(100), - ) - .unwrap(); - let aggregation_jobs = - Vec::from([AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - task_id, - aggregation_job_id, - AggregationParam(0), - (), - Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) - .unwrap(), - AggregationJobState::Finished, - AggregationJobRound::from(1), - )]); + let batch_id = random(); + let aggregation_jobs = Vec::from([AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( + task_id, + aggregation_job_id, + AggregationParam(0), + batch_id, + Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)).unwrap(), + AggregationJobState::Finished, + AggregationJobRound::from(1), + )]); let report_aggregations = Vec::from([ReportAggregation::<0, dummy_vdaf::Vdaf>::new( task_id, @@ -9197,10 +8930,10 @@ mod tests { ReportAggregationState::Start, )]); - let collection_job_test_cases = Vec::from([CollectionJobTestCase:: { + let collection_job_test_cases = Vec::from([CollectionJobTestCase:: { should_be_acquired: false, task_id, - batch_identifier: batch_interval, + batch_identifier: batch_id, agg_param: AggregationParam(0), collection_job_id: None, // collection job has already run to completion @@ -9211,7 +8944,7 @@ mod tests { &ds, CollectionJobAcquireTestCase { task_ids: Vec::from([task_id]), - query_type: task::QueryType::TimeInterval, + query_type: task::QueryType::FixedSize { max_batch_size: 10 }, reports, aggregation_jobs, report_aggregations, @@ -9237,27 +8970,23 @@ mod tests { ]); let aggregation_job_ids: [_; 2] = random(); - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(100), - ) - .unwrap(); + let batch_id = random(); let aggregation_jobs = Vec::from([ - AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( task_id, aggregation_job_ids[0], AggregationParam(0), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::Finished, AggregationJobRound::from(1), ), - AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( task_id, aggregation_job_ids[1], AggregationParam(0), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), // Aggregation job included in collect request is in progress @@ -9287,10 +9016,10 @@ mod tests { ), ]); - let collection_job_test_cases = Vec::from([CollectionJobTestCase:: { + let collection_job_test_cases = Vec::from([CollectionJobTestCase:: { should_be_acquired: false, task_id, - batch_identifier: batch_interval, + batch_identifier: batch_id, agg_param: AggregationParam(0), collection_job_id: None, state: CollectionJobTestCaseState::Start, @@ -9300,7 +9029,7 @@ mod tests { &ds, CollectionJobAcquireTestCase { task_ids: Vec::from([task_id]), - query_type: task::QueryType::TimeInterval, + query_type: task::QueryType::FixedSize { max_batch_size: 10 }, reports, aggregation_jobs, report_aggregations, @@ -9323,27 +9052,23 @@ mod tests { Time::from_seconds_since_epoch(0), )]); let aggregation_job_ids: [_; 2] = random(); - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(100), - ) - .unwrap(); + let batch_id = random(); let aggregation_jobs = Vec::from([ - AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( task_id, aggregation_job_ids[0], AggregationParam(0), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::Finished, AggregationJobRound::from(1), ), - AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( task_id, aggregation_job_ids[1], AggregationParam(1), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::Finished, @@ -9372,22 +9097,18 @@ mod tests { ]); let collection_job_test_cases = Vec::from([ - CollectionJobTestCase:: { + CollectionJobTestCase:: { should_be_acquired: true, task_id, - batch_identifier: batch_interval, + batch_identifier: batch_id, agg_param: AggregationParam(0), collection_job_id: None, state: CollectionJobTestCaseState::Collectable, }, - CollectionJobTestCase:: { + CollectionJobTestCase:: { should_be_acquired: true, task_id, - batch_identifier: Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(100), - ) - .unwrap(), + batch_identifier: batch_id, agg_param: AggregationParam(1), collection_job_id: None, state: CollectionJobTestCaseState::Collectable, @@ -9396,9 +9117,9 @@ mod tests { let test_case = setup_collection_job_acquire_test_case( &ds, - CollectionJobAcquireTestCase:: { + CollectionJobAcquireTestCase:: { task_ids: Vec::from([task_id]), - query_type: task::QueryType::TimeInterval, + query_type: task::QueryType::FixedSize { max_batch_size: 10 }, reports, aggregation_jobs, report_aggregations, @@ -9440,7 +9161,7 @@ mod tests { AcquiredCollectionJob::new( c.task_id, c.collection_job_id.unwrap(), - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, ), clock.now().as_naive_date_time().unwrap() @@ -9472,37 +9193,33 @@ mod tests { Time::from_seconds_since_epoch(0), )]); let aggregation_job_ids: [_; 3] = random(); - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(0), - Duration::from_seconds(100), - ) - .unwrap(); + let batch_id = random(); let aggregation_jobs = Vec::from([ - AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( task_id, aggregation_job_ids[0], AggregationParam(0), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::Finished, AggregationJobRound::from(1), ), - AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( task_id, aggregation_job_ids[1], AggregationParam(1), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::Finished, AggregationJobRound::from(1), ), - AggregationJob::<0, TimeInterval, dummy_vdaf::Vdaf>::new( + AggregationJob::<0, FixedSize, dummy_vdaf::Vdaf>::new( task_id, aggregation_job_ids[2], AggregationParam(2), - (), + batch_id, Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)) .unwrap(), AggregationJobState::Finished, @@ -9540,26 +9257,26 @@ mod tests { ]); let collection_job_test_cases = Vec::from([ - CollectionJobTestCase:: { + CollectionJobTestCase:: { should_be_acquired: true, task_id, - batch_identifier: batch_interval, + batch_identifier: batch_id, agg_param: AggregationParam(0), collection_job_id: None, state: CollectionJobTestCaseState::Finished, }, - CollectionJobTestCase:: { + CollectionJobTestCase:: { should_be_acquired: true, task_id, - batch_identifier: batch_interval, + batch_identifier: batch_id, agg_param: AggregationParam(1), collection_job_id: None, state: CollectionJobTestCaseState::Abandoned, }, - CollectionJobTestCase:: { + CollectionJobTestCase:: { should_be_acquired: true, task_id, - batch_identifier: batch_interval, + batch_identifier: batch_id, agg_param: AggregationParam(2), collection_job_id: None, state: CollectionJobTestCaseState::Deleted, @@ -9570,7 +9287,7 @@ mod tests { &ds, CollectionJobAcquireTestCase { task_ids: Vec::from([task_id]), - query_type: task::QueryType::TimeInterval, + query_type: task::QueryType::FixedSize { max_batch_size: 10 }, reports, aggregation_jobs, report_aggregations, @@ -9596,318 +9313,101 @@ mod tests { #[rstest_reuse::apply(schema_versions_template)] #[tokio::test] - async fn roundtrip_batch_aggregation_time_interval(ephemeral_datastore: EphemeralDatastore) { + async fn roundtrip_batch_aggregation_fixed_size(ephemeral_datastore: EphemeralDatastore) { install_test_trace_subscriber(); let ds = ephemeral_datastore.datastore(MockClock::default()).await; ds.run_tx(|tx| { Box::pin(async move { - let time_precision = Duration::from_seconds(100); let task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Leader, ) - .with_time_precision(time_precision) .build(); let other_task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Leader, ) .build(); let vdaf = dummy_vdaf::Vdaf::new(); + let batch_id = random(); let aggregate_share = AggregateShare(23); let aggregation_param = AggregationParam(12); tx.put_task(&task).await?; tx.put_task(&other_task).await?; - let first_batch_aggregation = - BatchAggregation::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - Interval::new(Time::from_seconds_since_epoch(100), time_precision).unwrap(), - aggregation_param, - 0, - BatchAggregationState::Aggregating, - Some(aggregate_share), - 0, - ReportIdChecksum::default(), - ); - - let second_batch_aggregation = - BatchAggregation::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - Interval::new(Time::from_seconds_since_epoch(200), time_precision).unwrap(), - aggregation_param, - 1, - BatchAggregationState::Collected, - None, - 0, - ReportIdChecksum::default(), - ); - - let third_batch_aggregation = - BatchAggregation::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - Interval::new(Time::from_seconds_since_epoch(300), time_precision).unwrap(), - aggregation_param, - 2, - BatchAggregationState::Aggregating, - Some(aggregate_share), - 0, - ReportIdChecksum::default(), - ); - - // Start of this aggregation's interval is before the interval queried below. - tx.put_batch_aggregation( - &BatchAggregation::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - Interval::new(Time::from_seconds_since_epoch(0), time_precision).unwrap(), - aggregation_param, - 3, - BatchAggregationState::Collected, - None, - 0, - ReportIdChecksum::default(), - ), - ) - .await?; + let batch_aggregation = BatchAggregation::<0, FixedSize, dummy_vdaf::Vdaf>::new( + *task.id(), + batch_id, + aggregation_param, + 0, + BatchAggregationState::Aggregating, + Some(aggregate_share), + 0, + ReportIdChecksum::default(), + ); - // Following three batches are within the interval queried below. - tx.put_batch_aggregation(&first_batch_aggregation).await?; - tx.put_batch_aggregation(&second_batch_aggregation).await?; - tx.put_batch_aggregation(&third_batch_aggregation).await?; + // Following batch aggregations have the batch ID queried below. + tx.put_batch_aggregation(&batch_aggregation).await?; assert_matches!( - tx.put_batch_aggregation(&first_batch_aggregation).await, + tx.put_batch_aggregation(&batch_aggregation).await, Err(Error::MutationTargetAlreadyExists) ); - // Aggregation parameter differs from the one queried below. - tx.put_batch_aggregation( - &BatchAggregation::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - Interval::new(Time::from_seconds_since_epoch(100), time_precision).unwrap(), - AggregationParam(13), - 4, - BatchAggregationState::Aggregating, - Some(aggregate_share), - 0, - ReportIdChecksum::default(), - ), - ) + // Wrong batch ID. + tx.put_batch_aggregation(&BatchAggregation::<0, FixedSize, dummy_vdaf::Vdaf>::new( + *task.id(), + random(), + AggregationParam(13), + 1, + BatchAggregationState::Collected, + None, + 0, + ReportIdChecksum::default(), + )) .await?; - // Start of this aggregation's interval is after the interval queried below. - tx.put_batch_aggregation( - &BatchAggregation::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *task.id(), - Interval::new(Time::from_seconds_since_epoch(400), time_precision).unwrap(), - aggregation_param, - 5, - BatchAggregationState::Collected, - None, - 0, - ReportIdChecksum::default(), - ), - ) + // Task ID differs from that queried below. + tx.put_batch_aggregation(&BatchAggregation::<0, FixedSize, dummy_vdaf::Vdaf>::new( + *other_task.id(), + batch_id, + aggregation_param, + 2, + BatchAggregationState::Aggregating, + Some(aggregate_share), + 0, + ReportIdChecksum::default(), + )) .await?; - // Task ID differs from that queried below. - tx.put_batch_aggregation( - &BatchAggregation::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *other_task.id(), - Interval::new(Time::from_seconds_since_epoch(200), time_precision).unwrap(), - aggregation_param, - 6, - BatchAggregationState::Aggregating, - Some(aggregate_share), - 0, - ReportIdChecksum::default(), - ), - ) + // Index differs from that queried below. + tx.put_batch_aggregation(&BatchAggregation::<0, FixedSize, dummy_vdaf::Vdaf>::new( + *task.id(), + batch_id, + aggregation_param, + 3, + BatchAggregationState::Collected, + None, + 0, + ReportIdChecksum::default(), + )) .await?; - let batch_aggregations = - TimeInterval::get_batch_aggregations_for_collection_identifier::< - 0, - dummy_vdaf::Vdaf, - _, - >( - tx, - &task, + let got_batch_aggregation = tx + .get_batch_aggregation::<0, FixedSize, dummy_vdaf::Vdaf>( &vdaf, - &Interval::new( - Time::from_seconds_since_epoch(100), - Duration::from_seconds(3 * time_precision.as_seconds()), - ) - .unwrap(), + task.id(), + &batch_id, &aggregation_param, + 0, ) .await?; - - assert_eq!(batch_aggregations.len(), 3, "{batch_aggregations:#?}"); - for batch_aggregation in [ - &first_batch_aggregation, - &second_batch_aggregation, - &third_batch_aggregation, - ] { - assert!( - batch_aggregations.contains(batch_aggregation), - "{batch_aggregations:#?}" - ); - } - - let first_batch_aggregation = - BatchAggregation::<0, TimeInterval, dummy_vdaf::Vdaf>::new( - *first_batch_aggregation.task_id(), - *first_batch_aggregation.batch_interval(), - *first_batch_aggregation.aggregation_parameter(), - first_batch_aggregation.ord(), - *first_batch_aggregation.state(), - Some(AggregateShare(92)), - 1, - ReportIdChecksum::get_decoded(&[1; 32]).unwrap(), - ); - tx.update_batch_aggregation(&first_batch_aggregation) - .await?; - - let batch_aggregations = - TimeInterval::get_batch_aggregations_for_collection_identifier::< - 0, - dummy_vdaf::Vdaf, - _, - >( - tx, - &task, - &vdaf, - &Interval::new( - Time::from_seconds_since_epoch(100), - Duration::from_seconds(3 * time_precision.as_seconds()), - ) - .unwrap(), - &aggregation_param, - ) - .await?; - - assert_eq!(batch_aggregations.len(), 3, "{batch_aggregations:#?}"); - for batch_aggregation in [ - &first_batch_aggregation, - &second_batch_aggregation, - &third_batch_aggregation, - ] { - assert!( - batch_aggregations.contains(batch_aggregation), - "{batch_aggregations:#?}" - ); - } - - Ok(()) - }) - }) - .await - .unwrap(); - } - - #[rstest_reuse::apply(schema_versions_template)] - #[tokio::test] - async fn roundtrip_batch_aggregation_fixed_size(ephemeral_datastore: EphemeralDatastore) { - install_test_trace_subscriber(); - - let ds = ephemeral_datastore.datastore(MockClock::default()).await; - - ds.run_tx(|tx| { - Box::pin(async move { - let task = TaskBuilder::new( - task::QueryType::FixedSize { max_batch_size: 10 }, - VdafInstance::Fake, - Role::Leader, - ) - .build(); - let other_task = TaskBuilder::new( - task::QueryType::FixedSize { max_batch_size: 10 }, - VdafInstance::Fake, - Role::Leader, - ) - .build(); - let vdaf = dummy_vdaf::Vdaf::new(); - let batch_id = random(); - let aggregate_share = AggregateShare(23); - let aggregation_param = AggregationParam(12); - - tx.put_task(&task).await?; - tx.put_task(&other_task).await?; - - let batch_aggregation = BatchAggregation::<0, FixedSize, dummy_vdaf::Vdaf>::new( - *task.id(), - batch_id, - aggregation_param, - 0, - BatchAggregationState::Aggregating, - Some(aggregate_share), - 0, - ReportIdChecksum::default(), - ); - - // Following batch aggregations have the batch ID queried below. - tx.put_batch_aggregation(&batch_aggregation).await?; - - assert_matches!( - tx.put_batch_aggregation(&batch_aggregation).await, - Err(Error::MutationTargetAlreadyExists) - ); - - // Wrong batch ID. - tx.put_batch_aggregation(&BatchAggregation::<0, FixedSize, dummy_vdaf::Vdaf>::new( - *task.id(), - random(), - AggregationParam(13), - 1, - BatchAggregationState::Collected, - None, - 0, - ReportIdChecksum::default(), - )) - .await?; - - // Task ID differs from that queried below. - tx.put_batch_aggregation(&BatchAggregation::<0, FixedSize, dummy_vdaf::Vdaf>::new( - *other_task.id(), - batch_id, - aggregation_param, - 2, - BatchAggregationState::Aggregating, - Some(aggregate_share), - 0, - ReportIdChecksum::default(), - )) - .await?; - - // Index differs from that queried below. - tx.put_batch_aggregation(&BatchAggregation::<0, FixedSize, dummy_vdaf::Vdaf>::new( - *task.id(), - batch_id, - aggregation_param, - 3, - BatchAggregationState::Collected, - None, - 0, - ReportIdChecksum::default(), - )) - .await?; - - let got_batch_aggregation = tx - .get_batch_aggregation::<0, FixedSize, dummy_vdaf::Vdaf>( - &vdaf, - task.id(), - &batch_id, - &aggregation_param, - 0, - ) - .await?; - assert_eq!(got_batch_aggregation.as_ref(), Some(&batch_aggregation)); + assert_eq!(got_batch_aggregation.as_ref(), Some(&batch_aggregation)); let batch_aggregation = BatchAggregation::<0, FixedSize, dummy_vdaf::Vdaf>::new( *batch_aggregation.task_id(), @@ -9947,61 +9447,52 @@ mod tests { ds.run_tx(|tx| { Box::pin(async move { - let task = - TaskBuilder::new(task::QueryType::TimeInterval, VdafInstance::Fake, Role::Helper) - .build(); + let task = TaskBuilder::new( + task::QueryType::FixedSize { max_batch_size: 10 }, + VdafInstance::Fake, + Role::Helper, + ) + .build(); tx.put_task(&task).await?; let vdaf = dummy_vdaf::Vdaf::new(); let aggregate_share = AggregateShare(42); - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(100), - Duration::from_seconds(100), - ) - .unwrap(); - let other_batch_interval = Interval::new( - Time::from_seconds_since_epoch(101), - Duration::from_seconds(101), - ) - .unwrap(); + let batch_id = random(); + let other_batch_id = random(); let report_count = 10; let checksum = ReportIdChecksum::get_decoded(&[1; 32]).unwrap(); let aggregation_param = AggregationParam(11); - let aggregate_share_job = - AggregateShareJob::new( - *task.id(), - batch_interval, - aggregation_param, - aggregate_share, - report_count, - checksum, - ); + let aggregate_share_job = AggregateShareJob::new( + *task.id(), + batch_id, + aggregation_param, + aggregate_share, + report_count, + checksum, + ); - tx.put_aggregate_share_job::<0, TimeInterval, dummy_vdaf::Vdaf>( - &aggregate_share_job, - ) - .await - .unwrap(); + tx.put_aggregate_share_job::<0, FixedSize, dummy_vdaf::Vdaf>(&aggregate_share_job) + .await + .unwrap(); let aggregate_share_job_again = tx - .get_aggregate_share_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_aggregate_share_job::<0, FixedSize, dummy_vdaf::Vdaf>( &vdaf, task.id(), - &batch_interval, + &batch_id, &aggregation_param, ) .await .unwrap() .unwrap(); - assert_eq!(aggregate_share_job, aggregate_share_job_again); assert!(tx - .get_aggregate_share_job::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_aggregate_share_job::<0, FixedSize, dummy_vdaf::Vdaf>( &vdaf, task.id(), - &other_batch_interval, + &other_batch_id, &aggregation_param, ) .await @@ -10009,37 +9500,15 @@ mod tests { .is_none()); let want_aggregate_share_jobs = Vec::from([aggregate_share_job]); - - let got_aggregate_share_jobs = tx - .get_aggregate_share_jobs_including_time::<0, dummy_vdaf::Vdaf>( - &vdaf, - task.id(), - &Time::from_seconds_since_epoch(150), - ) - .await?; - assert_eq!(got_aggregate_share_jobs, want_aggregate_share_jobs); - let got_aggregate_share_jobs = tx - .get_aggregate_share_jobs_intersecting_interval::<0, dummy_vdaf::Vdaf>( + .get_aggregate_share_jobs_by_batch_identifier::<0, FixedSize, dummy_vdaf::Vdaf>( &vdaf, task.id(), - &Interval::new( - Time::from_seconds_since_epoch(145), - Duration::from_seconds(10), - ) - .unwrap(), + &batch_id, ) .await?; assert_eq!(got_aggregate_share_jobs, want_aggregate_share_jobs); - let got_aggregate_share_jobs = tx - .get_aggregate_share_jobs_by_batch_identifier::<0, TimeInterval, dummy_vdaf::Vdaf>( - &vdaf, - task.id(), - &batch_interval - ).await?; - assert_eq!(got_aggregate_share_jobs, want_aggregate_share_jobs); - Ok(()) }) }) @@ -10378,45 +9847,6 @@ mod tests { ) -> Option<(TaskId, BatchId)>; } - #[async_trait] - impl ExpirationQueryTypeExt for TimeInterval { - fn batch_identifier_for_client_timestamps( - client_timestamps: &[Time], - ) -> Self::BatchIdentifier { - let min_client_timestamp = *client_timestamps.iter().min().unwrap(); - let max_client_timestamp = *client_timestamps.iter().max().unwrap(); - Interval::new( - min_client_timestamp, - Duration::from_seconds( - max_client_timestamp - .difference(&min_client_timestamp) - .unwrap() - .as_seconds() - + 1, - ), - ) - .unwrap() - } - - fn shortened_batch_identifier( - batch_identifier: &Self::BatchIdentifier, - ) -> Self::BatchIdentifier { - Interval::new( - *batch_identifier.start(), - Duration::from_seconds(batch_identifier.duration().as_seconds() / 2), - ) - .unwrap() - } - - async fn write_outstanding_batch( - _: &Transaction, - _: &TaskId, - _: &Self::BatchIdentifier, - ) -> Option<(TaskId, BatchId)> { - None - } - } - #[async_trait] impl ExpirationQueryTypeExt for FixedSize { fn batch_identifier_for_client_timestamps(_: &[Time]) -> Self::BatchIdentifier { @@ -10460,14 +9890,14 @@ mod tests { .run_tx(|tx| { Box::pin(async move { let task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Leader, ) .with_report_expiry_age(Some(report_expiry_age)) .build(); let other_task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Leader, ) @@ -10618,8 +10048,6 @@ mod tests { } let ( - leader_time_interval_task_id, - helper_time_interval_task_id, leader_fixed_size_task_id, helper_fixed_size_task_id, want_aggregation_job_ids, @@ -10627,20 +10055,6 @@ mod tests { ) = ds .run_tx(|tx| { Box::pin(async move { - let leader_time_interval_task = TaskBuilder::new( - task::QueryType::TimeInterval, - VdafInstance::Fake, - Role::Leader, - ) - .with_report_expiry_age(Some(REPORT_EXPIRY_AGE)) - .build(); - let helper_time_interval_task = TaskBuilder::new( - task::QueryType::TimeInterval, - VdafInstance::Fake, - Role::Helper, - ) - .with_report_expiry_age(Some(REPORT_EXPIRY_AGE)) - .build(); let leader_fixed_size_task = TaskBuilder::new( task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, @@ -10655,116 +10069,12 @@ mod tests { ) .with_report_expiry_age(Some(REPORT_EXPIRY_AGE)) .build(); - tx.put_task(&leader_time_interval_task).await?; - tx.put_task(&helper_time_interval_task).await?; tx.put_task(&leader_fixed_size_task).await?; tx.put_task(&helper_fixed_size_task).await?; let mut aggregation_job_ids = HashSet::new(); let mut all_report_ids = HashSet::new(); - // Leader, time-interval aggregation job with old reports [GC'ed]. - write_aggregation_artifacts::( - tx, - leader_time_interval_task.id(), - &[ - OLDEST_ALLOWED_REPORT_TIMESTAMP - .sub(&Duration::from_seconds(20)) - .unwrap(), - OLDEST_ALLOWED_REPORT_TIMESTAMP - .sub(&Duration::from_seconds(19)) - .unwrap(), - ], - ) - .await; - - // Leader, time-interval aggregation job with old & new reports [not GC'ed]. - let (_, aggregation_job_id, report_ids) = - write_aggregation_artifacts::( - tx, - leader_time_interval_task.id(), - &[ - OLDEST_ALLOWED_REPORT_TIMESTAMP - .sub(&Duration::from_seconds(5)) - .unwrap(), - OLDEST_ALLOWED_REPORT_TIMESTAMP - .add(&Duration::from_seconds(8)) - .unwrap(), - ], - ) - .await; - aggregation_job_ids.insert(aggregation_job_id); - all_report_ids.extend(report_ids); - - // Leader, time-interval aggregation job with new reports [not GC'ed]. - let (_, aggregation_job_id, report_ids) = - write_aggregation_artifacts::( - tx, - leader_time_interval_task.id(), - &[ - OLDEST_ALLOWED_REPORT_TIMESTAMP - .add(&Duration::from_seconds(19)) - .unwrap(), - OLDEST_ALLOWED_REPORT_TIMESTAMP - .add(&Duration::from_seconds(20)) - .unwrap(), - ], - ) - .await; - aggregation_job_ids.insert(aggregation_job_id); - all_report_ids.extend(report_ids); - - // Helper, time-interval aggregation job with old reports [GC'ed]. - write_aggregation_artifacts::( - tx, - helper_time_interval_task.id(), - &[ - OLDEST_ALLOWED_REPORT_TIMESTAMP - .sub(&Duration::from_seconds(20)) - .unwrap(), - OLDEST_ALLOWED_REPORT_TIMESTAMP - .sub(&Duration::from_seconds(19)) - .unwrap(), - ], - ) - .await; - - // Helper, time-interval task with old & new reports [not GC'ed]. - let (_, aggregation_job_id, report_ids) = - write_aggregation_artifacts::( - tx, - helper_time_interval_task.id(), - &[ - OLDEST_ALLOWED_REPORT_TIMESTAMP - .sub(&Duration::from_seconds(5)) - .unwrap(), - OLDEST_ALLOWED_REPORT_TIMESTAMP - .add(&Duration::from_seconds(8)) - .unwrap(), - ], - ) - .await; - aggregation_job_ids.insert(aggregation_job_id); - all_report_ids.extend(report_ids); - - // Helper, time-interval task with new reports [not GC'ed]. - let (_, aggregation_job_id, report_ids) = - write_aggregation_artifacts::( - tx, - helper_time_interval_task.id(), - &[ - OLDEST_ALLOWED_REPORT_TIMESTAMP - .add(&Duration::from_seconds(19)) - .unwrap(), - OLDEST_ALLOWED_REPORT_TIMESTAMP - .add(&Duration::from_seconds(20)) - .unwrap(), - ], - ) - .await; - aggregation_job_ids.insert(aggregation_job_id); - all_report_ids.extend(report_ids); - // Leader, fixed-size aggregation job with old reports [GC'ed]. write_aggregation_artifacts::( tx, @@ -10868,8 +10178,6 @@ mod tests { all_report_ids.extend(report_ids); Ok(( - *leader_time_interval_task.id(), - *helper_time_interval_task.id(), *leader_fixed_size_task.id(), *helper_fixed_size_task.id(), aggregation_job_ids, @@ -10886,10 +10194,6 @@ mod tests { // Run. ds.run_tx(|tx| { Box::pin(async move { - tx.delete_expired_aggregation_artifacts(&leader_time_interval_task_id) - .await?; - tx.delete_expired_aggregation_artifacts(&helper_time_interval_task_id) - .await?; tx.delete_expired_aggregation_artifacts(&leader_fixed_size_task_id) .await?; tx.delete_expired_aggregation_artifacts(&helper_fixed_size_task_id) @@ -10905,22 +10209,6 @@ mod tests { .run_tx(|tx| { let vdaf = vdaf.clone(); Box::pin(async move { - let leader_time_interval_aggregation_job_ids = tx - .get_aggregation_jobs_for_task::<0, TimeInterval, dummy_vdaf::Vdaf>( - &leader_time_interval_task_id, - ) - .await - .unwrap() - .into_iter() - .map(|job| *job.id()); - let helper_time_interval_aggregation_job_ids = tx - .get_aggregation_jobs_for_task::<0, TimeInterval, dummy_vdaf::Vdaf>( - &helper_time_interval_task_id, - ) - .await - .unwrap() - .into_iter() - .map(|job| *job.id()); let leader_fixed_size_aggregation_job_ids = tx .get_aggregation_jobs_for_task::<0, FixedSize, dummy_vdaf::Vdaf>( &leader_fixed_size_task_id, @@ -10937,28 +10225,10 @@ mod tests { .unwrap() .into_iter() .map(|job| *job.id()); - let got_aggregation_job_ids = leader_time_interval_aggregation_job_ids - .chain(helper_time_interval_aggregation_job_ids) - .chain(leader_fixed_size_aggregation_job_ids) + let got_aggregation_job_ids = leader_fixed_size_aggregation_job_ids .chain(helper_fixed_size_aggregation_job_ids) .collect(); - let leader_time_interval_report_aggregations = tx - .get_report_aggregations_for_task::<0, dummy_vdaf::Vdaf>( - &vdaf, - &Role::Leader, - &leader_time_interval_task_id, - ) - .await - .unwrap(); - let helper_time_interval_report_aggregations = tx - .get_report_aggregations_for_task::<0, dummy_vdaf::Vdaf>( - &vdaf, - &Role::Helper, - &helper_time_interval_task_id, - ) - .await - .unwrap(); let leader_fixed_size_report_aggregations = tx .get_report_aggregations_for_task::<0, dummy_vdaf::Vdaf>( &vdaf, @@ -10975,10 +10245,8 @@ mod tests { ) .await .unwrap(); - let got_report_ids = leader_time_interval_report_aggregations + let got_report_ids = leader_fixed_size_report_aggregations .into_iter() - .chain(helper_time_interval_report_aggregations) - .chain(leader_fixed_size_report_aggregations) .chain(helper_fixed_size_report_aggregations) .map(|report_aggregation| *report_aggregation.report_id()) .collect(); @@ -11079,8 +10347,6 @@ mod tests { const OLDEST_ALLOWED_REPORT_TIMESTAMP: Time = Time::from_seconds_since_epoch(1000); let ( - leader_time_interval_task_id, - helper_time_interval_task_id, leader_fixed_size_task_id, helper_fixed_size_task_id, other_task_id, @@ -11090,18 +10356,6 @@ mod tests { ) = ds .run_tx(|tx| { Box::pin(async move { - let leader_time_interval_task = TaskBuilder::new( - task::QueryType::TimeInterval, - VdafInstance::Fake, - Role::Leader, - ) - .build(); - let helper_time_interval_task = TaskBuilder::new( - task::QueryType::TimeInterval, - VdafInstance::Fake, - Role::Helper, - ) - .build(); let leader_fixed_size_task = TaskBuilder::new( task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, @@ -11115,13 +10369,11 @@ mod tests { ) .build(); let other_task = TaskBuilder::new( - task::QueryType::TimeInterval, + task::QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Fake, Role::Leader, ) .build(); - tx.put_task(&leader_time_interval_task).await?; - tx.put_task(&helper_time_interval_task).await?; tx.put_task(&leader_fixed_size_task).await?; tx.put_task(&helper_fixed_size_task).await?; tx.put_task(&other_task).await?; @@ -11130,112 +10382,6 @@ mod tests { let mut aggregate_share_job_ids = HashSet::new(); let mut outstanding_batch_ids = HashSet::new(); - // Leader, time-interval collection artifacts with old reports. [GC'ed] - write_collect_artifacts::( - tx, - &leader_time_interval_task, - &[ - OLDEST_ALLOWED_REPORT_TIMESTAMP - .sub(&Duration::from_seconds(10)) - .unwrap(), - OLDEST_ALLOWED_REPORT_TIMESTAMP - .sub(&Duration::from_seconds(9)) - .unwrap(), - ], - ) - .await; - - // Leader, time-interval collection artifacts with old & new reports. [not GC'ed] - let (collection_job_id, aggregate_share_job_id, outstanding_batch_id) = - write_collect_artifacts::( - tx, - &leader_time_interval_task, - &[ - OLDEST_ALLOWED_REPORT_TIMESTAMP - .sub(&Duration::from_seconds(5)) - .unwrap(), - OLDEST_ALLOWED_REPORT_TIMESTAMP - .add(&Duration::from_seconds(5)) - .unwrap(), - ], - ) - .await; - collection_job_ids.extend(collection_job_id); - aggregate_share_job_ids.extend(aggregate_share_job_id); - outstanding_batch_ids.extend(outstanding_batch_id); - - // Leader, time-interval collection artifacts with new reports. [not GC'ed] - let (collection_job_id, aggregate_share_job_id, outstanding_batch_id) = - write_collect_artifacts::( - tx, - &leader_time_interval_task, - &[ - OLDEST_ALLOWED_REPORT_TIMESTAMP - .add(&Duration::from_seconds(9)) - .unwrap(), - OLDEST_ALLOWED_REPORT_TIMESTAMP - .add(&Duration::from_seconds(10)) - .unwrap(), - ], - ) - .await; - collection_job_ids.extend(collection_job_id); - aggregate_share_job_ids.extend(aggregate_share_job_id); - outstanding_batch_ids.extend(outstanding_batch_id); - - // Helper, time-interval collection artifacts with old reports. [GC'ed] - write_collect_artifacts::( - tx, - &helper_time_interval_task, - &[ - OLDEST_ALLOWED_REPORT_TIMESTAMP - .sub(&Duration::from_seconds(10)) - .unwrap(), - OLDEST_ALLOWED_REPORT_TIMESTAMP - .sub(&Duration::from_seconds(9)) - .unwrap(), - ], - ) - .await; - - // Helper, time-interval collection artifacts with old & new reports. [not GC'ed] - let (collection_job_id, aggregate_share_job_id, outstanding_batch_id) = - write_collect_artifacts::( - tx, - &helper_time_interval_task, - &[ - OLDEST_ALLOWED_REPORT_TIMESTAMP - .sub(&Duration::from_seconds(5)) - .unwrap(), - OLDEST_ALLOWED_REPORT_TIMESTAMP - .add(&Duration::from_seconds(5)) - .unwrap(), - ], - ) - .await; - collection_job_ids.extend(collection_job_id); - aggregate_share_job_ids.extend(aggregate_share_job_id); - outstanding_batch_ids.extend(outstanding_batch_id); - - // Helper, time-interval collection artifacts with new reports. [not GC'ed] - let (collection_job_id, aggregate_share_job_id, outstanding_batch_id) = - write_collect_artifacts::( - tx, - &helper_time_interval_task, - &[ - OLDEST_ALLOWED_REPORT_TIMESTAMP - .add(&Duration::from_seconds(9)) - .unwrap(), - OLDEST_ALLOWED_REPORT_TIMESTAMP - .add(&Duration::from_seconds(10)) - .unwrap(), - ], - ) - .await; - collection_job_ids.extend(collection_job_id); - aggregate_share_job_ids.extend(aggregate_share_job_id); - outstanding_batch_ids.extend(outstanding_batch_id); - // Leader, fixed-size collection artifacts with old reports. [GC'ed] write_collect_artifacts::( tx, @@ -11322,7 +10468,7 @@ mod tests { // Collection artifacts for different task. [not GC'ed] let (collection_job_id, aggregate_share_job_id, outstanding_batch_id) = - write_collect_artifacts::( + write_collect_artifacts::( tx, &other_task, &[ @@ -11340,8 +10486,6 @@ mod tests { outstanding_batch_ids.extend(outstanding_batch_id); Ok(( - *leader_time_interval_task.id(), - *helper_time_interval_task.id(), *leader_fixed_size_task.id(), *helper_fixed_size_task.id(), *other_task.id(), @@ -11357,16 +10501,6 @@ mod tests { // Run. ds.run_tx(|tx| { Box::pin(async move { - tx.delete_expired_collection_artifacts( - &leader_time_interval_task_id, - OLDEST_ALLOWED_REPORT_TIMESTAMP, - ) - .await?; - tx.delete_expired_collection_artifacts( - &helper_time_interval_task_id, - OLDEST_ALLOWED_REPORT_TIMESTAMP, - ) - .await?; tx.delete_expired_collection_artifacts( &leader_fixed_size_task_id, OLDEST_ALLOWED_REPORT_TIMESTAMP, @@ -11388,24 +10522,6 @@ mod tests { .run_tx(|tx| { Box::pin(async move { let vdaf = dummy_vdaf::Vdaf::new(); - let leader_time_interval_collection_job_ids = tx - .get_collection_jobs_for_task::<0, TimeInterval, dummy_vdaf::Vdaf>( - &vdaf, - &leader_time_interval_task_id, - ) - .await - .unwrap() - .into_iter() - .map(|collection_job| *collection_job.id()); - let helper_time_interval_collection_job_ids = tx - .get_collection_jobs_for_task::<0, TimeInterval, dummy_vdaf::Vdaf>( - &vdaf, - &helper_time_interval_task_id, - ) - .await - .unwrap() - .into_iter() - .map(|collection_job| *collection_job.id()); let leader_fixed_size_collection_job_ids = tx .get_collection_jobs_for_task::<0, FixedSize, dummy_vdaf::Vdaf>( &vdaf, @@ -11425,7 +10541,7 @@ mod tests { .into_iter() .map(|collection_job| *collection_job.id()); let other_task_collection_job_ids = tx - .get_collection_jobs_for_task::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_collection_jobs_for_task::<0, FixedSize, dummy_vdaf::Vdaf>( &vdaf, &other_task_id, ) @@ -11433,31 +10549,11 @@ mod tests { .unwrap() .into_iter() .map(|collection_job| *collection_job.id()); - let got_collection_job_ids = leader_time_interval_collection_job_ids - .chain(helper_time_interval_collection_job_ids) - .chain(leader_fixed_size_collection_job_ids) + let got_collection_job_ids = leader_fixed_size_collection_job_ids .chain(helper_fixed_size_collection_job_ids) .chain(other_task_collection_job_ids) .collect(); - let leader_time_interval_aggregate_share_job_ids = tx - .get_aggregate_share_jobs_for_task::<0, TimeInterval, dummy_vdaf::Vdaf>( - &vdaf, - &leader_time_interval_task_id, - ) - .await - .unwrap() - .into_iter() - .map(|job| (*job.task_id(), job.batch_identifier().get_encoded())); - let helper_time_interval_aggregate_share_job_ids = tx - .get_aggregate_share_jobs_for_task::<0, TimeInterval, dummy_vdaf::Vdaf>( - &vdaf, - &helper_time_interval_task_id, - ) - .await - .unwrap() - .into_iter() - .map(|job| (*job.task_id(), job.batch_identifier().get_encoded())); let leader_fixed_size_aggregate_share_job_ids = tx .get_aggregate_share_jobs_for_task::<0, FixedSize, dummy_vdaf::Vdaf>( &vdaf, @@ -11477,33 +10573,19 @@ mod tests { .into_iter() .map(|job| (*job.task_id(), job.batch_identifier().get_encoded())); let other_task_aggregate_share_job_ids = tx - .get_aggregate_share_jobs_for_task::<0, TimeInterval, dummy_vdaf::Vdaf>( + .get_aggregate_share_jobs_for_task::<0, FixedSize, dummy_vdaf::Vdaf>( &vdaf, - &leader_time_interval_task_id, + &other_task_id, ) .await .unwrap() .into_iter() .map(|job| (*job.task_id(), job.batch_identifier().get_encoded())); - let got_aggregate_share_job_ids = leader_time_interval_aggregate_share_job_ids - .chain(helper_time_interval_aggregate_share_job_ids) - .chain(leader_fixed_size_aggregate_share_job_ids) + let got_aggregate_share_job_ids = leader_fixed_size_aggregate_share_job_ids .chain(helper_fixed_size_aggregate_share_job_ids) .chain(other_task_aggregate_share_job_ids) .collect(); - let leader_time_interval_outstanding_batch_ids = tx - .get_outstanding_batches_for_task(&leader_time_interval_task_id) - .await - .unwrap() - .into_iter() - .map(|batch| (*batch.task_id(), *batch.id())); - let helper_time_interval_outstanding_batch_ids = tx - .get_outstanding_batches_for_task(&helper_time_interval_task_id) - .await - .unwrap() - .into_iter() - .map(|batch| (*batch.task_id(), *batch.id())); let leader_fixed_size_outstanding_batch_ids = tx .get_outstanding_batches_for_task(&leader_fixed_size_task_id) .await @@ -11517,14 +10599,12 @@ mod tests { .into_iter() .map(|batch| (*batch.task_id(), *batch.id())); let other_task_outstanding_batch_ids = tx - .get_outstanding_batches_for_task(&helper_fixed_size_task_id) + .get_outstanding_batches_for_task(&other_task_id) .await .unwrap() .into_iter() .map(|batch| (*batch.task_id(), *batch.id())); - let got_outstanding_batch_ids = leader_time_interval_outstanding_batch_ids - .chain(helper_time_interval_outstanding_batch_ids) - .chain(leader_fixed_size_outstanding_batch_ids) + let got_outstanding_batch_ids = leader_fixed_size_outstanding_batch_ids .chain(helper_fixed_size_outstanding_batch_ids) .chain(other_task_outstanding_batch_ids) .collect(); diff --git a/aggregator_core/src/query_type.rs b/aggregator_core/src/query_type.rs index 6d6882d24..be2674e10 100644 --- a/aggregator_core/src/query_type.rs +++ b/aggregator_core/src/query_type.rs @@ -8,10 +8,10 @@ use crate::{ }; use async_trait::async_trait; use futures::future::try_join_all; -use janus_core::time::{Clock, TimeExt as _}; +use janus_core::time::Clock; use janus_messages::{ - query_type::{FixedSize, QueryType, TimeInterval}, - Duration, FixedSizeQuery, Interval, Query, ReportMetadata, TaskId, Time, + query_type::{FixedSize, QueryType}, + FixedSizeQuery, Interval, Query, ReportMetadata, TaskId, Time, }; use prio::vdaf; use std::iter; @@ -51,7 +51,7 @@ pub trait AccumulableQueryType: QueryType { batch_identifier: &Self::BatchIdentifier, ) -> Result>, datastore::Error>; - /// Some query types (e.g. [`TimeInterval`]) can represent their batch identifiers as an + /// Some query types (e.g. `TimeInterval`) can represent their batch identifiers as an /// interval. This method extracts the interval from such identifiers, or returns `None` if the /// query type does not represent batch identifiers as an interval. fn to_batch_interval(batch_identifier: &Self::BatchIdentifier) -> Option<&Interval>; @@ -70,74 +70,6 @@ pub trait AccumulableQueryType: QueryType { fn default_partial_batch_identifier() -> Option<&'static Self::PartialBatchIdentifier>; } -#[async_trait] -impl AccumulableQueryType for TimeInterval { - fn to_batch_identifier( - task: &Task, - _: &Self::PartialBatchIdentifier, - client_timestamp: &Time, - ) -> Result { - let batch_interval_start = client_timestamp - .to_batch_interval_start(task.time_precision()) - .map_err(|e| datastore::Error::User(e.into()))?; - Interval::new(batch_interval_start, *task.time_precision()) - .map_err(|e| datastore::Error::User(e.into())) - } - - async fn get_conflicting_aggregate_share_jobs< - const SEED_SIZE: usize, - C: Clock, - A: vdaf::Aggregator + Send + Sync, - >( - tx: &Transaction<'_, C>, - vdaf: &A, - task_id: &TaskId, - _: &Self::PartialBatchIdentifier, - report_metadata: &ReportMetadata, - ) -> Result>, datastore::Error> { - tx.get_aggregate_share_jobs_including_time::( - vdaf, - task_id, - report_metadata.time(), - ) - .await - } - - async fn get_collection_jobs_including< - const SEED_SIZE: usize, - C: Clock, - A: vdaf::Aggregator + Send + Sync, - >( - tx: &Transaction<'_, C>, - vdaf: &A, - task_id: &TaskId, - batch_identifier: &Self::BatchIdentifier, - ) -> Result>, datastore::Error> { - tx.get_collection_jobs_intersecting_interval(vdaf, task_id, batch_identifier) - .await - } - - fn to_batch_interval(collection_identifier: &Self::BatchIdentifier) -> Option<&Interval> { - Some(collection_identifier) - } - - fn downgrade_batch_identifier( - _batch_identifier: &Self::BatchIdentifier, - ) -> &Self::PartialBatchIdentifier { - &() - } - - fn upgrade_partial_batch_identifier( - _partial_batch_identifier: &Self::PartialBatchIdentifier, - ) -> Option<&Self::BatchIdentifier> { - None - } - - fn default_partial_batch_identifier() -> Option<&'static Self::PartialBatchIdentifier> { - Some(&()) - } -} - #[async_trait] impl AccumulableQueryType for FixedSize { fn to_batch_identifier( @@ -211,7 +143,7 @@ pub trait CollectableQueryType: AccumulableQueryType { query: &Query, ) -> Result, datastore::Error>; - /// Some query types (e.g. [`TimeInterval`]) can receive a batch identifier in collection + /// Some query types (e.g. `TimeInterval`) can receive a batch identifier in collection /// requests which refers to multiple batches. This method takes a batch identifier received in /// a collection request and provides an iterator over the individual batches' identifiers. fn batch_identifiers_for_collection_identifier( @@ -316,112 +248,6 @@ pub trait CollectableQueryType: AccumulableQueryType { ) -> Result<(), datastore::Error>; } -#[async_trait] -impl CollectableQueryType for TimeInterval { - type Iter = TimeIntervalBatchIdentifierIter; - - async fn collection_identifier_for_query( - _: &Transaction<'_, C>, - _: &Task, - query: &Query, - ) -> Result, datastore::Error> { - Ok(Some(*query.batch_interval())) - } - - fn batch_identifiers_for_collection_identifier( - task: &Task, - batch_interval: &Self::BatchIdentifier, - ) -> Self::Iter { - TimeIntervalBatchIdentifierIter::new(task, batch_interval) - } - - fn validate_collection_identifier( - task: &Task, - collection_identifier: &Self::BatchIdentifier, - ) -> bool { - // https://www.ietf.org/archive/id/draft-ietf-ppm-dap-02.html#section-4.5.6.1.1 - - // Batch interval should be greater than task's time precision - collection_identifier.duration().as_seconds() >= task.time_precision().as_seconds() - // Batch interval start must be a multiple of time precision - && collection_identifier.start().as_seconds_since_epoch() % task.time_precision().as_seconds() == 0 - // Batch interval duration must be a multiple of time precision - && collection_identifier.duration().as_seconds() % task.time_precision().as_seconds() == 0 - } - - async fn count_client_reports( - tx: &Transaction<'_, C>, - task: &Task, - batch_interval: &Self::BatchIdentifier, - ) -> Result { - tx.count_client_reports_for_interval(task.id(), batch_interval) - .await - } - - async fn acknowledge_collection( - _: &Transaction<'_, C>, - _: &TaskId, - _: &Self::BatchIdentifier, - ) -> Result<(), datastore::Error> { - Ok(()) // Purposeful no-op. - } -} - -// This type only exists because the CollectableQueryType trait requires specifying the type of the -// iterator explicitly (i.e. it cannot be inferred or replaced with an `impl Trait` expression), and -// the type of the iterator created via method chaining does not have a type which is expressible. -pub struct TimeIntervalBatchIdentifierIter { - step: u64, - - total_step_count: u64, - start: Time, - time_precision: Duration, -} - -impl TimeIntervalBatchIdentifierIter { - fn new(task: &Task, batch_interval: &Interval) -> Self { - // Sanity check that the given interval is of an appropriate length. We use an assert as - // this is expected to be checked before this method is used. - assert_eq!( - batch_interval.duration().as_seconds() % task.time_precision().as_seconds(), - 0 - ); - let total_step_count = - batch_interval.duration().as_seconds() / task.time_precision().as_seconds(); - - Self { - step: 0, - total_step_count, - start: *batch_interval.start(), - time_precision: *task.time_precision(), - } - } -} - -impl Iterator for TimeIntervalBatchIdentifierIter { - type Item = Interval; - - fn next(&mut self) -> Option { - if self.step == self.total_step_count { - return None; - } - // Unwrap safety: errors can only occur if the times being unwrapped cannot be represented - // as a Time. The relevant times can always be represented since they are internal to the - // batch interval used to create the iterator. - let interval = Interval::new( - self.start - .add(&Duration::from_seconds( - self.step * self.time_precision.as_seconds(), - )) - .unwrap(), - self.time_precision, - ) - .unwrap(); - self.step += 1; - Some(interval) - } -} - #[async_trait] impl CollectableQueryType for FixedSize { type Iter = iter::Once; @@ -468,82 +294,3 @@ impl CollectableQueryType for FixedSize { tx.delete_outstanding_batch(task_id, batch_identifier).await } } - -#[cfg(test)] -mod tests { - use crate::{ - query_type::CollectableQueryType, - task::{test_util::TaskBuilder, QueryType}, - }; - use janus_core::task::VdafInstance; - use janus_messages::{query_type::TimeInterval, Duration, Interval, Role, Time}; - - #[test] - fn validate_collect_identifier() { - let time_precision_secs = 3600; - let task = TaskBuilder::new(QueryType::TimeInterval, VdafInstance::Fake, Role::Leader) - .with_time_precision(Duration::from_seconds(time_precision_secs)) - .build(); - - struct TestCase { - name: &'static str, - input: Interval, - expected: bool, - } - - for test_case in Vec::from([ - TestCase { - name: "same duration as minimum", - input: Interval::new( - Time::from_seconds_since_epoch(time_precision_secs), - Duration::from_seconds(time_precision_secs), - ) - .unwrap(), - expected: true, - }, - TestCase { - name: "interval too short", - input: Interval::new( - Time::from_seconds_since_epoch(time_precision_secs), - Duration::from_seconds(time_precision_secs - 1), - ) - .unwrap(), - expected: false, - }, - TestCase { - name: "interval larger than minimum", - input: Interval::new( - Time::from_seconds_since_epoch(time_precision_secs), - Duration::from_seconds(time_precision_secs * 2), - ) - .unwrap(), - expected: true, - }, - TestCase { - name: "interval duration not aligned with minimum", - input: Interval::new( - Time::from_seconds_since_epoch(time_precision_secs), - Duration::from_seconds(time_precision_secs + 1800), - ) - .unwrap(), - expected: false, - }, - TestCase { - name: "interval start not aligned with minimum", - input: Interval::new( - Time::from_seconds_since_epoch(1800), - Duration::from_seconds(time_precision_secs), - ) - .unwrap(), - expected: false, - }, - ]) { - assert_eq!( - test_case.expected, - TimeInterval::validate_collection_identifier(&task, &test_case.input), - "test case: {}", - test_case.name - ); - } - } -} diff --git a/aggregator_core/src/task.rs b/aggregator_core/src/task.rs index bfbdb255b..43df9f533 100644 --- a/aggregator_core/src/task.rs +++ b/aggregator_core/src/task.rs @@ -36,9 +36,6 @@ pub enum Error { /// Identifiers for query types used by a task, along with query-type specific configuration. #[derive(Debug, Copy, Clone, PartialEq, Eq, PartialOrd, Ord, Serialize, Deserialize)] pub enum QueryType { - /// Time-interval: used to support a collection style based on fixed time intervals. - TimeInterval, - /// Fixed-size: used to support collection of batches as quickly as possible, without aligning /// to a fixed batch window. FixedSize { @@ -193,10 +190,9 @@ impl Task { if self.hpke_keys.is_empty() { return Err(Error::InvalidParameter("hpke_keys")); } - if let QueryType::FixedSize { max_batch_size } = self.query_type() { - if *max_batch_size < self.min_batch_size() { - return Err(Error::InvalidParameter("max_batch_size")); - } + let QueryType::FixedSize { max_batch_size } = self.query_type(); + if *max_batch_size < self.min_batch_size() { + return Err(Error::InvalidParameter("max_batch_size")); } Ok(()) } @@ -295,10 +291,6 @@ impl Task { /// pub fn validate_batch_size(&self, batch_size: u64) -> bool { match self.query_type { - QueryType::TimeInterval => { - // https://www.ietf.org/archive/id/draft-ietf-ppm-dap-02.html#section-4.5.6.1.2 - batch_size >= self.min_batch_size() - } QueryType::FixedSize { max_batch_size } => { // https://www.ietf.org/archive/id/draft-ietf-ppm-dap-02.html#section-4.5.6.2.2 batch_size >= self.min_batch_size() && batch_size <= max_batch_size @@ -784,7 +776,7 @@ mod tests { }; use janus_core::{ hpke::{test_util::generate_test_hpke_config_and_private_key, HpkeKeypair, HpkePrivateKey}, - task::{AuthenticationToken, DapAuthToken, PRIO3_VERIFY_KEY_LENGTH}, + task::{AuthenticationToken, PRIO3_VERIFY_KEY_LENGTH}, test_util::roundtrip_encoding, time::DurationExt, }; @@ -800,7 +792,7 @@ mod tests { fn task_serialization() { roundtrip_encoding( TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -822,7 +814,7 @@ mod tests { "http://leader_endpoint".parse().unwrap(), "http://helper_endpoint".parse().unwrap(), ]), - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, Vec::from([SecretBytes::new([0; PRIO3_VERIFY_KEY_LENGTH].into())]), @@ -846,7 +838,7 @@ mod tests { "http://leader_endpoint".parse().unwrap(), "http://helper_endpoint".parse().unwrap(), ]), - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, Vec::from([SecretBytes::new([0; PRIO3_VERIFY_KEY_LENGTH].into())]), @@ -870,7 +862,7 @@ mod tests { "http://leader_endpoint".parse().unwrap(), "http://helper_endpoint".parse().unwrap(), ]), - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Helper, Vec::from([SecretBytes::new([0; PRIO3_VERIFY_KEY_LENGTH].into())]), @@ -894,7 +886,7 @@ mod tests { "http://leader_endpoint".parse().unwrap(), "http://helper_endpoint".parse().unwrap(), ]), - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Helper, Vec::from([SecretBytes::new([0; PRIO3_VERIFY_KEY_LENGTH].into())]), @@ -920,7 +912,7 @@ mod tests { "http://leader_endpoint/foo/bar".parse().unwrap(), "http://helper_endpoint".parse().unwrap(), ]), - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, Vec::from([SecretBytes::new([0; PRIO3_VERIFY_KEY_LENGTH].into())]), @@ -952,7 +944,7 @@ mod tests { ( "", TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -961,7 +953,7 @@ mod tests { ( "/prefix", TaskBuilder::new( - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 10 }, VdafInstance::Prio3Count, Role::Leader, ) @@ -993,186 +985,6 @@ mod tests { #[test] fn task_serde() { - assert_tokens( - &Task::new( - TaskId::from([0; 32]), - Vec::from([ - "https://example.com/".parse().unwrap(), - "https://example.net/".parse().unwrap(), - ]), - QueryType::TimeInterval, - VdafInstance::Prio3Count, - Role::Leader, - Vec::from([SecretBytes::new(b"1234567812345678".to_vec())]), - 1, - None, - None, - 10, - Duration::from_seconds(3600), - Duration::from_seconds(60), - HpkeConfig::new( - HpkeConfigId::from(8), - HpkeKemId::X25519HkdfSha256, - HpkeKdfId::HkdfSha256, - HpkeAeadId::Aes128Gcm, - HpkePublicKey::from(b"collector hpke public key".to_vec()), - ), - Vec::from([AuthenticationToken::DapAuth( - DapAuthToken::try_from(b"aggregator token".to_vec()).unwrap(), - )]), - Vec::from([AuthenticationToken::Bearer(b"collector token".to_vec())]), - [HpkeKeypair::new( - HpkeConfig::new( - HpkeConfigId::from(255), - HpkeKemId::X25519HkdfSha256, - HpkeKdfId::HkdfSha256, - HpkeAeadId::Aes128Gcm, - HpkePublicKey::from(b"aggregator hpke public key".to_vec()), - ), - HpkePrivateKey::new(b"aggregator hpke private key".to_vec()), - )], - ) - .unwrap(), - &[ - Token::Struct { - name: "SerializedTask", - len: 16, - }, - Token::Str("task_id"), - Token::Some, - Token::Str("AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"), - Token::Str("aggregator_endpoints"), - Token::Seq { len: Some(2) }, - Token::Str("https://example.com/"), - Token::Str("https://example.net/"), - Token::SeqEnd, - Token::Str("query_type"), - Token::UnitVariant { - name: "QueryType", - variant: "TimeInterval", - }, - Token::Str("vdaf"), - Token::UnitVariant { - name: "VdafInstance", - variant: "Prio3Count", - }, - Token::Str("role"), - Token::UnitVariant { - name: "Role", - variant: "Leader", - }, - Token::Str("vdaf_verify_keys"), - Token::Seq { len: Some(1) }, - Token::Str("MTIzNDU2NzgxMjM0NTY3OA"), - Token::SeqEnd, - Token::Str("max_batch_query_count"), - Token::U64(1), - Token::Str("task_expiration"), - Token::None, - Token::Str("report_expiry_age"), - Token::None, - Token::Str("min_batch_size"), - Token::U64(10), - Token::Str("time_precision"), - Token::NewtypeStruct { name: "Duration" }, - Token::U64(3600), - Token::Str("tolerable_clock_skew"), - Token::NewtypeStruct { name: "Duration" }, - Token::U64(60), - Token::Str("collector_hpke_config"), - Token::Struct { - name: "HpkeConfig", - len: 5, - }, - Token::Str("id"), - Token::NewtypeStruct { - name: "HpkeConfigId", - }, - Token::U8(8), - Token::Str("kem_id"), - Token::UnitVariant { - name: "HpkeKemId", - variant: "X25519HkdfSha256", - }, - Token::Str("kdf_id"), - Token::UnitVariant { - name: "HpkeKdfId", - variant: "HkdfSha256", - }, - Token::Str("aead_id"), - Token::UnitVariant { - name: "HpkeAeadId", - variant: "Aes128Gcm", - }, - Token::Str("public_key"), - Token::Str("Y29sbGVjdG9yIGhwa2UgcHVibGljIGtleQ"), - Token::StructEnd, - Token::Str("aggregator_auth_tokens"), - Token::Seq { len: Some(1) }, - Token::Struct { - name: "AuthenticationToken", - len: 2, - }, - Token::Str("type"), - Token::Str("DapAuth"), - Token::Str("token"), - Token::Str("YWdncmVnYXRvciB0b2tlbg"), - Token::StructEnd, - Token::SeqEnd, - Token::Str("collector_auth_tokens"), - Token::Seq { len: Some(1) }, - Token::Struct { - name: "AuthenticationToken", - len: 2, - }, - Token::Str("type"), - Token::Str("Bearer"), - Token::Str("token"), - Token::Str("Y29sbGVjdG9yIHRva2Vu"), - Token::StructEnd, - Token::SeqEnd, - Token::Str("hpke_keys"), - Token::Seq { len: Some(1) }, - Token::Struct { - name: "HpkeKeypair", - len: 2, - }, - Token::Str("config"), - Token::Struct { - name: "HpkeConfig", - len: 5, - }, - Token::Str("id"), - Token::NewtypeStruct { - name: "HpkeConfigId", - }, - Token::U8(255), - Token::Str("kem_id"), - Token::UnitVariant { - name: "HpkeKemId", - variant: "X25519HkdfSha256", - }, - Token::Str("kdf_id"), - Token::UnitVariant { - name: "HpkeKdfId", - variant: "HkdfSha256", - }, - Token::Str("aead_id"), - Token::UnitVariant { - name: "HpkeAeadId", - variant: "Aes128Gcm", - }, - Token::Str("public_key"), - Token::Str("YWdncmVnYXRvciBocGtlIHB1YmxpYyBrZXk"), - Token::StructEnd, - Token::Str("private_key"), - Token::Str("YWdncmVnYXRvciBocGtlIHByaXZhdGUga2V5"), - Token::StructEnd, - Token::SeqEnd, - Token::StructEnd, - ], - ); - assert_tokens( &Task::new( TaskId::from([255; 32]), diff --git a/collector/src/lib.rs b/collector/src/lib.rs index 0aa17dcd1..e537fd6c4 100644 --- a/collector/src/lib.rs +++ b/collector/src/lib.rs @@ -10,8 +10,8 @@ //! use janus_collector::{AuthenticationToken, Collector, CollectorParameters, default_http_client}; //! use janus_core::{hpke::generate_hpke_config_and_private_key}; //! use janus_messages::{ -//! Duration, HpkeAeadId, HpkeConfig, HpkeConfigId, HpkeKdfId, HpkeKemId, Interval, TaskId, -//! Time, Query, +//! Duration, FixedSizeQuery, HpkeAeadId, HpkeConfig, HpkeConfigId, HpkeKdfId, HpkeKemId, +//! Interval, TaskId, Time, Query //! }; //! use prio::vdaf::prio3::Prio3; //! use rand::random; @@ -40,14 +40,8 @@ //! let http_client = default_http_client().unwrap(); //! let collector = Collector::new(parameters, vdaf, http_client); //! -//! // Specify the time interval over which the aggregation should be calculated. -//! let interval = Interval::new( -//! Time::from_seconds_since_epoch(1_656_000_000), -//! Duration::from_seconds(3600), -//! ) -//! .unwrap(); -//! // Make the requests and retrieve the aggregated statistic. -//! let aggregation_result = collector.collect(Query::new_time_interval(interval), &()).await.unwrap(); +//! // Make the request and retrieve the aggregated statistic. +//! let aggregation_result = collector.collect(Query::new_fixed_size(FixedSizeQuery::CurrentBatch), &()).await.unwrap(); //! # } //! ``` @@ -67,7 +61,7 @@ use janus_core::{ }; use janus_messages::{ problem_type::DapProblemType, - query_type::{QueryType, TimeInterval}, + query_type::{FixedSize, QueryType}, AggregateShareAad, BatchSelector, Collection as CollectionMessage, CollectionJobId, CollectionReq, HpkeConfig, PartialBatchSelector, Query, Role, TaskId, }; @@ -404,7 +398,7 @@ impl Collector { self.parameters.authentication.request_authentication(); self.http_client .put(collection_job_url.clone()) - .header(CONTENT_TYPE, CollectionReq::::MEDIA_TYPE) + .header(CONTENT_TYPE, CollectionReq::::MEDIA_TYPE) .body(collect_request.get_encoded()) .header(auth_header, auth_value) .send() @@ -496,7 +490,7 @@ impl Collector { .headers() .get(CONTENT_TYPE) .ok_or(Error::BadContentType(None))?; - if content_type != CollectionMessage::::MEDIA_TYPE { + if content_type != CollectionMessage::::MEDIA_TYPE { return Err(Error::BadContentType(Some(content_type.clone()))); } @@ -676,11 +670,9 @@ mod tests { test_util::{install_test_trace_subscriber, run_vdaf, VdafTranscript}, }; use janus_messages::{ - problem_type::DapProblemType, - query_type::{FixedSize, TimeInterval}, - AggregateShareAad, BatchId, BatchSelector, Collection as CollectionMessage, - CollectionJobId, CollectionReq, Duration, FixedSizeQuery, HpkeCiphertext, Interval, - PartialBatchSelector, Query, Role, TaskId, Time, + problem_type::DapProblemType, query_type::FixedSize, AggregateShareAad, BatchId, + BatchSelector, Collection as CollectionMessage, CollectionJobId, CollectionReq, Duration, + FixedSizeQuery, HpkeCiphertext, Interval, PartialBatchSelector, Query, Role, TaskId, Time, }; use mockito::Matcher; use prio::{ @@ -721,46 +713,6 @@ mod tests { )) } - fn build_collect_response_time>( - transcript: &VdafTranscript, - parameters: &CollectorParameters, - batch_interval: Interval, - ) -> CollectionMessage { - let associated_data = AggregateShareAad::new( - parameters.task_id, - BatchSelector::new_time_interval(batch_interval), - ); - CollectionMessage::new( - PartialBatchSelector::new_time_interval(), - 1, - batch_interval, - Vec::::from([ - hpke::seal( - ¶meters.hpke_config, - &HpkeApplicationInfo::new( - &Label::AggregateShare, - &Role::Leader, - &Role::Collector, - ), - &transcript.aggregate_shares[0].get_encoded(), - &associated_data.get_encoded(), - ) - .unwrap(), - hpke::seal( - ¶meters.hpke_config, - &HpkeApplicationInfo::new( - &Label::AggregateShare, - &Role::Helper, - &Role::Collector, - ), - &transcript.aggregate_shares[1].get_encoded(), - &associated_data.get_encoded(), - ) - .unwrap(), - ]), - ) - } - fn build_collect_response_fixed>( transcript: &VdafTranscript, parameters: &CollectorParameters, @@ -771,7 +723,11 @@ mod tests { CollectionMessage::new( PartialBatchSelector::new_fixed_size(batch_id), 1, - Interval::new(Time::from_seconds_since_epoch(0), Duration::from_seconds(1)).unwrap(), + Interval::new( + Time::from_seconds_since_epoch(0), + Duration::from_seconds(3600), + ) + .unwrap(), Vec::::from([ hpke::seal( ¶meters.hpke_config, @@ -840,20 +796,16 @@ mod tests { collector.parameters.authentication.request_authentication(); let auth_value = String::from_utf8(auth_value).unwrap(); - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(1_000_000), - Duration::from_seconds(3600), - ) - .unwrap(); + let batch_id = random(); let collect_resp = - build_collect_response_time(&transcript, &collector.parameters, batch_interval); + build_collect_response_fixed(&transcript, &collector.parameters, batch_id); let matcher = collection_uri_regex_matcher(&collector.parameters.task_id); let mocked_collect_start_error = server .mock("PUT", matcher.clone()) .match_header( CONTENT_TYPE.as_str(), - CollectionReq::::MEDIA_TYPE, + CollectionReq::::MEDIA_TYPE, ) .with_status(500) .expect(1) @@ -863,7 +815,7 @@ mod tests { .mock("PUT", matcher) .match_header( CONTENT_TYPE.as_str(), - CollectionReq::::MEDIA_TYPE, + CollectionReq::::MEDIA_TYPE, ) .match_header(auth_header, auth_value.as_str()) .with_status(201) @@ -872,14 +824,20 @@ mod tests { .await; let job = collector - .start_collection(Query::new_time_interval(batch_interval), &()) + .start_collection( + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id }), + &(), + ) .await; mocked_collect_start_error.assert_async().await; mocked_collect_start_success.assert_async().await; let job = job.unwrap(); - assert_eq!(job.query.batch_interval(), &batch_interval); + assert_eq!( + job.query.query_body(), + &FixedSizeQuery::ByBatchId { batch_id } + ); let mocked_collect_error = server .mock("POST", job.collection_job_url.path()) @@ -899,7 +857,7 @@ mod tests { .with_status(200) .with_header( CONTENT_TYPE.as_str(), - CollectionMessage::::MEDIA_TYPE, + CollectionMessage::::MEDIA_TYPE, ) .with_body(collect_resp.get_encoded()) .expect(1) @@ -913,11 +871,11 @@ mod tests { assert_eq!( collection, Collection::new( - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(batch_id), 1, ( DateTime::::from_utc( - NaiveDateTime::from_timestamp_opt(1_000_000, 0).unwrap(), + NaiveDateTime::from_timestamp_opt(0, 0).unwrap(), Utc ), chrono::Duration::seconds(3600), @@ -939,20 +897,16 @@ mod tests { let transcript = run_vdaf(&vdaf, &random(), &(), &random(), &144); let collector = setup_collector(&mut server, vdaf); - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(1_000_000), - Duration::from_seconds(3600), - ) - .unwrap(); + let batch_id = random(); let collect_resp = - build_collect_response_time(&transcript, &collector.parameters, batch_interval); + build_collect_response_fixed(&transcript, &collector.parameters, batch_id); let matcher = collection_uri_regex_matcher(&collector.parameters.task_id); let mocked_collect_start_success = server .mock("PUT", matcher) .match_header( CONTENT_TYPE.as_str(), - CollectionReq::::MEDIA_TYPE, + CollectionReq::::MEDIA_TYPE, ) .with_status(201) .expect(1) @@ -960,10 +914,16 @@ mod tests { .await; let job = collector - .start_collection(Query::new_time_interval(batch_interval), &()) + .start_collection( + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id }), + &(), + ) .await .unwrap(); - assert_eq!(job.query.batch_interval(), &batch_interval); + assert_eq!( + job.query.query_body(), + &FixedSizeQuery::ByBatchId { batch_id } + ); mocked_collect_start_success.assert_async().await; let mocked_collect_complete = server @@ -971,7 +931,7 @@ mod tests { .with_status(200) .with_header( CONTENT_TYPE.as_str(), - CollectionMessage::::MEDIA_TYPE, + CollectionMessage::::MEDIA_TYPE, ) .with_body(collect_resp.get_encoded()) .expect(1) @@ -982,11 +942,11 @@ mod tests { assert_eq!( collection, Collection::new( - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(batch_id), 1, ( DateTime::::from_utc( - NaiveDateTime::from_timestamp_opt(1_000_000, 0).unwrap(), + NaiveDateTime::from_timestamp_opt(0, 0).unwrap(), Utc ), chrono::Duration::seconds(3600), @@ -1006,20 +966,16 @@ mod tests { let transcript = run_vdaf(&vdaf, &random(), &(), &random(), &80); let collector = setup_collector(&mut server, vdaf); - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(1_000_000), - Duration::from_seconds(3600), - ) - .unwrap(); + let batch_id = random(); let collect_resp = - build_collect_response_time(&transcript, &collector.parameters, batch_interval); + build_collect_response_fixed(&transcript, &collector.parameters, batch_id); let matcher = collection_uri_regex_matcher(&collector.parameters.task_id); let mocked_collect_start_success = server .mock("PUT", matcher) .match_header( CONTENT_TYPE.as_str(), - CollectionReq::::MEDIA_TYPE, + CollectionReq::::MEDIA_TYPE, ) .with_status(201) .expect(1) @@ -1027,10 +983,16 @@ mod tests { .await; let job = collector - .start_collection(Query::new_time_interval(batch_interval), &()) + .start_collection( + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id }), + &(), + ) .await .unwrap(); - assert_eq!(job.query.batch_interval(), &batch_interval); + assert_eq!( + job.query.query_body(), + &FixedSizeQuery::ByBatchId { batch_id } + ); mocked_collect_start_success.assert_async().await; @@ -1039,7 +1001,7 @@ mod tests { .with_status(200) .with_header( CONTENT_TYPE.as_str(), - CollectionMessage::::MEDIA_TYPE, + CollectionMessage::::MEDIA_TYPE, ) .with_body(collect_resp.get_encoded()) .expect(1) @@ -1050,11 +1012,11 @@ mod tests { assert_eq!( collection, Collection::new( - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(batch_id), 1, ( DateTime::::from_utc( - NaiveDateTime::from_timestamp_opt(1_000_000, 0).unwrap(), + NaiveDateTime::from_timestamp_opt(0, 0).unwrap(), Utc ), chrono::Duration::seconds(3600), @@ -1083,74 +1045,6 @@ mod tests { ); let collector = setup_collector(&mut server, vdaf); - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(1_000_000), - Duration::from_seconds(3600), - ) - .unwrap(); - let collect_resp = - build_collect_response_time(&transcript, &collector.parameters, batch_interval); - let matcher = collection_uri_regex_matcher(&collector.parameters.task_id); - - let mocked_collect_start_success = server - .mock("PUT", matcher) - .match_header( - CONTENT_TYPE.as_str(), - CollectionReq::::MEDIA_TYPE, - ) - .with_status(201) - .expect(1) - .create_async() - .await; - - let job = collector - .start_collection(Query::new_time_interval(batch_interval), &()) - .await - .unwrap(); - assert_eq!(job.query.batch_interval(), &batch_interval); - - mocked_collect_start_success.assert_async().await; - - let mocked_collect_complete = server - .mock("POST", job.collection_job_url.path()) - .with_status(200) - .with_header( - CONTENT_TYPE.as_str(), - CollectionMessage::::MEDIA_TYPE, - ) - .with_body(collect_resp.get_encoded()) - .expect(1) - .create_async() - .await; - - let agg_result = collector.poll_until_complete(&job).await.unwrap(); - assert_eq!( - agg_result, - Collection::new( - PartialBatchSelector::new_time_interval(), - 1, - ( - DateTime::::from_utc( - NaiveDateTime::from_timestamp_opt(1_000_000, 0).unwrap(), - Utc - ), - chrono::Duration::seconds(3600), - ), - Vec::from([0.0625, 0.125, 0.25]) - ) - ); - - mocked_collect_complete.assert_async().await; - } - - #[tokio::test] - async fn successful_collect_fixed_size() { - install_test_trace_subscriber(); - let mut server = mockito::Server::new_async().await; - let vdaf = Prio3::new_count(2).unwrap(); - let transcript = run_vdaf(&vdaf, &random(), &(), &random(), &1); - let collector = setup_collector(&mut server, vdaf); - let batch_id = random(); let collect_resp = build_collect_response_fixed(&transcript, &collector.parameters, batch_id); @@ -1175,7 +1069,7 @@ mod tests { .await .unwrap(); assert_eq!( - job.query.fixed_size_query(), + job.query.query_body(), &FixedSizeQuery::ByBatchId { batch_id } ); @@ -1193,9 +1087,9 @@ mod tests { .create_async() .await; - let collection = collector.poll_until_complete(&job).await.unwrap(); + let agg_result = collector.poll_until_complete(&job).await.unwrap(); assert_eq!( - collection, + agg_result, Collection::new( PartialBatchSelector::new_fixed_size(batch_id), 1, @@ -1204,9 +1098,9 @@ mod tests { NaiveDateTime::from_timestamp_opt(0, 0).unwrap(), Utc ), - chrono::Duration::seconds(1), + chrono::Duration::seconds(3600), ), - 1 + Vec::from([0.0625, 0.125, 0.25]) ) ); @@ -1232,20 +1126,16 @@ mod tests { .with_collect_poll_backoff(test_http_request_exponential_backoff()); let collector = Collector::new(parameters, vdaf, default_http_client().unwrap()); - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(1_000_000), - Duration::from_seconds(3600), - ) - .unwrap(); + let batch_id = random(); let collect_resp = - build_collect_response_time(&transcript, &collector.parameters, batch_interval); + build_collect_response_fixed(&transcript, &collector.parameters, batch_id); let matcher = collection_uri_regex_matcher(&collector.parameters.task_id); let mocked_collect_start_success = server .mock("PUT", matcher) .match_header( CONTENT_TYPE.as_str(), - CollectionReq::::MEDIA_TYPE, + CollectionReq::::MEDIA_TYPE, ) .match_header(AUTHORIZATION.as_str(), "Bearer QUFBQUFBQUFBQUFBQUFBQQ==") .with_status(201) @@ -1254,10 +1144,16 @@ mod tests { .await; let job = collector - .start_collection(Query::new_time_interval(batch_interval), &()) + .start_collection( + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id }), + &(), + ) .await .unwrap(); - assert_eq!(job.query.batch_interval(), &batch_interval); + assert_eq!( + job.query.query_body(), + &FixedSizeQuery::ByBatchId { batch_id } + ); mocked_collect_start_success.assert_async().await; @@ -1267,7 +1163,7 @@ mod tests { .with_status(200) .with_header( CONTENT_TYPE.as_str(), - CollectionMessage::::MEDIA_TYPE, + CollectionMessage::::MEDIA_TYPE, ) .with_body(collect_resp.get_encoded()) .expect(1) @@ -1278,11 +1174,11 @@ mod tests { assert_eq!( agg_result, Collection::new( - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(batch_id), 1, ( DateTime::::from_utc( - NaiveDateTime::from_timestamp_opt(1_000_000, 0).unwrap(), + NaiveDateTime::from_timestamp_opt(0, 0).unwrap(), Utc ), chrono::Duration::seconds(3600), @@ -1306,20 +1202,19 @@ mod tests { .mock("PUT", matcher.clone()) .match_header( CONTENT_TYPE.as_str(), - CollectionReq::::MEDIA_TYPE, + CollectionReq::::MEDIA_TYPE, ) .with_status(500) .expect_at_least(1) .create_async() .await; - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(1_000_000), - Duration::from_seconds(3600), - ) - .unwrap(); + let batch_id = random(); let error = collector - .start_collection(Query::new_time_interval(batch_interval), &()) + .start_collection( + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id }), + &(), + ) .await .unwrap_err(); assert_matches!(error, Error::Http { problem_details, dap_problem_type } => { @@ -1333,7 +1228,7 @@ mod tests { .mock("PUT", matcher.clone()) .match_header( CONTENT_TYPE.as_str(), - CollectionReq::::MEDIA_TYPE, + CollectionReq::::MEDIA_TYPE, ) .with_status(500) .with_header("Content-Type", "application/problem+json") @@ -1343,7 +1238,10 @@ mod tests { .await; let error = collector - .start_collection(Query::new_time_interval(batch_interval), &()) + .start_collection( + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id }), + &(), + ) .await .unwrap_err(); assert_matches!(error, Error::Http { problem_details, dap_problem_type } => { @@ -1358,7 +1256,7 @@ mod tests { .mock("PUT", matcher) .match_header( CONTENT_TYPE.as_str(), - CollectionReq::::MEDIA_TYPE, + CollectionReq::::MEDIA_TYPE, ) .with_status(400) .with_header("Content-Type", "application/problem+json") @@ -1372,7 +1270,10 @@ mod tests { .await; let error = collector - .start_collection(Query::new_time_interval(batch_interval), &()) + .start_collection( + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id }), + &(), + ) .await .unwrap_err(); assert_matches!(error, Error::Http { problem_details, dap_problem_type } => { @@ -1397,7 +1298,7 @@ mod tests { .mock("PUT", matcher.clone()) .match_header( CONTENT_TYPE.as_str(), - CollectionReq::::MEDIA_TYPE, + CollectionReq::::MEDIA_TYPE, ) .with_status(201) .expect(1) @@ -1410,13 +1311,17 @@ mod tests { .create_async() .await; + let batch_id = random(); let batch_interval = Interval::new( Time::from_seconds_since_epoch(1_000_000), Duration::from_seconds(3600), ) .unwrap(); let job = collector - .start_collection(Query::new_time_interval(batch_interval), &()) + .start_collection( + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id }), + &(), + ) .await .unwrap(); let error = collector.poll_once(&job).await.unwrap_err(); @@ -1476,7 +1381,7 @@ mod tests { .with_status(200) .with_header( CONTENT_TYPE.as_str(), - CollectionMessage::::MEDIA_TYPE, + CollectionMessage::::MEDIA_TYPE, ) .with_body(b"") .expect_at_least(1) @@ -1487,17 +1392,16 @@ mod tests { assert_matches!(error, Error::Codec(_)); mock_collection_job_bad_message_bytes.assert_async().await; - let mock_collection_job_bad_share_count = server .mock("POST", job.collection_job_url.path()) .with_status(200) .with_header( CONTENT_TYPE.as_str(), - CollectionMessage::::MEDIA_TYPE, + CollectionMessage::::MEDIA_TYPE, ) .with_body( CollectionMessage::new( - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(batch_id), 0, batch_interval, Vec::new(), @@ -1518,11 +1422,11 @@ mod tests { .with_status(200) .with_header( CONTENT_TYPE.as_str(), - CollectionMessage::::MEDIA_TYPE, + CollectionMessage::::MEDIA_TYPE, ) .with_body( CollectionMessage::new( - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(batch_id), 1, batch_interval, Vec::from([ @@ -1551,10 +1455,10 @@ mod tests { let associated_data = AggregateShareAad::new( collector.parameters.task_id, - BatchSelector::new_time_interval(batch_interval), + BatchSelector::new_fixed_size(batch_id), ); let collect_resp = CollectionMessage::new( - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(batch_id), 1, batch_interval, Vec::from([ @@ -1587,7 +1491,7 @@ mod tests { .with_status(200) .with_header( CONTENT_TYPE.as_str(), - CollectionMessage::::MEDIA_TYPE, + CollectionMessage::::MEDIA_TYPE, ) .with_body(collect_resp.get_encoded()) .expect_at_least(1) @@ -1600,7 +1504,7 @@ mod tests { mock_collection_job_bad_shares.assert_async().await; let collect_resp = CollectionMessage::new( - PartialBatchSelector::new_time_interval(), + PartialBatchSelector::new_fixed_size(batch_id), 1, batch_interval, Vec::from([ @@ -1638,7 +1542,7 @@ mod tests { .with_status(200) .with_header( CONTENT_TYPE.as_str(), - CollectionMessage::::MEDIA_TYPE, + CollectionMessage::::MEDIA_TYPE, ) .with_body(collect_resp.get_encoded()) .expect_at_least(1) @@ -1676,19 +1580,18 @@ mod tests { .mock("PUT", matcher.clone()) .match_header( CONTENT_TYPE.as_str(), - CollectionReq::::MEDIA_TYPE, + CollectionReq::::MEDIA_TYPE, ) .with_status(201) .expect(1) .create_async() .await; - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(1_000_000), - Duration::from_seconds(3600), - ) - .unwrap(); + let batch_id = random(); let job = collector - .start_collection(Query::new_time_interval(batch_interval), &()) + .start_collection( + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id }), + &(), + ) .await .unwrap(); mock_collect_start.assert_async().await; @@ -1754,14 +1657,10 @@ mod tests { ); let collection_job_url = format!("{}{collection_job_path}", server.url()); - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(1_000_000), - Duration::from_seconds(3600), - ) - .unwrap(); + let batch_id = random(); let job = CollectionJob::new( collection_job_url.parse().unwrap(), - Query::new_time_interval(batch_interval), + Query::new_fixed_size(FixedSizeQuery::ByBatchId { batch_id }), (), ); diff --git a/docs/samples/tasks.yaml b/docs/samples/tasks.yaml index 1d381cd03..9bf7bde8f 100644 --- a/docs/samples/tasks.yaml +++ b/docs/samples/tasks.yaml @@ -11,8 +11,10 @@ - "https://example.com/" - "https://example.net/" - # The DAP query type. See below for an example of a fixed-size task - query_type: TimeInterval + # For tasks using the fixed size query type, an additional `max_batch_size` + # parameter must be provided. + query_type: !FixedSize + max_batch_size: 100 # The task's VDAF. Each VDAF requires its own set of parameters. vdaf: !Prio3Sum @@ -100,42 +102,3 @@ aead_id: Aes128Gcm public_key: bK5esYAgvRb7eWto4IlzDV5fMpmdTeO0K56sV-rf9xo private_key: wFRYwiypcHC-mkGP1u3XQgIvtnlkQlUfZjgtM_zRsnI - -- task_id: "D-hCKPuqL2oTf7ZVRVyMP5VGt43EAEA8q34mDf6p1JE" - aggregator_endpoints: - - "https://example.org/" - - "https://example.com/" - # For tasks using the fixed size query type, an additional `max_batch_size` - # parameter must be provided. - query_type: !FixedSize - max_batch_size: 100 - vdaf: Prio3Count - role: Helper - vdaf_verify_keys: - - "ZXtE4kLqtsCOr8h_pNUeoQ" - max_batch_query_count: 1 - task_expiration: 1704088800 - report_expiry_age: null - min_batch_size: 100 - time_precision: 300 - tolerable_clock_skew: 60 - collector_hpke_config: - id: 80 - kem_id: X25519HkdfSha256 - kdf_id: HkdfSha256 - aead_id: Aes128Gcm - public_key: KHRLcWgfWxli8cdOLPsgsZPttHXh0ho3vLVLrW-63lE - aggregator_auth_tokens: - - type: "Bearer" - token: "YWdncmVnYXRvci1jZmE4NDMyZjdkMzllMjZiYjU3OGUzMzY5Mzk1MWQzNQ==" - # Note that this task does not have any collector authentication tokens, since - # it is a helper role task. - collector_auth_tokens: [] - hpke_keys: - - config: - id: 37 - kem_id: X25519HkdfSha256 - kdf_id: HkdfSha256 - aead_id: Aes128Gcm - public_key: nvoVceq50ScadLoeE3E4tgFkzF85UfdiEQOPZSKVx0Y - private_key: oA38bVlfuTvi_rg6ciYI1S0tWQuwwDhBDzHSBXKCYVc diff --git a/integration_tests/tests/common/mod.rs b/integration_tests/tests/common/mod.rs index 3542bef9e..8168f62a5 100644 --- a/integration_tests/tests/common/mod.rs +++ b/integration_tests/tests/common/mod.rs @@ -6,9 +6,7 @@ use janus_collector::{ }; use janus_core::{ hpke::test_util::generate_test_hpke_config_and_private_key, - retries::test_http_request_exponential_backoff, - task::VdafInstance, - time::{Clock, RealClock, TimeExt}, + retries::test_http_request_exponential_backoff, task::VdafInstance, }; use janus_integration_tests::{ client::{ClientBackend, ClientImplementation, InteropClientEncoding}, @@ -17,7 +15,7 @@ use janus_integration_tests::{ use janus_messages::{ problem_type::DapProblemType, query_type::{self, FixedSize}, - Duration, FixedSizeQuery, Interval, Query, Role, + FixedSizeQuery, Query, Role, }; use prio::vdaf::{self, prio3::Prio3}; use rand::{random, thread_rng, Rng}; @@ -121,9 +119,7 @@ pub async fn submit_measurements_and_verify_aggregate_generic( V: vdaf::Client<16> + vdaf::Collector + InteropClientEncoding, V::AggregateResult: PartialEq, { - // Submit some measurements, recording a timestamp before measurement upload to allow us to - // determine the correct collect interval. (for time interval tasks) - let before_timestamp = RealClock::default().now(); + // Submit some measurements. for measurement in test_case.measurements.iter() { client_implementation.upload(measurement).await.unwrap(); } @@ -154,32 +150,6 @@ pub async fn submit_measurements_and_verify_aggregate_generic( // Send a collect request and verify that we got the correct result. match &task_parameters.query_type { - QueryType::TimeInterval => { - let batch_interval = Interval::new( - before_timestamp - .to_batch_interval_start(&task_parameters.time_precision) - .unwrap(), - // Use two time precisions as the interval duration in order to avoid a race condition if - // this test happens to run very close to the end of a batch window. - Duration::from_seconds(2 * task_parameters.time_precision.as_seconds()), - ) - .unwrap(); - let collection = collect_generic( - &collector, - Query::new_time_interval(batch_interval), - &test_case.aggregation_parameter, - "127.0.0.1", - leader_port, - ) - .await - .unwrap(); - - assert_eq!( - collection.report_count(), - u64::try_from(test_case.measurements.len()).unwrap() - ); - assert_eq!(collection.aggregate_result(), &test_case.aggregate_result); - } QueryType::FixedSize { .. } => { let mut requests = 0; let collection = loop { diff --git a/integration_tests/tests/daphne.rs b/integration_tests/tests/daphne.rs index e351e6070..3d8289bca 100644 --- a/integration_tests/tests/daphne.rs +++ b/integration_tests/tests/daphne.rs @@ -18,8 +18,10 @@ async fn daphne_janus() { // Start servers. let network = generate_network_name(); - let (mut task_parameters, leader_task, helper_task) = - test_task_builders(VdafInstance::Prio3Count, QueryType::TimeInterval); + let (mut task_parameters, leader_task, helper_task) = test_task_builders( + VdafInstance::Prio3Count, + QueryType::FixedSize { max_batch_size: 46 }, + ); // Daphne is hardcoded to serve from a path starting with /v04/. task_parameters.endpoint_fragments.leader_endpoint_path = "/v04/".to_string(); @@ -54,8 +56,10 @@ async fn janus_daphne() { // Start servers. let network = generate_network_name(); - let (mut task_parameters, leader_task, helper_task) = - test_task_builders(VdafInstance::Prio3Count, QueryType::TimeInterval); + let (mut task_parameters, leader_task, helper_task) = test_task_builders( + VdafInstance::Prio3Count, + QueryType::FixedSize { max_batch_size: 46 }, + ); // Daphne is hardcoded to serve from a path starting with /v04/. task_parameters.endpoint_fragments.helper_endpoint_path = "/v04/".to_string(); diff --git a/integration_tests/tests/divviup_ts.rs b/integration_tests/tests/divviup_ts.rs index 3aecb6648..109c75915 100644 --- a/integration_tests/tests/divviup_ts.rs +++ b/integration_tests/tests/divviup_ts.rs @@ -17,7 +17,7 @@ mod common; async fn run_divviup_ts_integration_test(container_client: &Cli, vdaf: VdafInstance) { let (task_parameters, leader_task, helper_task) = - test_task_builders(vdaf, QueryType::TimeInterval); + test_task_builders(vdaf, QueryType::FixedSize { max_batch_size: 46 }); let network = generate_network_name(); let leader = Janus::new(container_client, &network, &leader_task.build()).await; let helper = Janus::new(container_client, &network, &helper_task.build()).await; diff --git a/integration_tests/tests/in_cluster.rs b/integration_tests/tests/in_cluster.rs index f84ad96a5..c7f7c6680 100644 --- a/integration_tests/tests/in_cluster.rs +++ b/integration_tests/tests/in_cluster.rs @@ -151,7 +151,6 @@ impl InClusterJanusPair { vdaf: task.vdaf().try_into().unwrap(), min_batch_size: task.min_batch_size(), max_batch_size: match task.query_type() { - QueryType::TimeInterval => None, QueryType::FixedSize { max_batch_size } => Some(*max_batch_size), }, expiration: "3000-01-01T00:00:00Z".to_owned(), @@ -227,8 +226,11 @@ async fn in_cluster_count() { install_test_trace_subscriber(); // Start port forwards and set up task. - let janus_pair = - InClusterJanusPair::new(VdafInstance::Prio3Count, QueryType::TimeInterval).await; + let janus_pair = InClusterJanusPair::new( + VdafInstance::Prio3Count, + QueryType::FixedSize { max_batch_size: 46 }, + ) + .await; // Run the behavioral test. submit_measurements_and_verify_aggregate( @@ -244,8 +246,11 @@ async fn in_cluster_sum() { install_test_trace_subscriber(); // Start port forwards and set up task. - let janus_pair = - InClusterJanusPair::new(VdafInstance::Prio3Sum { bits: 16 }, QueryType::TimeInterval).await; + let janus_pair = InClusterJanusPair::new( + VdafInstance::Prio3Sum { bits: 16 }, + QueryType::FixedSize { max_batch_size: 46 }, + ) + .await; // Run the behavioral test. submit_measurements_and_verify_aggregate( @@ -264,7 +269,7 @@ async fn in_cluster_histogram() { let buckets = Vec::from([3, 6, 8]); let janus_pair = InClusterJanusPair::new( VdafInstance::Prio3Histogram { buckets }, - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 46 }, ) .await; diff --git a/integration_tests/tests/janus.rs b/integration_tests/tests/janus.rs index 0eea059be..39f5a2386 100644 --- a/integration_tests/tests/janus.rs +++ b/integration_tests/tests/janus.rs @@ -54,7 +54,7 @@ async fn janus_janus_count() { let janus_pair = JanusPair::new( &container_client, VdafInstance::Prio3Count, - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 46 }, ) .await; @@ -77,7 +77,7 @@ async fn janus_janus_sum_16() { let janus_pair = JanusPair::new( &container_client, VdafInstance::Prio3Sum { bits: 16 }, - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 46 }, ) .await; @@ -102,7 +102,7 @@ async fn janus_janus_histogram_4_buckets() { let janus_pair = JanusPair::new( &container_client, VdafInstance::Prio3Histogram { buckets }, - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 46 }, ) .await; @@ -125,30 +125,7 @@ async fn janus_janus_count_vec_15() { let janus_pair = JanusPair::new( &container_client, VdafInstance::Prio3CountVec { length: 15 }, - QueryType::TimeInterval, - ) - .await; - - // Run the behavioral test. - submit_measurements_and_verify_aggregate( - &janus_pair.task_parameters, - (janus_pair.leader.port(), janus_pair.helper.port()), - &ClientBackend::InProcess, - ) - .await; -} - -/// This test exercises the fixed-size query type with Janus as both the leader and the helper. -#[tokio::test(flavor = "multi_thread")] -async fn janus_janus_fixed_size() { - install_test_trace_subscriber(); - - // Start servers. - let container_client = container_client(); - let janus_pair = JanusPair::new( - &container_client, - VdafInstance::Prio3Count, - QueryType::FixedSize { max_batch_size: 50 }, + QueryType::FixedSize { max_batch_size: 46 }, ) .await; @@ -173,7 +150,7 @@ async fn janus_janus_sum_vec() { bits: 16, length: 15, }, - QueryType::TimeInterval, + QueryType::FixedSize { max_batch_size: 46 }, ) .await; diff --git a/interop_binaries/src/bin/janus_interop_aggregator.rs b/interop_binaries/src/bin/janus_interop_aggregator.rs index a8a96335e..b5b570d42 100644 --- a/interop_binaries/src/bin/janus_interop_aggregator.rs +++ b/interop_binaries/src/bin/janus_interop_aggregator.rs @@ -75,7 +75,6 @@ async fn handle_add_task( let hpke_keypair = keyring.lock().await.get_random_keypair(); let query_type = match request.query_type { - 1 => task::QueryType::TimeInterval, 2 => task::QueryType::FixedSize { max_batch_size: request .max_batch_size diff --git a/interop_binaries/src/bin/janus_interop_collector.rs b/interop_binaries/src/bin/janus_interop_collector.rs index 94f64c7d1..4130d3854 100644 --- a/interop_binaries/src/bin/janus_interop_collector.rs +++ b/interop_binaries/src/bin/janus_interop_collector.rs @@ -19,8 +19,7 @@ use janus_interop_binaries::{ ErrorHandler, HpkeConfigRegistry, NumberAsString, VdafObject, }; use janus_messages::{ - query_type::QueryType, BatchId, Duration, FixedSizeQuery, HpkeConfig, Interval, - PartialBatchSelector, Query, TaskId, Time, + query_type::QueryType, BatchId, FixedSizeQuery, HpkeConfig, PartialBatchSelector, Query, TaskId, }; #[cfg(feature = "fpvec_bounded_l2")] use prio::vdaf::prio3::Prio3FixedPointBoundedL2VecSumMultithreaded; @@ -63,8 +62,6 @@ struct AddTaskResponse { struct RequestQuery { #[serde(rename = "type")] query_type: u8, - batch_interval_start: Option, - batch_interval_duration: Option, subtype: Option, batch_id: Option, } @@ -213,7 +210,6 @@ where } enum ParsedQuery { - TimeInterval(Interval), FixedSize(FixedSizeQuery), } @@ -260,23 +256,6 @@ async fn handle_collection_start( ); let query = match request.query.query_type { - 1 => { - let start = Time::from_seconds_since_epoch( - request - .query - .batch_interval_start - .context("\"batch_interval_start\" was missing")?, - ); - let duration = Duration::from_seconds( - request - .query - .batch_interval_duration - .context("\"batch_interval_duration\" was missing")?, - ); - let interval = - Interval::new(start, duration).context("invalid batch interval specification")?; - ParsedQuery::TimeInterval(interval) - } 2 => match request.query.subtype { Some(0) => { let batch_id_bytes = URL_SAFE_NO_PAD @@ -299,157 +278,6 @@ async fn handle_collection_start( let vdaf_instance = task_state.vdaf.clone().into(); let task_handle = match (query, vdaf_instance) { - (ParsedQuery::TimeInterval(batch_interval), VdafInstance::Prio3Count {}) => { - let vdaf = Prio3::new_count(2).context("failed to construct Prio3Count VDAF")?; - handle_collect_generic( - http_client, - collector_params, - Query::new_time_interval(batch_interval), - vdaf, - &agg_param, - |_| None, - |result| AggregationResult::Number(NumberAsString((*result).into())), - ) - .await? - } - - (ParsedQuery::TimeInterval(batch_interval), VdafInstance::Prio3CountVec { length }) => { - let vdaf = Prio3::new_sum_vec_multithreaded(2, 1, length) - .context("failed to construct Prio3CountVec VDAF")?; - handle_collect_generic( - http_client, - collector_params, - Query::new_time_interval(batch_interval), - vdaf, - &agg_param, - |_| None, - |result| { - let converted = result.iter().cloned().map(NumberAsString).collect(); - AggregationResult::NumberVec(converted) - }, - ) - .await? - } - - (ParsedQuery::TimeInterval(batch_interval), VdafInstance::Prio3Sum { bits }) => { - let vdaf = Prio3::new_sum(2, bits).context("failed to construct Prio3Sum VDAF")?; - handle_collect_generic( - http_client, - collector_params, - Query::new_time_interval(batch_interval), - vdaf, - &agg_param, - |_| None, - |result| AggregationResult::Number(NumberAsString(*result)), - ) - .await? - } - - (ParsedQuery::TimeInterval(batch_interval), VdafInstance::Prio3SumVec { bits, length }) => { - let vdaf = Prio3::new_sum_vec_multithreaded(2, bits, length) - .context("failed to construct Prio3SumVec VDAF")?; - handle_collect_generic( - http_client, - collector_params, - Query::new_time_interval(batch_interval), - vdaf, - &agg_param, - |_| None, - |result| { - let converted = result.iter().cloned().map(NumberAsString).collect(); - AggregationResult::NumberVec(converted) - }, - ) - .await? - } - - (ParsedQuery::TimeInterval(batch_interval), VdafInstance::Prio3Histogram { buckets }) => { - let vdaf = Prio3::new_histogram(2, &buckets) - .context("failed to construct Prio3Histogram VDAF")?; - handle_collect_generic( - http_client, - collector_params, - Query::new_time_interval(batch_interval), - vdaf, - &agg_param, - |_| None, - |result| { - let converted = result.iter().cloned().map(NumberAsString).collect(); - AggregationResult::NumberVec(converted) - }, - ) - .await? - } - - #[cfg(feature = "fpvec_bounded_l2")] - ( - ParsedQuery::TimeInterval(batch_interval), - janus_core::task::VdafInstance::Prio3FixedPoint16BitBoundedL2VecSum { length }, - ) => { - let vdaf: Prio3FixedPointBoundedL2VecSumMultithreaded> = - Prio3::new_fixedpoint_boundedl2_vec_sum_multithreaded(2, length) - .context("failed to construct Prio3FixedPoint16BitBoundedL2VecSum VDAF")?; - handle_collect_generic( - http_client, - collector_params, - Query::new_time_interval(batch_interval), - vdaf, - &agg_param, - |_| None, - |result| { - let converted = result.iter().cloned().map(NumberAsString).collect(); - AggregationResult::FloatVec(converted) - }, - ) - .await? - } - - #[cfg(feature = "fpvec_bounded_l2")] - ( - ParsedQuery::TimeInterval(batch_interval), - janus_core::task::VdafInstance::Prio3FixedPoint32BitBoundedL2VecSum { length }, - ) => { - let vdaf: Prio3FixedPointBoundedL2VecSumMultithreaded> = - Prio3::new_fixedpoint_boundedl2_vec_sum_multithreaded(2, length) - .context("failed to construct Prio3FixedPoint32BitBoundedL2VecSum VDAF")?; - handle_collect_generic( - http_client, - collector_params, - Query::new_time_interval(batch_interval), - vdaf, - &agg_param, - |_| None, - |result| { - let converted = result.iter().cloned().map(NumberAsString).collect(); - AggregationResult::FloatVec(converted) - }, - ) - .await? - } - - #[cfg(feature = "fpvec_bounded_l2")] - ( - ParsedQuery::TimeInterval(batch_interval), - janus_core::task::VdafInstance::Prio3FixedPoint64BitBoundedL2VecSum { length }, - ) => { - let vdaf: Prio3FixedPointBoundedL2VecSumMultithreaded> = - Prio3::new_fixedpoint_boundedl2_vec_sum_multithreaded(2, length) - .context("failed to construct Prio3FixedPoint64BitBoundedL2VecSum VDAF")?; - handle_collect_generic( - http_client, - collector_params, - Query::new_time_interval(batch_interval), - vdaf, - &agg_param, - |_| None, - |result| { - let converted = result.iter().cloned().map(NumberAsString).collect(); - AggregationResult::FloatVec(converted) - }, - ) - .await? - } - (ParsedQuery::FixedSize(fixed_size_query), VdafInstance::Prio3Count {}) => { let vdaf = Prio3::new_count(2).context("failed to construct Prio3Count VDAF")?; handle_collect_generic( diff --git a/interop_binaries/src/lib.rs b/interop_binaries/src/lib.rs index 5347ad64a..41a28b269 100644 --- a/interop_binaries/src/lib.rs +++ b/interop_binaries/src/lib.rs @@ -5,7 +5,7 @@ use janus_core::{ task::VdafInstance, }; use janus_messages::{ - query_type::{FixedSize, QueryType as _, TimeInterval}, + query_type::{FixedSize, QueryType as _}, HpkeAeadId, HpkeConfigId, HpkeKdfId, HpkeKemId, Role, TaskId, Time, }; use prio::codec::Encode; @@ -280,7 +280,6 @@ pub struct AddTaskResponse { impl From for AggregatorAddTaskRequest { fn from(task: Task) -> Self { let (query_type, max_batch_size) = match task.query_type() { - QueryType::TimeInterval => (TimeInterval::CODE as u8, None), QueryType::FixedSize { max_batch_size } => { (FixedSize::CODE as u8, Some(*max_batch_size)) } diff --git a/interop_binaries/tests/end_to_end.rs b/interop_binaries/tests/end_to_end.rs index 50cbf21fe..9cbeb7b65 100644 --- a/interop_binaries/tests/end_to_end.rs +++ b/interop_binaries/tests/end_to_end.rs @@ -4,7 +4,6 @@ use futures::future::join_all; use janus_core::{ task::PRIO3_VERIFY_KEY_LENGTH, test_util::{install_test_trace_subscriber, testcontainers::container_client}, - time::{Clock, RealClock, TimeExt}, }; use janus_interop_binaries::{ test_util::{await_ready_ok, generate_network_name, generate_unique_name}, @@ -12,8 +11,8 @@ use janus_interop_binaries::{ ContainerLogsDropGuard, }; use janus_messages::{ - query_type::{FixedSize, QueryType, TimeInterval}, - Duration, TaskId, + query_type::{FixedSize, QueryType}, + TaskId, }; use prio::codec::Encode; use rand::random; @@ -30,7 +29,6 @@ const JSON_MEDIA_TYPE: &str = "application/json"; const TIME_PRECISION: u64 = 3600; enum QueryKind { - TimeInterval, FixedSize, } @@ -46,10 +44,6 @@ async fn run( install_test_trace_subscriber(); let (query_type_json, max_batch_size) = match query_kind { - QueryKind::TimeInterval => { - let query_type = json!(TimeInterval::CODE as u8); - (query_type, None) - } QueryKind::FixedSize => { let query_type = json!(FixedSize::CODE as u8); (query_type, Some(json!(measurements.len()))) @@ -361,9 +355,6 @@ async fn run( helper_add_task_response_object.get("error"), ); - // Record the time before generating reports, for use in calculating time interval queries. - let start_timestamp = RealClock::default().now(); - // Send one or more /internal/test/upload requests to the client. for measurement in measurements { let upload_response = http_client @@ -399,20 +390,6 @@ async fn run( } let query_json = match query_kind { - QueryKind::TimeInterval => { - let batch_interval_start = start_timestamp - .to_batch_interval_start(&Duration::from_seconds(TIME_PRECISION)) - .unwrap() - .as_seconds_since_epoch(); - // Span the aggregation over two time precisions, just in case our measurements spilled over a - // batch boundary. - let batch_interval_duration = TIME_PRECISION * 2; - json!({ - "type": query_type_json, - "batch_interval_start": batch_interval_start, - "batch_interval_duration": batch_interval_duration, - }) - } QueryKind::FixedSize => { json!({ "type": query_type_json, @@ -427,7 +404,6 @@ async fn run( .with_initial_interval(StdDuration::from_secs(1)) .with_max_interval(StdDuration::from_secs(1)) .with_max_elapsed_time(match query_kind { - QueryKind::TimeInterval => Some(StdDuration::from_secs(0)), QueryKind::FixedSize => Some(StdDuration::from_secs(15)), }) .build(); @@ -536,14 +512,12 @@ async fn run( "error: {:?}", collection_poll_response_object.get("error"), ); - if let QueryKind::FixedSize = query_kind { - let batch_id_encoded = collection_poll_response_object - .get("batch_id") - .expect("completed collection_poll response is missing \"batch_id\"") - .as_str() - .expect("\"batch_id\" value is not a string"); - URL_SAFE_NO_PAD.decode(batch_id_encoded).unwrap(); - } + let batch_id_encoded = collection_poll_response_object + .get("batch_id") + .expect("completed collection_poll response is missing \"batch_id\"") + .as_str() + .expect("\"batch_id\" value is not a string"); + URL_SAFE_NO_PAD.decode(batch_id_encoded).unwrap(); collection_poll_response_object .get("report_count") .expect("completed collection_poll response is missing \"report_count\"") @@ -569,7 +543,7 @@ async fn run( #[tokio::test] async fn e2e_prio3_count() { let result = run( - QueryKind::TimeInterval, + QueryKind::FixedSize, json!({"type": "Prio3Count"}), &[ json!("0"), @@ -600,7 +574,7 @@ async fn e2e_prio3_count() { #[tokio::test] async fn e2e_prio3_sum() { let result = run( - QueryKind::TimeInterval, + QueryKind::FixedSize, json!({"type": "Prio3Sum", "bits": "64"}), &[ json!("0"), @@ -620,7 +594,7 @@ async fn e2e_prio3_sum() { #[tokio::test] async fn e2e_prio3_sum_vec() { let result = run( - QueryKind::TimeInterval, + QueryKind::FixedSize, json!({"type": "Prio3SumVec", "bits": "64", "length": "4"}), &[ json!(["0", "0", "0", "10"]), @@ -639,7 +613,7 @@ async fn e2e_prio3_sum_vec() { #[tokio::test] async fn e2e_prio3_histogram() { let result = run( - QueryKind::TimeInterval, + QueryKind::FixedSize, json!({ "type": "Prio3Histogram", "buckets": ["0", "1", "10", "100", "1000", "10000", "100000"], @@ -670,7 +644,7 @@ async fn e2e_prio3_histogram() { #[tokio::test] async fn e2e_prio3_count_vec() { let result = run( - QueryKind::TimeInterval, + QueryKind::FixedSize, json!({"type": "Prio3CountVec", "length": "4"}), &[ json!(["0", "0", "0", "1"]), @@ -691,114 +665,6 @@ async fn e2e_prio3_count_vec() { #[tokio::test] async fn e2e_prio3_fixed16vec() { - let fp16_4_inv = fixed!(0.25: I1F15); - let fp16_8_inv = fixed!(0.125: I1F15); - let fp16_16_inv = fixed!(0.0625: I1F15); - let result = run( - QueryKind::TimeInterval, - json!({"type": "Prio3FixedPoint16BitBoundedL2VecSum", "length": "3"}), - &[ - json!([ - fp16_4_inv.to_string(), - fp16_8_inv.to_string(), - fp16_8_inv.to_string() - ]), - json!([ - fp16_16_inv.to_string(), - fp16_8_inv.to_string(), - fp16_16_inv.to_string() - ]), - json!([ - fp16_8_inv.to_string(), - fp16_8_inv.to_string(), - fp16_4_inv.to_string() - ]), - json!([ - fp16_16_inv.to_string(), - fp16_8_inv.to_string(), - fp16_4_inv.to_string() - ]), - ], - b"", - ) - .await; - assert_eq!(result, json!(["0.5", "0.5", "0.6875"])); -} - -#[tokio::test] -async fn e2e_prio3_fixed32vec() { - let fp32_4_inv = fixed!(0.25: I1F31); - let fp32_8_inv = fixed!(0.125: I1F31); - let fp32_16_inv = fixed!(0.0625: I1F31); - let result = run( - QueryKind::TimeInterval, - json!({"type": "Prio3FixedPoint32BitBoundedL2VecSum", "length": "3"}), - &[ - json!([ - fp32_4_inv.to_string(), - fp32_8_inv.to_string(), - fp32_8_inv.to_string() - ]), - json!([ - fp32_16_inv.to_string(), - fp32_8_inv.to_string(), - fp32_16_inv.to_string() - ]), - json!([ - fp32_8_inv.to_string(), - fp32_8_inv.to_string(), - fp32_4_inv.to_string() - ]), - json!([ - fp32_16_inv.to_string(), - fp32_8_inv.to_string(), - fp32_4_inv.to_string() - ]), - ], - b"", - ) - .await; - assert_eq!(result, json!(["0.5", "0.5", "0.6875"])); -} - -#[tokio::test] -async fn e2e_prio3_fixed64vec() { - let fp64_4_inv = fixed!(0.25: I1F63); - let fp64_8_inv = fixed!(0.125: I1F63); - let fp64_16_inv = fixed!(0.0625: I1F63); - let result = run( - QueryKind::TimeInterval, - json!({"type": "Prio3FixedPoint64BitBoundedL2VecSum", "length": "3"}), - &[ - json!([ - fp64_4_inv.to_string(), - fp64_8_inv.to_string(), - fp64_8_inv.to_string() - ]), - json!([ - fp64_16_inv.to_string(), - fp64_8_inv.to_string(), - fp64_16_inv.to_string() - ]), - json!([ - fp64_8_inv.to_string(), - fp64_8_inv.to_string(), - fp64_4_inv.to_string() - ]), - json!([ - fp64_16_inv.to_string(), - fp64_8_inv.to_string(), - fp64_4_inv.to_string() - ]), - ], - b"", - ) - .await; - assert_eq!(result, json!(["0.5", "0.5", "0.6875"])); -} - -#[tokio::test] -async fn e2e_prio3_fixed16vec_fixed_size() { let fp16_4_inv = fixed!(0.25: I1F15); let fp16_8_inv = fixed!(0.125: I1F15); let fp16_16_inv = fixed!(0.0625: I1F15); @@ -834,7 +700,7 @@ async fn e2e_prio3_fixed16vec_fixed_size() { } #[tokio::test] -async fn e2e_prio3_fixed32vec_fixed_size() { +async fn e2e_prio3_fixed32vec() { let fp32_4_inv = fixed!(0.25: I1F31); let fp32_8_inv = fixed!(0.125: I1F31); let fp32_16_inv = fixed!(0.0625: I1F31); @@ -870,7 +736,7 @@ async fn e2e_prio3_fixed32vec_fixed_size() { } #[tokio::test] -async fn e2e_prio3_fixed64vec_fixed_size() { +async fn e2e_prio3_fixed64vec() { let fp64_4_inv = fixed!(0.25: I1F63); let fp64_8_inv = fixed!(0.125: I1F63); let fp64_16_inv = fixed!(0.0625: I1F63); @@ -904,26 +770,3 @@ async fn e2e_prio3_fixed64vec_fixed_size() { .await; assert_eq!(result, json!(["0.5", "0.5", "0.6875"])); } - -#[tokio::test] -async fn e2e_prio3_count_fixed_size() { - let result = run( - QueryKind::FixedSize, - json!({"type": "Prio3Count"}), - &[ - json!("0"), - json!("1"), - json!("1"), - json!("1"), - json!("0"), - json!("1"), - json!("0"), - json!("1"), - json!("0"), - json!("0"), - ], - b"", - ) - .await; - assert!(result.is_string()); -} diff --git a/messages/src/lib.rs b/messages/src/lib.rs index d59af2684..4a170d7cf 100644 --- a/messages/src/lib.rs +++ b/messages/src/lib.rs @@ -3,7 +3,7 @@ //! //! [dap]: https://datatracker.ietf.org/doc/draft-ietf-ppm-dap/ -use self::query_type::{FixedSize, QueryType, TimeInterval}; +use self::query_type::{FixedSize, QueryType}; use anyhow::anyhow; use base64::{display::Base64Display, engine::general_purpose::URL_SAFE_NO_PAD, Engine}; use derivative::Derivative; @@ -1440,18 +1440,6 @@ impl Query { } } -impl Query { - /// Constructs a new query for a time-interval task. - pub fn new_time_interval(batch_interval: Interval) -> Self { - Self::new(batch_interval) - } - - /// Gets the batch interval associated with this query. - pub fn batch_interval(&self) -> &Interval { - self.query_body() - } -} - impl Query { /// Constructs a new query for a fixed-size task. pub fn new_fixed_size(fixed_size_query: FixedSizeQuery) -> Self { @@ -1566,13 +1554,6 @@ impl PartialBatchSelector { } } -impl PartialBatchSelector { - /// Constructs a new partial batch selector for a time-interval task. - pub fn new_time_interval() -> Self { - Self::new(()) - } -} - impl PartialBatchSelector { /// Constructs a new partial batch selector for a fixed-size task. pub fn new_fixed_size(batch_id: BatchId) -> Self { @@ -1868,7 +1849,7 @@ impl Decode for AggregateShareAad { pub mod query_type { use crate::{Collection, FixedSizeQuery, Query}; - use super::{BatchId, Interval}; + use super::BatchId; use anyhow::anyhow; use num_enum::TryFromPrimitive; use prio::codec::{CodecError, Decode, Encode}; @@ -1925,29 +1906,6 @@ pub mod query_type { ) -> Self::BatchIdentifier; } - /// Represents a `time-interval` DAP query type. - #[derive(Copy, Clone, Debug, PartialEq, Eq)] - pub struct TimeInterval; - - impl QueryType for TimeInterval { - const CODE: Code = Code::TimeInterval; - - type BatchIdentifier = Interval; - type PartialBatchIdentifier = (); - type QueryBody = Interval; - - fn partial_batch_identifier(_: &Self::BatchIdentifier) -> &Self::PartialBatchIdentifier { - &() - } - - fn batch_identifier_for_collection( - query: &Query, - _: &Collection, - ) -> Self::BatchIdentifier { - *query.batch_interval() - } - } - /// Represents a `fixed-size` DAP query type. #[derive(Copy, Clone, Debug, PartialEq, Eq)] pub struct FixedSize; @@ -2534,18 +2492,6 @@ impl BatchSelector { } } -impl BatchSelector { - /// Constructs a new batch selector for time-interval tasks. - pub fn new_time_interval(batch_interval: Interval) -> Self { - Self::new(batch_interval) - } - - /// Gets the batch interval associated with this batch selector. - pub fn batch_interval(&self) -> &Interval { - self.batch_identifier() - } -} - impl BatchSelector { /// Constructs a new batch selector for fixed-size tasks. pub fn new_fixed_size(batch_id: BatchId) -> Self { @@ -2745,7 +2691,7 @@ mod tests { HpkeConfig, HpkeConfigId, HpkeKdfId, HpkeKemId, HpkePublicKey, InputShareAad, Interval, PartialBatchSelector, PlaintextInputShare, PrepareStep, PrepareStepResult, Query, Report, ReportId, ReportIdChecksum, ReportMetadata, ReportShare, ReportShareError, Role, TaskId, - Time, TimeInterval, Url, + Time, Url, }; use assert_matches::assert_matches; use prio::codec::{CodecError, Decode, Encode}; @@ -3352,44 +3298,6 @@ mod tests { #[test] fn roundtrip_query() { - // TimeInterval. - roundtrip_encoding(&[ - ( - Query:: { - query_body: Interval::new( - Time::from_seconds_since_epoch(54321), - Duration::from_seconds(12345), - ) - .unwrap(), - }, - concat!( - "01", // query_type - concat!( - // query_body - "000000000000D431", // start - "0000000000003039", // duration - ), - ), - ), - ( - Query:: { - query_body: Interval::new( - Time::from_seconds_since_epoch(48913), - Duration::from_seconds(44721), - ) - .unwrap(), - }, - concat!( - "01", // query_type - concat!( - // query_body - "000000000000BF11", // start - "000000000000AEB1", // duration - ), - ), - ), - ]); - // FixedSize. roundtrip_encoding(&[ ( @@ -3424,67 +3332,6 @@ mod tests { #[test] fn roundtrip_collection_req() { - // TimeInterval. - roundtrip_encoding(&[ - ( - CollectionReq:: { - query: Query { - query_body: Interval::new( - Time::from_seconds_since_epoch(54321), - Duration::from_seconds(12345), - ) - .unwrap(), - }, - aggregation_parameter: Vec::new(), - }, - concat!( - concat!( - // query - "01", // query_type - concat!( - // query_body - "000000000000D431", // start - "0000000000003039", // duration - ), - ), - concat!( - // aggregation_parameter - "00000000", // length - "", // opaque data - ), - ), - ), - ( - CollectionReq:: { - query: Query { - query_body: Interval::new( - Time::from_seconds_since_epoch(48913), - Duration::from_seconds(44721), - ) - .unwrap(), - }, - aggregation_parameter: Vec::from("012345"), - }, - concat!( - concat!( - // query - "01", // query_type - concat!( - // batch_interval - "000000000000BF11", // start - "000000000000AEB1", // duration - ), - ), - concat!( - // aggregation_parameter - "00000006", // length - "303132333435", // opaque data - ), - ), - ), - ]); - - // FixedSize. roundtrip_encoding(&[ ( CollectionReq:: { @@ -3538,15 +3385,6 @@ mod tests { #[test] fn roundtrip_partial_batch_selector() { - // TimeInterval. - roundtrip_encoding(&[( - PartialBatchSelector::new_time_interval(), - concat!( - "01", // query_type - ), - )]); - - // FixedSize. roundtrip_encoding(&[ ( PartialBatchSelector::new_fixed_size(BatchId::from([3u8; 32])), @@ -3571,96 +3409,6 @@ mod tests { start: Time::from_seconds_since_epoch(54321), duration: Duration::from_seconds(12345), }; - // TimeInterval. - roundtrip_encoding(&[ - ( - Collection { - partial_batch_selector: PartialBatchSelector::new_time_interval(), - report_count: 0, - interval, - encrypted_aggregate_shares: Vec::new(), - }, - concat!( - concat!( - // partial_batch_selector - "01", // query_type - ), - "0000000000000000", // report_count - concat!( - // interval - "000000000000D431", // start - "0000000000003039", // duration - ), - concat!( - // encrypted_aggregate_shares - "00000000", // length - ) - ), - ), - ( - Collection { - partial_batch_selector: PartialBatchSelector::new_time_interval(), - report_count: 23, - interval, - encrypted_aggregate_shares: Vec::from([ - HpkeCiphertext::new( - HpkeConfigId::from(10), - Vec::from("0123"), - Vec::from("4567"), - ), - HpkeCiphertext::new( - HpkeConfigId::from(12), - Vec::from("01234"), - Vec::from("567"), - ), - ]), - }, - concat!( - concat!( - // partial_batch_selector - "01", // query_type - ), - "0000000000000017", // report_count - concat!( - // interval - "000000000000D431", // start - "0000000000003039", // duration - ), - concat!( - // encrypted_aggregate_shares - "0000001E", // length - concat!( - "0A", // config_id - concat!( - // encapsulated_context - "0004", // length - "30313233", // opaque data - ), - concat!( - // payload - "00000004", // length - "34353637", // opaque data - ), - ), - concat!( - "0C", // config_id - concat!( - // encapsulated_context - "0005", // length - "3031323334", // opaque data - ), - concat!( - // payload - "00000003", // length - "353637", // opaque data - ), - ) - ) - ), - ), - ]); - - // FixedSize. roundtrip_encoding(&[ ( Collection { @@ -3825,107 +3573,6 @@ mod tests { #[test] fn roundtrip_aggregation_job_initialize_req() { - // TimeInterval. - roundtrip_encoding(&[( - AggregationJobInitializeReq { - aggregation_parameter: Vec::from("012345"), - partial_batch_selector: PartialBatchSelector::new_time_interval(), - report_shares: Vec::from([ - ReportShare { - metadata: ReportMetadata::new( - ReportId::from([1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16]), - Time::from_seconds_since_epoch(54321), - ), - public_share: Vec::new(), - encrypted_input_share: HpkeCiphertext::new( - HpkeConfigId::from(42), - Vec::from("012345"), - Vec::from("543210"), - ), - }, - ReportShare { - metadata: ReportMetadata::new( - ReportId::from([16, 15, 14, 13, 12, 11, 10, 9, 8, 7, 6, 5, 4, 3, 2, 1]), - Time::from_seconds_since_epoch(73542), - ), - public_share: Vec::from("0123"), - encrypted_input_share: HpkeCiphertext::new( - HpkeConfigId::from(13), - Vec::from("abce"), - Vec::from("abfd"), - ), - }, - ]), - }, - concat!( - concat!( - // aggregation_parameter - "00000006", // length - "303132333435", // opaque data - ), - concat!( - // partial_batch_selector - "01", // query_type - ), - concat!( - // report_shares - "0000005E", // length - concat!( - concat!( - // metadata - "0102030405060708090A0B0C0D0E0F10", // report_id - "000000000000D431", // time - ), - concat!( - // public_share - "00000000", // length - "", // opaque data - ), - concat!( - // encrypted_input_share - "2A", // config_id - concat!( - // encapsulated_context - "0006", // length - "303132333435", // opaque data - ), - concat!( - // payload - "00000006", // length - "353433323130", // opaque data - ), - ), - ), - concat!( - concat!( - // metadata - "100F0E0D0C0B0A090807060504030201", // report_id - "0000000000011F46", // time - ), - concat!( - "00000004", // payload - "30313233", // opaque data - ), - concat!( - // encrypted_input_share - "0D", // config_id - concat!( - // encapsulated_context - "0004", // length - "61626365", // opaque data - ), - concat!( - // payload - "00000004", // length - "61626664", // opaque data - ), - ), - ), - ), - ), - )]); - - // FixedSize. roundtrip_encoding(&[( AggregationJobInitializeReq:: { aggregation_parameter: Vec::from("012345"), @@ -4114,45 +3761,6 @@ mod tests { #[test] fn roundtrip_batch_selector() { - // TimeInterval. - roundtrip_encoding(&[ - ( - BatchSelector:: { - batch_identifier: Interval::new( - Time::from_seconds_since_epoch(54321), - Duration::from_seconds(12345), - ) - .unwrap(), - }, - concat!( - "01", // query_type - concat!( - // batch_interval - "000000000000D431", // start - "0000000000003039", // duration - ), - ), - ), - ( - BatchSelector:: { - batch_identifier: Interval::new( - Time::from_seconds_since_epoch(50821), - Duration::from_seconds(84354), - ) - .unwrap(), - }, - concat!( - "01", // query_type - concat!( - // batch_interval - "000000000000C685", // start - "0000000000014982", // duration - ), - ), - ), - ]); - - // FixedSize. roundtrip_encoding(&[ ( BatchSelector:: { @@ -4178,75 +3786,6 @@ mod tests { #[test] fn roundtrip_aggregate_share_req() { - // TimeInterval. - roundtrip_encoding(&[ - ( - AggregateShareReq:: { - batch_selector: BatchSelector { - batch_identifier: Interval::new( - Time::from_seconds_since_epoch(54321), - Duration::from_seconds(12345), - ) - .unwrap(), - }, - aggregation_parameter: Vec::new(), - report_count: 439, - checksum: ReportIdChecksum::get_decoded(&[u8::MIN; 32]).unwrap(), - }, - concat!( - concat!( - // batch_selector - "01", // query_type - concat!( - // batch_interval - "000000000000D431", // start - "0000000000003039", // duration - ), - ), - concat!( - // aggregation_parameter - "00000000", // length - "", // opaque data - ), - "00000000000001B7", // report_count - "0000000000000000000000000000000000000000000000000000000000000000", // checksum - ), - ), - ( - AggregateShareReq:: { - batch_selector: BatchSelector { - batch_identifier: Interval::new( - Time::from_seconds_since_epoch(50821), - Duration::from_seconds(84354), - ) - .unwrap(), - }, - aggregation_parameter: Vec::from("012345"), - report_count: 8725, - checksum: ReportIdChecksum::get_decoded(&[u8::MAX; 32]).unwrap(), - }, - concat!( - concat!( - // batch_selector - "01", // query_type - concat!( - // batch_interval - "000000000000C685", // start - "0000000000014982", // duration - ), - ), - concat!( - // aggregation_parameter - "00000006", // length - "303132333435", // opaque data - ), - "0000000000002215", // report_count - "FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF", // checksum - ), - ), - ]); - - // FixedSize. roundtrip_encoding(&[ ( AggregateShareReq:: { @@ -4379,33 +3918,6 @@ mod tests { #[test] fn roundtrip_aggregate_share_aad() { - // TimeInterval. - roundtrip_encoding(&[( - AggregateShareAad:: { - task_id: TaskId::from([12u8; 32]), - batch_selector: BatchSelector { - batch_identifier: Interval::new( - Time::from_seconds_since_epoch(54321), - Duration::from_seconds(12345), - ) - .unwrap(), - }, - }, - concat!( - "0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C0C", // task_id - concat!( - // batch_selector - "01", // query_type - concat!( - // batch_interval - "000000000000D431", // start - "0000000000003039", // duration - ), - ), - ), - )]); - - // FixedSize. roundtrip_encoding(&[( AggregateShareAad:: { task_id: TaskId::from([u8::MIN; 32]), diff --git a/tools/src/bin/collect.rs b/tools/src/bin/collect.rs index 80991f553..a9dd85a7d 100644 --- a/tools/src/bin/collect.rs +++ b/tools/src/bin/collect.rs @@ -15,9 +15,8 @@ use fixed::{FixedI16, FixedI32, FixedI64}; use janus_collector::{default_http_client, AuthenticationToken, Collector, CollectorParameters}; use janus_core::{hpke::HpkePrivateKey, task::DapAuthToken}; use janus_messages::{ - query_type::{FixedSize, QueryType, TimeInterval}, - BatchId, Duration, FixedSizeQuery, HpkeConfig, Interval, PartialBatchSelector, Query, TaskId, - Time, + query_type::{FixedSize, QueryType}, + BatchId, FixedSizeQuery, HpkeConfig, PartialBatchSelector, Query, TaskId, }; #[cfg(feature = "fpvec_bounded_l2")] use prio::vdaf::prio3::Prio3FixedPointBoundedL2VecSumMultithreaded; @@ -283,26 +282,11 @@ struct AuthenticationOptions { #[derive(Debug, Args, PartialEq, Eq)] #[group(required = true)] struct QueryOptions { - /// Start of the collection batch interval, as the number of seconds since the Unix epoch - #[clap( - long, - requires = "batch_interval_duration", - help_heading = "Collect Request Parameters (Time Interval)" - )] - batch_interval_start: Option, - /// Duration of the collection batch interval, in seconds - #[clap( - long, - requires = "batch_interval_start", - help_heading = "Collect Request Parameters (Time Interval)" - )] - batch_interval_duration: Option, - /// Batch identifier, encoded with base64url #[clap( long, value_parser = BatchIdValueParser::new(), - conflicts_with_all = ["batch_interval_start", "batch_interval_duration", "current_batch"], + conflicts_with_all = ["current_batch"], help_heading = "Collect Request Parameters (Fixed Size)", )] batch_id: Option, @@ -310,7 +294,7 @@ struct QueryOptions { #[clap( long, action = ArgAction::SetTrue, - conflicts_with_all = ["batch_interval_start", "batch_interval_duration", "batch_id"], + conflicts_with_all = ["batch_id"], help_heading = "Collect Request Parameters (Fixed Size)", )] current_batch: bool, @@ -410,21 +394,8 @@ async fn main() -> anyhow::Result<()> { // This function is broken out from `main()` for the sake of testing its argument handling. async fn run(options: Options) -> Result<(), Error> { - match ( - &options.query.batch_interval_start, - &options.query.batch_interval_duration, - &options.query.batch_id, - options.query.current_batch, - ) { - (Some(batch_interval_start), Some(batch_interval_duration), None, false) => { - let batch_interval = Interval::new( - Time::from_seconds_since_epoch(*batch_interval_start), - Duration::from_seconds(*batch_interval_duration), - ) - .map_err(|err| Error::Anyhow(err.into()))?; - run_with_query(options, Query::new_time_interval(batch_interval)).await - } - (None, None, Some(batch_id), false) => { + match (&options.query.batch_id, options.query.current_batch) { + (Some(batch_id), false) => { let batch_id = *batch_id; run_with_query( options, @@ -432,7 +403,7 @@ async fn run(options: Options) -> Result<(), Error> { ) .await } - (None, None, None, true) => { + (None, true) => { run_with_query(options, Query::new_fixed_size(FixedSizeQuery::CurrentBatch)).await } _ => unreachable!(), @@ -584,14 +555,6 @@ trait QueryTypeExt: QueryType { -> String; } -impl QueryTypeExt for TimeInterval { - const IS_PARTIAL_BATCH_SELECTOR_TRIVIAL: bool = true; - - fn format_partial_batch_selector(_: &PartialBatchSelector) -> String { - "()".to_string() - } -} - impl QueryTypeExt for FixedSize { const IS_PARTIAL_BATCH_SELECTOR_TRIVIAL: bool = false; @@ -605,10 +568,8 @@ impl QueryTypeExt for FixedSize { #[cfg(test)] mod tests { use crate::{ - run, AuthenticationOptions, AuthenticationToken, DapAuthToken, Error, Options, - QueryOptions, VdafType, + AuthenticationOptions, AuthenticationToken, DapAuthToken, Options, QueryOptions, VdafType, }; - use assert_matches::assert_matches; use base64::{engine::general_purpose::URL_SAFE_NO_PAD, Engine}; use clap::{error::ErrorKind, CommandFactory, Parser}; use janus_core::hpke::test_util::generate_test_hpke_config_and_private_key; @@ -622,261 +583,6 @@ mod tests { Options::command().debug_assert(); } - #[tokio::test] - async fn argument_handling() { - let hpke_keypair = generate_test_hpke_config_and_private_key(); - let encoded_hpke_config = URL_SAFE_NO_PAD.encode(hpke_keypair.config().get_encoded()); - let encoded_private_key = URL_SAFE_NO_PAD.encode(hpke_keypair.private_key().as_ref()); - - let task_id = random(); - let leader = Url::parse("https://example.com/dap/").unwrap(); - - let expected = Options { - task_id, - leader: leader.clone(), - authentication: AuthenticationOptions { - dap_auth_token: Some(AuthenticationToken::DapAuth( - DapAuthToken::try_from(b"collector-authentication-token".to_vec()).unwrap(), - )), - authorization_bearer_token: None, - }, - hpke_config: hpke_keypair.config().clone(), - hpke_private_key: hpke_keypair.private_key().clone(), - vdaf: VdafType::Count, - length: None, - bits: None, - buckets: None, - query: QueryOptions { - batch_interval_start: Some(1_000_000), - batch_interval_duration: Some(1_000), - batch_id: None, - current_batch: false, - }, - }; - let task_id_encoded = URL_SAFE_NO_PAD.encode(task_id.get_encoded()); - let correct_arguments = [ - "collect", - &format!("--task-id={task_id_encoded}"), - "--leader", - leader.as_str(), - "--dap-auth-token", - "collector-authentication-token", - &format!("--hpke-config={encoded_hpke_config}"), - &format!("--hpke-private-key={encoded_private_key}"), - "--vdaf", - "count", - "--batch-interval-start", - "1000000", - "--batch-interval-duration", - "1000", - ]; - match Options::try_parse_from(correct_arguments) { - Ok(got) => assert_eq!(got, expected), - Err(e) => panic!("{}\narguments were {:?}", e, correct_arguments), - } - - assert_eq!( - Options::try_parse_from(["collect"]).unwrap_err().kind(), - ErrorKind::MissingRequiredArgument, - ); - - let mut bad_arguments = correct_arguments; - bad_arguments[1] = "--task-id=not valid base64"; - assert_eq!( - Options::try_parse_from(bad_arguments).unwrap_err().kind(), - ErrorKind::ValueValidation, - ); - - let mut bad_arguments = correct_arguments; - let short_encoded = URL_SAFE_NO_PAD.encode("too short"); - let bad_argument = format!("--task-id={short_encoded}"); - bad_arguments[1] = &bad_argument; - assert_eq!( - Options::try_parse_from(bad_arguments).unwrap_err().kind(), - ErrorKind::ValueValidation, - ); - - let mut bad_arguments = correct_arguments; - bad_arguments[3] = "http:bad:url:///dap@"; - assert_eq!( - Options::try_parse_from(bad_arguments).unwrap_err().kind(), - ErrorKind::ValueValidation, - ); - - let mut bad_arguments = correct_arguments; - bad_arguments[6] = "--hpke-config=not valid base64"; - assert_eq!( - Options::try_parse_from(bad_arguments).unwrap_err().kind(), - ErrorKind::ValueValidation, - ); - - let mut bad_arguments = correct_arguments; - bad_arguments[7] = "--hpke-private-key=not valid base64"; - assert_eq!( - Options::try_parse_from(bad_arguments).unwrap_err().kind(), - ErrorKind::ValueValidation, - ); - - let base_arguments = Vec::from([ - "collect".to_string(), - format!("--task-id={task_id_encoded}"), - "--leader".to_string(), - leader.to_string(), - "--dap-auth-token".to_string(), - "collector-authentication-token".to_string(), - format!("--hpke-config={encoded_hpke_config}"), - format!("--hpke-private-key={encoded_private_key}"), - "--batch-interval-start".to_string(), - "1000000".to_string(), - "--batch-interval-duration".to_string(), - "1000".to_string(), - ]); - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend(["--vdaf=count".to_string(), "--buckets=1,2,3,4".to_string()]); - let bad_options = Options::try_parse_from(bad_arguments).unwrap(); - assert_matches!( - run(bad_options).await.unwrap_err(), - Error::Clap(err) => assert_eq!(err.kind(), ErrorKind::ArgumentConflict) - ); - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend(["--vdaf=sum".to_string(), "--buckets=1,2,3,4".to_string()]); - let bad_options = Options::try_parse_from(bad_arguments).unwrap(); - assert_matches!( - run(bad_options).await.unwrap_err(), - Error::Clap(err) => assert_eq!(err.kind(), ErrorKind::ArgumentConflict) - ); - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend([ - "--vdaf=countvec".to_string(), - "--buckets=1,2,3,4".to_string(), - ]); - let bad_options = Options::try_parse_from(bad_arguments).unwrap(); - assert_matches!( - run(bad_options).await.unwrap_err(), - Error::Clap(err) => assert_eq!(err.kind(), ErrorKind::ArgumentConflict) - ); - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend(["--vdaf=countvec".to_string(), "--bits=3".to_string()]); - let bad_options = Options::try_parse_from(bad_arguments).unwrap(); - assert_matches!( - run(bad_options).await.unwrap_err(), - Error::Clap(err) => assert_eq!(err.kind(), ErrorKind::ArgumentConflict) - ); - - #[cfg(feature = "fpvec_bounded_l2")] - { - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend([ - "--vdaf=fixedpoint16bitboundedl2vecsum".to_string(), - "--bits=3".to_string(), - ]); - let bad_options = Options::try_parse_from(bad_arguments).unwrap(); - assert_matches!( - run(bad_options).await.unwrap_err(), - Error::Clap(err) => assert_eq!(err.kind(), ErrorKind::ArgumentConflict) - ); - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend([ - "--vdaf=fixedpoint32bitboundedl2vecsum".to_string(), - "--bits=3".to_string(), - ]); - let bad_options = Options::try_parse_from(bad_arguments).unwrap(); - assert_matches!( - run(bad_options).await.unwrap_err(), - Error::Clap(err) => assert_eq!(err.kind(), ErrorKind::ArgumentConflict) - ); - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend([ - "--vdaf=fixedpoint64bitboundedl2vecsum".to_string(), - "--bits=3".to_string(), - ]); - let bad_options = Options::try_parse_from(bad_arguments).unwrap(); - assert_matches!( - run(bad_options).await.unwrap_err(), - Error::Clap(err) => assert_eq!(err.kind(), ErrorKind::ArgumentConflict) - ); - } - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend([ - "--vdaf=histogram".to_string(), - "--buckets=1,2,3,4,apple".to_string(), - ]); - assert_eq!( - Options::try_parse_from(bad_arguments).unwrap_err().kind(), - ErrorKind::ValueValidation - ); - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend(["--vdaf=histogram".to_string()]); - let bad_options = Options::try_parse_from(bad_arguments).unwrap(); - assert_matches!( - run(bad_options).await.unwrap_err(), - Error::Clap(err) => assert_eq!(err.kind(), ErrorKind::ArgumentConflict) - ); - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend(["--vdaf=sum".to_string()]); - let bad_options = Options::try_parse_from(bad_arguments).unwrap(); - assert_matches!( - run(bad_options).await.unwrap_err(), - Error::Clap(err) => assert_eq!(err.kind(), ErrorKind::ArgumentConflict) - ); - - let mut good_arguments = base_arguments.clone(); - good_arguments.extend(["--vdaf=countvec".to_string(), "--length=10".to_string()]); - Options::try_parse_from(good_arguments).unwrap(); - - let mut good_arguments = base_arguments.clone(); - good_arguments.extend(["--vdaf=sum".to_string(), "--bits=8".to_string()]); - Options::try_parse_from(good_arguments).unwrap(); - - let mut good_arguments = base_arguments.clone(); - good_arguments.extend([ - "--vdaf=sumvec".to_string(), - "--bits=8".to_string(), - "--length=10".to_string(), - ]); - Options::try_parse_from(good_arguments).unwrap(); - - let mut good_arguments = base_arguments.clone(); - good_arguments.extend([ - "--vdaf=histogram".to_string(), - "--buckets=1,2,3,4".to_string(), - ]); - Options::try_parse_from(good_arguments).unwrap(); - - #[cfg(feature = "fpvec_bounded_l2")] - { - let mut good_arguments = base_arguments.clone(); - good_arguments.extend([ - "--vdaf=fixedpoint16bitboundedl2vecsum".to_string(), - "--length=10".to_string(), - ]); - Options::try_parse_from(good_arguments).unwrap(); - - let mut good_arguments = base_arguments.clone(); - good_arguments.extend([ - "--vdaf=fixedpoint32bitboundedl2vecsum".to_string(), - "--length=10".to_string(), - ]); - Options::try_parse_from(good_arguments).unwrap(); - - let mut good_arguments = base_arguments.clone(); - good_arguments.extend([ - "--vdaf=fixedpoint64bitboundedl2vecsum".to_string(), - "--length=10".to_string(), - ]); - Options::try_parse_from(good_arguments).unwrap(); - } - } - #[test] fn batch_arguments() { let task_id: TaskId = random(); @@ -906,8 +612,6 @@ mod tests { bits: None, buckets: None, query: QueryOptions { - batch_interval_start: None, - batch_interval_duration: None, batch_id: None, current_batch: true, }, @@ -947,8 +651,6 @@ mod tests { bits: None, buckets: None, query: QueryOptions { - batch_interval_start: None, - batch_interval_duration: None, batch_id: Some(batch_id), current_batch: false, }, @@ -998,71 +700,6 @@ mod tests { ErrorKind::MissingRequiredArgument ); - let mut bad_arguments = base_arguments.clone(); - bad_arguments.push("--batch-interval-start=1".to_string()); - assert_eq!( - Options::try_parse_from(bad_arguments).unwrap_err().kind(), - ErrorKind::MissingRequiredArgument - ); - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.push("--batch-interval-duration=1".to_string()); - assert_eq!( - Options::try_parse_from(bad_arguments).unwrap_err().kind(), - ErrorKind::MissingRequiredArgument - ); - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend([ - "--batch-interval-start=1".to_string(), - "--batch-interval-duration=1".to_string(), - "--batch-id=AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA".to_string(), - ]); - assert_eq!( - Options::try_parse_from(bad_arguments).unwrap_err().kind(), - ErrorKind::ArgumentConflict - ); - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend([ - "--batch-interval-start=1".to_string(), - "--batch-id=AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA".to_string(), - ]); - assert_eq!( - Options::try_parse_from(bad_arguments).unwrap_err().kind(), - ErrorKind::ArgumentConflict - ); - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend([ - "--batch-interval-duration=1".to_string(), - "--batch-id=AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA".to_string(), - ]); - assert_eq!( - Options::try_parse_from(bad_arguments).unwrap_err().kind(), - ErrorKind::ArgumentConflict - ); - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend([ - "--batch-interval-start=1".to_string(), - "--current-batch".to_string(), - ]); - assert_eq!( - Options::try_parse_from(bad_arguments).unwrap_err().kind(), - ErrorKind::ArgumentConflict - ); - - let mut bad_arguments = base_arguments.clone(); - bad_arguments.extend([ - "--batch-interval-duration=1".to_string(), - "--current-batch".to_string(), - ]); - assert_eq!( - Options::try_parse_from(bad_arguments).unwrap_err().kind(), - ErrorKind::ArgumentConflict - ); - let mut bad_arguments = base_arguments; bad_arguments.extend([ "--batch-id=AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA".to_string(), @@ -1090,10 +727,7 @@ mod tests { "https://example.com/dap/".to_string(), format!("--hpke-config={encoded_hpke_config}"), format!("--hpke-private-key={encoded_private_key}"), - "--batch-interval-start".to_string(), - "1000000".to_string(), - "--batch-interval-duration".to_string(), - "1000".to_string(), + "--current-batch".to_string(), "--vdaf=count".to_string(), ]); let dap_auth_token_arguments = Vec::from([ diff --git a/tools/src/bin/dap_decode.rs b/tools/src/bin/dap_decode.rs index c7fe5502d..0d9ed0e9f 100644 --- a/tools/src/bin/dap_decode.rs +++ b/tools/src/bin/dap_decode.rs @@ -1,9 +1,9 @@ use anyhow::Result; use clap::{Parser, ValueEnum}; use janus_messages::{ - query_type::{FixedSize, TimeInterval}, - AggregateShare, AggregateShareReq, AggregationJobContinueReq, AggregationJobInitializeReq, - AggregationJobResp, Collection, CollectionReq, HpkeConfig, HpkeConfigList, Report, + query_type::FixedSize, AggregateShare, AggregateShareReq, AggregationJobContinueReq, + AggregationJobInitializeReq, AggregationJobResp, Collection, CollectionReq, HpkeConfig, + HpkeConfigList, Report, }; use prio::codec::Decode; use std::{ @@ -47,16 +47,9 @@ fn decode_dap_message(message_file: &str, media_type: &MediaType) -> Result { - if let Ok(decoded) = - AggregationJobInitializeReq::::get_decoded(&message_buf) - { - let message: AggregationJobInitializeReq = decoded; - Box::new(message) - } else { - let message: AggregationJobInitializeReq = - AggregationJobInitializeReq::::get_decoded(&message_buf)?; - Box::new(message) - } + let message: AggregationJobInitializeReq = + AggregationJobInitializeReq::::get_decoded(&message_buf)?; + Box::new(message) } MediaType::AggregationJobContinueReq => { let message: AggregationJobContinueReq = @@ -68,38 +61,23 @@ fn decode_dap_message(message_file: &str, media_type: &MediaType) -> Result { - if let Ok(decoded) = AggregateShareReq::::get_decoded(&message_buf) { - let message: AggregateShareReq = decoded; - Box::new(message) - } else { - let message: AggregateShareReq = - AggregateShareReq::::get_decoded(&message_buf)?; - Box::new(message) - } + let message: AggregateShareReq = + AggregateShareReq::::get_decoded(&message_buf)?; + Box::new(message) } MediaType::AggregateShare => { let message: AggregateShare = AggregateShare::get_decoded(&message_buf)?; Box::new(message) } MediaType::CollectionReq => { - if let Ok(decoded) = CollectionReq::::get_decoded(&message_buf) { - let message: CollectionReq = decoded; - Box::new(message) - } else { - let message: CollectionReq = - CollectionReq::::get_decoded(&message_buf)?; - Box::new(message) - } + let message: CollectionReq = + CollectionReq::::get_decoded(&message_buf)?; + Box::new(message) } MediaType::Collection => { - if let Ok(decoded) = Collection::::get_decoded(&message_buf) { - let message: Collection = decoded; - Box::new(message) - } else { - let message: Collection = - Collection::::get_decoded(&message_buf)?; - Box::new(message) - } + let message: Collection = + Collection::::get_decoded(&message_buf)?; + Box::new(message) } }; diff --git a/tools/tests/cmd/collect.trycmd b/tools/tests/cmd/collect.trycmd index 70e4bae7d..680bf71b1 100644 --- a/tools/tests/cmd/collect.trycmd +++ b/tools/tests/cmd/collect.trycmd @@ -2,7 +2,7 @@ $ collect --help Command-line DAP-PPM collector from ISRG's Divvi Up -Usage: collect [OPTIONS] --task-id --leader --hpke-config --hpke-private-key --vdaf <--dap-auth-token |--authorization-bearer-token > <--batch-interval-start |--batch-interval-duration |--batch-id |--current-batch> +Usage: collect [OPTIONS] --task-id --leader --hpke-config --hpke-private-key --vdaf <--dap-auth-token |--authorization-bearer-token > <--batch-id |--current-batch> Options: -h, --help @@ -57,13 +57,6 @@ VDAF Algorithm and Parameters: --buckets Comma-separated list of bucket boundaries, for use with --vdaf=histogram -Collect Request Parameters (Time Interval): - --batch-interval-start - Start of the collection batch interval, as the number of seconds since the Unix epoch - - --batch-interval-duration - Duration of the collection batch interval, in seconds - Collect Request Parameters (Fixed Size): --batch-id Batch identifier, encoded with base64url