From 524af05d3d7feb15f3f9e5f98d47ed4cd62297f9 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 9 Aug 2023 14:56:37 -0400 Subject: [PATCH 01/14] Prototype TopK operator --- datafusion/core/src/physical_plan/mod.rs | 2 + .../core/src/physical_plan/sorts/sort.rs | 77 ++- datafusion/core/src/physical_plan/topk/mod.rs | 516 ++++++++++++++++++ .../tests/sqllogictests/test_files/aal.slt | 202 +++++++ .../tests/sqllogictests/test_files/window.slt | 101 ++-- 5 files changed, 825 insertions(+), 73 deletions(-) create mode 100644 datafusion/core/src/physical_plan/topk/mod.rs create mode 100644 datafusion/core/tests/sqllogictests/test_files/aal.slt diff --git a/datafusion/core/src/physical_plan/mod.rs b/datafusion/core/src/physical_plan/mod.rs index c73e61aea1904..c60dbd6a44c5d 100644 --- a/datafusion/core/src/physical_plan/mod.rs +++ b/datafusion/core/src/physical_plan/mod.rs @@ -17,6 +17,7 @@ //! Traits for physical query plan, supporting parallel execution for partitioned relations. +mod topk; mod visitor; pub use self::metrics::Metric; use self::metrics::MetricsSet; @@ -27,6 +28,7 @@ use crate::datasource::physical_plan::FileScanConfig; use crate::physical_plan::expressions::PhysicalSortExpr; use datafusion_common::Result; pub use datafusion_common::{ColumnStatistics, Statistics}; +pub use topk::TopK; pub use visitor::{accept, visit_execution_plan, ExecutionPlanVisitor}; use arrow::datatypes::SchemaRef; diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 52936dc55e6e1..9f6ff0fb6d03f 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -26,6 +26,7 @@ use crate::physical_plan::metrics::{ }; use crate::physical_plan::sorts::merge::streaming_merge; use crate::physical_plan::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; +use crate::physical_plan::topk::TopK; use crate::physical_plan::{ DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, @@ -759,7 +760,12 @@ impl DisplayAs for SortExec { let expr: Vec = self.expr.iter().map(|e| e.to_string()).collect(); match self.fetch { Some(fetch) => { - write!(f, "SortExec: fetch={fetch}, expr=[{}]", expr.join(",")) + write!( + f, + // TODO should this say topk? + "SortExec: fetch={fetch}, expr=[{}]", + expr.join(",") + ) } None => write!(f, "SortExec: expr=[{}]", expr.join(",")), } @@ -847,29 +853,54 @@ impl ExecutionPlan for SortExec { trace!("End SortExec's input.execute for partition: {}", partition); - let mut sorter = ExternalSorter::new( - partition, - input.schema(), - self.expr.clone(), - context.session_config().batch_size(), - self.fetch, - execution_options.sort_spill_reservation_bytes, - execution_options.sort_in_place_threshold_bytes, - &self.metrics_set, - context.runtime_env(), - ); + if let Some(fetch) = self.fetch.as_ref() { + let mut topk = TopK::try_new( + partition, + input.schema(), + self.expr.clone(), + *fetch, + context.session_config().batch_size(), + context.runtime_env(), + &self.metrics_set, + partition, + )?; + + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + futures::stream::once(async move { + while let Some(batch) = input.next().await { + let batch = batch?; + topk.insert_batch(batch)?; + } + topk.emit() + }) + .try_flatten(), + ))) + } else { + let mut sorter = ExternalSorter::new( + partition, + input.schema(), + self.expr.clone(), + context.session_config().batch_size(), + self.fetch, + execution_options.sort_spill_reservation_bytes, + execution_options.sort_in_place_threshold_bytes, + &self.metrics_set, + context.runtime_env(), + ); - Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema(), - futures::stream::once(async move { - while let Some(batch) = input.next().await { - let batch = batch?; - sorter.insert_batch(batch).await?; - } - sorter.sort() - }) - .try_flatten(), - ))) + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + futures::stream::once(async move { + while let Some(batch) = input.next().await { + let batch = batch?; + sorter.insert_batch(batch).await?; + } + sorter.sort() + }) + .try_flatten(), + ))) + } } fn metrics(&self) -> Option { diff --git a/datafusion/core/src/physical_plan/topk/mod.rs b/datafusion/core/src/physical_plan/topk/mod.rs new file mode 100644 index 0000000000000..d626f0806698b --- /dev/null +++ b/datafusion/core/src/physical_plan/topk/mod.rs @@ -0,0 +1,516 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! TopK: Combination of Sort / LIMIT + +use arrow::{ + compute::interleave, + row::{OwnedRow, RowConverter, Rows, SortField}, +}; +use std::{cmp::Ordering, sync::Arc}; + +use arrow_array::{Array, ArrayRef, RecordBatch}; +use arrow_schema::SchemaRef; +use datafusion_common::Result; +use datafusion_execution::{ + memory_pool::{MemoryConsumer, MemoryReservation}, + runtime_env::RuntimeEnv, +}; +use datafusion_physical_expr::PhysicalSortExpr; +use hashbrown::HashMap; + +use crate::physical_plan::{stream::RecordBatchStreamAdapter, SendableRecordBatchStream}; + +use super::metrics::{BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder}; + +/// Global TopK +/// +/// # Background +/// +/// "Top K" is a common query optimization used for queries such as +/// "find the top 3 customers by revenue". The (simplified) SQL for +/// such a query might be: +/// +/// ```sql +/// SELECT customer_id, revenue FROM 'sales.csv' ORDER BY revenue DESC limit 3; +/// ``` +/// +/// The simple plan would be: +/// +/// ``` +/// > explain SELECT customer_id, revenue FROM sales ORDER BY revenue DESC limit 3; +/// +--------------+----------------------------------------+ +/// | plan_type | plan | +/// +--------------+----------------------------------------+ +/// | logical_plan | Limit: 3 | +/// | | Sort: revenue DESC NULLS FIRST | +/// | | Projection: customer_id, revenue | +/// | | TableScan: sales | +/// +--------------+----------------------------------------+ +/// ``` +/// +/// While this plan produces the correct answer, it will fully sorts the +/// input before discarding everything other than the top 3 elements. +/// +/// The same answer can be produced by simply keeping track of the top +/// N elements, reducing the total amount of required buffer memory. +/// +/// # Structure +/// +/// This operator tracks the top K items using a `TopKHeap`. +pub struct TopK { + /// schema of the output (and the input) + schema: SchemaRef, + /// Runtime metrics + metrics: TopKMetrics, + /// Reservation + reservation: MemoryReservation, + /// The target number of rows for output batches + batch_size: usize, + /// sort expressions + expr: Arc<[PhysicalSortExpr]>, + /// row converter, for sort keys + row_converter: RowConverter, + /// scratch space for converting rows + scratch_rows: Rows, + /// stores the top k values and their sort key values, in order + heap: TopKHeap, +} + +impl TopK { + /// Create a new [`TopK`] that stores the top `k` values, as + /// defined by the sort expressions in `expr`. + // TOOD: make a builder or some other nicer API to avoid the + // clippy warning + #[allow(clippy::too_many_arguments)] + pub fn try_new( + partition_id: usize, + schema: SchemaRef, + expr: Vec, + k: usize, + batch_size: usize, + runtime: Arc, + metrics: &ExecutionPlanMetricsSet, + partition: usize, + ) -> Result { + let reservation = MemoryConsumer::new(format!("TopK[{partition_id}]")) + .register(&runtime.memory_pool); + + let expr: Arc<[PhysicalSortExpr]> = expr.into(); + + let sort_fields: Vec<_> = expr + .iter() + .map(|e| { + Ok(SortField::new_with_options( + e.expr.data_type(&schema)?, + e.options, + )) + }) + .collect::>()?; + + let row_converter = RowConverter::new(sort_fields)?; + let scratch_rows = row_converter.empty_rows( + batch_size, + 20 * batch_size, // guestimate 20 bytes per row + ); + + Ok(Self { + schema, + metrics: TopKMetrics::new(metrics, partition), + reservation, + batch_size, + expr, + row_converter, + scratch_rows, + heap: TopKHeap::new(k), + }) + } + + /// Insert `batch`, remembering it if any of its values are among + /// the top k seen so far. + pub fn insert_batch(&mut self, batch: RecordBatch) -> Result<()> { + // Updates on drop + let _timer = self.metrics.baseline.elapsed_compute().timer(); + + let sort_keys: Vec = self + .expr + .iter() + .map(|expr| { + let value = expr.expr.evaluate(&batch)?; + Ok(value.into_array(batch.num_rows())) + }) + .collect::>>()?; + + // reuse existing `Rows` to avoid reallocations + let rows = &mut self.scratch_rows; + rows.clear(); + self.row_converter.append(rows, &sort_keys)?; + + // TODO make this algorithmically better?: + // 1. only check topk values in rows + // 2. only do one update through top_k + + let mut batch_entry = self.heap.register_batch(batch); + for (index, row) in rows.iter().enumerate() { + match self.heap.k_largest() { + // heap has k items, and the current row is not + // smaller than the curret smallest k value, skip + Some(largest) if largest.row.row() <= row => {} + // don't yet have k items or new item is greater than + // current min top k + None | Some(_) => { + self.heap.add(&mut batch_entry, row.owned(), index); + self.metrics.row_replacements.add(1); + } + } + } + self.heap.insert_batch_entry(batch_entry); + + // update memory reservation + self.reservation.try_resize(self.size())?; + Ok(()) + } + + /// Returns the top k results broken into `batch_size` [`RecordBatch`]es + pub fn emit(self) -> Result { + let Self { + schema, + metrics, + reservation: _, + batch_size, + expr: _, + row_converter: _, + scratch_rows: _, + heap, + } = self; + let _timer = metrics.baseline.elapsed_compute().timer(); // time updated on drop + + let mut batch = heap.emit(schema.clone())?; + metrics.baseline.output_rows().add(batch.num_rows()); + + // break into record batches as needed + let mut batches = vec![]; + loop { + if batch.num_rows() < batch_size { + batches.push(Ok(batch)); + break; + } else { + batches.push(Ok(batch.slice(0, batch_size))); + batch = batch.slice(batch_size, batch.num_rows()); + } + } + Ok(Box::pin(RecordBatchStreamAdapter::new( + schema, + futures::stream::iter(batches), + ))) + } + + /// return the size of memory used by this operator, in bytes + fn size(&self) -> usize { + std::mem::size_of::() + + self.row_converter.size() + + self.scratch_rows.size() + + self.heap.size() + } +} + +struct TopKMetrics { + /// metrics + pub baseline: BaselineMetrics, + + /// count of how many rows were replaced in the heap + pub row_replacements: Count, +} + +impl TopKMetrics { + fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self { + Self { + baseline: BaselineMetrics::new(metrics, partition), + row_replacements: MetricBuilder::new(metrics) + .counter("row_replacements", partition), + } + } +} + +/// This structure keeps at most the *smallest* k items, using the +/// [arrow::row] format for sort keys. While it is called "topK" for +/// values like `1, 2, 3, 4, 5` the "top 3" really means the +/// *smallest* 3 , `1, 2, 3`, not the *largest* 3 `3, 4, 5`. +/// +/// Using the `Row` format handles things such as ascending vs +/// descending and nulls first vs nulls last. +/// +/// It doesn't use `BinaryHeap` in the Rust standard library because +/// it is important to check the current minimum value in the heap +/// prior to creating a new value to insert. +struct TopKHeap { + /// The maximum size of this heap. + k: usize, + /// Storage for up at most `k` items, in ascending + /// order. `inner[0]` holds the smallest value of the smallest k + /// so far, `inner[len-1]` holds the largest value smallest k so far. + inner: Vec, + /// Storage the original row values (TopKRow only has the sort key) + store: RecordBatchStore, + /// The size of all `OwnedRows`s held by this heap + owned_row_bytes: usize, +} + +impl TopKHeap { + fn new(k: usize) -> Self { + assert!(k > 0); + Self { + k, + inner: Vec::with_capacity(k), + store: RecordBatchStore::new(), + owned_row_bytes: 0, + } + } + + /// Register a [`RecordBatch`] with the heap, returning the + /// appropriate entry + pub fn register_batch(&mut self, batch: RecordBatch) -> RecordBatchEntry { + self.store.register(batch) + } + + /// Insert a [`RecordBatchEntry`] created by a previous call to + /// [`Self::register_batch`] into storage. + pub fn insert_batch_entry(&mut self, entry: RecordBatchEntry) { + self.store.insert(entry) + } + + /// Returns the largest value stored by the heap if there are k + /// items, otherwise returns None + fn k_largest(&self) -> Option<&TopKRow> { + if self.inner.len() < self.k { + None + } else { + self.inner.last() + } + } + + /// Adds `row` to this heap. If inserting this new item would + /// increase the size past `k`, removes the previously smallest + /// item. + fn add(&mut self, batch_entry: &mut RecordBatchEntry, row: OwnedRow, index: usize) { + assert!(self.inner.len() <= self.k); + + batch_entry.uses += 1; + + self.owned_row_bytes += owned_row_size(&row); + + // put the new row into the correct location to maintain that + // self.inner is sorted in descending order + let insertion_point = self + .inner + .partition_point(|current_row| current_row.row <= row); + self.inner.insert( + insertion_point, + TopKRow { + row, + batch_id: batch_entry.id, + index, + }, + ); + + // limit size to k items + if self.inner.len() > self.k { + // If there was a previous minimum value, decrement its use + if let Some(prev_min) = self.inner.pop() { + if prev_min.batch_id == batch_entry.id { + batch_entry.uses -= 1; + } else { + self.store.unuse(prev_min.batch_id); + } + // update memory accounting + let prev_size = owned_row_size(&prev_min.row); + assert!(self.owned_row_bytes >= prev_size); + self.owned_row_bytes -= prev_size; + } + } + } + + /// Returns the values stored in this heap, from values low to high, as a single + /// [`RecordBatch`] + pub fn emit(&self, schema: SchemaRef) -> Result { + // Indicies for each row within its respective RecordBatch + let indicies: Vec<_> = self + .inner + .iter() + .enumerate() + .map(|(i, k)| (i, k.index)) + .collect(); + + let num_columns = { + let Some(first_value) = self.inner.get(0) else { + return Ok(RecordBatch::new_empty(schema)); + }; + self.store + .get(first_value.batch_id) + .expect("invalid batch id") + .batch + .num_columns() + }; + + // build the output columns one at time, using the + // `interleave` kernel to pick rows from different arrays + let output_columns: Vec<_> = (0..num_columns) + .map(|col| { + let input_arrays: Vec<_> = self + .inner + .iter() + .map(|k| { + let entry = + self.store.get(k.batch_id).expect("invalid stored batch id"); + entry.batch.column(col) as &dyn Array + }) + .collect(); + + // at this point `indices` contains indexes within the + // rows and `input_arrays` contains a reference to the + // relevant Array for that index. `interleave` pulls + // them together into a single new array + Ok(interleave(&input_arrays, &indicies)?) + }) + .collect::>()?; + + Ok(RecordBatch::try_new(schema, output_columns)?) + } + + /// return the size of memory used by this heap, in bytes + fn size(&self) -> usize { + std::mem::size_of::() + + (self.inner.capacity() * std::mem::size_of::()) + + self.store.size() + + self.owned_row_bytes + } +} + +/// Size of memory owned by `row` until row::size() is available +/// TODO file upstream ticket in arrow-rs to add this +fn owned_row_size(row: &OwnedRow) -> usize { + std::mem::size_of_val(row) + row.as_ref().len() // underlying data, doesn't account for capacity +} + +/// Represents one of the top K rows. Orders according to `OwnedRow` +#[derive(Debug, PartialEq)] +struct TopKRow { + /// the value of the sort key for this row + row: OwnedRow, + /// the index in this record batch the row came from + index: usize, + /// the RecordBatch this row came from: an id into a [`RecordBatchStore`] + batch_id: u32, +} + +impl Eq for TopKRow {} + +impl PartialOrd for TopKRow { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for TopKRow { + fn cmp(&self, other: &Self) -> Ordering { + self.row.cmp(&other.row) + } +} + +#[derive(Debug)] +struct RecordBatchEntry { + id: u32, + batch: RecordBatch, + // for this batch, how many times has it been used + uses: usize, +} + +/// This structure tracks [`RecordBatch`] by an id so that: +/// +/// 1. The baches can be tracked via an id that can be copied cheaply +/// 2. The total memory held by all batches is tracked +#[derive(Debug)] +struct RecordBatchStore { + /// id generator + next_id: u32, + /// storage + batches: HashMap, + /// total size of all record batches tracked by this store + batches_size: usize, +} + +impl RecordBatchStore { + fn new() -> Self { + Self { + next_id: 0, + batches: HashMap::new(), + batches_size: 0, + } + } + + /// Register this batch with the store and assign an ID. No + /// attempt is made to compare this batch to other batches + pub fn register(&mut self, batch: RecordBatch) -> RecordBatchEntry { + let id = self.next_id; + self.next_id += 1; + RecordBatchEntry { id, batch, uses: 0 } + } + + /// Insert a record batch entry into this store, tracking its + /// memory use, if it has any uses + pub fn insert(&mut self, entry: RecordBatchEntry) { + // uses of 0 means that none of the rows in the batch were stored in the topk + if entry.uses > 0 { + self.batches_size += entry.batch.get_array_memory_size(); + self.batches.insert(entry.id, entry); + } + } + + fn get(&self, id: u32) -> Option<&RecordBatchEntry> { + self.batches.get(&id) + } + + /// remove a use from the specified batch id. If the use count + /// reaches zero the batch entry is removed from the store + /// + /// panics if there were no remaining uses of id + pub fn unuse(&mut self, id: u32) { + let remove = if let Some(batch_entry) = self.batches.get_mut(&id) { + batch_entry.uses = batch_entry.uses.checked_sub(1).expect("underflow"); + batch_entry.uses == 0 + } else { + panic!("No entry for id {id}"); + }; + + if remove { + let old_entry = self.batches.remove(&id).unwrap(); + self.batches_size = self + .batches_size + .checked_sub(old_entry.batch.get_array_memory_size()) + .unwrap(); + } + } + + /// returns the size of memory used by this store, including all + /// referenced `RecordBatch`es, in bytes + pub fn size(&self) -> usize { + std::mem::size_of::() + + self.batches.capacity() + * (std::mem::size_of::() + std::mem::size_of::()) + + self.batches_size + } +} diff --git a/datafusion/core/tests/sqllogictests/test_files/aal.slt b/datafusion/core/tests/sqllogictests/test_files/aal.slt new file mode 100644 index 0000000000000..f19c79e8d1cb1 --- /dev/null +++ b/datafusion/core/tests/sqllogictests/test_files/aal.slt @@ -0,0 +1,202 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +# Tests for development + +statement ok +create table aal(x int) as values (10), (2), (3), (0), (5), (4), (3), (2), (1), (3), (8); + +query I +select * from aal order by x; +---- +0 +1 +2 +2 +3 +3 +3 +4 +5 +8 +10 + +query I +select * from aal order by x limit 3; +---- +0 +1 +2 + +query I +select * from aal order by x desc limit 3; +---- +10 +8 +5 + + + + +statement ok +CREATE EXTERNAL TABLE aggregate_test_100 ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 BIGINT UNSIGNED NOT NULL, + c10 VARCHAR NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL +) +STORED AS CSV +WITH HEADER ROW +LOCATION '../../testing/data/csv/aggregate_test_100.csv' + +query TT +explain select * from aggregate_test_100 ORDER BY c13 desc limit 5; +---- +logical_plan +Limit: skip=0, fetch=5 +--Sort: aggregate_test_100.c13 DESC NULLS FIRST, fetch=5 +----TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13] +physical_plan +GlobalLimitExec: skip=0, fetch=5 +--SortExec: fetch=5, expr=[c13@12 DESC] +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true + + + + +query T +select c13 from aggregate_test_100 ORDER BY c13; +---- +0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm +0keZ5G8BffGwgF2RwQD59TFzMStxCB +0og6hSkhbX8AC1ktFS4kounvTzy8Vo +1aOcrEGd0cOqZe2I5XBOm0nDcwtBZO +2T3wSlHdEmASmO0xcXHnndkKEt6bz8 +3BEOHQsMEFZ58VcNTOJYShTBpAPzbt +4HX6feIvmNXBN7XGqgO4YVBkhu8GDI +4JznSdBajNWhu4hRQwjV1FjTTxY68i +52mKlRE3aHCBZtjECq6sY9OqVf8Dze +56MZa5O1hVtX4c5sbnCfxuX5kDChqI +6FPJlLAcaQ5uokyOWZ9HGdLZObFvOZ +6WfVFBVGJSQb7FhA7E0lBwdvjfZnSW +6oIXZuIPIqEoPBvFmbt2Nxy3tryGUE +6x93sxYioWuq5c9Kkk8oTAAORM7cH0 +802bgTGl6Bk5TlkPYYTxp5JkKyaYUA +8LIh0b6jmDGm87BmIyjdxNIpX4ugjD +90gAtmGEeIqUTbo1ZrxCvWtsseukXC +9UbObCsVkmYpJGcGrgfK90qOnwb2Lj +AFGCj7OWlEB5QfniEFgonMq90Tq5uH +ALuRhobVWbnQTTWZdSOk0iVe8oYFhW +Amn2K87Db5Es3dFQO9cw9cvpAM6h35 +AyYVExXK6AR2qUTxNZ7qRHQOVGMLcz +BJqx5WokrmrrezZA0dUbleMYkG5U2O +BPtQMxnuSPpxMExYV9YkDa6cAN7GP3 +BsM5ZAYifRh5Lw3Y8X1r53I0cTJnfE +C2GT5KVyOPZpgKVl110TyZO0NcJ434 +DuJNG8tufSqW0ZstHqWj3aGvFLMg4A +EcCuckwsF3gV1Ecgmh5v4KM8g1ozif +ErJFw6hzZ5fmI5r8bhE4JzlscnhKZU +F7NSTjWvQJyBburN7CXRUlbgp2dIrA +Fi4rJeTQq4eXj8Lxg3Hja5hBVTVV5u +H5j5ZHy1FGesOAHjkQEDYCucbpKWRu +HKSMQ9nTnwXCJIte1JrM1dtYnDtJ8g +IWl0G3ZlMNf7WT8yjIB49cx7MmYOmr +IZTkHMLvIKuiLjhDjYMmIHxh166we4 +Ig1QcuKsjHXkproePdERo2w0mYzIqd +JHNgc2UCaiXOdmkxwDDyGhRlO0mnBQ +JN0VclewmjwYlSl8386MlWv5rEhWCz +JafwVLSVk5AVoXFuzclesQ000EE2k1 +KJFcmTVjdkCMv94wYCtfHMFhzyRsmH +Ktb7GQ0N1DrxwkCkEUsTaIXk0xYinn +Ld2ej8NEv5zNcqU60FwpHeZKBhfpiV +LiEBxds3X0Uw0lxiYjDqrkAaAwoiIW +MXhhH1Var3OzzJCtI9VNyYvA0q8UyJ +MeSTAXq8gVxVjbEjgkvU9YLte0X9uE +NEhyk8uIx4kEULJGa8qIyFjjBcP2G6 +O66j6PaYuZhEUtqV6fuU7TyjM2WxC5 +OF7fQ37GzaZ5ikA2oMyvleKtgnLjXh +OPwBqCEK5PWTjWaiOyL45u2NLTaDWv +Oq6J4Rx6nde0YlhOIJkFsX2MsSvAQ0 +Ow5PGpfTm4dXCfTDsXAOTatXRoAydR +QEHVvcP8gxI6EMJIrvcnIhgzPNjIvv +QJYm7YRA3YetcBHI5wkMZeLXVmfuNy +QYlaIAnJA6r8rlAb6f59wcxvcPcWFf +RilTlL1tKkPOUFuzmLydHAVZwv1OGl +Sfx0vxv1skzZWT1PqVdoRDdO6Sb6xH +TTQUwpMNSXZqVBKAFvXu7OlWvKXJKX +TtDKUZxzVxsq758G6AWPSYuZgVgbcl +VDhtJkYjAYPykCgOU9x3v7v3t4SO1a +VY0zXmXeksCT8BzvpzpPLbmU9Kp9Y4 +Vp3gmWunM5A7wOC9YW2JroFqTWjvTi +WHmjWk2AY4c6m7DA4GitUx6nmb1yYS +XemNcT1xp61xcM1Qz3wZ1VECCnq06O +Z2sWcQr0qyCJRMHDpRy3aQr7PkHtkK +aDxBtor7Icd9C5hnTvvw5NrIre740e +akiiY5N0I44CMwEnBL6RTBk7BRkxEj +b3b9esRhTzFEawbs6XhpKnD9ojutHB +bgK1r6v3BCTh0aejJUhkA1Hn6idXGp +cBGc0kSm32ylBDnxogG727C0uhZEYZ +cq4WSAIFwx3wwTUS5bp1wCe71R6U5I +dVdvo6nUD5FgCgsbOZLds28RyGTpnx +e2Gh6Ov8XkXoFdJWhl0EjwEHlMDYyG +f9ALCzwDAKmdu7Rk2msJaB1wxe5IBX +fuyvs0w7WsKSlXqJ1e6HFSoLmx03AG +gTpyQnEODMcpsPnJMZC66gh33i3m0b +gpo8K5qtYePve6jyPt6xgJx4YOVjms +gxfHWUF8XgY2KdFxigxvNEXe2V2XMl +i6RQVXKUh7MzuGMDaNclUYnFUAireU +ioEncce3mPOXD2hWhpZpCPWGATG6GU +jQimhdepw3GKmioWUlVSWeBVRKFkY3 +l7uwDoTepWwnAP0ufqtHJS3CRi7RfP +lqhzgLsXZ8JhtpeeUWWNbMz8PHI705 +m6jD0LBIQWaMfenwRCTANI9eOdyyto +mhjME0zBHbrK6NMkytMTQzOssOa1gF +mzbkwXKrPeZnxg2Kn1LRF5hYSsmksS +nYVJnVicpGRqKZibHyBAmtmzBXAFfT +oHJMNvWuunsIMIWFnYG31RCfkOo2V7 +oLZ21P2JEDooxV1pU31cIxQHEeeoLu +okOkcWflkNXIy4R8LzmySyY1EC3sYd +pLk3i59bZwd5KBZrI1FiweYTd5hteG +pTeu0WMjBRTaNRT15rLCuEh3tBJVc5 +qnPOOmslCJaT45buUisMRnM0rc77EK +t6fQUjJejPcjc04wHvHTPe55S65B4V +ukOiFGGFnQJDHFgZxHMpvhD3zybF0M +ukyD7b0Efj7tNlFSRmzZ0IqkEzg2a8 +waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs +wwXqSGKLyBQyPkonlzBNYUJTCo4LRS +xipQ93429ksjNcXPX5326VSg1xJZcW +y7C453hRWd4E7ImjNDWlpexB8nUqjh +ydkwycaISlYSlEq3TlkS2m15I2pcp8 + + +query TIIIIIIIITRRT +select * from aggregate_test_100 ORDER BY c13 desc limit 5; +---- +a 4 -38 20744 762932956 308913475857409919 7 45465 1787652631 878137512938218976 0.7459874 0.021825780392 ydkwycaISlYSlEq3TlkS2m15I2pcp8 +d 1 -98 13630 -1991133944 1184110014998006843 220 2986 225513085 9634106610243643486 0.89651865 0.164088254508 y7C453hRWd4E7ImjNDWlpexB8nUqjh +e 2 52 -12056 -1090239422 9011500141803970147 238 4168 2013662838 12565360638488684051 0.6694766 0.391444365692 xipQ93429ksjNcXPX5326VSg1xJZcW +d 1 -72 25590 1188089983 3090286296481837049 241 832 3542840110 5885937420286765261 0.41980565 0.215354023438 wwXqSGKLyBQyPkonlzBNYUJTCo4LRS +a 1 -5 12636 794623392 2909750622865366631 15 24022 2669374863 4776679784701509574 0.29877836 0.253725340799 waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs diff --git a/datafusion/core/tests/sqllogictests/test_files/window.slt b/datafusion/core/tests/sqllogictests/test_files/window.slt index cd257aaa92de7..45a3bb5834503 100644 --- a/datafusion/core/tests/sqllogictests/test_files/window.slt +++ b/datafusion/core/tests/sqllogictests/test_files/window.slt @@ -2597,6 +2597,7 @@ SELECT # test_source_sorted_builtin query TT EXPLAIN SELECT + ts, FIRST_VALUE(inc_col) OVER(ORDER BY ts RANGE BETWEEN 10 PRECEDING and 1 FOLLOWING) as fv1, FIRST_VALUE(inc_col) OVER(ORDER BY ts ROWS BETWEEN 10 PRECEDING and 1 FOLLOWING) as fv2, LAST_VALUE(inc_col) OVER(ORDER BY ts RANGE BETWEEN 10 PRECEDING and 1 FOLLOWING) as lv1, @@ -2626,24 +2627,23 @@ EXPLAIN SELECT LIMIT 5; ---- logical_plan -Projection: fv1, fv2, lv1, lv2, nv1, nv2, rn1, rn2, rank1, rank2, dense_rank1, dense_rank2, lag1, lag2, lead1, lead2, fvr1, fvr2, lvr1, lvr2, lagr1, lagr2, leadr1, leadr2 ---Limit: skip=0, fetch=5 -----Sort: annotated_data_finite.ts DESC NULLS FIRST, fetch=5 -------Projection: FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS leadr2, annotated_data_finite.ts ---------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] -----------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] -------------TableScan: annotated_data_finite projection=[ts, inc_col] +Limit: skip=0, fetch=5 +--Sort: annotated_data_finite.ts DESC NULLS FIRST, fetch=5 +----Projection: annotated_data_finite.ts, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS leadr2 +------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] +--------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] +----------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan -ProjectionExec: expr=[fv1@0 as fv1, fv2@1 as fv2, lv1@2 as lv1, lv2@3 as lv2, nv1@4 as nv1, nv2@5 as nv2, rn1@6 as rn1, rn2@7 as rn2, rank1@8 as rank1, rank2@9 as rank2, dense_rank1@10 as dense_rank1, dense_rank2@11 as dense_rank2, lag1@12 as lag1, lag2@13 as lag2, lead1@14 as lead1, lead2@15 as lead2, fvr1@16 as fvr1, fvr2@17 as fvr2, lvr1@18 as lvr1, lvr2@19 as lvr2, lagr1@20 as lagr1, lagr2@21 as lagr2, leadr1@22 as leadr1, leadr2@23 as leadr2] ---GlobalLimitExec: skip=0, fetch=5 -----SortExec: fetch=5, expr=[ts@24 DESC] -------ProjectionExec: expr=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2, ts@0 as ts] ---------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }], mode=[Sorted] -----------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }], mode=[Sorted] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true +GlobalLimitExec: skip=0, fetch=5 +--SortExec: fetch=5, expr=[ts@0 DESC] +----ProjectionExec: expr=[ts@0 as ts, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2] +------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }], mode=[Sorted] +--------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }], mode=[Sorted] +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true -query IIIIIIIIIIIIIIIIIIIIIIII +query IIIIIIIIIIIIIIIIIIIIIIIII SELECT + ts, FIRST_VALUE(inc_col) OVER(ORDER BY ts RANGE BETWEEN 10 PRECEDING and 1 FOLLOWING) as fv1, FIRST_VALUE(inc_col) OVER(ORDER BY ts ROWS BETWEEN 10 PRECEDING and 1 FOLLOWING) as fv2, LAST_VALUE(inc_col) OVER(ORDER BY ts RANGE BETWEEN 10 PRECEDING and 1 FOLLOWING) as lv1, @@ -2672,11 +2672,11 @@ SELECT ORDER BY ts DESC LIMIT 5; ---- -289 269 305 305 305 283 100 100 99 99 86 86 301 296 301 1004 305 305 301 301 1001 1002 1001 289 -289 266 305 305 305 278 99 99 99 99 86 86 296 291 296 1004 305 305 301 296 305 1002 305 286 -289 261 296 301 NULL 275 98 98 98 98 85 85 291 289 291 1004 305 305 296 291 301 305 301 283 -286 259 291 296 NULL 272 97 97 97 97 84 84 289 286 289 1004 305 305 291 289 296 301 296 278 -275 254 289 291 289 269 96 96 96 96 83 83 286 283 286 305 305 305 289 286 291 296 291 275 +264 289 266 305 305 305 278 99 99 99 99 86 86 296 291 296 1004 305 305 301 296 305 1002 305 286 +264 289 269 305 305 305 283 100 100 99 99 86 86 301 296 301 1004 305 305 301 301 1001 1002 1001 289 +262 289 261 296 301 NULL 275 98 98 98 98 85 85 291 289 291 1004 305 305 296 291 301 305 301 283 +258 286 259 291 296 NULL 272 97 97 97 97 84 84 289 286 289 1004 305 305 291 289 296 301 296 278 +254 275 254 289 291 289 269 96 96 96 96 83 83 286 283 286 305 305 305 289 286 291 296 291 275 # test_source_sorted_unbounded_preceding @@ -3197,8 +3197,9 @@ drop table annotated_data_infinite2 # window3 spec is not used in window functions. # The query should still work. -query RR +query IRR SELECT + C3, MAX(c12) OVER window1, MIN(c12) OVER window2 as max1 FROM aggregate_test_100 @@ -3208,14 +3209,15 @@ SELECT ORDER BY C3 LIMIT 5 ---- -0.970671228336 0.970671228336 -0.850672105305 0.850672105305 -0.152498292972 0.152498292972 -0.369363046006 0.369363046006 -0.56535284223 0.56535284223 +-117 0.850672105305 0.850672105305 +-117 0.970671228336 0.970671228336 +-111 0.152498292972 0.152498292972 +-107 0.369363046006 0.369363046006 +-106 0.56535284223 0.56535284223 query TT EXPLAIN SELECT + C3, MAX(c12) OVER window1 as min1, MIN(c12) OVER window2 as max1 FROM aggregate_test_100 @@ -3226,30 +3228,29 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: min1, max1 ---Limit: skip=0, fetch=5 -----Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 -------Projection: MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max1, aggregate_test_100.c3 ---------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------Projection: aggregate_test_100.c3, aggregate_test_100.c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING -------------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] ---------------TableScan: aggregate_test_100 projection=[c3, c11, c12] +Limit: skip=0, fetch=5 +--Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 +----Projection: aggregate_test_100.c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max1 +------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +--------Projection: aggregate_test_100.c3, aggregate_test_100.c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING +----------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +------------TableScan: aggregate_test_100 projection=[c3, c11, c12] physical_plan -ProjectionExec: expr=[min1@0 as min1, max1@1 as max1] ---GlobalLimitExec: skip=0, fetch=5 -----SortExec: fetch=5, expr=[c3@2 ASC NULLS LAST] -------ProjectionExec: expr=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as max1, c3@0 as c3] ---------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow }], mode=[Sorted] -----------SortExec: expr=[c12@1 ASC NULLS LAST] -------------ProjectionExec: expr=[c3@0 as c3, c12@2 as c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] ---------------WindowAggExec: wdw=[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }] -----------------SortExec: expr=[c11@1 ASC NULLS LAST] -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], has_header=true +GlobalLimitExec: skip=0, fetch=5 +--SortExec: fetch=5, expr=[c3@0 ASC NULLS LAST] +----ProjectionExec: expr=[c3@0 as c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as max1] +------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--------SortExec: expr=[c12@1 ASC NULLS LAST] +----------ProjectionExec: expr=[c3@0 as c3, c12@2 as c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] +------------WindowAggExec: wdw=[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }] +--------------SortExec: expr=[c11@1 ASC NULLS LAST] +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], has_header=true # window1 spec is used multiple times under different aggregations. # The query should still work. -query RR +query IRR SELECT + C3, MAX(c12) OVER window1 as min1, MIN(c12) OVER window1 as max1 FROM aggregate_test_100 @@ -3257,11 +3258,11 @@ SELECT ORDER BY C3 LIMIT 5 ---- -0.970671228336 0.014793053078 -0.850672105305 0.014793053078 -0.152498292972 0.014793053078 -0.369363046006 0.014793053078 -0.56535284223 0.014793053078 +-117 0.850672105305 0.014793053078 +-117 0.970671228336 0.014793053078 +-111 0.152498292972 0.014793053078 +-107 0.369363046006 0.014793053078 +-106 0.56535284223 0.014793053078 query TT EXPLAIN SELECT From d4c09f283397fa817ee48c1ae285fea11c6de661 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 14 Aug 2023 14:45:28 -0400 Subject: [PATCH 02/14] Avoid use of Row --- datafusion/core/src/physical_plan/topk/mod.rs | 141 ++++++++++++------ 1 file changed, 94 insertions(+), 47 deletions(-) diff --git a/datafusion/core/src/physical_plan/topk/mod.rs b/datafusion/core/src/physical_plan/topk/mod.rs index d626f0806698b..06d3ad33c161c 100644 --- a/datafusion/core/src/physical_plan/topk/mod.rs +++ b/datafusion/core/src/physical_plan/topk/mod.rs @@ -19,7 +19,7 @@ use arrow::{ compute::interleave, - row::{OwnedRow, RowConverter, Rows, SortField}, + row::{RowConverter, Rows, SortField}, }; use std::{cmp::Ordering, sync::Arc}; @@ -169,11 +169,11 @@ impl TopK { match self.heap.k_largest() { // heap has k items, and the current row is not // smaller than the curret smallest k value, skip - Some(largest) if largest.row.row() <= row => {} + Some(largest) if largest.row.as_slice() <= row.as_ref() => {} // don't yet have k items or new item is greater than // current min top k None | Some(_) => { - self.heap.add(&mut batch_entry, row.owned(), index); + self.heap.add(&mut batch_entry, row, index); self.metrics.row_replacements.add(1); } } @@ -210,7 +210,8 @@ impl TopK { break; } else { batches.push(Ok(batch.slice(0, batch_size))); - batch = batch.slice(batch_size, batch.num_rows()); + let remaining_length = batch.num_rows() - batch_size; + batch = batch.slice(batch_size, remaining_length); } } Ok(Box::pin(RecordBatchStreamAdapter::new( @@ -266,8 +267,8 @@ struct TopKHeap { inner: Vec, /// Storage the original row values (TopKRow only has the sort key) store: RecordBatchStore, - /// The size of all `OwnedRows`s held by this heap - owned_row_bytes: usize, + /// The size of all owned data held by this heap + owned_bytes: usize, } impl TopKHeap { @@ -277,7 +278,7 @@ impl TopKHeap { k, inner: Vec::with_capacity(k), store: RecordBatchStore::new(), - owned_row_bytes: 0, + owned_bytes: 0, } } @@ -306,42 +307,44 @@ impl TopKHeap { /// Adds `row` to this heap. If inserting this new item would /// increase the size past `k`, removes the previously smallest /// item. - fn add(&mut self, batch_entry: &mut RecordBatchEntry, row: OwnedRow, index: usize) { + fn add( + &mut self, + batch_entry: &mut RecordBatchEntry, + row: impl AsRef<[u8]>, + index: usize, + ) { + let batch_id = batch_entry.id; + batch_entry.uses += 1; + assert!(self.inner.len() <= self.k); + let row = row.as_ref(); - batch_entry.uses += 1; + // Reuse storage for evicted item if possible + let new_top_k = if self.inner.len() == self.k { + let prev_min = self.inner.pop().unwrap(); + + // Update batch use + if prev_min.batch_id == batch_entry.id { + batch_entry.uses -= 1; + } else { + self.store.unuse(prev_min.batch_id); + } - self.owned_row_bytes += owned_row_size(&row); + // update memory accounting + self.owned_bytes -= prev_min.owned_size(); + prev_min.with_new_row(row, batch_id, index) + } else { + TopKRow::new(row, batch_id, index) + }; + + self.owned_bytes += new_top_k.owned_size(); // put the new row into the correct location to maintain that // self.inner is sorted in descending order let insertion_point = self .inner - .partition_point(|current_row| current_row.row <= row); - self.inner.insert( - insertion_point, - TopKRow { - row, - batch_id: batch_entry.id, - index, - }, - ); - - // limit size to k items - if self.inner.len() > self.k { - // If there was a previous minimum value, decrement its use - if let Some(prev_min) = self.inner.pop() { - if prev_min.batch_id == batch_entry.id { - batch_entry.uses -= 1; - } else { - self.store.unuse(prev_min.batch_id); - } - // update memory accounting - let prev_size = owned_row_size(&prev_min.row); - assert!(self.owned_row_bytes >= prev_size); - self.owned_row_bytes -= prev_size; - } - } + .partition_point(|current_row| current_row.row() <= row.as_ref()); + self.inner.insert(insertion_point, new_top_k); } /// Returns the values stored in this heap, from values low to high, as a single @@ -396,25 +399,69 @@ impl TopKHeap { std::mem::size_of::() + (self.inner.capacity() * std::mem::size_of::()) + self.store.size() - + self.owned_row_bytes + + self.owned_bytes } } -/// Size of memory owned by `row` until row::size() is available -/// TODO file upstream ticket in arrow-rs to add this -fn owned_row_size(row: &OwnedRow) -> usize { - std::mem::size_of_val(row) + row.as_ref().len() // underlying data, doesn't account for capacity -} - -/// Represents one of the top K rows. Orders according to `OwnedRow` +/// Represents one of the top K rows held in this heap. Orders +/// according to memcmp of row (e.g. the arrow Row format, but could +/// also be primtive values) +/// +/// Reuses allocations to minimize runtime overhead of creating new Vecs #[derive(Debug, PartialEq)] struct TopKRow { - /// the value of the sort key for this row - row: OwnedRow, - /// the index in this record batch the row came from - index: usize, + /// the value of the sort key for this row. This contains the + /// bytes that could be stored in `OwnedRow` but uses `Vec` to + /// reuse allocations. + row: Vec, /// the RecordBatch this row came from: an id into a [`RecordBatchStore`] batch_id: u32, + /// the index in this record batch the row came from + index: usize, +} + +impl TopKRow { + /// Create a new TopKRow with new allocation + fn new(row: impl AsRef<[u8]>, batch_id: u32, index: usize) -> Self { + Self { + row: row.as_ref().to_vec(), + batch_id, + index, + } + } + + /// Create a new TopKRow reusing the existing allocation + fn with_new_row( + self, + new_row: impl AsRef<[u8]>, + batch_id: u32, + index: usize, + ) -> Self { + let Self { + mut row, + batch_id: _, + index: _, + } = self; + row.clear(); + row.extend_from_slice(new_row.as_ref()); + + Self { + row, + batch_id, + index, + } + } + + /// Returns the number of bytes owned by this row in the heap (not + /// including itself) + fn owned_size(&self) -> usize { + self.row.capacity() + } + + /// Returns a slice to the owned row value + fn row(&self) -> &[u8] { + self.row.as_slice() + } } impl Eq for TopKRow {} From 948c1a2b6578749beecc8b3456a8af96b82ecd8b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 22 Aug 2023 14:26:00 -0400 Subject: [PATCH 03/14] start working on compaction --- datafusion/core/src/physical_plan/mod.rs | 2 +- datafusion/core/src/physical_plan/topk/mod.rs | 47 ++++++++++++------- 2 files changed, 30 insertions(+), 19 deletions(-) diff --git a/datafusion/core/src/physical_plan/mod.rs b/datafusion/core/src/physical_plan/mod.rs index f544d7cce5e53..ce13e46a7ec6a 100644 --- a/datafusion/core/src/physical_plan/mod.rs +++ b/datafusion/core/src/physical_plan/mod.rs @@ -27,8 +27,8 @@ use self::{ use crate::datasource::physical_plan::FileScanConfig; use crate::physical_plan::expressions::PhysicalSortExpr; use datafusion_common::Result; -pub use topk::TopK; pub use datafusion_common::{internal_err, ColumnStatistics, Statistics}; +pub use topk::TopK; pub use visitor::{accept, visit_execution_plan, ExecutionPlanVisitor}; use arrow::datatypes::SchemaRef; diff --git a/datafusion/core/src/physical_plan/topk/mod.rs b/datafusion/core/src/physical_plan/topk/mod.rs index 06d3ad33c161c..2efb65fefff2d 100644 --- a/datafusion/core/src/physical_plan/topk/mod.rs +++ b/datafusion/core/src/physical_plan/topk/mod.rs @@ -129,14 +129,14 @@ impl TopK { ); Ok(Self { - schema, + schema: schema.clone(), metrics: TopKMetrics::new(metrics, partition), reservation, batch_size, expr, row_converter, scratch_rows, - heap: TopKHeap::new(k), + heap: TopKHeap::new(k, schema), }) } @@ -199,7 +199,7 @@ impl TopK { } = self; let _timer = metrics.baseline.elapsed_compute().timer(); // time updated on drop - let mut batch = heap.emit(schema.clone())?; + let mut batch = heap.emit()?; metrics.baseline.output_rows().add(batch.num_rows()); // break into record batches as needed @@ -259,7 +259,7 @@ impl TopKMetrics { /// it is important to check the current minimum value in the heap /// prior to creating a new value to insert. struct TopKHeap { - /// The maximum size of this heap. + /// The maximum number of elemenents to store in this heap. k: usize, /// Storage for up at most `k` items, in ascending /// order. `inner[0]` holds the smallest value of the smallest k @@ -272,12 +272,12 @@ struct TopKHeap { } impl TopKHeap { - fn new(k: usize) -> Self { + fn new(k: usize, schema: SchemaRef) -> Self { assert!(k > 0); Self { k, inner: Vec::with_capacity(k), - store: RecordBatchStore::new(), + store: RecordBatchStore::new(schema), owned_bytes: 0, } } @@ -349,7 +349,9 @@ impl TopKHeap { /// Returns the values stored in this heap, from values low to high, as a single /// [`RecordBatch`] - pub fn emit(&self, schema: SchemaRef) -> Result { + pub fn emit(&self) -> Result { + let schema = self.store.schema().clone(); + // Indicies for each row within its respective RecordBatch let indicies: Vec<_> = self .inner @@ -358,16 +360,7 @@ impl TopKHeap { .map(|(i, k)| (i, k.index)) .collect(); - let num_columns = { - let Some(first_value) = self.inner.get(0) else { - return Ok(RecordBatch::new_empty(schema)); - }; - self.store - .get(first_value.batch_id) - .expect("invalid batch id") - .batch - .num_columns() - }; + let num_columns = schema.fields().len(); // build the output columns one at time, using the // `interleave` kernel to pick rows from different arrays @@ -394,6 +387,11 @@ impl TopKHeap { Ok(RecordBatch::try_new(schema, output_columns)?) } + /// Compact this heap, rewriting all stored batches + fn compact(&mut self) { + //let new_batch = self.emit( + } + /// return the size of memory used by this heap, in bytes fn size(&self) -> usize { std::mem::size_of::() @@ -498,14 +496,17 @@ struct RecordBatchStore { batches: HashMap, /// total size of all record batches tracked by this store batches_size: usize, + /// schema of the batches + schema: SchemaRef, } impl RecordBatchStore { - fn new() -> Self { + fn new(schema: SchemaRef) -> Self { Self { next_id: 0, batches: HashMap::new(), batches_size: 0, + schema, } } @@ -531,6 +532,16 @@ impl RecordBatchStore { self.batches.get(&id) } + /// returns the total number of batches stored in this store + fn len(&self) -> usize { + self.batches.len() + } + + /// return the schema of batches stored + fn schema(&self) -> &SchemaRef { + &self.schema + } + /// remove a use from the specified batch id. If the use count /// reaches zero the batch entry is removed from the store /// From 354d687caf8ad0e75fa27e28779844842a535724 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 22 Aug 2023 15:03:31 -0400 Subject: [PATCH 04/14] checkpoint --- datafusion/core/src/physical_plan/topk/mod.rs | 56 ++++++++++++++++++- datafusion/sqllogictest/test_files/aal.slt | 15 +++++ 2 files changed, 68 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_plan/topk/mod.rs b/datafusion/core/src/physical_plan/topk/mod.rs index 2efb65fefff2d..e6993b70d4c60 100644 --- a/datafusion/core/src/physical_plan/topk/mod.rs +++ b/datafusion/core/src/physical_plan/topk/mod.rs @@ -143,6 +143,8 @@ impl TopK { /// Insert `batch`, remembering it if any of its values are among /// the top k seen so far. pub fn insert_batch(&mut self, batch: RecordBatch) -> Result<()> { + use log::info; + info!("INSERTING {} rows", batch.num_rows()); // Updates on drop let _timer = self.metrics.baseline.elapsed_compute().timer(); @@ -180,6 +182,9 @@ impl TopK { } self.heap.insert_batch_entry(batch_entry); + // conserve memory + self.heap.maybe_compact()?; + // update memory reservation self.reservation.try_resize(self.size())?; Ok(()) @@ -352,6 +357,10 @@ impl TopKHeap { pub fn emit(&self) -> Result { let schema = self.store.schema().clone(); + if self.store.is_empty() { + return Ok(RecordBatch::new_empty(schema)); + } + // Indicies for each row within its respective RecordBatch let indicies: Vec<_> = self .inner @@ -387,9 +396,39 @@ impl TopKHeap { Ok(RecordBatch::try_new(schema, output_columns)?) } - /// Compact this heap, rewriting all stored batches - fn compact(&mut self) { - //let new_batch = self.emit( + /// Compact this heap, rewriting all stored batches into a single + /// input batch + pub fn maybe_compact(&mut self) -> Result<()>{ + // don't compact if the store has less than ten batches + if self.store.len() <= 10 { + return Ok(()); + } + + panic!("Disco"); + + // at first, compact the entire thing always into a new batch + // (maybe we can get fancier in the future about ignoring + // batches that have a high usage ratio already + + // Note: new batch is in the same order as inner + let new_batch = self.emit()?; + + // clear all old entires in store (this invalidates all + // store_ids in `inner`) + self.store.clear(); + + let mut batch_entry = self.register_batch(new_batch); + batch_entry.uses = self.inner.len(); + + // rewrite all existing entries to use the new batch, and + // remove old entries. The sortedness and their relative + // position do not change + for (i, topk_row) in self.inner.iter_mut().enumerate() { + topk_row.batch_id = batch_entry.id; + topk_row.index = i; + } + self.insert_batch_entry(batch_entry); + Ok(()) } /// return the size of memory used by this heap, in bytes @@ -528,6 +567,12 @@ impl RecordBatchStore { } } + /// Clear all values in this store, invalidating all previous batch ids + fn clear(&mut self) { + self.batches.clear(); + self.batches_size = 0; + } + fn get(&self, id: u32) -> Option<&RecordBatchEntry> { self.batches.get(&id) } @@ -537,6 +582,11 @@ impl RecordBatchStore { self.batches.len() } + /// returns true if the store has nothing stored + fn is_empty(&self) -> bool { + self.batches.is_empty() + } + /// return the schema of batches stored fn schema(&self) -> &SchemaRef { &self.schema diff --git a/datafusion/sqllogictest/test_files/aal.slt b/datafusion/sqllogictest/test_files/aal.slt index f19c79e8d1cb1..4d8346edf23ee 100644 --- a/datafusion/sqllogictest/test_files/aal.slt +++ b/datafusion/sqllogictest/test_files/aal.slt @@ -200,3 +200,18 @@ d 1 -98 13630 -1991133944 1184110014998006843 220 2986 225513085 963410661024364 e 2 52 -12056 -1090239422 9011500141803970147 238 4168 2013662838 12565360638488684051 0.6694766 0.391444365692 xipQ93429ksjNcXPX5326VSg1xJZcW d 1 -72 25590 1188089983 3090286296481837049 241 832 3542840110 5885937420286765261 0.41980565 0.215354023438 wwXqSGKLyBQyPkonlzBNYUJTCo4LRS a 1 -5 12636 794623392 2909750622865366631 15 24022 2669374863 4776679784701509574 0.29877836 0.253725340799 waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs + + + +## -- make tiny batches to trigger batch compaction +statement ok +set datafusion.execution.batch_size = 7 + +query TIIIIIIIITRRT +select * from aggregate_test_100 ORDER BY c13 desc limit 5; +---- +a 4 -38 20744 762932956 308913475857409919 7 45465 1787652631 878137512938218976 0.7459874 0.021825780392 ydkwycaISlYSlEq3TlkS2m15I2pcp8 +d 1 -98 13630 -1991133944 1184110014998006843 220 2986 225513085 9634106610243643486 0.89651865 0.164088254508 y7C453hRWd4E7ImjNDWlpexB8nUqjh +e 2 52 -12056 -1090239422 9011500141803970147 238 4168 2013662838 12565360638488684051 0.6694766 0.391444365692 xipQ93429ksjNcXPX5326VSg1xJZcW +d 1 -72 25590 1188089983 3090286296481837049 241 832 3542840110 5885937420286765261 0.41980565 0.215354023438 wwXqSGKLyBQyPkonlzBNYUJTCo4LRS +a 1 -5 12636 794623392 2909750622865366631 15 24022 2669374863 4776679784701509574 0.29877836 0.253725340799 waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs From afea7d3e38faf0ec149c78849177a20d640bb541 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 22 Aug 2023 15:10:11 -0400 Subject: [PATCH 05/14] update --- datafusion/core/src/physical_plan/topk/mod.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_plan/topk/mod.rs b/datafusion/core/src/physical_plan/topk/mod.rs index e6993b70d4c60..64d2090da866d 100644 --- a/datafusion/core/src/physical_plan/topk/mod.rs +++ b/datafusion/core/src/physical_plan/topk/mod.rs @@ -399,13 +399,14 @@ impl TopKHeap { /// Compact this heap, rewriting all stored batches into a single /// input batch pub fn maybe_compact(&mut self) -> Result<()>{ + use log::info; + info!("Have {} batches in store", self.store.len()); // don't compact if the store has less than ten batches - if self.store.len() <= 10 { + //if self.store.len() <= 10 { + if self.store.len() <= 2 { return Ok(()); } - panic!("Disco"); - // at first, compact the entire thing always into a new batch // (maybe we can get fancier in the future about ignoring // batches that have a high usage ratio already @@ -428,6 +429,7 @@ impl TopKHeap { topk_row.index = i; } self.insert_batch_entry(batch_entry); + info!("COMPACTION DONE: Have {} batches in store", self.store.len()); Ok(()) } From 69b86ab0bce7aa469f6912ab673af1d3f2135873 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 23 Aug 2023 08:03:56 -0400 Subject: [PATCH 06/14] checkpoint --- datafusion/core/src/physical_plan/topk/mod.rs | 30 +++++++++++++++---- 1 file changed, 25 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/physical_plan/topk/mod.rs b/datafusion/core/src/physical_plan/topk/mod.rs index 64d2090da866d..8fdefae544f64 100644 --- a/datafusion/core/src/physical_plan/topk/mod.rs +++ b/datafusion/core/src/physical_plan/topk/mod.rs @@ -399,13 +399,20 @@ impl TopKHeap { /// Compact this heap, rewriting all stored batches into a single /// input batch pub fn maybe_compact(&mut self) -> Result<()>{ - use log::info; - info!("Have {} batches in store", self.store.len()); - // don't compact if the store has less than ten batches - //if self.store.len() <= 10 { - if self.store.len() <= 2 { + + // we compact if the number of "unused" rows in the store is + // past some pre-defined threshold. Target holding up to + // around 20 batches, but handle cases of large k where some + // batches might be partially full + let target_batch_size = 8024; + let max_unused_rows = 20 * target_batch_size + self.k; + + // don't compact if the store has only one batch or + if self.store.len() <= 2 || self.store.unused_rows() < max_unused_rows { return Ok(()); } + use log::info; + info!("Have {} batches in store, COMPACTING", self.store.len()); // at first, compact the entire thing always into a new batch // (maybe we can get fancier in the future about ignoring @@ -442,6 +449,8 @@ impl TopKHeap { } } + + /// Represents one of the top K rows held in this heap. Orders /// according to memcmp of row (e.g. the arrow Row format, but could /// also be primtive values) @@ -584,6 +593,17 @@ impl RecordBatchStore { self.batches.len() } + /// Returns the total number of rows in batches minus the number + /// which are in use + fn unused_rows(&self) -> usize { + self.batches + .values() + .map(|batch_entry| { + batch_entry.batch.num_rows() - batch_entry.uses + }) + .sum() + } + /// returns true if the store has nothing stored fn is_empty(&self) -> bool { self.batches.is_empty() From c8b415c1e9da90921f89c56e576c370a50a1a0cc Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 23 Aug 2023 08:58:21 -0400 Subject: [PATCH 07/14] fmt --- datafusion/core/src/physical_plan/topk/mod.rs | 155 +++++++++--------- 1 file changed, 77 insertions(+), 78 deletions(-) diff --git a/datafusion/core/src/physical_plan/topk/mod.rs b/datafusion/core/src/physical_plan/topk/mod.rs index 8fdefae544f64..5e242746e6c09 100644 --- a/datafusion/core/src/physical_plan/topk/mod.rs +++ b/datafusion/core/src/physical_plan/topk/mod.rs @@ -21,7 +21,7 @@ use arrow::{ compute::interleave, row::{RowConverter, Rows, SortField}, }; -use std::{cmp::Ordering, sync::Arc}; +use std::{cmp::Ordering, collections::BinaryHeap, sync::Arc}; use arrow_array::{Array, ArrayRef, RecordBatch}; use arrow_schema::SchemaRef; @@ -143,8 +143,6 @@ impl TopK { /// Insert `batch`, remembering it if any of its values are among /// the top k seen so far. pub fn insert_batch(&mut self, batch: RecordBatch) -> Result<()> { - use log::info; - info!("INSERTING {} rows", batch.num_rows()); // Updates on drop let _timer = self.metrics.baseline.elapsed_compute().timer(); @@ -168,12 +166,11 @@ impl TopK { let mut batch_entry = self.heap.register_batch(batch); for (index, row) in rows.iter().enumerate() { - match self.heap.k_largest() { - // heap has k items, and the current row is not - // smaller than the curret smallest k value, skip - Some(largest) if largest.row.as_slice() <= row.as_ref() => {} - // don't yet have k items or new item is greater than - // current min top k + match self.heap.max() { + // heap has k items, and the new row is greater than the + // current max in the heap ==> it is not a new topk + Some(max_row) if row.as_ref() >= max_row.row.as_slice() => {} + // don't yet have k items or new item is lower than the currently k low values None | Some(_) => { self.heap.add(&mut batch_entry, row, index); self.metrics.row_replacements.add(1); @@ -190,7 +187,7 @@ impl TopK { Ok(()) } - /// Returns the top k results broken into `batch_size` [`RecordBatch`]es + /// Returns the top k results broken into `batch_size` [`RecordBatch`]es, consuming the heap pub fn emit(self) -> Result { let Self { schema, @@ -200,7 +197,7 @@ impl TopK { expr: _, row_converter: _, scratch_rows: _, - heap, + mut heap, } = self; let _timer = metrics.baseline.elapsed_compute().timer(); // time updated on drop @@ -266,10 +263,9 @@ impl TopKMetrics { struct TopKHeap { /// The maximum number of elemenents to store in this heap. k: usize, - /// Storage for up at most `k` items, in ascending - /// order. `inner[0]` holds the smallest value of the smallest k - /// so far, `inner[len-1]` holds the largest value smallest k so far. - inner: Vec, + /// Storage for up at most `k` items using a BinaryHeap. Reverserd + /// so that the smallest k so far is on the top + inner: BinaryHeap, /// Storage the original row values (TopKRow only has the sort key) store: RecordBatchStore, /// The size of all owned data held by this heap @@ -281,7 +277,7 @@ impl TopKHeap { assert!(k > 0); Self { k, - inner: Vec::with_capacity(k), + inner: BinaryHeap::new(), store: RecordBatchStore::new(schema), owned_bytes: 0, } @@ -300,12 +296,13 @@ impl TopKHeap { } /// Returns the largest value stored by the heap if there are k - /// items, otherwise returns None - fn k_largest(&self) -> Option<&TopKRow> { + /// items, otherwise returns None. Remember this structure is + /// keeping the "smallest" k values + fn max(&self) -> Option<&TopKRow> { if self.inner.len() < self.k { None } else { - self.inner.last() + self.inner.peek() } } @@ -344,26 +341,33 @@ impl TopKHeap { self.owned_bytes += new_top_k.owned_size(); - // put the new row into the correct location to maintain that - // self.inner is sorted in descending order - let insertion_point = self - .inner - .partition_point(|current_row| current_row.row() <= row.as_ref()); - self.inner.insert(insertion_point, new_top_k); + // put the new row into the heap + self.inner.push(new_top_k) } - /// Returns the values stored in this heap, from values low to high, as a single - /// [`RecordBatch`] - pub fn emit(&self) -> Result { + /// Returns the values stored in this heap, from values low to + /// high, as a single [`RecordBatch`], resetting the inner heap + pub fn emit(&mut self) -> Result { + Ok(self.emit_with_state()?.0) + } + + /// Returns the values stored in this heap, from values low to + /// high, as a single [`RecordBatch`], and a sorted vec of heap contents + + pub fn emit_with_state(&mut self) -> Result<(RecordBatch, Vec)> { let schema = self.store.schema().clone(); + let mut topk_rows = std::mem::take(&mut self.inner).into_vec(); + + // sort low to high (reverse the reverse) + topk_rows.sort(); + if self.store.is_empty() { - return Ok(RecordBatch::new_empty(schema)); + return Ok((RecordBatch::new_empty(schema), topk_rows)); } // Indicies for each row within its respective RecordBatch - let indicies: Vec<_> = self - .inner + let indicies: Vec<_> = topk_rows .iter() .enumerate() .map(|(i, k)| (i, k.index)) @@ -375,8 +379,7 @@ impl TopKHeap { // `interleave` kernel to pick rows from different arrays let output_columns: Vec<_> = (0..num_columns) .map(|col| { - let input_arrays: Vec<_> = self - .inner + let input_arrays: Vec<_> = topk_rows .iter() .map(|k| { let entry = @@ -393,50 +396,50 @@ impl TopKHeap { }) .collect::>()?; - Ok(RecordBatch::try_new(schema, output_columns)?) + let new_batch = RecordBatch::try_new(schema, output_columns)?; + Ok((new_batch, topk_rows)) } /// Compact this heap, rewriting all stored batches into a single /// input batch - pub fn maybe_compact(&mut self) -> Result<()>{ - - // we compact if the number of "unused" rows in the store is - // past some pre-defined threshold. Target holding up to - // around 20 batches, but handle cases of large k where some - // batches might be partially full - let target_batch_size = 8024; - let max_unused_rows = 20 * target_batch_size + self.k; - - // don't compact if the store has only one batch or - if self.store.len() <= 2 || self.store.unused_rows() < max_unused_rows { - return Ok(()); - } - use log::info; - info!("Have {} batches in store, COMPACTING", self.store.len()); - - // at first, compact the entire thing always into a new batch - // (maybe we can get fancier in the future about ignoring - // batches that have a high usage ratio already - - // Note: new batch is in the same order as inner - let new_batch = self.emit()?; - - // clear all old entires in store (this invalidates all - // store_ids in `inner`) - self.store.clear(); - - let mut batch_entry = self.register_batch(new_batch); - batch_entry.uses = self.inner.len(); - - // rewrite all existing entries to use the new batch, and - // remove old entries. The sortedness and their relative - // position do not change - for (i, topk_row) in self.inner.iter_mut().enumerate() { - topk_row.batch_id = batch_entry.id; - topk_row.index = i; - } - self.insert_batch_entry(batch_entry); - info!("COMPACTION DONE: Have {} batches in store", self.store.len()); + pub fn maybe_compact(&mut self) -> Result<()> { + // // we compact if the number of "unused" rows in the store is + // // past some pre-defined threshold. Target holding up to + // // around 20 batches, but handle cases of large k where some + // // batches might be partially full + // let target_batch_size = 8024; + // let max_unused_rows = 20 * target_batch_size + self.k; + + // // don't compact if the store has only one batch or + // if self.store.len() <= 2 || self.store.unused_rows() < max_unused_rows { + // return Ok(()); + // } + // use log::info; + // info!("Have {} batches in store, COMPACTING", self.store.len()); + + // // at first, compact the entire thing always into a new batch + // // (maybe we can get fancier in the future about ignoring + // // batches that have a high usage ratio already + + // // Note: new batch is in the same order as inner + // let new_batch = self.emit()?; + + // // clear all old entires in store (this invalidates all + // // store_ids in `inner`) + // self.store.clear(); + + // let mut batch_entry = self.register_batch(new_batch); + // batch_entry.uses = self.inner.len(); + + // // rewrite all existing entries to use the new batch, and + // // remove old entries. The sortedness and their relative + // // position do not change + // for (i, topk_row) in self.inner.iter_mut().enumerate() { + // topk_row.batch_id = batch_entry.id; + // topk_row.index = i; + // } + // self.insert_batch_entry(batch_entry); + // info!("COMPACTION DONE: Have {} batches in store", self.store.len()); Ok(()) } @@ -449,8 +452,6 @@ impl TopKHeap { } } - - /// Represents one of the top K rows held in this heap. Orders /// according to memcmp of row (e.g. the arrow Row format, but could /// also be primtive values) @@ -598,9 +599,7 @@ impl RecordBatchStore { fn unused_rows(&self) -> usize { self.batches .values() - .map(|batch_entry| { - batch_entry.batch.num_rows() - batch_entry.uses - }) + .map(|batch_entry| batch_entry.batch.num_rows() - batch_entry.uses) .sum() } From 0337e310cd3d998cdfb105b6cba41aa28a34b8b1 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 23 Aug 2023 11:35:47 -0400 Subject: [PATCH 08/14] Fix compaction --- datafusion/core/src/physical_plan/topk/mod.rs | 94 +++++++++++-------- datafusion/sqllogictest/test_files/aal.slt | 2 +- 2 files changed, 55 insertions(+), 41 deletions(-) diff --git a/datafusion/core/src/physical_plan/topk/mod.rs b/datafusion/core/src/physical_plan/topk/mod.rs index 5e242746e6c09..faf68bcd5ac25 100644 --- a/datafusion/core/src/physical_plan/topk/mod.rs +++ b/datafusion/core/src/physical_plan/topk/mod.rs @@ -136,7 +136,7 @@ impl TopK { expr, row_converter, scratch_rows, - heap: TopKHeap::new(k, schema), + heap: TopKHeap::new(k, batch_size, schema), }) } @@ -169,7 +169,7 @@ impl TopK { match self.heap.max() { // heap has k items, and the new row is greater than the // current max in the heap ==> it is not a new topk - Some(max_row) if row.as_ref() >= max_row.row.as_slice() => {} + Some(max_row) if row.as_ref() >= max_row.row() => {} // don't yet have k items or new item is lower than the currently k low values None | Some(_) => { self.heap.add(&mut batch_entry, row, index); @@ -263,6 +263,8 @@ impl TopKMetrics { struct TopKHeap { /// The maximum number of elemenents to store in this heap. k: usize, + /// The target number of rows for output batches + batch_size: usize, /// Storage for up at most `k` items using a BinaryHeap. Reverserd /// so that the smallest k so far is on the top inner: BinaryHeap, @@ -273,10 +275,15 @@ struct TopKHeap { } impl TopKHeap { - fn new(k: usize, schema: SchemaRef) -> Self { + fn new( + k: usize, + batch_size: usize, + schema: SchemaRef + ) -> Self { assert!(k > 0); Self { k, + batch_size, inner: BinaryHeap::new(), store: RecordBatchStore::new(schema), owned_bytes: 0, @@ -403,43 +410,50 @@ impl TopKHeap { /// Compact this heap, rewriting all stored batches into a single /// input batch pub fn maybe_compact(&mut self) -> Result<()> { - // // we compact if the number of "unused" rows in the store is - // // past some pre-defined threshold. Target holding up to - // // around 20 batches, but handle cases of large k where some - // // batches might be partially full - // let target_batch_size = 8024; - // let max_unused_rows = 20 * target_batch_size + self.k; - - // // don't compact if the store has only one batch or - // if self.store.len() <= 2 || self.store.unused_rows() < max_unused_rows { - // return Ok(()); - // } - // use log::info; - // info!("Have {} batches in store, COMPACTING", self.store.len()); - - // // at first, compact the entire thing always into a new batch - // // (maybe we can get fancier in the future about ignoring - // // batches that have a high usage ratio already - - // // Note: new batch is in the same order as inner - // let new_batch = self.emit()?; - - // // clear all old entires in store (this invalidates all - // // store_ids in `inner`) - // self.store.clear(); - - // let mut batch_entry = self.register_batch(new_batch); - // batch_entry.uses = self.inner.len(); - - // // rewrite all existing entries to use the new batch, and - // // remove old entries. The sortedness and their relative - // // position do not change - // for (i, topk_row) in self.inner.iter_mut().enumerate() { - // topk_row.batch_id = batch_entry.id; - // topk_row.index = i; - // } - // self.insert_batch_entry(batch_entry); - // info!("COMPACTION DONE: Have {} batches in store", self.store.len()); + // we compact if the number of "unused" rows in the store is + // past some pre-defined threshold. Target holding up to + // around 20 batches, but handle cases of large k where some + // batches might be partially full + let max_unused_rows = (20 * self.batch_size) + self.k; + let unused_rows = self.store.unused_rows(); + use log::info; + //info!("{} batches in store, unused rows in store: {}, max unused rows: {}", + //self.store.len(), unused_rows, max_unused_rows); + + // don't compact if the store has only one batch or + if self.store.len() <= 2 || unused_rows < max_unused_rows { + //if self.store.len() <= 2 { + return Ok(()); + } + info!("Have {} batches in store, COMPACTING", self.store.len()); + + // at first, compact the entire thing always into a new batch + // (maybe we can get fancier in the future about ignoring + // batches that have a high usage ratio already + + // Note: new batch is in the same order as inner + let num_rows = self.inner.len(); + let (new_batch, mut topk_rows) = self.emit_with_state()?; + + // clear all old entires in store (this invalidates all + // store_ids in `inner`) + self.store.clear(); + + let mut batch_entry = self.register_batch(new_batch); + batch_entry.uses = num_rows; + + // rewrite all existing entries to use the new batch, and + // remove old entries. The sortedness and their relative + // position do not change + for (i, topk_row) in topk_rows.iter_mut().enumerate() { + topk_row.batch_id = batch_entry.id; + topk_row.index = i; + } + self.insert_batch_entry(batch_entry); + // restore the heap + self.inner = BinaryHeap::from(topk_rows); + + info!("COMPACTION DONE: Have {} batches in store", self.store.len()); Ok(()) } diff --git a/datafusion/sqllogictest/test_files/aal.slt b/datafusion/sqllogictest/test_files/aal.slt index 4d8346edf23ee..bbab912956e18 100644 --- a/datafusion/sqllogictest/test_files/aal.slt +++ b/datafusion/sqllogictest/test_files/aal.slt @@ -205,7 +205,7 @@ a 1 -5 12636 794623392 2909750622865366631 15 24022 2669374863 47766797847015095 ## -- make tiny batches to trigger batch compaction statement ok -set datafusion.execution.batch_size = 7 +set datafusion.execution.batch_size = 2 query TIIIIIIIITRRT select * from aggregate_test_100 ORDER BY c13 desc limit 5; From db196fb1a13520b336df9a18171986d59eb7b2d1 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 24 Aug 2023 16:02:52 -0400 Subject: [PATCH 09/14] add location for re-encoding --- datafusion/core/src/physical_plan/topk/mod.rs | 30 ++++++++++++++++--- datafusion/sqllogictest/test_files/aal.slt | 15 ++++++++++ 2 files changed, 41 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/physical_plan/topk/mod.rs b/datafusion/core/src/physical_plan/topk/mod.rs index faf68bcd5ac25..e2868c3b7fa3e 100644 --- a/datafusion/core/src/physical_plan/topk/mod.rs +++ b/datafusion/core/src/physical_plan/topk/mod.rs @@ -18,13 +18,13 @@ //! TopK: Combination of Sort / LIMIT use arrow::{ - compute::interleave, + error::ArrowError, row::{RowConverter, Rows, SortField}, }; use std::{cmp::Ordering, collections::BinaryHeap, sync::Arc}; use arrow_array::{Array, ArrayRef, RecordBatch}; -use arrow_schema::SchemaRef; +use arrow_schema::{DataType, SchemaRef}; use datafusion_common::Result; use datafusion_execution::{ memory_pool::{MemoryConsumer, MemoryReservation}, @@ -359,8 +359,8 @@ impl TopKHeap { } /// Returns the values stored in this heap, from values low to - /// high, as a single [`RecordBatch`], and a sorted vec of heap contents - + /// high, as a single [`RecordBatch`], and a sorted vec of the + /// current heap's contents pub fn emit_with_state(&mut self) -> Result<(RecordBatch, Vec)> { let schema = self.store.schema().clone(); @@ -657,3 +657,25 @@ impl RecordBatchStore { + self.batches_size } } + + +/// wrapper over [`arrow::compute::interleave`] that re-encodes +/// dictionaries that have a low usage (values referenced) + fn interleave( + values: &[&dyn Array], + indices: &[(usize, usize)], +) -> Result { + // for now, always re-encode only string dictionaries + if !values.is_empty() { + match values[0].data_type() { + DataType::Dictionary(_key_type, value_type) if value_type.as_ref() == &DataType::Utf8 => { + + //todo!() + return arrow::compute::interleave(values, indices); + } + _ => { } + } + } + // fallback to arrow + arrow::compute::interleave(values, indices) + } diff --git a/datafusion/sqllogictest/test_files/aal.slt b/datafusion/sqllogictest/test_files/aal.slt index bbab912956e18..36dc0d9fdcf95 100644 --- a/datafusion/sqllogictest/test_files/aal.slt +++ b/datafusion/sqllogictest/test_files/aal.slt @@ -215,3 +215,18 @@ d 1 -98 13630 -1991133944 1184110014998006843 220 2986 225513085 963410661024364 e 2 52 -12056 -1090239422 9011500141803970147 238 4168 2013662838 12565360638488684051 0.6694766 0.391444365692 xipQ93429ksjNcXPX5326VSg1xJZcW d 1 -72 25590 1188089983 3090286296481837049 241 832 3542840110 5885937420286765261 0.41980565 0.215354023438 wwXqSGKLyBQyPkonlzBNYUJTCo4LRS a 1 -5 12636 794623392 2909750622865366631 15 24022 2669374863 4776679784701509574 0.29877836 0.253725340799 waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs + + +## make an example for + +statement ok +create table dict as select c1, c2, c3, c13, arrow_cast(c13, 'Dictionary(Int32, Utf8)') as c13_dict from aggregate_test_100; + +query TIIT? +select * from dict order by c13 desc limit 5; +---- +a 4 -38 ydkwycaISlYSlEq3TlkS2m15I2pcp8 ydkwycaISlYSlEq3TlkS2m15I2pcp8 +d 1 -98 y7C453hRWd4E7ImjNDWlpexB8nUqjh y7C453hRWd4E7ImjNDWlpexB8nUqjh +e 2 52 xipQ93429ksjNcXPX5326VSg1xJZcW xipQ93429ksjNcXPX5326VSg1xJZcW +d 1 -72 wwXqSGKLyBQyPkonlzBNYUJTCo4LRS wwXqSGKLyBQyPkonlzBNYUJTCo4LRS +a 1 -5 waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs From f12307596f6c0f0255d40c682c8a456d13dff980 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 24 Aug 2023 16:16:32 -0400 Subject: [PATCH 10/14] Start sketching dictionary interleave --- datafusion/core/src/physical_plan/topk/mod.rs | 60 ++++++++++++------- datafusion/core/tests/sql/order.rs | 4 +- .../simplify_expressions/expr_simplifier.rs | 4 +- .../src/simplify_expressions/regex.rs | 4 +- datafusion/sql/src/statement.rs | 4 +- .../substrait/src/logical_plan/consumer.rs | 26 ++++---- .../substrait/src/logical_plan/producer.rs | 5 +- 7 files changed, 65 insertions(+), 42 deletions(-) diff --git a/datafusion/core/src/physical_plan/topk/mod.rs b/datafusion/core/src/physical_plan/topk/mod.rs index e2868c3b7fa3e..3b8546cc876ca 100644 --- a/datafusion/core/src/physical_plan/topk/mod.rs +++ b/datafusion/core/src/physical_plan/topk/mod.rs @@ -23,7 +23,7 @@ use arrow::{ }; use std::{cmp::Ordering, collections::BinaryHeap, sync::Arc}; -use arrow_array::{Array, ArrayRef, RecordBatch}; +use arrow_array::{downcast_dictionary_array, Array, ArrayRef, RecordBatch}; use arrow_schema::{DataType, SchemaRef}; use datafusion_common::Result; use datafusion_execution::{ @@ -275,11 +275,7 @@ struct TopKHeap { } impl TopKHeap { - fn new( - k: usize, - batch_size: usize, - schema: SchemaRef - ) -> Self { + fn new(k: usize, batch_size: usize, schema: SchemaRef) -> Self { assert!(k > 0); Self { k, @@ -453,7 +449,10 @@ impl TopKHeap { // restore the heap self.inner = BinaryHeap::from(topk_rows); - info!("COMPACTION DONE: Have {} batches in store", self.store.len()); + info!( + "COMPACTION DONE: Have {} batches in store", + self.store.len() + ); Ok(()) } @@ -658,24 +657,39 @@ impl RecordBatchStore { } } - /// wrapper over [`arrow::compute::interleave`] that re-encodes /// dictionaries that have a low usage (values referenced) - fn interleave( +fn interleave( + values: &[&dyn Array], + indices: &[(usize, usize)], +) -> Result { + // for now, always re-encode only string dictionaries + if !values.is_empty() { + match values[0].data_type() { + DataType::Dictionary(_key_type, value_type) + if value_type.as_ref() == &DataType::Utf8 => + { + return interleave_dictionary(values, indices); + } + _ => {} + } + } + // fallback to arrow + arrow::compute::interleave(values, indices) +} + +// we don't need specialized version for each index type, simply need +fn interleave_dictionary( values: &[&dyn Array], indices: &[(usize, usize)], ) -> Result { - // for now, always re-encode only string dictionaries - if !values.is_empty() { - match values[0].data_type() { - DataType::Dictionary(_key_type, value_type) if value_type.as_ref() == &DataType::Utf8 => { - - //todo!() - return arrow::compute::interleave(values, indices); - } - _ => { } - } - } - // fallback to arrow - arrow::compute::interleave(values, indices) - } + todo!() +} + +/// returns a reference to the values of this dictioanry +fn values(array: &ArrayRef) -> &ArrayRef { + downcast_dictionary_array!( + array => return array.values(), + _ => unreachable!("Non dictionary type") + ) +} diff --git a/datafusion/core/tests/sql/order.rs b/datafusion/core/tests/sql/order.rs index 3981fbaa4d7ab..a400a78fc9146 100644 --- a/datafusion/core/tests/sql/order.rs +++ b/datafusion/core/tests/sql/order.rs @@ -48,7 +48,9 @@ async fn sort_with_lots_of_repetition_values() -> Result<()> { async fn create_external_table_with_order() -> Result<()> { let ctx = SessionContext::new(); let sql = "CREATE EXTERNAL TABLE dt (a_id integer, a_str string, a_bool boolean) STORED AS CSV WITH ORDER (a_id ASC) LOCATION 'file://path/to/table';"; - let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = ctx.state().create_logical_plan(sql).await? else { + let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = + ctx.state().create_logical_plan(sql).await? + else { panic!("Wrong command") }; diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 895432026b483..3cf564f367bab 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -412,7 +412,9 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { }) if list.len() == 1 && matches!(list.first(), Some(Expr::ScalarSubquery { .. })) => { - let Expr::ScalarSubquery(subquery) = list.remove(0) else { unreachable!() }; + let Expr::ScalarSubquery(subquery) = list.remove(0) else { + unreachable!() + }; Expr::InSubquery(InSubquery::new(expr, subquery, negated)) } diff --git a/datafusion/optimizer/src/simplify_expressions/regex.rs b/datafusion/optimizer/src/simplify_expressions/regex.rs index 5094623b82c08..b9d9821b43f09 100644 --- a/datafusion/optimizer/src/simplify_expressions/regex.rs +++ b/datafusion/optimizer/src/simplify_expressions/regex.rs @@ -203,7 +203,9 @@ fn anchored_literal_to_expr(v: &[Hir]) -> Option { match v.len() { 2 => Some(lit("")), 3 => { - let HirKind::Literal(l) = v[1].kind() else { return None }; + let HirKind::Literal(l) = v[1].kind() else { + return None; + }; like_str_from_literal(l).map(lit) } _ => None, diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index c9a34bfaf2203..8676e2a6e76ab 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -499,10 +499,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { "DELETE FROM only supports single table, got: joins".to_string(), )); } - let TableFactor::Table{name, ..} = table_factor.relation else { + let TableFactor::Table { name, .. } = table_factor.relation else { return Err(DataFusionError::NotImplemented(format!( "DELETE FROM only supports single table, got: {table_factor:?}" - ))) + ))); }; Ok(name) diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 4e4d71ddb6048..54f1facb4adae 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -573,9 +573,9 @@ pub async fn from_substrait_sorts( Some(k) => match k { Direction(d) => { let Some(direction) = SortDirection::from_i32(*d) else { - return Err(DataFusionError::NotImplemented( - format!("Unsupported Substrait SortDirection value {d}"), - )) + return Err(DataFusionError::NotImplemented(format!( + "Unsupported Substrait SortDirection value {d}" + ))); }; match direction { @@ -1313,27 +1313,27 @@ async fn make_datafusion_like( } let Some(ArgType::Value(expr_substrait)) = &f.arguments[0].arg_type else { - return Err(DataFusionError::NotImplemented( - format!("Invalid arguments type for `{fn_name}` expr") - )) + return Err(DataFusionError::NotImplemented(format!( + "Invalid arguments type for `{fn_name}` expr" + ))); }; let expr = from_substrait_rex(expr_substrait, input_schema, extensions) .await? .as_ref() .clone(); let Some(ArgType::Value(pattern_substrait)) = &f.arguments[1].arg_type else { - return Err(DataFusionError::NotImplemented( - format!("Invalid arguments type for `{fn_name}` expr") - )) + return Err(DataFusionError::NotImplemented(format!( + "Invalid arguments type for `{fn_name}` expr" + ))); }; let pattern = from_substrait_rex(pattern_substrait, input_schema, extensions) .await? .as_ref() .clone(); let Some(ArgType::Value(escape_char_substrait)) = &f.arguments[2].arg_type else { - return Err(DataFusionError::NotImplemented( - format!("Invalid arguments type for `{fn_name}` expr") - )) + return Err(DataFusionError::NotImplemented(format!( + "Invalid arguments type for `{fn_name}` expr" + ))); }; let escape_char_expr = from_substrait_rex(escape_char_substrait, input_schema, extensions) @@ -1343,7 +1343,7 @@ async fn make_datafusion_like( let Expr::Literal(ScalarValue::Utf8(escape_char)) = escape_char_expr else { return Err(DataFusionError::Substrait(format!( "Expect Utf8 literal for escape char, but found {escape_char_expr:?}", - ))) + ))); }; Ok(Arc::new(Expr::Like(Like { diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 79cd8995c6c61..d1f46c9858a0e 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -1664,7 +1664,10 @@ mod test { println!("Checking round trip of {scalar:?}"); let substrait = to_substrait_literal(&scalar)?; - let Expression { rex_type: Some(RexType::Literal(substrait_literal)) } = substrait else { + let Expression { + rex_type: Some(RexType::Literal(substrait_literal)), + } = substrait + else { panic!("Expected Literal expression, got {substrait:?}"); }; From 157379a21820db8e4da636c7bb4adaca7693f282 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 24 Aug 2023 16:41:39 -0400 Subject: [PATCH 11/14] checkpoint --- datafusion/core/src/physical_plan/topk/mod.rs | 82 ++++++++++++++++--- 1 file changed, 71 insertions(+), 11 deletions(-) diff --git a/datafusion/core/src/physical_plan/topk/mod.rs b/datafusion/core/src/physical_plan/topk/mod.rs index 3b8546cc876ca..f48ba32025d93 100644 --- a/datafusion/core/src/physical_plan/topk/mod.rs +++ b/datafusion/core/src/physical_plan/topk/mod.rs @@ -18,12 +18,11 @@ //! TopK: Combination of Sort / LIMIT use arrow::{ - error::ArrowError, row::{RowConverter, Rows, SortField}, }; use std::{cmp::Ordering, collections::BinaryHeap, sync::Arc}; -use arrow_array::{downcast_dictionary_array, Array, ArrayRef, RecordBatch}; +use arrow_array::{downcast_dictionary_array, Array, ArrayRef, RecordBatch, builder::StringBuilder, cast::AsArray, StringArray, Int32Array, types::Int32Type, DictionaryArray}; use arrow_schema::{DataType, SchemaRef}; use datafusion_common::Result; use datafusion_execution::{ @@ -31,7 +30,7 @@ use datafusion_execution::{ runtime_env::RuntimeEnv, }; use datafusion_physical_expr::PhysicalSortExpr; -use hashbrown::HashMap; +use hashbrown::{HashMap, HashSet}; use crate::physical_plan::{stream::RecordBatchStreamAdapter, SendableRecordBatchStream}; @@ -662,32 +661,93 @@ impl RecordBatchStore { fn interleave( values: &[&dyn Array], indices: &[(usize, usize)], -) -> Result { +) -> Result { // for now, always re-encode only string dictionaries if !values.is_empty() { match values[0].data_type() { DataType::Dictionary(_key_type, value_type) if value_type.as_ref() == &DataType::Utf8 => { - return interleave_dictionary(values, indices); + return interleave_and_repack_dictionary(values, indices); } _ => {} } } // fallback to arrow - arrow::compute::interleave(values, indices) + Ok(arrow::compute::interleave(values, indices)?) } -// we don't need specialized version for each index type, simply need -fn interleave_dictionary( +/// Special interleave kernel that re-creates the dictionary values, +/// ensuring no unused space +fn interleave_and_repack_dictionary( values: &[&dyn Array], indices: &[(usize, usize)], -) -> Result { - todo!() +) -> Result { + let existing_values = HashSet::new(); + + let data_type = values[0].data_type(); + + // repack to a new StringArray + let mut new_values = StringBuilder::new(); + // we could specialize this and avoid the copy of the index, but + // that seems like a lot of codegen overhead + let mut new_keys = vec![]; + + for (array_idx, row_idx) in indices { + // look up value, + let array = values[*array_idx]; + downcast_dictionary_array!( + array=> { + if let Some(key) = array.key(*row_idx) { + let values: &StringArray = array.values().as_string(); + if values.is_valid(key) { + let current_value = values.value(key); + println!("Current value is {current_value}"); + todo!(); + } else { + new_keys.push(None) + } + } + else { + new_keys.push(None); + } + + + } + _ => unreachable!("Non dictionary type") + + ) + } + + // form the output + let DataType::Dictionary(key_type, value_type) = data_type else { + unreachable!("non dictionary type"); + }; + + let new_values: ArrayRef = Arc::new(new_values.finish()); + match key_type.as_ref() { + DataType::Int32 => { + // check the keys will fit in this array + if new_values.len() >= i32::MAX as usize { + panic!("todo make a real error message"); + } + + let new_keys: Int32Array = new_keys.iter().map(|v| v.map(|v| v as i32)).collect(); + + Ok(Arc::new(DictionaryArray::try_new(new_keys, new_values)?)) + } + _ => { + // handle other keys + todo!() + } + } + + + } /// returns a reference to the values of this dictioanry -fn values(array: &ArrayRef) -> &ArrayRef { +fn get_dict_values(array: &ArrayRef) -> &ArrayRef { downcast_dictionary_array!( array => return array.values(), _ => unreachable!("Non dictionary type") From 682127af5898de0f6dec81d037a5efbe54c07f5b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 24 Aug 2023 16:50:29 -0400 Subject: [PATCH 12/14] initial specialized dictionary --- datafusion/core/src/physical_plan/topk/mod.rs | 36 ++++++++----------- 1 file changed, 15 insertions(+), 21 deletions(-) diff --git a/datafusion/core/src/physical_plan/topk/mod.rs b/datafusion/core/src/physical_plan/topk/mod.rs index f48ba32025d93..e5ba0a69e392a 100644 --- a/datafusion/core/src/physical_plan/topk/mod.rs +++ b/datafusion/core/src/physical_plan/topk/mod.rs @@ -22,7 +22,7 @@ use arrow::{ }; use std::{cmp::Ordering, collections::BinaryHeap, sync::Arc}; -use arrow_array::{downcast_dictionary_array, Array, ArrayRef, RecordBatch, builder::StringBuilder, cast::AsArray, StringArray, Int32Array, types::Int32Type, DictionaryArray}; +use arrow_array::{downcast_dictionary_array, Array, ArrayRef, RecordBatch, builder::StringBuilder, cast::AsArray, StringArray, Int32Array, DictionaryArray}; use arrow_schema::{DataType, SchemaRef}; use datafusion_common::Result; use datafusion_execution::{ @@ -30,7 +30,7 @@ use datafusion_execution::{ runtime_env::RuntimeEnv, }; use datafusion_physical_expr::PhysicalSortExpr; -use hashbrown::{HashMap, HashSet}; +use hashbrown::{HashMap}; use crate::physical_plan::{stream::RecordBatchStreamAdapter, SendableRecordBatchStream}; @@ -683,14 +683,11 @@ fn interleave_and_repack_dictionary( values: &[&dyn Array], indices: &[(usize, usize)], ) -> Result { - let existing_values = HashSet::new(); - let data_type = values[0].data_type(); - // repack to a new StringArray + // maps strings to new keys ( indexes) + let mut new_value_to_key = HashMap::new(); let mut new_values = StringBuilder::new(); - // we could specialize this and avoid the copy of the index, but - // that seems like a lot of codegen overhead let mut new_keys = vec![]; for (array_idx, row_idx) in indices { @@ -702,8 +699,16 @@ fn interleave_and_repack_dictionary( let values: &StringArray = array.values().as_string(); if values.is_valid(key) { let current_value = values.value(key); - println!("Current value is {current_value}"); - todo!(); + if let Some(new_key) = new_value_to_key.get(current_value) { + // value was already in the set + new_keys.push(Some(*new_key)) + } else { + // value not yet seen + let new_key = new_value_to_key.len(); + new_values.append_value(current_value); + new_keys.push(Some(new_key)); + new_value_to_key.insert(current_value, new_key); + } } else { new_keys.push(None) } @@ -720,7 +725,7 @@ fn interleave_and_repack_dictionary( } // form the output - let DataType::Dictionary(key_type, value_type) = data_type else { + let DataType::Dictionary(key_type, _value_type) = data_type else { unreachable!("non dictionary type"); }; @@ -741,15 +746,4 @@ fn interleave_and_repack_dictionary( todo!() } } - - - -} - -/// returns a reference to the values of this dictioanry -fn get_dict_values(array: &ArrayRef) -> &ArrayRef { - downcast_dictionary_array!( - array => return array.values(), - _ => unreachable!("Non dictionary type") - ) } From a1ea62ecac4ecc4bfad96eee1068ffa02e49f9ae Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 24 Aug 2023 16:54:05 -0400 Subject: [PATCH 13/14] finish initial special interleave --- datafusion/core/src/physical_plan/topk/mod.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/datafusion/core/src/physical_plan/topk/mod.rs b/datafusion/core/src/physical_plan/topk/mod.rs index e5ba0a69e392a..8169471472bf7 100644 --- a/datafusion/core/src/physical_plan/topk/mod.rs +++ b/datafusion/core/src/physical_plan/topk/mod.rs @@ -716,8 +716,6 @@ fn interleave_and_repack_dictionary( else { new_keys.push(None); } - - } _ => unreachable!("Non dictionary type") @@ -736,7 +734,6 @@ fn interleave_and_repack_dictionary( if new_values.len() >= i32::MAX as usize { panic!("todo make a real error message"); } - let new_keys: Int32Array = new_keys.iter().map(|v| v.map(|v| v as i32)).collect(); Ok(Arc::new(DictionaryArray::try_new(new_keys, new_values)?)) From 5e65130adab1b90cad8d785bed766230632d4f38 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 24 Aug 2023 17:58:55 -0400 Subject: [PATCH 14/14] Complete dictionary order --- datafusion/core/src/physical_plan/topk/mod.rs | 53 ++++++++++++------- 1 file changed, 35 insertions(+), 18 deletions(-) diff --git a/datafusion/core/src/physical_plan/topk/mod.rs b/datafusion/core/src/physical_plan/topk/mod.rs index 8169471472bf7..5e1ec8d5152ad 100644 --- a/datafusion/core/src/physical_plan/topk/mod.rs +++ b/datafusion/core/src/physical_plan/topk/mod.rs @@ -17,12 +17,14 @@ //! TopK: Combination of Sort / LIMIT -use arrow::{ - row::{RowConverter, Rows, SortField}, -}; +use arrow::row::{RowConverter, Rows, SortField}; use std::{cmp::Ordering, collections::BinaryHeap, sync::Arc}; -use arrow_array::{downcast_dictionary_array, Array, ArrayRef, RecordBatch, builder::StringBuilder, cast::AsArray, StringArray, Int32Array, DictionaryArray}; +use arrow_array::{ + builder::StringBuilder, cast::AsArray, downcast_dictionary_array, Array, ArrayRef, + DictionaryArray, Int16Array, Int32Array, Int64Array, Int8Array, RecordBatch, + StringArray, UInt16Array, UInt32Array, UInt64Array, UInt8Array, +}; use arrow_schema::{DataType, SchemaRef}; use datafusion_common::Result; use datafusion_execution::{ @@ -30,7 +32,7 @@ use datafusion_execution::{ runtime_env::RuntimeEnv, }; use datafusion_physical_expr::PhysicalSortExpr; -use hashbrown::{HashMap}; +use hashbrown::HashMap; use crate::physical_plan::{stream::RecordBatchStreamAdapter, SendableRecordBatchStream}; @@ -658,10 +660,7 @@ impl RecordBatchStore { /// wrapper over [`arrow::compute::interleave`] that re-encodes /// dictionaries that have a low usage (values referenced) -fn interleave( - values: &[&dyn Array], - indices: &[(usize, usize)], -) -> Result { +fn interleave(values: &[&dyn Array], indices: &[(usize, usize)]) -> Result { // for now, always re-encode only string dictionaries if !values.is_empty() { match values[0].data_type() { @@ -688,7 +687,7 @@ fn interleave_and_repack_dictionary( // maps strings to new keys ( indexes) let mut new_value_to_key = HashMap::new(); let mut new_values = StringBuilder::new(); - let mut new_keys = vec![]; + let mut new_keys = vec![]; for (array_idx, row_idx) in indices { // look up value, @@ -728,19 +727,37 @@ fn interleave_and_repack_dictionary( }; let new_values: ArrayRef = Arc::new(new_values.finish()); - match key_type.as_ref() { - DataType::Int32 => { + + // creates a $ARRAY_TYPE array from $NEW_KEYS ad $NEW_VALUES + use datafusion_common::DataFusionError; + macro_rules! make_keys { + ($PRIM_TYPE:ty, $ARRAY_TYPE:ty, $NEW_KEYS:ident, $NEW_VALUES:ident) => {{ // check the keys will fit in this array - if new_values.len() >= i32::MAX as usize { - panic!("todo make a real error message"); + if $NEW_VALUES.len() >= <$PRIM_TYPE>::MAX as usize { + return Err(DataFusionError::Execution(format!( + "keys did not fit in prim type -- TODO MAKE BETTER" + ))); } - let new_keys: Int32Array = new_keys.iter().map(|v| v.map(|v| v as i32)).collect(); - + let new_keys: $ARRAY_TYPE = new_keys + .iter() + .map(|v| v.map(|v| v as $PRIM_TYPE)) + .collect(); Ok(Arc::new(DictionaryArray::try_new(new_keys, new_values)?)) - } + }}; + } + + match key_type.as_ref() { + DataType::Int8 => make_keys!(i8, Int8Array, new_keys, new_values), + DataType::Int16 => make_keys!(i16, Int16Array, new_keys, new_values), + DataType::Int32 => make_keys!(i32, Int32Array, new_keys, new_values), + DataType::Int64 => make_keys!(i64, Int64Array, new_keys, new_values), + DataType::UInt8 => make_keys!(u8, UInt8Array, new_keys, new_values), + DataType::UInt16 => make_keys!(u16, UInt16Array, new_keys, new_values), + DataType::UInt32 => make_keys!(u32, UInt32Array, new_keys, new_values), + DataType::UInt64 => make_keys!(u64, UInt64Array, new_keys, new_values), _ => { // handle other keys - todo!() + unreachable!("unvalid key type"); } } }