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
// Copyright 2023 Lance Developers.
//
// 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::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};

use arrow_array::{cast::as_primitive_array, RecordBatch, UInt64Array};
use arrow_schema::{Schema as ArrowSchema, SchemaRef};
use datafusion::error::{DataFusionError, Result};
use datafusion::physical_plan::{
    DisplayAs, DisplayFormatType, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream,
};
use futures::stream::{self, Stream, StreamExt, TryStreamExt};
use futures::{Future, FutureExt};
use tokio::sync::mpsc::{self, Receiver};
use tokio::task::JoinHandle;
use tracing::{instrument, Instrument};

use crate::dataset::{Dataset, ROW_ID};
use crate::datatypes::Schema;
use crate::{arrow::*, Error};

/// Dataset Take Node.
///
/// [Take] node takes the filtered batch from the child node.
///
/// It uses the `_rowid` to random access on [Dataset] to gather the final results.
pub struct Take {
    rx: Receiver<Result<RecordBatch>>,
    bg_thread: Option<JoinHandle<()>>,

    output_schema: SchemaRef,
}

impl Take {
    /// Create a Take node with
    ///
    ///  - Dataset: the dataset to read from
    ///  - projection: extra columns to take from the dataset.
    ///  - output_schema: the output schema of the take node.
    ///  - child: the upstream stream to feed data in.
    ///  - batch_readahead: max number of batches to readahead, potentially concurrently
    #[instrument(level = "debug", skip_all, name = "Take::new")]
    fn new(
        dataset: Arc<Dataset>,
        projection: Arc<Schema>,
        output_schema: SchemaRef,
        child: SendableRecordBatchStream,
        batch_readahead: usize,
    ) -> Self {
        let (tx, rx) = mpsc::channel(4);

        let bg_thread = tokio::spawn(
            async move {
                if let Err(e) = child
                    .zip(stream::repeat_with(|| {
                        (dataset.clone(), projection.clone())
                    }))
                    .map(|(batch, (dataset, extra))| async move {
                        Self::take_batch(batch?, dataset, extra).await
                    })
                    .buffered(batch_readahead)
                    .map(|r| r.map_err(|e| DataFusionError::Execution(e.to_string())))
                    .try_for_each(|b| async {
                        if tx.send(Ok(b)).await.is_err() {
                        // If channel is closed, make sure we return an error to end the stream. 
                        return Err(DataFusionError::Internal(
                            "ExecNode(Take): channel closed".to_string(),
                        ));
                        }
                        Ok(())
                    })
                    .await
                {
                    if let Err(e) = tx.send(Err(e)).await {
                        if let Err(e) = e.0 {
                            // if channel was closed, it was cancelled by the receiver.
                            // But if there was a different error we should send it
                            // or log it.
                            if !e.to_string().contains("channel closed") {
                                log::error!("channel was closed by receiver, but error occurred in background thread: {:?}", e);
                            }
                        }
                    }
                }
                drop(tx)
            }
            .in_current_span(),
        );

        Self {
            rx,
            bg_thread: Some(bg_thread),
            output_schema,
        }
    }

    /// Given a batch with a _rowid column, retrieve extra columns from dataset.
    // This method mostly exists to annotate the Send bound so the compiler
    // doesn't produce a higher-order lifetime error.
    // manually implemented async for Send bound
    #[allow(clippy::manual_async_fn)]
    #[instrument(level = "debug", skip_all)]
    fn take_batch(
        batch: RecordBatch,
        dataset: Arc<Dataset>,
        extra: Arc<Schema>,
    ) -> impl Future<Output = Result<RecordBatch, Error>> + Send {
        async move {
            let row_id_arr = batch.column_by_name(ROW_ID).unwrap();
            let row_ids: &UInt64Array = as_primitive_array(row_id_arr);
            let rows = if extra.fields.is_empty() {
                batch
            } else {
                let new_columns = dataset.take_rows(row_ids.values(), &extra).await?;
                debug_assert_eq!(batch.num_rows(), new_columns.num_rows());
                batch.merge(&new_columns)?
            };
            Ok::<RecordBatch, Error>(rows)
        }
        .in_current_span()
    }
}

impl Stream for Take {
    type Item = Result<RecordBatch>;

    fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
        let this = Pin::into_inner(self);
        // We need to check the JoinHandle to make sure the thread hasn't panicked.
        let bg_thread_completed = if let Some(bg_thread) = &mut this.bg_thread {
            match bg_thread.poll_unpin(cx) {
                Poll::Ready(Ok(())) => true,
                Poll::Ready(Err(join_error)) => {
                    return Poll::Ready(Some(Err(DataFusionError::Execution(format!(
                        "ExecNode(Take): thread panicked: {}",
                        join_error
                    )))));
                }
                Poll::Pending => false,
            }
        } else {
            false
        };
        if bg_thread_completed {
            // Need to take it, since we aren't allowed to poll if again after.
            this.bg_thread.take();
        }
        // this.rx.
        this.rx.poll_recv(cx)
    }
}

impl RecordBatchStream for Take {
    fn schema(&self) -> SchemaRef {
        self.output_schema.clone()
    }
}

/// [`TakeExec`] is a [`ExecutionPlan`] that enriches the input [`RecordBatch`]
/// with extra columns from [`Dataset`].
///
/// The rows are identified by the inexplicit row IDs from `input` plan.
///
/// The output schema will be the input schema, merged with extra schemas from the dataset.
#[derive(Debug)]
pub struct TakeExec {
    /// Dataset to read from.
    dataset: Arc<Dataset>,

    pub(crate) extra_schema: Arc<Schema>,

    input: Arc<dyn ExecutionPlan>,

    /// Output schema is the merged schema between input schema and extra schema.
    output_schema: Schema,

    batch_readahead: usize,
}

impl DisplayAs for TakeExec {
    fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
        match t {
            DisplayFormatType::Default | DisplayFormatType::Verbose => {
                let columns = self
                    .output_schema
                    .fields
                    .iter()
                    .map(|f| f.name.as_str())
                    .collect::<Vec<_>>()
                    .join(", ");
                write!(f, "Take: columns={:?}", columns)
            }
        }
    }
}

impl TakeExec {
    /// Create a [`TakeExec`] node.
    ///
    /// - dataset: the dataset to read from
    /// - input: the upstream [`ExecutionPlan`] to feed data in.
    /// - extra_schema: the extra schema to take / read from the dataset.
    pub fn try_new(
        dataset: Arc<Dataset>,
        input: Arc<dyn ExecutionPlan>,
        extra_schema: Arc<Schema>,
        batch_readahead: usize,
    ) -> Result<Self> {
        if input.schema().column_with_name(ROW_ID).is_none() {
            return Err(DataFusionError::Plan(
                "TakeExec requires the input plan to have a column named '_rowid'".to_string(),
            ));
        }

        let input_schema = Schema::try_from(input.schema().as_ref())?;
        let output_schema = input_schema.merge(extra_schema.as_ref())?;

        let remaining_schema = extra_schema.exclude(&input_schema)?;

        Ok(Self {
            dataset,
            extra_schema: Arc::new(remaining_schema),
            input,
            output_schema,
            batch_readahead,
        })
    }
}

impl ExecutionPlan for TakeExec {
    fn as_any(&self) -> &dyn std::any::Any {
        self
    }

    fn schema(&self) -> SchemaRef {
        ArrowSchema::try_from(&self.output_schema).unwrap().into()
    }

    fn output_partitioning(&self) -> datafusion::physical_plan::Partitioning {
        self.input.output_partitioning()
    }

    fn output_ordering(&self) -> Option<&[datafusion::physical_expr::PhysicalSortExpr]> {
        self.input.output_ordering()
    }

    fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
        vec![self.input.clone()]
    }

    fn with_new_children(
        self: Arc<Self>,
        _children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        Ok(Arc::new(Self {
            dataset: self.dataset.clone(),
            extra_schema: self.extra_schema.clone(),
            input: _children[0].clone(),
            output_schema: self.output_schema.clone(),
            batch_readahead: self.batch_readahead,
        }))
    }

    fn execute(
        &self,
        partition: usize,
        context: Arc<datafusion::execution::context::TaskContext>,
    ) -> Result<SendableRecordBatchStream> {
        let input_stream = self.input.execute(partition, context)?;
        Ok(Box::pin(Take::new(
            self.dataset.clone(),
            self.extra_schema.clone(),
            self.schema(),
            input_stream,
            self.batch_readahead,
        )))
    }

    fn statistics(&self) -> datafusion::physical_plan::Statistics {
        self.input.statistics()
    }
}

#[cfg(test)]
mod tests {
    use super::*;

    use arrow_array::{ArrayRef, Float32Array, Int32Array, RecordBatchIterator, StringArray};
    use arrow_schema::{DataType, Field};
    use tempfile::tempdir;

    use crate::{dataset::WriteParams, io::exec::LanceScanExec};

    async fn create_dataset() -> Arc<Dataset> {
        let schema = Arc::new(ArrowSchema::new(vec![
            Field::new("i", DataType::Int32, false),
            Field::new("f", DataType::Float32, false),
            Field::new("s", DataType::Utf8, false),
        ]));

        // Write 3 batches.
        let expected_batches: Vec<RecordBatch> = (0..3)
            .map(|batch_id| {
                let value_range = batch_id * 10..batch_id * 10 + 10;
                let columns: Vec<ArrayRef> = vec![
                    Arc::new(Int32Array::from_iter_values(value_range.clone())),
                    Arc::new(Float32Array::from_iter(
                        value_range.clone().map(|v| v as f32),
                    )),
                    Arc::new(StringArray::from_iter_values(
                        value_range.map(|v| format!("str-{v}")),
                    )),
                ];
                RecordBatch::try_new(schema.clone(), columns).unwrap()
            })
            .collect();

        let test_dir = tempdir().unwrap();
        let test_uri = test_dir.path().to_str().unwrap();
        let params = WriteParams {
            max_rows_per_group: 10,
            ..Default::default()
        };
        let reader =
            RecordBatchIterator::new(expected_batches.clone().into_iter().map(Ok), schema.clone());
        Dataset::write(reader, test_uri, Some(params))
            .await
            .unwrap();

        Arc::new(Dataset::open(test_uri).await.unwrap())
    }

    #[tokio::test]
    async fn test_take_schema() {
        let dataset = create_dataset().await;

        let scan_arrow_schema = ArrowSchema::new(vec![Field::new("i", DataType::Int32, false)]);
        let scan_schema = Arc::new(Schema::try_from(&scan_arrow_schema).unwrap());

        let extra_arrow_schema = ArrowSchema::new(vec![Field::new("s", DataType::Int32, false)]);
        let extra_schema = Arc::new(Schema::try_from(&extra_arrow_schema).unwrap());

        // With row id
        let input = Arc::new(LanceScanExec::new(
            dataset.clone(),
            dataset.fragments().clone(),
            scan_schema,
            10,
            10,
            4,
            true,
            false,
            true,
        ));
        let take_exec = TakeExec::try_new(dataset, input, extra_schema, 10).unwrap();
        let schema = take_exec.schema();
        assert_eq!(
            schema.fields.iter().map(|f| f.name()).collect::<Vec<_>>(),
            vec!["i", ROW_ID, "s"]
        );
    }

    #[tokio::test]
    async fn test_take_no_extra_columns() {
        let dataset = create_dataset().await;

        let scan_arrow_schema = ArrowSchema::new(vec![
            Field::new("i", DataType::Int32, false),
            Field::new("s", DataType::Int32, false),
        ]);
        let scan_schema = Arc::new(Schema::try_from(&scan_arrow_schema).unwrap());

        // Extra column is already read.
        let extra_arrow_schema = ArrowSchema::new(vec![Field::new("s", DataType::Int32, false)]);
        let extra_schema = Arc::new(Schema::try_from(&extra_arrow_schema).unwrap());

        let input = Arc::new(LanceScanExec::new(
            dataset.clone(),
            dataset.fragments().clone(),
            scan_schema,
            10,
            10,
            4,
            true,
            false,
            true,
        ));
        let take_exec = TakeExec::try_new(dataset, input, extra_schema, 10).unwrap();
        let schema = take_exec.schema();
        assert_eq!(
            schema.fields.iter().map(|f| f.name()).collect::<Vec<_>>(),
            vec!["i", "s", ROW_ID]
        );
    }

    #[tokio::test]
    async fn test_take_no_row_id() {
        let dataset = create_dataset().await;

        let scan_arrow_schema = ArrowSchema::new(vec![
            Field::new("i", DataType::Int32, false),
            Field::new("s", DataType::Int32, false),
        ]);
        let scan_schema = Arc::new(Schema::try_from(&scan_arrow_schema).unwrap());

        let extra_arrow_schema = ArrowSchema::new(vec![Field::new("s", DataType::Int32, false)]);
        let extra_schema = Arc::new(Schema::try_from(&extra_arrow_schema).unwrap());

        // No row ID
        let input = Arc::new(LanceScanExec::new(
            dataset.clone(),
            dataset.fragments().clone(),
            scan_schema,
            10,
            10,
            4,
            false,
            false,
            true,
        ));
        assert!(TakeExec::try_new(dataset, input, extra_schema, 10).is_err());
    }
}