re_datafusion/
grpc_streaming_provider.rs

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
use std::{any::Any, pin::Pin, sync::Arc};

use async_trait::async_trait;

use arrow::{array::RecordBatch, datatypes::SchemaRef};
use datafusion::{
    catalog::{Session, TableProvider},
    error::{DataFusionError, Result as DataFusionResult},
    execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext},
    physical_plan::{
        streaming::{PartitionStream, StreamingTableExec},
        ExecutionPlan,
    },
    prelude::Expr,
};
use futures_util::StreamExt as _;
use tokio_stream::Stream;

#[async_trait]
pub trait GrpcStreamToTable:
    std::fmt::Debug + 'static + Send + Sync + Clone + std::marker::Unpin
{
    type GrpcStreamData;

    async fn fetch_schema(&mut self) -> DataFusionResult<SchemaRef>;

    fn process_response(&mut self, response: Self::GrpcStreamData)
        -> DataFusionResult<RecordBatch>;

    async fn send_streaming_request(
        &mut self,
    ) -> DataFusionResult<tonic::Response<tonic::Streaming<Self::GrpcStreamData>>>;
}

#[derive(Debug)]
pub struct GrpcStreamProvider<T: GrpcStreamToTable> {
    schema: SchemaRef,
    client: T,
}

impl<T: GrpcStreamToTable> GrpcStreamProvider<T> {
    pub async fn prepare(mut client: T) -> Result<Arc<Self>, DataFusionError> {
        let schema = client.fetch_schema().await?;
        Ok(Arc::new(Self { schema, client }))
    }
}

#[async_trait]
impl<T> TableProvider for GrpcStreamProvider<T>
where
    T: GrpcStreamToTable + Send + 'static,
    T::GrpcStreamData: Send + 'static,
{
    fn as_any(&self) -> &dyn Any {
        self
    }

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

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

    async fn scan(
        &self,
        _state: &dyn Session,
        projection: Option<&Vec<usize>>,
        _filters: &[Expr],
        _limit: Option<usize>,
    ) -> DataFusionResult<Arc<dyn ExecutionPlan>> {
        StreamingTableExec::try_new(
            self.schema.clone(),
            vec![Arc::new(GrpcStreamPartitionStream::new(
                &self.schema,
                self.client.clone(),
            ))],
            projection,
            Vec::default(),
            false,
            None,
        )
        .map(|e| Arc::new(e) as Arc<dyn ExecutionPlan>)
    }
}

#[derive(Debug)]
pub struct GrpcStreamPartitionStream<T: GrpcStreamToTable> {
    schema: SchemaRef,
    client: T,
}

impl<T: GrpcStreamToTable> GrpcStreamPartitionStream<T> {
    fn new(schema: &SchemaRef, client: T) -> Self {
        Self {
            schema: Arc::clone(schema),
            client,
        }
    }
}

impl<T> PartitionStream for GrpcStreamPartitionStream<T>
where
    T: GrpcStreamToTable + Send + 'static,
    T::GrpcStreamData: Send + 'static,
{
    fn schema(&self) -> &SchemaRef {
        &self.schema
    }

    fn execute(&self, _ctx: Arc<TaskContext>) -> SendableRecordBatchStream {
        Box::pin(GrpcStream::execute(&self.schema, self.client.clone()))
    }
}

pub struct GrpcStream {
    schema: SchemaRef,
    adapted_stream: Pin<Box<dyn Stream<Item = datafusion::common::Result<RecordBatch>> + Send>>,
}

impl GrpcStream {
    fn execute<T>(schema: &SchemaRef, mut client: T) -> Self
    where
        T::GrpcStreamData: Send + 'static,
        T: GrpcStreamToTable + Send + 'static,
    {
        let adapted_stream = Box::pin(async_stream::try_stream! {
            let mut stream = client.send_streaming_request().await.map_err(|err| DataFusionError::External(Box::new(
                tonic::Status::internal(err.to_string())
            )))?.into_inner();

            while let Some(msg) = stream.next().await {
                let msg = msg.map_err(|err| DataFusionError::External(Box::new(err)))?;
                let processed = client
                    .process_response(msg)
                    .map_err(|err| DataFusionError::External(Box::new(
                        tonic::Status::internal(err.to_string())
                    )))?;
                yield processed;
            }
        });

        Self {
            schema: Arc::clone(schema),
            adapted_stream,
        }
    }
}

impl RecordBatchStream for GrpcStream {
    fn schema(&self) -> SchemaRef {
        Arc::clone(&self.schema)
    }
}

impl Stream for GrpcStream {
    type Item = DataFusionResult<RecordBatch>;

    fn poll_next(
        mut self: Pin<&mut Self>,
        cx: &mut std::task::Context<'_>,
    ) -> std::task::Poll<Option<Self::Item>> {
        self.adapted_stream.poll_next_unpin(cx)
    }
}