re_datafusion 0.31.0-rc.1

High-level query APIs
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
use crate::batch_coalescer::coalesce_exec::SizedCoalesceBatchesExec;
use crate::batch_coalescer::coalescer::CoalescerOptions;
use crate::pushdown_expressions::{apply_filter_expr_to_queries, filter_expr_is_supported};
use ahash::HashSet;
use arrow::array::{
    Array as _, ArrayRef, DurationNanosecondArray, FixedSizeBinaryArray, Int64Array, RecordBatch,
    StringArray, TimestampMicrosecondArray, TimestampMillisecondArray, TimestampNanosecondArray,
    TimestampSecondArray, UInt32Array,
};
use arrow::compute::concat_batches;
use arrow::datatypes::{DataType, Field, Int64Type, Schema, SchemaRef, TimeUnit};
use arrow::record_batch::RecordBatchOptions;
use async_trait::async_trait;
use datafusion::catalog::{Session, TableProvider};
use datafusion::common::{Column, DataFusionError, downcast_value, exec_datafusion_err};
use datafusion::datasource::TableType;
use datafusion::logical_expr::{Expr, Operator, TableProviderFilterPushDown};
use datafusion::physical_plan::ExecutionPlan;
use futures::StreamExt as _;
use re_dataframe::external::re_chunk_store::ChunkStore;
use re_dataframe::{Index, IndexValue, QueryExpression};
use re_log_types::EntryId;
use re_protos::cloud::v1alpha1::ext::{Query, QueryDatasetRequest, QueryLatestAt, QueryRange};
use re_protos::cloud::v1alpha1::{
    FetchChunksRequest, GetDatasetSchemaRequest, GetDatasetSchemaResponse, QueryDatasetResponse,
    ScanSegmentTableResponse,
};
use re_protos::common::v1alpha1::ext::ScanParameters;
use re_protos::headers::RerunHeadersInjectorExt as _;
use re_redap_client::{ConnectionClient, ConnectionRegistryHandle};
use re_sorbet::{BatchType, ChunkColumnDescriptors, ColumnKind, ComponentColumnSelector};
use re_uri::Origin;
use std::any::Any;
use std::cmp::Ordering;
use std::collections::{BTreeMap, BTreeSet};
use std::str::FromStr as _;
use std::sync::Arc;

/// Sets the size for output record batches in rows. The last batch will likely be smaller.
/// The default for Data Fusion is 8192, which leads to a 256Kb record batch on average for
/// rows with 32b of data. We are setting this lower as a reasonable first guess to avoid
/// the pitfall of executing a single row at a time, but we will likely want to consider
/// at some point moving to a dynamic sizing.
pub(crate) const DEFAULT_BATCH_BYTES: u64 = 200 * 1024 * 1024;
pub(crate) const DEFAULT_BATCH_ROWS: usize = 2048;

/// Mapping of `rerun_segment_id` to set of `IndexValues` to be used for querying
/// a specific set of index values per segment. If the option is None, then
/// `using_index_values` will not be applied to the dataset queries.
pub(crate) type IndexValuesMap = Option<Arc<BTreeMap<String, BTreeSet<IndexValue>>>>;

#[derive(Debug)]
pub struct DataframeQueryTableProvider<T: DataframeClientAPI> {
    pub schema: SchemaRef,
    query_expression: QueryExpression,
    query_dataset_request: QueryDatasetRequest,
    sort_index: Option<Index>,
    dataset_id: EntryId,
    client: T,
    index_values: IndexValuesMap,

    /// passing trace headers between phases of execution pipeline helps keep
    /// the entire operation under a single trace.
    #[cfg(not(target_arch = "wasm32"))]
    trace_headers: Option<crate::TraceHeaders>,
}

/// This trait provides the specific methods used when interacting with the
/// gRPC services for the datafusion client services. By implementing this
/// as a trait we can provide an alternative implementation in our testing
/// facility to remove all gRPC layers and test the server responses
/// more directly.
#[async_trait]
pub trait DataframeClientAPI: std::fmt::Debug + Clone + Send + Sync + Unpin + 'static {
    async fn get_dataset_schema(
        &mut self,
        request: tonic::Request<GetDatasetSchemaRequest>,
    ) -> tonic::Result<tonic::Response<GetDatasetSchemaResponse>>;

    async fn query_dataset(
        &mut self,
        request: tonic::Request<re_protos::cloud::v1alpha1::QueryDatasetRequest>,
    ) -> tonic::Result<tonic::Response<tonic::codec::Streaming<QueryDatasetResponse>>>;

    async fn fetch_chunks(
        &mut self,
        request: tonic::Request<re_protos::cloud::v1alpha1::FetchChunksRequest>,
    ) -> tonic::Result<
        tonic::Response<tonic::codec::Streaming<re_protos::cloud::v1alpha1::FetchChunksResponse>>,
    >;
}

#[async_trait]
impl DataframeClientAPI for ConnectionClient {
    async fn get_dataset_schema(
        &mut self,
        request: tonic::Request<GetDatasetSchemaRequest>,
    ) -> tonic::Result<tonic::Response<GetDatasetSchemaResponse>> {
        self.inner().get_dataset_schema(request).await
    }

    async fn query_dataset(
        &mut self,
        request: tonic::Request<re_protos::cloud::v1alpha1::QueryDatasetRequest>,
    ) -> tonic::Result<tonic::Response<tonic::codec::Streaming<QueryDatasetResponse>>> {
        self.inner().query_dataset(request).await
    }

    async fn fetch_chunks(
        &mut self,
        request: tonic::Request<re_protos::cloud::v1alpha1::FetchChunksRequest>,
    ) -> tonic::Result<
        tonic::Response<tonic::codec::Streaming<re_protos::cloud::v1alpha1::FetchChunksResponse>>,
    > {
        self.inner().fetch_chunks(request).await
    }
}

impl DataframeQueryTableProvider<ConnectionClient> {
    /// Create a table provider for a gRPC query. This function is async
    /// because we need to make gRPC calls to determine the schema at the
    /// creation of the table provider.
    #[tracing::instrument(level = "info", skip_all)]
    pub async fn new(
        origin: Origin,
        connection: ConnectionRegistryHandle,
        dataset_id: EntryId,
        query_expression: &QueryExpression,
        segment_ids: &[impl AsRef<str> + Sync],
        index_values: IndexValuesMap,
        #[cfg(not(target_arch = "wasm32"))] trace_headers: Option<crate::TraceHeaders>,
    ) -> Result<Self, DataFusionError> {
        let client = connection
            .client(origin)
            .await
            .map_err(|err| exec_datafusion_err!("{err}"))?;

        Self::new_from_client(
            client,
            dataset_id,
            query_expression,
            segment_ids,
            index_values,
            #[cfg(not(target_arch = "wasm32"))]
            trace_headers,
        )
        .await
    }
}

impl<T: DataframeClientAPI> DataframeQueryTableProvider<T> {
    #[tracing::instrument(level = "info", skip_all)]
    pub async fn new_from_client(
        mut client: T,
        dataset_id: EntryId,
        query_expression: &QueryExpression,
        segment_ids: &[impl AsRef<str> + Sync],
        index_values: IndexValuesMap,
        #[cfg(not(target_arch = "wasm32"))] trace_headers: Option<crate::TraceHeaders>,
    ) -> Result<Self, DataFusionError> {
        let schema = client
            .get_dataset_schema(
                tonic::Request::new(GetDatasetSchemaRequest {})
                    .with_entry_id(dataset_id)
                    .map_err(|err| exec_datafusion_err!("{err}"))?,
            )
            .await
            .map_err(|err| exec_datafusion_err!("{err}"))?
            .into_inner()
            .schema()
            .map_err(|err| exec_datafusion_err!("{err}"))?;

        let schema = compute_schema_for_query(&schema, query_expression)?;

        let select_all_entity_paths = false;

        let entity_paths = query_expression
            .view_contents
            .as_ref()
            .map_or(vec![], |contents| contents.keys().collect::<Vec<_>>());

        let query = query_from_query_expression(query_expression);
        let fuzzy_descriptors: Vec<String> = query_expression
            .view_contents
            .as_ref()
            .map_or(BTreeSet::new(), |contents| {
                contents
                    .values()
                    .filter_map(|opt_set| opt_set.as_ref())
                    .flat_map(|set| set.iter().copied())
                    .collect::<BTreeSet<_>>()
            })
            .into_iter()
            .map(|ident| ident.to_string())
            .collect();

        let query_dataset_request = QueryDatasetRequest {
            segment_ids: segment_ids
                .iter()
                .map(|id| id.as_ref().to_owned().into())
                .collect(),
            chunk_ids: vec![],
            entity_paths: entity_paths.into_iter().map(|p| (*p).clone()).collect(),
            select_all_entity_paths,
            fuzzy_descriptors,
            exclude_static_data: false,
            exclude_temporal_data: false,
            query: Some(query),
            scan_parameters: Some(ScanParameters {
                columns: FetchChunksRequest::required_column_names(),
                ..Default::default()
            }),
        };

        let schema = Arc::new(prepend_string_column_schema(
            &schema,
            ScanSegmentTableResponse::FIELD_SEGMENT_ID,
        ));

        Ok(Self {
            schema,
            query_expression: query_expression.to_owned(),
            query_dataset_request,
            sort_index: query_expression.filtered_index,
            dataset_id,
            client,
            index_values,
            #[cfg(not(target_arch = "wasm32"))]
            trace_headers,
        })
    }

    fn selector_from_column(column: &Column) -> Option<ComponentColumnSelector> {
        ComponentColumnSelector::from_str(column.name()).ok()
    }

    fn is_neq_null(expr: &Expr) -> Option<&Column> {
        match expr {
            Expr::IsNotNull(inner) => {
                if let Expr::Column(col) = inner.as_ref() {
                    return Some(col);
                }
            }
            Expr::Not(inner) => {
                if let Expr::IsNull(col_expr) = inner.as_ref()
                    && let Expr::Column(col) = col_expr.as_ref()
                {
                    return Some(col);
                }
            }
            Expr::BinaryExpr(binary) => {
                if binary.op == Operator::NotEq
                    && let (Expr::Column(col), Expr::Literal(sv, _))
                    | (Expr::Literal(sv, _), Expr::Column(col)) =
                        (binary.left.as_ref(), binary.right.as_ref())
                    && sv.is_null()
                {
                    return Some(col);
                }
            }
            _ => {}
        }

        None
    }

    /// For a given input expression, check to see if it can match the supported
    /// row filtering. We can currently filter out rows for which a specific
    /// component of one entity is not null. We do this by checking the column
    /// name matches the entity path and component naming conventions, which
    /// should always be true at the level of this call. We attempt to match
    /// a few different logically equivalent variants the user may pass.
    fn compute_column_is_neq_null_filter(
        filters: &[&Expr],
    ) -> Vec<Option<ComponentColumnSelector>> {
        filters
            .iter()
            .map(|expr| Self::is_neq_null(expr).and_then(Self::selector_from_column))
            .collect()
    }
}

#[async_trait]
impl<T: DataframeClientAPI> TableProvider for DataframeQueryTableProvider<T> {
    fn as_any(&self) -> &dyn Any {
        self
    }

    fn schema(&self) -> SchemaRef {
        Arc::clone(&self.schema)
    }

    fn table_type(&self) -> TableType {
        TableType::Base
    }

    #[tracing::instrument(level = "info", skip_all)]
    async fn scan(
        &self,
        state: &dyn Session,
        projection: Option<&Vec<usize>>,
        filters: &[Expr],
        limit: Option<usize>,
    ) -> datafusion::common::Result<Arc<dyn ExecutionPlan>> {
        let mut dataset_queries = vec![self.query_dataset_request.clone()];
        for filter in filters {
            if let Some(updated_queries) =
                apply_filter_expr_to_queries(dataset_queries.clone(), filter, &self.schema)?
            {
                dataset_queries = updated_queries;
            }
        }

        let mut query_expression = self.query_expression.clone();

        let mut chunk_info_batches = Vec::with_capacity(dataset_queries.len());

        for dataset_query in dataset_queries {
            let response_stream = self
                .client
                .clone()
                .query_dataset(
                    tonic::Request::new(dataset_query.into())
                        .with_entry_id(self.dataset_id)
                        .map_err(|err| exec_datafusion_err!("{err}"))?,
                )
                .await
                .map_err(|err| exec_datafusion_err!("{err}"))?
                .into_inner();

            let batches: Vec<RecordBatch> = response_stream
                .collect::<Vec<_>>()
                .await
                .into_iter()
                .collect::<Result<Vec<_>, _>>()
                .map_err(|err| exec_datafusion_err!("{err}"))?
                .into_iter()
                .filter_map(|response| response.data)
                .map(|dataframe_part| {
                    dataframe_part
                        .try_into()
                        .map_err(|err| exec_datafusion_err!("{err}"))
                })
                .collect::<Result<Vec<_>, _>>()?;

            chunk_info_batches.push(batches);
        }
        let chunk_info_batches = Arc::new(compute_unique_chunk_info_ids(chunk_info_batches)?);

        // Find the first column selection that is a component
        if query_expression.filtered_is_not_null.is_none() {
            let filters = filters.iter().collect::<Vec<_>>();
            query_expression.filtered_is_not_null =
                Self::compute_column_is_neq_null_filter(&filters)
                    .into_iter()
                    .flatten()
                    .next();
        }

        crate::SegmentStreamExec::try_new(
            &self.schema,
            self.sort_index,
            projection,
            state.config().target_partitions(),
            chunk_info_batches,
            query_expression,
            self.index_values.clone(),
            self.client.clone(),
            #[cfg(not(target_arch = "wasm32"))]
            self.trace_headers.clone(),
        )
        .map(Arc::new)
        .map(|exec| {
            Arc::new(SizedCoalesceBatchesExec::new(
                exec,
                CoalescerOptions {
                    target_batch_rows: DEFAULT_BATCH_ROWS,
                    target_batch_bytes: DEFAULT_BATCH_BYTES,
                    max_rows: limit,
                },
            )) as Arc<dyn ExecutionPlan>
        })
    }

    fn supports_filters_pushdown(
        &self,
        filters: &[&Expr],
    ) -> datafusion::common::Result<Vec<TableProviderFilterPushDown>> {
        let filter_columns = Self::compute_column_is_neq_null_filter(filters);
        let non_null_columns = filter_columns.iter().flatten().collect::<Vec<_>>();
        if let Some(col) = non_null_columns.first() {
            let col = *col;
            Ok(filter_columns
                .iter()
                .zip(filters)
                .map(|(column_selector, filter_expr)| {
                    if Some(col) == column_selector.as_ref() {
                        Ok(TableProviderFilterPushDown::Exact)
                    } else {
                        filter_expr_is_supported(
                            filter_expr,
                            &self.query_dataset_request,
                            &self.schema,
                        )
                    }
                })
                .collect::<Result<Vec<_>, DataFusionError>>()?)
        } else {
            Ok(filters
                .iter()
                .map(|filter_expr| {
                    filter_expr_is_supported(filter_expr, &self.query_dataset_request, &self.schema)
                })
                .collect::<Result<Vec<_>, DataFusionError>>()?)
        }
    }
}

/// Compute the output schema for a query on a dataset. When we call `get_dataset_schema`
/// on the Data Platform, we will get the schema for all entities and all components. This
/// method is used to down select from that full schema based on `query_expression`.
#[tracing::instrument(level = "trace", skip_all)]
fn compute_schema_for_query(
    dataset_schema: &Schema,
    query_expression: &QueryExpression,
) -> Result<SchemaRef, DataFusionError> {
    // Short circuit for empty datasets. Needed because `ChunkColumnDescriptors::try_from_arrow_fields`
    // needs row ids, which we only have for non-empty datasets.
    if dataset_schema.fields.is_empty() {
        return Ok(Arc::new(Schema::empty()));
    }

    // Schema returned from `get_dataset_schema` does not match the required ChunkColumnDescriptors ordering
    // which is row id, then time, then data. We don't need perfect ordering other than that.
    let mut fields = dataset_schema
        .fields()
        .iter()
        .map(Arc::clone)
        .collect::<Vec<_>>();
    fields.sort_by(|a, b| {
        let Ok(a) = ColumnKind::try_from(a.as_ref()) else {
            return Ordering::Equal;
        };
        let Ok(b) = ColumnKind::try_from(b.as_ref()) else {
            return Ordering::Equal;
        };

        match (a, b) {
            (ColumnKind::RowId, _) => Ordering::Less,
            (_, ColumnKind::RowId) => Ordering::Greater,
            (ColumnKind::Index, _) => Ordering::Less,
            (_, ColumnKind::Index) => Ordering::Greater,
            _ => Ordering::Equal,
        }
    });
    let fields: arrow::datatypes::Fields = fields.into();

    let column_descriptors = ChunkColumnDescriptors::try_from_arrow_fields(None, &fields)
        .map_err(|err| exec_datafusion_err!("col desc {err}"))?;

    // Create the actual filter to apply to the column descriptors
    let filter = ChunkStore::create_component_filter_from_query(query_expression);

    // When we call QueryDataset we will not return row_id, so we only select indices and
    // components from the column descriptors.
    let filtered_fields = column_descriptors
        .filter_components(filter)
        .indices_and_components()
        .into_iter()
        .map(|cd| cd.to_arrow_field(BatchType::Dataframe))
        .collect::<Vec<_>>();

    Ok(Arc::new(Schema::new_with_metadata(
        filtered_fields,
        dataset_schema.metadata().clone(),
    )))
}

#[tracing::instrument(level = "info", skip_all)]
pub(crate) fn prepend_string_column_schema(schema: &Schema, column_name: &str) -> Schema {
    let mut fields = vec![Field::new(column_name, DataType::Utf8, false)];
    fields.extend(schema.fields().iter().map(|f| (**f).clone()));
    Schema::new_with_metadata(fields, schema.metadata.clone())
}

/// We need to create `num_partitions` of DataFusion partition stream outputs, each of
/// which will be fed from multiple `rerun_segment_id` sources. The partitioning
/// output is a hash of the `rerun_segment_id`. We will reuse some of the
/// underlying execution code from `DataFusion`'s `RepartitionExec` to compute
/// these DataFusion partition IDs, just to be certain they match partitioning generated
/// from sources other than Rerun gRPC services.
/// This function will do the relevant grouping of chunk infos by chunk's segment id,
/// and we will eventually fire individual queries for each group. Segments must be ordered,
/// see `SegmentStreamExec::try_new` for more details.
#[tracing::instrument(level = "trace", skip_all)]
pub(crate) fn group_chunk_infos_by_segment_id(
    chunk_info_batches: &Arc<Vec<RecordBatch>>,
) -> Result<Arc<BTreeMap<String, Vec<RecordBatch>>>, DataFusionError> {
    let mut results = BTreeMap::new();

    for batch in chunk_info_batches.as_ref() {
        let segment_ids = batch
            .column_by_name(QueryDatasetResponse::FIELD_CHUNK_SEGMENT_ID)
            .ok_or(exec_datafusion_err!(
                "Unable to find {} column",
                QueryDatasetResponse::FIELD_CHUNK_SEGMENT_ID
            ))?
            .as_any()
            .downcast_ref::<StringArray>()
            .ok_or(exec_datafusion_err!(
                "{} must be string type",
                QueryDatasetResponse::FIELD_CHUNK_SEGMENT_ID
            ))?;

        // group rows by segment ID
        let mut segment_rows: BTreeMap<String, Vec<usize>> = BTreeMap::new();
        for (row_idx, segment_id) in segment_ids.iter().enumerate() {
            let sid = segment_id.ok_or(exec_datafusion_err!(
                "Found null segment id in {} column at row {row_idx}",
                QueryDatasetResponse::FIELD_CHUNK_SEGMENT_ID
            ))?;
            segment_rows
                .entry(sid.to_owned())
                .or_default()
                .push(row_idx);
        }

        for (segment_id, row_indices) in segment_rows {
            if row_indices.is_empty() {
                continue;
            }

            let segment_batch = re_arrow_util::take_record_batch(batch, &row_indices)?;

            results
                .entry(segment_id)
                .or_insert_with(Vec::new)
                .push(segment_batch);
        }
    }

    Ok(Arc::new(results))
}

#[tracing::instrument(level = "trace", skip_all)]
#[expect(dead_code)]
pub(crate) fn time_array_ref_to_i64(time_array: &ArrayRef) -> Result<Int64Array, DataFusionError> {
    Ok(match time_array.data_type() {
        DataType::Int64 => downcast_value!(time_array, Int64Array).reinterpret_cast::<Int64Type>(),
        DataType::Timestamp(TimeUnit::Second, _) => {
            let nano_array = downcast_value!(time_array, TimestampSecondArray);
            nano_array.reinterpret_cast::<Int64Type>()
        }
        DataType::Timestamp(TimeUnit::Millisecond, _) => {
            let nano_array = downcast_value!(time_array, TimestampMillisecondArray);
            nano_array.reinterpret_cast::<Int64Type>()
        }
        DataType::Timestamp(TimeUnit::Microsecond, _) => {
            let nano_array = downcast_value!(time_array, TimestampMicrosecondArray);
            nano_array.reinterpret_cast::<Int64Type>()
        }
        DataType::Timestamp(TimeUnit::Nanosecond, _) => {
            let nano_array = downcast_value!(time_array, TimestampNanosecondArray);
            nano_array.reinterpret_cast::<Int64Type>()
        }
        DataType::Duration(TimeUnit::Nanosecond) => {
            let duration_array = downcast_value!(time_array, DurationNanosecondArray);
            duration_array.reinterpret_cast::<Int64Type>()
        }
        _ => {
            return Err(exec_datafusion_err!(
                "Unexpected type for time column {}",
                time_array.data_type()
            ));
        }
    })
}

pub fn query_from_query_expression(query_expression: &QueryExpression) -> Query {
    let latest_at = if query_expression.is_static() {
        Some(QueryLatestAt::new_static())
    } else {
        query_expression
            .min_latest_at()
            .map(|latest_at| QueryLatestAt {
                index: Some(latest_at.timeline().to_string()),
                at: latest_at.at(),
            })
    };

    Query {
        latest_at,
        range: query_expression.max_range().map(|range| QueryRange {
            index: range.timeline().to_string(),
            index_range: range.range,
        }),
        columns_always_include_everything: false,
        columns_always_include_entity_paths: false,
        columns_always_include_byte_offsets: false,
        columns_always_include_static_indexes: false,
        columns_always_include_global_indexes: false,
        columns_always_include_component_indexes: false,
    }
}

fn compute_unique_chunk_info_ids(
    chunk_info_batches: Vec<Vec<RecordBatch>>,
) -> Result<Vec<RecordBatch>, DataFusionError> {
    let batches: Vec<_> = chunk_info_batches.into_iter().flatten().collect();
    if batches.is_empty() {
        return Ok(vec![]);
    }

    let schema = batches[0].schema();
    let combined = concat_batches(&schema, &batches)?;

    // Find the chunk_id column
    let chunk_id_col = combined
        .column_by_name("chunk_id")
        .ok_or(exec_datafusion_err!("chunk_id column not found"))?;

    let chunk_id_array = chunk_id_col
        .as_any()
        .downcast_ref::<FixedSizeBinaryArray>()
        .ok_or(exec_datafusion_err!("chunk_id is not FixedSizeBinary"))?;

    let mut indices_to_keep = Vec::new();
    let mut seen: HashSet<[u8; 16]> = HashSet::default();

    for row_idx in 0..combined.num_rows() {
        let chunk_id = chunk_id_array.value(row_idx);
        let chunk_id_fixed: [u8; 16] = chunk_id
            .try_into()
            .expect("chunk_id should be exactly 16 bytes");

        if seen.insert(chunk_id_fixed) {
            indices_to_keep.push(row_idx as u32);
        }
    }

    let indices = UInt32Array::from(indices_to_keep);

    let distinct_columns = arrow::compute::take_arrays(combined.columns(), &indices, None)?;

    Ok(vec![RecordBatch::try_new_with_options(
        schema,
        distinct_columns,
        &RecordBatchOptions::default(),
    )?])
}

#[cfg(test)]
mod tests {
    use std::collections::HashMap;

    use arrow::array::{Array as _, FixedSizeBinaryArray, FixedSizeBinaryBuilder};

    use super::*;

    #[test]
    fn test_batches_grouping() {
        let schema = Arc::new(Schema::new_with_metadata(
            vec![
                QueryDatasetResponse::field_chunk_segment_id(),
                QueryDatasetResponse::field_chunk_id(),
            ],
            HashMap::default(),
        ));

        let capacity = 4;
        let byte_width = 16;
        let mut chunk_id_builder = FixedSizeBinaryBuilder::with_capacity(capacity, byte_width);
        chunk_id_builder.append_value([0u8; 16]).unwrap();
        chunk_id_builder.append_value([1u8; 16]).unwrap();
        chunk_id_builder.append_value([2u8; 16]).unwrap();
        chunk_id_builder.append_value([3u8; 16]).unwrap();
        let chunk_id_array = Arc::new(chunk_id_builder.finish());

        let batch1 = RecordBatch::try_new_with_options(
            schema.clone(),
            vec![
                Arc::new(StringArray::from(vec![
                    Some("A"),
                    Some("B"),
                    Some("A"),
                    Some("C"),
                ])),
                chunk_id_array,
            ],
            &RecordBatchOptions::new().with_row_count(Some(4)),
        )
        .unwrap();

        let mut chunk_id_builder = FixedSizeBinaryBuilder::with_capacity(capacity, byte_width);
        chunk_id_builder.append_value([4u8; 16]).unwrap();
        chunk_id_builder.append_value([5u8; 16]).unwrap();
        chunk_id_builder.append_value([6u8; 16]).unwrap();
        let chunk_id_array = Arc::new(chunk_id_builder.finish());

        let batch2 = RecordBatch::try_new_with_options(
            schema.clone(),
            vec![
                Arc::new(StringArray::from(vec![Some("B"), Some("C"), Some("D")])),
                chunk_id_array,
            ],
            &RecordBatchOptions::new().with_row_count(Some(3)),
        )
        .unwrap();

        let chunk_info_batches = Arc::new(vec![batch1, batch2]);

        let grouped = group_chunk_infos_by_segment_id(&chunk_info_batches).unwrap();

        assert_eq!(grouped.len(), 4);

        let group_a = grouped.get("A").unwrap();
        assert_eq!(group_a.len(), 1);
        let chunk_ids_a = group_a[0]
            .column_by_name("chunk_id")
            .unwrap()
            .as_any()
            .downcast_ref::<FixedSizeBinaryArray>()
            .unwrap();
        assert_eq!(chunk_ids_a.len(), 2);
        assert_eq!(chunk_ids_a.value(0), [0u8; 16]);
        assert_eq!(chunk_ids_a.value(1), [2u8; 16]);

        let group_b = grouped.get("B").unwrap();
        assert_eq!(group_b.len(), 2);
        let chunk_ids_b1 = group_b[0]
            .column_by_name("chunk_id")
            .unwrap()
            .as_any()
            .downcast_ref::<FixedSizeBinaryArray>()
            .unwrap();
        assert_eq!(chunk_ids_b1.len(), 1);
        assert_eq!(chunk_ids_b1.value(0), [1u8; 16]);
        let chunk_ids_b2 = group_b[1]
            .column_by_name("chunk_id")
            .unwrap()
            .as_any()
            .downcast_ref::<FixedSizeBinaryArray>()
            .unwrap();
        assert_eq!(chunk_ids_b2.len(), 1);
        assert_eq!(chunk_ids_b2.value(0), [4u8; 16]);

        let group_c = grouped.get("C").unwrap();
        assert_eq!(group_c.len(), 2);
        let chunk_ids_c1 = group_c[0]
            .column_by_name("chunk_id")
            .unwrap()
            .as_any()
            .downcast_ref::<FixedSizeBinaryArray>()
            .unwrap();
        assert_eq!(chunk_ids_c1.len(), 1);
        assert_eq!(chunk_ids_c1.value(0), [3u8; 16]);
        let chunk_ids_c2 = group_c[1]
            .column_by_name("chunk_id")
            .unwrap()
            .as_any()
            .downcast_ref::<FixedSizeBinaryArray>()
            .unwrap();
        assert_eq!(chunk_ids_c2.len(), 1);
        assert_eq!(chunk_ids_c2.value(0), [5u8; 16]);

        let group_d = grouped.get("D").unwrap();
        assert_eq!(group_d.len(), 1);
        let chunk_ids_d = group_d[0]
            .column_by_name("chunk_id")
            .unwrap()
            .as_any()
            .downcast_ref::<FixedSizeBinaryArray>()
            .unwrap();
        assert_eq!(chunk_ids_d.len(), 1);
        assert_eq!(chunk_ids_d.value(0), [6u8; 16]);
    }
}