Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
// Copyright 2024 The Drasi Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::{
    collections::HashMap,
    fmt::Debug,
    hash::{Hash, Hasher},
    sync::Arc,
    time::Duration,
};

use drasi_query_ast::ast::Query;
use hashers::jenkins::spooky_hash::SpookyHasher;
use tokio::{
    select,
    sync::{Mutex, Notify},
    task::JoinHandle,
};

use crate::{
    evaluation::{
        context::{ChangeContext, QueryPartEvaluationContext, QueryVariables},
        EvaluationError, ExpressionEvaluationContext, ExpressionEvaluator, InstantQueryClock,
        QueryPartEvaluator,
    },
    interface::{
        ElementIndex, FutureQueue, FutureQueueConsumer, MiddlewareError, QueryClock,
        SessionControl, SessionGuard,
    },
    middleware::SourceMiddlewarePipelineCollection,
    models::{Element, SourceChange},
    path_solver::{
        match_path::{MatchPath, SlotElementSpec},
        solution::{MatchPathSolution, SolutionSignature},
        MatchPathSolver, MatchSolveContext,
    },
};

/// Result of processing a due future item.
/// Contains the evaluation results and the source_id from the popped future's element_ref,
/// needed by the lib crate to record provenance in QueryResult metadata.
pub struct DueFutureResult {
    pub results: Vec<QueryPartEvaluationContext>,
    /// The source_id from the popped future's element_ref.
    pub source_id: Arc<str>,
}

pub struct ContinuousQuery {
    expression_evaluator: Arc<ExpressionEvaluator>,
    part_evaluator: Arc<QueryPartEvaluator>,
    element_index: Arc<dyn ElementIndex>,
    path_solver: Arc<MatchPathSolver>,
    match_path: Arc<MatchPath>,
    query: Arc<Query>,
    future_consumer_shutdown_request: Arc<Notify>,
    future_queue: Arc<dyn FutureQueue>,
    future_queue_task: Mutex<Option<JoinHandle<()>>>,
    change_lock: Mutex<()>,
    source_pipelines: SourceMiddlewarePipelineCollection,
    session_control: Arc<dyn SessionControl>,
}

impl ContinuousQuery {
    #[allow(clippy::too_many_arguments)]
    pub fn new(
        query: Arc<Query>,
        match_path: Arc<MatchPath>,
        expression_evaluator: Arc<ExpressionEvaluator>,
        element_index: Arc<dyn ElementIndex>,
        path_solver: Arc<MatchPathSolver>,
        part_evaluator: Arc<QueryPartEvaluator>,
        future_queue: Arc<dyn FutureQueue>,
        source_pipelines: SourceMiddlewarePipelineCollection,
        session_control: Arc<dyn SessionControl>,
    ) -> Self {
        Self {
            expression_evaluator,
            element_index,
            path_solver,
            match_path,
            part_evaluator,
            query,
            future_consumer_shutdown_request: Arc::new(Notify::new()),
            future_queue,
            future_queue_task: Mutex::new(None),
            change_lock: Mutex::new(()),
            source_pipelines,
            session_control,
        }
    }

    #[tracing::instrument(skip_all, err, level = "debug")]
    pub async fn process_source_change(
        &self,
        change: SourceChange,
    ) -> Result<Vec<QueryPartEvaluationContext>, EvaluationError> {
        let _lock = self.change_lock.lock().await;
        let guard = SessionGuard::begin(self.session_control.clone()).await?;

        let changes = self.execute_source_middleware(change).await?;
        let result = self.process_changes_inner(changes).await?;

        guard.commit().await?;
        Ok(result)
    }

    /// Atomically pop a due future from the queue and process it within a single session.
    ///
    /// Returns `Ok(None)` when the queue is empty (stale peek).
    /// Returns `Ok(Some(DueFutureResult))` with results and the original source_id.
    ///
    /// Pop happens inside the session → atomic with all downstream index writes.
    /// If a crash occurs before commit, the pop rolls back and the item stays in the queue.
    #[tracing::instrument(skip_all, err, level = "debug")]
    pub async fn process_due_futures(&self) -> Result<Option<DueFutureResult>, EvaluationError> {
        let _lock = self.change_lock.lock().await;
        let guard = SessionGuard::begin(self.session_control.clone()).await?;

        let future_ref = match self.future_queue.pop().await {
            Ok(Some(fr)) => fr,
            Ok(None) => {
                guard.commit().await?;
                return Ok(None);
            }
            Err(e) => return Err(EvaluationError::from(e)),
        };

        let source_id = future_ref.element_ref.source_id.clone();
        let change = SourceChange::Future { future_ref };
        let changes = self.execute_source_middleware(change).await?;
        let results = self.process_changes_inner(changes).await?;
        guard.commit().await?;
        Ok(Some(DueFutureResult { results, source_id }))
    }

    /// Expose the ContinuousQuery's future queue for external polling.
    pub fn future_queue(&self) -> Arc<dyn FutureQueue> {
        self.future_queue.clone()
    }

    /// Inner processing logic shared by `process_source_change` and `process_due_futures`.
    /// Must be called within an active session and while holding `change_lock`.
    #[tracing::instrument(skip_all, err, level = "debug")]
    async fn process_changes_inner(
        &self,
        changes: Vec<SourceChange>,
    ) -> Result<Vec<QueryPartEvaluationContext>, EvaluationError> {
        let mut result = Vec::new();

        for change in changes {
            let base_variables = QueryVariables::new(); //todo: get query parameters
            let after_clock = Arc::new(InstantQueryClock::from_source_change(&change));

            let solution_changes = self
                .build_solution_changes(&base_variables, change, after_clock.clone())
                .await?;
            let before_clock = match solution_changes.before_clock {
                Some(before_clock) => before_clock,
                None => after_clock.clone(),
            };

            let mut aggregation_results = CollapsedAggregationResults::new();

            for (solution_signature, part_context) in solution_changes.changes {
                let change_results = match self
                    .project_solution(
                        part_context,
                        &ChangeContext {
                            solution_signature,
                            before_clock: before_clock.clone(),
                            after_clock: after_clock.clone(),
                            before_anchor_element: solution_changes.before_anchor_element.clone(),
                            after_anchor_element: solution_changes.anchor_element.clone(),
                            is_future_reprocess: solution_changes.is_future_reprocess,
                            before_grouping_hash: solution_signature,
                            after_grouping_hash: solution_signature,
                        },
                    )
                    .await
                {
                    Ok(results) => results,
                    Err(EvaluationError::DivideByZero) => {
                        log::debug!("Skipping solution due to DivideByZero in projection");
                        continue;
                    }
                    Err(e) => return Err(e),
                };
                change_results.into_iter().for_each(|ctx| {
                    match &ctx {
                        QueryPartEvaluationContext::Aggregation {
                            before,
                            after,
                            default_before,
                            ..
                        } => {
                            if let Some(before) = before {
                                if before == after && !default_before {
                                    return;
                                }
                            }

                            aggregation_results.insert(ctx);
                        }
                        QueryPartEvaluationContext::Updating { before, after, .. } => {
                            if before == after {
                                return;
                            }
                            result.push(ctx);
                        }
                        _ => result.push(ctx),
                    };
                });
            }

            for ctx in aggregation_results.into_result_vec() {
                result.push(ctx);
            }
        }

        Ok(result)
    }

    #[tracing::instrument(skip_all, err, level = "debug")]
    async fn build_solution_changes(
        &self,
        base_variables: &QueryVariables,
        change: SourceChange,
        clock: Arc<dyn QueryClock>,
    ) -> Result<SolutionChangesResult, EvaluationError> {
        let mut result = SolutionChangesResult::new();
        let mut before_change_solutions = HashMap::new();
        let mut after_change_solutions = HashMap::new();

        match change {
            SourceChange::Insert { element } => {
                let element = Arc::new(element);
                let affinity_slots = self
                    .get_slots_with_affinity(base_variables, element.clone(), clock.clone())
                    .await?;
                let solutions = self
                    .resolve_solutions(element.clone(), affinity_slots, true)
                    .await?;

                for (signature, solution) in solutions {
                    if let Some(blank_optional_solution) =
                        solution.get_empty_optional_solution(&self.match_path)
                    {
                        before_change_solutions.insert(signature, blank_optional_solution);
                    }
                    after_change_solutions.insert(signature, solution);
                }

                result.anchor_element = Some(element);
            }
            SourceChange::Update { mut element } => {
                if let Some(prev_version) = self
                    .element_index
                    .get_element(element.get_reference())
                    .await?
                {
                    let prev_timestamp = prev_version.get_effective_from();
                    let before_clock =
                        Arc::new(InstantQueryClock::new(prev_timestamp, clock.get_realtime()));
                    let affinity_slots = self
                        .get_slots_with_affinity(
                            base_variables,
                            prev_version.clone(),
                            before_clock.clone(),
                        )
                        .await?;
                    let solutions = self
                        .resolve_solutions(prev_version.clone(), affinity_slots, false)
                        .await?;
                    for (signature, solution) in solutions {
                        before_change_solutions.insert(signature, solution);
                    }
                    element.merge_missing_properties(prev_version.as_ref());
                    result.before_clock = Some(before_clock);
                    result.before_anchor_element = Some(prev_version);
                }

                let element = Arc::new(element);
                let affinity_slots = self
                    .get_slots_with_affinity(base_variables, element.clone(), clock.clone())
                    .await?;
                let solutions = self
                    .resolve_solutions(element.clone(), affinity_slots, true)
                    .await?;

                for (signature, solution) in solutions {
                    after_change_solutions.insert(signature, solution);
                }

                result.anchor_element = Some(element);
            }
            SourceChange::Delete { metadata } => {
                if let Some(element) = self.element_index.get_element(&metadata.reference).await? {
                    let prev_timestamp = element.get_effective_from();
                    let before_clock =
                        Arc::new(InstantQueryClock::new(prev_timestamp, clock.get_realtime()));
                    let affinity_slots = self
                        .get_slots_with_affinity(
                            base_variables,
                            element.clone(),
                            before_clock.clone(),
                        )
                        .await?;
                    let solutions = self
                        .resolve_solutions(element.clone(), affinity_slots, false)
                        .await?;
                    for (signature, solution) in solutions {
                        if let Some(blank_optional_solution) =
                            solution.get_empty_optional_solution(&self.match_path)
                        {
                            after_change_solutions.insert(signature, blank_optional_solution);
                        }

                        before_change_solutions.insert(signature, solution);
                    }
                    result.before_clock = Some(before_clock);
                    result.before_anchor_element = Some(element);

                    match self.element_index.delete_element(&metadata.reference).await {
                        Ok(_) => {}
                        Err(e) => return Err(EvaluationError::from(e)),
                    }
                }
            }
            SourceChange::Future { future_ref } => {
                result.is_future_reprocess = true;
                if let Some(element) = self
                    .element_index
                    .get_element(&future_ref.element_ref)
                    .await?
                {
                    let prev_timestamp = element.get_effective_from();
                    if prev_timestamp >= future_ref.due_time {
                        // element already processed with due time expired, don't duplicate
                        return Ok(result);
                    }

                    let before_clock =
                        Arc::new(InstantQueryClock::new(prev_timestamp, prev_timestamp));

                    let affinity_slots = self
                        .get_slots_with_affinity(
                            base_variables,
                            element.clone(),
                            before_clock.clone(),
                        )
                        .await?;

                    let before_solutions = self
                        .resolve_solutions(element.clone(), affinity_slots, false)
                        .await?;
                    for (signature, solution) in before_solutions {
                        before_change_solutions.insert(signature, solution);
                    }

                    result.before_clock = Some(before_clock);
                    result.before_anchor_element = Some(element.clone());

                    let affinity_slots = self
                        .get_slots_with_affinity(base_variables, element.clone(), clock.clone())
                        .await?;

                    let after_solutions = self
                        .resolve_solutions(element.clone(), affinity_slots, false)
                        .await?;
                    for (signature, solution) in after_solutions {
                        after_change_solutions.insert(signature, solution);
                    }

                    result.anchor_element = Some(element);
                }
            }
        }

        for (sig, before_sol) in &before_change_solutions {
            match after_change_solutions.get(sig) {
                Some(after_sol) => result.changes.push((
                    *sig,
                    QueryPartEvaluationContext::Updating {
                        before: before_sol.into_query_variables(&self.match_path, base_variables),
                        after: after_sol.into_query_variables(&self.match_path, base_variables),
                        row_signature: 0,
                    },
                )),
                None => result.changes.push((
                    *sig,
                    QueryPartEvaluationContext::Removing {
                        before: before_sol.into_query_variables(&self.match_path, base_variables),
                        row_signature: 0,
                    },
                )),
            }
        }

        for (sig, after_sol) in &after_change_solutions {
            if !before_change_solutions.contains_key(sig) {
                result.changes.push((
                    *sig,
                    QueryPartEvaluationContext::Adding {
                        after: after_sol.into_query_variables(&self.match_path, base_variables),
                        row_signature: 0,
                    },
                ))
            }
        }

        Ok(result)
    }

    async fn resolve_solutions(
        &self,
        anchor_element: Arc<Element>,
        affinity_slots: Vec<usize>,
        update_index: bool,
    ) -> Result<HashMap<u64, MatchPathSolution>, EvaluationError> {
        if update_index {
            self.element_index
                .set_element(anchor_element.as_ref(), &affinity_slots)
                .await?;
        }

        let mut result = HashMap::new();

        for slot_num in affinity_slots {
            let solution = self
                .path_solver
                .solve(self.match_path.clone(), anchor_element.clone(), slot_num)
                .await?;
            result.extend(solution.into_iter());
        }

        Ok(result)
    }

    async fn get_slots_with_affinity(
        &self,
        variables: &QueryVariables,
        anchor_element: Arc<Element>,
        clock: Arc<dyn QueryClock>,
    ) -> Result<Vec<usize>, EvaluationError> {
        let context = MatchSolveContext::new(variables, clock);

        let mut affinity_slots = Vec::new();

        for (slot_num, slot) in self.match_path.slots.iter().enumerate() {
            if self
                .match_element_to_slot(&context, &slot.spec, anchor_element.clone())
                .await?
            {
                affinity_slots.push(slot_num);
            }
        }

        Ok(affinity_slots)
    }

    async fn match_element_to_slot(
        &self,
        context: &MatchSolveContext<'_>,
        element_spec: &SlotElementSpec,
        element: Arc<Element>,
    ) -> Result<bool, EvaluationError> {
        let metadata = element.get_metadata();
        let mut label_match = element_spec.labels.is_empty();

        for label in &element_spec.labels {
            if metadata.labels.contains(label) {
                label_match = true;
                break;
            }
        }

        if !label_match {
            return Ok(false);
        }

        let mut variables = context.variables.clone();

        let element_variable = element.to_expression_variable();

        if element_spec.annotation.is_some() {
            variables.insert(
                element_spec
                    .annotation
                    .clone()
                    .unwrap()
                    .to_string()
                    .into_boxed_str(),
                element_variable.clone(),
            );
        }

        variables.insert("".into(), element_variable);

        let eval_context = ExpressionEvaluationContext::from_slot(
            &variables,
            context.clock.clone(),
            &metadata.reference,
        );

        for predicate in &element_spec.predicates {
            let result = self
                .expression_evaluator
                .evaluate_predicate(&eval_context, predicate)
                .await?;
            if !result {
                return Ok(false);
            }
        }

        Ok(true)
    }

    #[tracing::instrument(skip_all, err, level = "debug")]
    async fn project_solution(
        &self,
        part_context: QueryPartEvaluationContext,
        change_context: &ChangeContext,
    ) -> Result<Vec<QueryPartEvaluationContext>, EvaluationError> {
        let mut result = Vec::new();
        let mut contexts = vec![(part_context, change_context.clone())];

        let mut part_num = 0;

        for part in &self.query.parts {
            part_num += 1;
            result.clear();

            for (part_context, change_context) in &contexts {
                let new_contexts = self
                    .part_evaluator
                    .evaluate(part_context.clone(), part_num, part, change_context)
                    .await?;

                let mut aggregation_results = CollapsedAggregationResults::new();

                new_contexts.into_iter().for_each(|ctx| match &ctx {
                    QueryPartEvaluationContext::Aggregation { .. } => {
                        aggregation_results.insert(ctx)
                    }
                    QueryPartEvaluationContext::Noop => (),
                    _ => result.push((ctx, change_context.clone())),
                });

                for actx in aggregation_results.into_vec_with_context(change_context) {
                    result.push(actx);
                }
            }
            contexts = result.clone();
        }

        Ok(result
            .into_iter()
            .map(|(ctx, cc)| match ctx {
                QueryPartEvaluationContext::Adding { after, .. } => {
                    QueryPartEvaluationContext::Adding {
                        after,
                        row_signature: cc.solution_signature,
                    }
                }
                QueryPartEvaluationContext::Updating { before, after, .. } => {
                    QueryPartEvaluationContext::Updating {
                        before,
                        after,
                        row_signature: cc.solution_signature,
                    }
                }
                QueryPartEvaluationContext::Removing { before, .. } => {
                    QueryPartEvaluationContext::Removing {
                        before,
                        row_signature: cc.solution_signature,
                    }
                }
                QueryPartEvaluationContext::Aggregation {
                    before,
                    after,
                    grouping_keys,
                    default_before,
                    default_after,
                    ..
                } => QueryPartEvaluationContext::Aggregation {
                    before,
                    after,
                    grouping_keys,
                    default_before,
                    default_after,
                    row_signature: cc.after_grouping_hash,
                },
                QueryPartEvaluationContext::Noop => QueryPartEvaluationContext::Noop,
            })
            .collect())
    }

    #[tracing::instrument(skip_all, err, level = "debug")]
    async fn execute_source_middleware(
        &self,
        change: SourceChange,
    ) -> Result<Vec<SourceChange>, MiddlewareError> {
        let source_id = change.get_reference().source_id.clone();
        let mut source_changes = vec![change];

        let pipeline = match self.source_pipelines.get(source_id) {
            Some(pipeline) => pipeline,
            None => return Ok(source_changes),
        };

        let mut new_source_changes = Vec::new();
        for source_change in source_changes {
            new_source_changes.append(
                &mut pipeline
                    .process(source_change, self.element_index.clone())
                    .await?,
            );
        }

        source_changes = new_source_changes;

        Ok(source_changes)
    }

    pub async fn set_future_consumer(&self, consumer: Arc<dyn FutureQueueConsumer>) {
        let mut future_queue_task = self.future_queue_task.lock().await;
        if let Some(c) = future_queue_task.take() {
            c.abort();
        }

        let queue = self.future_queue.clone();
        let shutdown_request = self.future_consumer_shutdown_request.clone();

        let task = tokio::spawn(async move {
            let idle_interval = Duration::from_secs(1);
            let error_interval = Duration::from_secs(5);
            loop {
                select! {
                    _ = shutdown_request.notified() => {
                        log::info!("Future queue consumer shutting down");
                        break;
                    }
                    peek = queue.peek_due_time() => {
                        match peek {
                            Ok(Some(due_time)) => {
                                if due_time > consumer.now() {
                                    tokio::time::sleep(idle_interval).await;
                                    continue;
                                }
                            }
                            Ok(None) => {
                                tokio::time::sleep(idle_interval).await;
                                continue;
                            }
                            Err(e) => {
                                log::error!("Future queue consumer error: {e:?}");
                                tokio::time::sleep(error_interval).await;
                                continue;
                            }
                        };

                        // Items are due — delegate to consumer which calls process_due_futures()
                        match consumer.on_items_due().await {
                            Ok(_) => {}
                            Err(e) => {
                                log::error!("Future queue consumer error: {e:?}");
                                consumer.on_error(e).await;
                                tokio::time::sleep(error_interval).await;
                            }
                        }
                    }
                }
            }
        });

        _ = future_queue_task.insert(task);
    }

    pub async fn terminate_future_consumer(&self) {
        let mut future_queue_task = self.future_queue_task.lock().await;
        if let Some(task) = future_queue_task.take() {
            self.future_consumer_shutdown_request.notify_one();
            select! {
                _ = task => {
                    log::info!("Future queue consumer terminated");
                }
                _ = tokio::time::sleep(Duration::from_secs(10)) => {
                    log::error!("Future queue consumer termination timeout");
                }
            }
        }
    }

    pub fn get_query(&self) -> Arc<Query> {
        self.query.clone()
    }
}

impl Drop for ContinuousQuery {
    fn drop(&mut self) {
        self.future_consumer_shutdown_request.notify_one();
    }
}

impl Debug for ContinuousQuery {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("ContinuousQuery")
            .field("query", &self.query)
            .finish()
    }
}

struct SolutionChangesResult {
    pub changes: Vec<(SolutionSignature, QueryPartEvaluationContext)>,
    pub anchor_element: Option<Arc<Element>>,
    pub before_clock: Option<Arc<dyn QueryClock>>,
    pub before_anchor_element: Option<Arc<Element>>,
    pub is_future_reprocess: bool,
}

impl SolutionChangesResult {
    fn new() -> Self {
        Self {
            changes: Vec::new(),
            before_clock: None,
            anchor_element: None,
            before_anchor_element: None,
            is_future_reprocess: false,
        }
    }
}

struct CollapsedAggregationResults {
    // [hash of after change grouping keys] -> (context, hash of before change grouping keys)
    data: HashMap<u64, (QueryPartEvaluationContext, u64)>,
}

impl CollapsedAggregationResults {
    fn new() -> Self {
        Self {
            data: HashMap::new(),
        }
    }

    fn insert(&mut self, context: QueryPartEvaluationContext) {
        if let QueryPartEvaluationContext::Aggregation {
            before,
            after,
            grouping_keys,
            default_before,
            default_after,
            ..
        } = context
        {
            let after_key = extract_grouping_value_hash(&grouping_keys, &after);
            let before_key = match &before {
                Some(before) => extract_grouping_value_hash(&grouping_keys, before),
                None => after_key,
            };

            match self.data.remove(&after_key) {
                Some((existing, before_key)) => {
                    if let QueryPartEvaluationContext::Aggregation {
                        before: existing_before,
                        ..
                    } = existing
                    {
                        self.data.insert(
                            after_key,
                            (
                                QueryPartEvaluationContext::Aggregation {
                                    before: existing_before,
                                    default_before,
                                    default_after,
                                    after,
                                    grouping_keys,
                                    row_signature: after_key,
                                },
                                before_key,
                            ),
                        );
                    }
                }
                None => {
                    self.data.insert(
                        after_key,
                        (
                            QueryPartEvaluationContext::Aggregation {
                                before,
                                after,
                                grouping_keys,
                                default_before,
                                default_after,
                                row_signature: after_key,
                            },
                            before_key,
                        ),
                    );
                }
            }
        }
    }

    fn into_vec_with_context(
        self,
        change_context: &ChangeContext,
    ) -> Vec<(QueryPartEvaluationContext, ChangeContext)> {
        self.data
            .into_iter()
            .map(|(after_key, (v, before_key))| {
                let mut change_context = change_context.clone();
                change_context.before_grouping_hash = before_key;
                change_context.after_grouping_hash = after_key;
                (v, change_context)
            })
            .collect()
    }

    fn into_result_vec(self) -> Vec<QueryPartEvaluationContext> {
        self.data
            .into_iter()
            .map(|(after_key, (ctx, _))| match ctx {
                QueryPartEvaluationContext::Aggregation {
                    before,
                    after,
                    grouping_keys,
                    default_before,
                    default_after,
                    ..
                } => QueryPartEvaluationContext::Aggregation {
                    before,
                    after,
                    grouping_keys,
                    default_before,
                    default_after,
                    row_signature: after_key,
                },
                other => other,
            })
            .collect()
    }
}

fn extract_grouping_value_hash(grouping_keys: &Vec<String>, variables: &QueryVariables) -> u64 {
    let mut hasher = SpookyHasher::default();

    for key in grouping_keys {
        match variables.get(key.as_str()) {
            Some(v) => v.hash_for_groupby(&mut hasher),
            None => 0.hash(&mut hasher),
        };
    }
    hasher.finish()
}