fix: Always return a PartitionRecord which maybe streams record batches
Connects to #6421. Even if the ingester doesn't have data in memory for a query, we need to send back metadata about the ingester UUID and the number of files persisted so that the querier can decide whether it needs to refresh the cache.pull/24376/head
parent
473ce7a268
commit
07772e8d22
|
@ -1041,8 +1041,9 @@ mod tests {
|
||||||
let partition = partitions.pop().unwrap();
|
let partition = partitions.pop().unwrap();
|
||||||
|
|
||||||
// Perform the partition read
|
// Perform the partition read
|
||||||
let batches =
|
let batches = datafusion::physical_plan::common::collect(
|
||||||
datafusion::physical_plan::common::collect(partition.into_record_batch_stream())
|
partition.into_record_batch_stream().unwrap(),
|
||||||
|
)
|
||||||
.await
|
.await
|
||||||
.expect("failed to collate query results");
|
.expect("failed to collate query results");
|
||||||
|
|
||||||
|
|
|
@ -267,7 +267,7 @@ where
|
||||||
);
|
);
|
||||||
|
|
||||||
// Gather the partition data from all of the partitions in this table.
|
// Gather the partition data from all of the partitions in this table.
|
||||||
let partitions = self.partitions().into_iter().filter_map(move |p| {
|
let partitions = self.partitions().into_iter().map(move |p| {
|
||||||
let mut span = SpanRecorder::new(span.clone().map(|s| s.child("partition read")));
|
let mut span = SpanRecorder::new(span.clone().map(|s| s.child("partition read")));
|
||||||
|
|
||||||
let (id, completed_persistence_count, data) = {
|
let (id, completed_persistence_count, data) = {
|
||||||
|
@ -275,9 +275,12 @@ where
|
||||||
(
|
(
|
||||||
p.partition_id(),
|
p.partition_id(),
|
||||||
p.completed_persistence_count(),
|
p.completed_persistence_count(),
|
||||||
p.get_query_data()?,
|
p.get_query_data(),
|
||||||
)
|
)
|
||||||
};
|
};
|
||||||
|
|
||||||
|
let ret = match data {
|
||||||
|
Some(data) => {
|
||||||
assert_eq!(id, data.partition_id());
|
assert_eq!(id, data.partition_id());
|
||||||
|
|
||||||
// Project the data if necessary
|
// Project the data if necessary
|
||||||
|
@ -288,17 +291,16 @@ where
|
||||||
Projection::Some(columns.as_ref())
|
Projection::Some(columns.as_ref())
|
||||||
};
|
};
|
||||||
|
|
||||||
let ret = PartitionResponse::new(
|
let data = Box::pin(MemoryStream::new(
|
||||||
Box::pin(MemoryStream::new(
|
|
||||||
data.project_selection(selection).into_iter().collect(),
|
data.project_selection(selection).into_iter().collect(),
|
||||||
)),
|
));
|
||||||
id,
|
PartitionResponse::new(data, id, None, completed_persistence_count)
|
||||||
None,
|
}
|
||||||
completed_persistence_count,
|
None => PartitionResponse::new_no_batches(id, None, completed_persistence_count),
|
||||||
);
|
};
|
||||||
|
|
||||||
span.ok("read partition data");
|
span.ok("read partition data");
|
||||||
Some(ret)
|
ret
|
||||||
});
|
});
|
||||||
|
|
||||||
Ok(PartitionStream::new(futures::stream::iter(partitions)))
|
Ok(PartitionStream::new(futures::stream::iter(partitions)))
|
||||||
|
|
|
@ -8,7 +8,7 @@ use datafusion::physical_plan::SendableRecordBatchStream;
|
||||||
/// Response data for a single partition.
|
/// Response data for a single partition.
|
||||||
pub(crate) struct PartitionResponse {
|
pub(crate) struct PartitionResponse {
|
||||||
/// Stream of snapshots.
|
/// Stream of snapshots.
|
||||||
batches: SendableRecordBatchStream,
|
batches: Option<SendableRecordBatchStream>,
|
||||||
|
|
||||||
/// Partition ID.
|
/// Partition ID.
|
||||||
id: PartitionId,
|
id: PartitionId,
|
||||||
|
@ -42,7 +42,20 @@ impl PartitionResponse {
|
||||||
completed_persistence_count: u64,
|
completed_persistence_count: u64,
|
||||||
) -> Self {
|
) -> Self {
|
||||||
Self {
|
Self {
|
||||||
batches,
|
batches: Some(batches),
|
||||||
|
id,
|
||||||
|
max_persisted_sequence_number,
|
||||||
|
completed_persistence_count,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
pub(crate) fn new_no_batches(
|
||||||
|
id: PartitionId,
|
||||||
|
max_persisted_sequence_number: Option<SequenceNumber>,
|
||||||
|
completed_persistence_count: u64,
|
||||||
|
) -> Self {
|
||||||
|
Self {
|
||||||
|
batches: None,
|
||||||
id,
|
id,
|
||||||
max_persisted_sequence_number,
|
max_persisted_sequence_number,
|
||||||
completed_persistence_count,
|
completed_persistence_count,
|
||||||
|
@ -61,7 +74,7 @@ impl PartitionResponse {
|
||||||
self.completed_persistence_count
|
self.completed_persistence_count
|
||||||
}
|
}
|
||||||
|
|
||||||
pub(crate) fn into_record_batch_stream(self) -> SendableRecordBatchStream {
|
pub(crate) fn into_record_batch_stream(self) -> Option<SendableRecordBatchStream> {
|
||||||
self.batches
|
self.batches
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -51,6 +51,7 @@ impl QueryResponse {
|
||||||
/// Reduce the [`QueryResponse`] to a stream of [`RecordBatch`].
|
/// Reduce the [`QueryResponse`] to a stream of [`RecordBatch`].
|
||||||
pub(crate) fn into_record_batches(self) -> impl Stream<Item = Result<RecordBatch, ArrowError>> {
|
pub(crate) fn into_record_batches(self) -> impl Stream<Item = Result<RecordBatch, ArrowError>> {
|
||||||
self.into_partition_stream()
|
self.into_partition_stream()
|
||||||
.flat_map(|partition| partition.into_record_batch_stream())
|
.flat_map(|partition| futures::stream::iter(partition.into_record_batch_stream()))
|
||||||
|
.flatten()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -322,11 +322,13 @@ impl From<QueryResponse> for FlatIngesterQueryResponseStream {
|
||||||
completed_persistence_count,
|
completed_persistence_count,
|
||||||
})
|
})
|
||||||
});
|
});
|
||||||
let tail = partition
|
|
||||||
.into_record_batch_stream()
|
match partition.into_record_batch_stream() {
|
||||||
.flat_map(|snapshot_res| match snapshot_res {
|
Some(stream) => {
|
||||||
|
let tail = stream.flat_map(|snapshot_res| match snapshot_res {
|
||||||
Ok(snapshot) => {
|
Ok(snapshot) => {
|
||||||
let schema = Arc::new(prepare_schema_for_flight(&snapshot.schema()));
|
let schema =
|
||||||
|
Arc::new(prepare_schema_for_flight(&snapshot.schema()));
|
||||||
|
|
||||||
let schema_captured = Arc::clone(&schema);
|
let schema_captured = Arc::clone(&schema);
|
||||||
let head = futures::stream::once(async {
|
let head = futures::stream::once(async {
|
||||||
|
@ -335,8 +337,10 @@ impl From<QueryResponse> for FlatIngesterQueryResponseStream {
|
||||||
})
|
})
|
||||||
});
|
});
|
||||||
|
|
||||||
let tail =
|
let tail = match prepare_batch_for_flight(
|
||||||
match prepare_batch_for_flight(&snapshot, Arc::clone(&schema)) {
|
&snapshot,
|
||||||
|
Arc::clone(&schema),
|
||||||
|
) {
|
||||||
Ok(batch) => {
|
Ok(batch) => {
|
||||||
futures::stream::iter(split_batch_for_grpc_response(batch))
|
futures::stream::iter(split_batch_for_grpc_response(batch))
|
||||||
.map(|batch| {
|
.map(|batch| {
|
||||||
|
@ -353,6 +357,9 @@ impl From<QueryResponse> for FlatIngesterQueryResponseStream {
|
||||||
});
|
});
|
||||||
|
|
||||||
head.chain(tail).boxed()
|
head.chain(tail).boxed()
|
||||||
|
}
|
||||||
|
None => head.boxed(),
|
||||||
|
}
|
||||||
})
|
})
|
||||||
.boxed()
|
.boxed()
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue