Skip to content

Commit

Permalink
Fix writing aggregation jobs touching GC'ed batches.
Browse files Browse the repository at this point in the history
This issue should only exist in the time-interval query type, as
fixed-size is arranged such that aggregation jobs touching a given batch
must be GC'ed before the batch. I include a guard to ensure that the new
codepath is only taken in the expected case of an already-GC'ed batch
for a time-interval query, as otherwise we might drop batch writes if we
fell into it unexpectedly.
  • Loading branch information
branlwyd committed Jan 10, 2024
1 parent edc4c05 commit ef95f87
Show file tree
Hide file tree
Showing 3 changed files with 357 additions and 5 deletions.
308 changes: 308 additions & 0 deletions aggregator/src/aggregator/aggregation_job_driver.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1895,6 +1895,314 @@ mod tests {
assert_eq!(want_batch, got_batch);
}

#[tokio::test]
async fn step_time_interval_aggregation_job_init_partially_garbage_collected() {
// This is a regression test for https://github.com/divviup/janus/issues/2464.
const OLDEST_ALLOWED_REPORT_TIMESTAMP: Time = Time::from_seconds_since_epoch(1000);
const REPORT_EXPIRY_AGE: Duration = Duration::from_seconds(500);
const TIME_PRECISION: Duration = Duration::from_seconds(10);

// Setup: insert an "old" and "new" client report, and add them to a new aggregation job.
install_test_trace_subscriber();
let mut server = mockito::Server::new_async().await;
let clock = MockClock::new(OLDEST_ALLOWED_REPORT_TIMESTAMP);
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)
.with_helper_aggregator_endpoint(server.url().parse().unwrap())
.with_report_expiry_age(Some(REPORT_EXPIRY_AGE))
.with_time_precision(TIME_PRECISION)
.build();

let leader_task = task.leader_view().unwrap();

let first_time = OLDEST_ALLOWED_REPORT_TIMESTAMP
.sub(&Duration::from_seconds(3 * TIME_PRECISION.as_seconds()))
.unwrap()
.to_batch_interval_start(&TIME_PRECISION)
.unwrap();
let first_batch_identifier =
TimeInterval::to_batch_identifier(&leader_task, &(), &first_time).unwrap();
let first_report_metadata = ReportMetadata::new(random(), first_time);

let second_time = OLDEST_ALLOWED_REPORT_TIMESTAMP
.add(&Duration::from_seconds(3 * TIME_PRECISION.as_seconds()))
.unwrap()
.to_batch_interval_start(&TIME_PRECISION)
.unwrap();
let second_batch_identifier =
TimeInterval::to_batch_identifier(&leader_task, &(), &second_time).unwrap();
let second_report_metadata = ReportMetadata::new(random(), second_time);

let verify_key: VerifyKey<VERIFY_KEY_LENGTH> = task.vdaf_verify_key().unwrap();

let first_transcript = run_vdaf(
vdaf.as_ref(),
verify_key.as_bytes(),
&(),
first_report_metadata.id(),
&0,
);
let second_transcript = run_vdaf(
vdaf.as_ref(),
verify_key.as_bytes(),
&(),
second_report_metadata.id(),
&0,
);

let agg_auth_token = task.aggregator_auth_token();
let helper_hpke_keypair = generate_test_hpke_config_and_private_key();
let first_report = generate_report::<VERIFY_KEY_LENGTH, Prio3Count>(
*task.id(),
first_report_metadata,
helper_hpke_keypair.config(),
first_transcript.public_share.clone(),
Vec::new(),
&first_transcript.leader_input_share,
&first_transcript.helper_input_share,
);
let second_report = generate_report::<VERIFY_KEY_LENGTH, Prio3Count>(
*task.id(),
second_report_metadata,
helper_hpke_keypair.config(),
second_transcript.public_share.clone(),
Vec::new(),
&second_transcript.leader_input_share,
&second_transcript.helper_input_share,
);

let aggregation_job_id = random();

let lease = ds
.run_unnamed_tx(|tx| {
let vdaf = Arc::clone(&vdaf);
let leader_task = leader_task.clone();
let first_report = first_report.clone();
let second_report = second_report.clone();

Box::pin(async move {
tx.put_aggregator_task(&leader_task).await?;
tx.put_client_report(vdaf.borrow(), &first_report).await?;
tx.put_client_report(vdaf.borrow(), &second_report).await?;

tx.put_aggregation_job(&AggregationJob::<
VERIFY_KEY_LENGTH,
TimeInterval,
Prio3Count,
>::new(
*leader_task.id(),
aggregation_job_id,
(),
(),
Interval::new(first_time, second_time.difference(&first_time).unwrap())
.unwrap(),
AggregationJobState::InProgress,
AggregationJobStep::from(0),
))
.await
.unwrap();
tx.put_report_aggregation(
&ReportAggregation::<VERIFY_KEY_LENGTH, Prio3Count>::new(
*leader_task.id(),
aggregation_job_id,
*first_report.metadata().id(),
*first_report.metadata().time(),
0,
None,
ReportAggregationState::Start,
),
)
.await
.unwrap();
tx.put_report_aggregation(
&ReportAggregation::<VERIFY_KEY_LENGTH, Prio3Count>::new(
*leader_task.id(),
aggregation_job_id,
*second_report.metadata().id(),
*second_report.metadata().time(),
1,
None,
ReportAggregationState::Start,
),
)
.await
.unwrap();

tx.put_batch(&Batch::<VERIFY_KEY_LENGTH, TimeInterval, Prio3Count>::new(
*leader_task.id(),
first_batch_identifier,
(),
BatchState::Closing,
1,
Interval::from_time(&first_time).unwrap(),
))
.await
.unwrap();
tx.put_batch(&Batch::<VERIFY_KEY_LENGTH, TimeInterval, Prio3Count>::new(
*leader_task.id(),
second_batch_identifier,
(),
BatchState::Closing,
1,
Interval::from_time(&second_time).unwrap(),
))
.await
.unwrap();

Ok(tx
.acquire_incomplete_aggregation_jobs(&StdDuration::from_secs(60), 1)
.await
.unwrap()
.remove(0))
})
})
.await
.unwrap();
assert_eq!(lease.leased().task_id(), task.id());
assert_eq!(lease.leased().aggregation_job_id(), &aggregation_job_id);

// Advance the clock to "enable" report expiry.
clock.advance(&REPORT_EXPIRY_AGE);

// Setup: prepare mocked HTTP response.
let leader_request = AggregationJobInitializeReq::new(
().get_encoded(),
PartialBatchSelector::new_time_interval(),
Vec::from([PrepareInit::new(
ReportShare::new(
second_report.metadata().clone(),
second_report.public_share().get_encoded(),
second_report.helper_encrypted_input_share().clone(),
),
second_transcript.leader_prepare_transitions[0]
.message
.clone(),
)]),
);
let helper_response = AggregationJobResp::new(Vec::from([PrepareResp::new(
*second_report.metadata().id(),
PrepareStepResult::Continue {
message: second_transcript.helper_prepare_transitions[0]
.message
.clone(),
},
)]));
let (header, value) = agg_auth_token.request_authentication();
let mocked_aggregate_init = server
.mock(
"PUT",
task.aggregation_job_uri(&aggregation_job_id)
.unwrap()
.path(),
)
.match_header(header, value.as_str())
.match_header(
CONTENT_TYPE.as_str(),
AggregationJobInitializeReq::<TimeInterval>::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.
let aggregation_job_driver = AggregationJobDriver::new(
reqwest::Client::builder().build().unwrap(),
&noop_meter(),
32,
);
aggregation_job_driver
.step_aggregation_job(ds.clone(), Arc::new(lease))
.await
.unwrap();

// Verify.
mocked_aggregate_init.assert_async().await;

let want_aggregation_job =
AggregationJob::<VERIFY_KEY_LENGTH, TimeInterval, Prio3Count>::new(
*task.id(),
aggregation_job_id,
(),
(),
Interval::new(first_time, second_time.difference(&first_time).unwrap()).unwrap(),
AggregationJobState::Finished,
AggregationJobStep::from(1),
);

let want_first_report_aggregation = ReportAggregation::<VERIFY_KEY_LENGTH, Prio3Count>::new(
*task.id(),
aggregation_job_id,
*first_report.metadata().id(),
*first_report.metadata().time(),
0,
None,
ReportAggregationState::Failed(PrepareError::ReportDropped),
);
let want_second_report_aggregation =
ReportAggregation::<VERIFY_KEY_LENGTH, Prio3Count>::new(
*task.id(),
aggregation_job_id,
*second_report.metadata().id(),
*second_report.metadata().time(),
1,
None,
ReportAggregationState::Finished,
);
let want_report_aggregations = Vec::from([
want_first_report_aggregation,
want_second_report_aggregation,
]);

let want_batch = Batch::<VERIFY_KEY_LENGTH, TimeInterval, Prio3Count>::new(
*task.id(),
second_batch_identifier,
(),
BatchState::Closing,
0,
Interval::from_time(&second_time).unwrap(),
);
let want_batches = Vec::from([want_batch]);

let (got_aggregation_job, got_report_aggregations, got_batches) = ds
.run_unnamed_tx(|tx| {
let vdaf = Arc::clone(&vdaf);
let task = task.clone();
Box::pin(async move {
let aggregation_job = tx
.get_aggregation_job::<VERIFY_KEY_LENGTH, TimeInterval, Prio3Count>(
task.id(),
&aggregation_job_id,
)
.await
.unwrap()
.unwrap();
let report_aggregations = tx
.get_report_aggregations_for_aggregation_job(
vdaf.as_ref(),
&Role::Leader,
task.id(),
&aggregation_job_id,
)
.await
.unwrap();
let batches = tx.get_batches_for_task(task.id()).await.unwrap();
Ok((aggregation_job, report_aggregations, batches))
})
})
.await
.unwrap();

assert_eq!(want_aggregation_job, got_aggregation_job);
assert_eq!(want_report_aggregations, got_report_aggregations);
assert_eq!(want_batches, got_batches);
}

#[tokio::test]
async fn step_fixed_size_aggregation_job_init_single_step() {
// Setup: insert a client report and add it to a new aggregation job.
Expand Down
49 changes: 44 additions & 5 deletions aggregator/src/aggregator/aggregation_job_writer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ use std::{
sync::{Arc, Mutex},
};
use tokio::try_join;
use tracing::{debug, error};

/// AggregationJobWriter contains the logic used to write aggregation jobs, both initially &
/// on updates. It is used only by the Leader.
Expand Down Expand Up @@ -300,7 +301,7 @@ impl<const SEED_SIZE: usize, Q: CollectableQueryType, A: vdaf::Aggregator<SEED_S
.by_batch_identifier_index
.iter()
.flat_map(|(batch_identifier, by_aggregation_job_index)| {
let (operation, mut batch) = match batches.remove(batch_identifier) {
let (batch_op, mut batch) = match batches.remove(batch_identifier) {
Some(batch) => (Operation::Update, batch),
None => (
Operation::Put,
Expand Down Expand Up @@ -332,15 +333,53 @@ impl<const SEED_SIZE: usize, Q: CollectableQueryType, A: vdaf::Aggregator<SEED_S
for (aggregation_job_id, report_aggregation_ords) in by_aggregation_job_index.iter()
{
// unwrap safety: index lookup
let (op, agg_job, report_aggs) =
let (agg_job_op, agg_job, report_aggs) =
by_aggregation_job.get(aggregation_job_id).unwrap();
if op == &Operation::Put
if agg_job_op == &Operation::Put
&& matches!(agg_job.state(), AggregationJobState::InProgress)
{
outstanding_aggregation_jobs += 1;
} else if op == &Operation::Update
} else if agg_job_op == &Operation::Update
&& !matches!(agg_job.state(), AggregationJobState::InProgress)
{
// GC hack: if we are Putting the batch, that means that it does not exist
// in the datastore. But since we first write the batch when an aggregation
// job referencing that batch is created, and we are completing the
// aggregation job here, we must have deleted the batch at some point
// between creating & completing this aggregation job. This should only be
// possible if the batch is GC'ed, as part of a time-interval task. In that
// case, it is acceptable to skip writing the batch entirely; and indeed, we
// must do so, since otherwise we might underflow the
// outstanding_aggregation_jobs counter.
//
// See https://github.com/divviup/janus/issues/2464 for more detail.
if batch_op == Operation::Put {
// Guard to ensure we are in the situation we think we're in:
// `to_batch_interval` returns a value only for the time-interval query
// type; this code effectively checks that the query type is
// time-interval, and the batch interval is past the GC window.
if !Q::to_batch_interval(batch_identifier)
.map(|interval| interval.end() < tx.clock().now())
.unwrap_or(false)
{
error!(
task_id = ?self.task.id(),
batch_id = ?batch_identifier,
?aggregation_job_id,
"Unexpectedly missing batch while writing completed aggregation job"
);
panic!("Unexpectedly missing batch while writing completed aggregation job");
}

debug!(
task_id = ?self.task.id(),
batch_id = ?batch_identifier,
?aggregation_job_id,
"Skipping batch write for GC'ed batch"
);
return None;
}

outstanding_aggregation_jobs -= 1;
}

Expand All @@ -366,7 +405,7 @@ impl<const SEED_SIZE: usize, Q: CollectableQueryType, A: vdaf::Aggregator<SEED_S
Some(Ok((
batch_identifier,
(
operation,
batch_op,
batch
.with_outstanding_aggregation_jobs(outstanding_aggregation_jobs)
.with_client_timestamp_interval(client_timestamp_interval),
Expand Down
Loading

0 comments on commit ef95f87

Please sign in to comment.