Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 2 additions & 2 deletions datafusion-examples/examples/dataframe/cache_factory.rs
Original file line number Diff line number Diff line change
Expand Up @@ -223,7 +223,7 @@ impl CacheManager {
self.cache.insert(k, v);
}

pub fn get(&self, k: &LogicalPlan) -> Option<&Vec<Vec<RecordBatch>>> {
self.cache.get(k)
pub fn get(&self, k: &LogicalPlan) -> Option<&[Vec<RecordBatch>]> {
self.cache.get(k).map(Vec::as_slice)
}
}
4 changes: 2 additions & 2 deletions datafusion/catalog-listing/src/table.rs
Original file line number Diff line number Diff line change
Expand Up @@ -274,7 +274,7 @@ impl ListingTable {
}

/// Get paths ref
pub fn table_paths(&self) -> &Vec<ListingTableUrl> {
pub fn table_paths(&self) -> &[ListingTableUrl] {
&self.table_paths
}

Expand Down Expand Up @@ -662,7 +662,7 @@ impl TableProvider for ListingTable {
let config = FileSinkConfig {
original_url: String::default(),
object_store_url: self.table_paths()[0].object_store(),
table_paths: self.table_paths().clone(),
table_paths: self.table_paths().to_vec(),
file_group,
output_schema: self.schema(),
table_partition_cols: self.options.table_partition_cols.clone(),
Expand Down
8 changes: 4 additions & 4 deletions datafusion/catalog/src/information_schema.rs
Original file line number Diff line number Diff line change
Expand Up @@ -318,7 +318,7 @@ impl InformationSchemaConfig {
let catalog_name = &config_options.catalog.default_catalog;
let schema_name = &config_options.catalog.default_schema;
let mut add_parameters = |func_name: &str,
args: Option<&Vec<(String, String)>>,
args: Option<&[(String, String)]>,
arg_types: Vec<String>,
return_type: Option<String>,
is_variadic: bool,
Expand Down Expand Up @@ -361,7 +361,7 @@ impl InformationSchemaConfig {
for (rid, (arg_types, return_type)) in combinations.into_iter().enumerate() {
add_parameters(
func_name,
args.as_ref(),
args.as_deref(),
arg_types,
return_type,
Self::is_variadic(udf.signature()),
Expand All @@ -376,7 +376,7 @@ impl InformationSchemaConfig {
for (rid, (arg_types, return_type)) in combinations.into_iter().enumerate() {
add_parameters(
func_name,
args.as_ref(),
args.as_deref(),
arg_types,
return_type,
Self::is_variadic(udaf.signature()),
Expand All @@ -391,7 +391,7 @@ impl InformationSchemaConfig {
for (rid, (arg_types, return_type)) in combinations.into_iter().enumerate() {
add_parameters(
func_name,
args.as_ref(),
args.as_deref(),
arg_types,
return_type,
Self::is_variadic(udwf.signature()),
Expand Down
2 changes: 1 addition & 1 deletion datafusion/catalog/src/stream.rs
Original file line number Diff line number Diff line change
Expand Up @@ -336,7 +336,7 @@ impl TableProvider for StreamTable {
Ok(Arc::new(StreamingTableExec::try_new(
Arc::clone(self.0.source.schema()),
vec![Arc::new(StreamRead(Arc::clone(&self.0))) as _],
projection,
projection.map(Vec::as_slice),
projected_schema,
true,
limit,
Expand Down
2 changes: 1 addition & 1 deletion datafusion/catalog/src/streaming.rs
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,7 @@ impl TableProvider for StreamingTable {
Ok(Arc::new(StreamingTableExec::try_new(
Arc::clone(&self.schema),
self.partitions.clone(),
projection,
projection.map(Vec::as_slice),
LexOrdering::new(physical_sort),
self.infinite,
limit,
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/bin/print_functions_docs.rs
Original file line number Diff line number Diff line change
Expand Up @@ -300,7 +300,7 @@ impl DocProvider for Arc<dyn HigherOrderUDF> {
}

#[expect(clippy::borrowed_box)]
fn get_names_and_aliases(functions: &Vec<&Box<dyn DocProvider>>) -> Vec<String> {
fn get_names_and_aliases(functions: &[&Box<dyn DocProvider>]) -> Vec<String> {
functions
.iter()
.flat_map(|f| {
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/datasource/file_format/parquet.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1594,7 +1594,7 @@ mod tests {

fn assert_file_metadata(
parquet_meta_data: ParquetMetaData,
expected_kv: &Vec<KeyValue>,
expected_kv: &[KeyValue],
) {
let file_metadata = parquet_meta_data.file_metadata();
let schema_descr = file_metadata.schema_descr();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -561,7 +561,7 @@ fn test_streaming_table_after_projection() -> Result<()> {
Field::new("e", DataType::Int32, true),
])),
}) as _],
Some(&vec![0_usize, 2, 4, 3]),
Some(&[0_usize, 2, 4, 3]),
vec![
[
PhysicalSortExpr {
Expand Down
2 changes: 1 addition & 1 deletion datafusion/datasource-arrow/src/file_format.rs
Original file line number Diff line number Diff line change
Expand Up @@ -199,7 +199,7 @@ impl FileFormat for ArrowFormat {

let table_schema = TableSchema::new(
Arc::clone(conf.file_schema()),
conf.table_partition_cols().clone(),
conf.table_partition_cols().to_vec(),
);

let mut source: Arc<dyn FileSource> =
Expand Down
2 changes: 1 addition & 1 deletion datafusion/datasource-parquet/src/row_group_filter.rs
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ impl RowGroupAccessPlanFilter {
}

/// Returns the is_fully_matched vector
pub fn is_fully_matched(&self) -> &Vec<bool> {
pub fn is_fully_matched(&self) -> &[bool] {
&self.is_fully_matched
}

Expand Down
2 changes: 1 addition & 1 deletion datafusion/datasource/src/file_scan_config/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1068,7 +1068,7 @@ impl FileScanConfig {
}

/// Get the table partition columns
pub fn table_partition_cols(&self) -> &Vec<FieldRef> {
pub fn table_partition_cols(&self) -> &[FieldRef] {
self.file_source.table_schema().table_partition_cols()
}

Expand Down
2 changes: 1 addition & 1 deletion datafusion/datasource/src/table_schema.rs
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ impl TableSchema {
///
/// These are the columns derived from the directory structure that
/// will be appended to each row during query execution.
pub fn table_partition_cols(&self) -> &Vec<FieldRef> {
pub fn table_partition_cols(&self) -> &[FieldRef] {
&self.table_partition_cols
}

Expand Down
5 changes: 3 additions & 2 deletions datafusion/functions/src/crypto/basic.rs
Original file line number Diff line number Diff line change
Expand Up @@ -225,8 +225,9 @@ pub(crate) fn digest_process(
.digest_scalar(a.as_ref().map(|s: &String| s.as_bytes()))),
ScalarValue::Binary(a)
| ScalarValue::LargeBinary(a)
| ScalarValue::BinaryView(a) => Ok(digest_algorithm
.digest_scalar(a.as_ref().map(|v: &Vec<u8>| v.as_slice()))),
| ScalarValue::BinaryView(a) => {
Ok(digest_algorithm.digest_scalar(a.as_deref()))
}
other => exec_err!(
"Unsupported data type {other:?} for function {digest_algorithm}"
),
Expand Down
6 changes: 3 additions & 3 deletions datafusion/physical-expr/src/expressions/dynamic_filters.rs
Original file line number Diff line number Diff line change
Expand Up @@ -213,7 +213,7 @@ impl DynamicFilterPhysicalExpr {

fn remap_children(
children: &[Arc<dyn PhysicalExpr>],
remapped_children: Option<&Vec<Arc<dyn PhysicalExpr>>>,
remapped_children: Option<&[Arc<dyn PhysicalExpr>]>,
expr: Arc<dyn PhysicalExpr>,
) -> Result<Arc<dyn PhysicalExpr>> {
if let Some(remapped_children) = remapped_children {
Expand Down Expand Up @@ -250,7 +250,7 @@ impl DynamicFilterPhysicalExpr {
/// remapped to match calls to [`PhysicalExpr::with_new_children`].
pub fn current(&self) -> Result<Arc<dyn PhysicalExpr>> {
let expr = Arc::clone(self.inner.read().expr());
Self::remap_children(&self.children, self.remapped_children.as_ref(), expr)
Self::remap_children(&self.children, self.remapped_children.as_deref(), expr)
}

/// Update the current expression and notify all waiters.
Expand All @@ -266,7 +266,7 @@ impl DynamicFilterPhysicalExpr {
// and the same externally facing `PhysicalExpr` is used for both `with_new_children` and `update()`.`
let new_expr = Self::remap_children(
&self.children,
self.remapped_children.as_ref(),
self.remapped_children.as_deref(),
new_expr,
)?;

Expand Down
2 changes: 1 addition & 1 deletion datafusion/physical-optimizer/src/join_selection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -587,7 +587,7 @@ pub(crate) fn swap_join_according_to_unboundedness(
/// auxiliary boundedness information, is in the `PipelineStatePropagator` object.
fn apply_subrules(
mut input: Arc<dyn ExecutionPlan>,
subrules: &Vec<Box<PipelineFixerSubrule>>,
subrules: &[Box<PipelineFixerSubrule>],
config_options: &ConfigOptions,
) -> Result<Transformed<Arc<dyn ExecutionPlan>>> {
let original = Arc::clone(&input);
Expand Down
4 changes: 2 additions & 2 deletions datafusion/physical-plan/benches/aggregate_vectorized.rs
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,7 @@ fn bytes_bench(
group: &mut BenchmarkGroup<WallTime>,
bench_prefix: &str,
size: usize,
rows: &Vec<usize>,
rows: &[usize],
null_density: f32,
input: &ArrayRef,
) {
Expand Down Expand Up @@ -182,7 +182,7 @@ fn primitive_vectorized_append(c: &mut Criterion) {
fn bench_single_primitive<const NULLABLE: bool>(
group: &mut BenchmarkGroup<WallTime>,
size: usize,
rows: &Vec<usize>,
rows: &[usize],
null_density: f32,
) {
if !NULLABLE {
Expand Down
14 changes: 7 additions & 7 deletions datafusion/physical-plan/src/execution_plan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1467,7 +1467,7 @@ pub fn execute_input_stream(
/// is returned.
pub fn check_not_null_constraints(
batch: RecordBatch,
column_indices: &Vec<usize>,
column_indices: &[usize],
) -> Result<RecordBatch> {
for &index in column_indices {
if batch.num_columns() <= index {
Expand Down Expand Up @@ -1862,7 +1862,7 @@ mod tests {
Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, true)])),
vec![Arc::new(Int32Array::from(vec![Some(1), Some(2), Some(3)]))],
)?,
&vec![0],
&[0],
)?;
Ok(())
}
Expand All @@ -1874,7 +1874,7 @@ mod tests {
Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, true)])),
vec![Arc::new(Int32Array::from(vec![Some(1), None, Some(3)]))],
)?,
&vec![0],
&[0],
);
assert!(result.is_err());
assert_eq!(
Expand All @@ -1899,7 +1899,7 @@ mod tests {
)])),
vec![Arc::new(run_end_array)],
)?,
&vec![0],
&[0],
);
assert!(result.is_err());
assert_eq!(
Expand All @@ -1923,7 +1923,7 @@ mod tests {
)])),
vec![Arc::new(dictionary)],
)?,
&vec![0],
&[0],
);
assert!(result.is_err());
assert_eq!(
Expand Down Expand Up @@ -1953,7 +1953,7 @@ mod tests {
)])),
vec![Arc::new(dictionary)],
)?,
&vec![0],
&[0],
)?;
Ok(())
}
Expand All @@ -1966,7 +1966,7 @@ mod tests {
Arc::new(Schema::new(vec![Field::new("a", DataType::Null, true)])),
vec![Arc::new(NullArray::new(3))],
)?,
&vec![0],
&[0],
);
assert!(result.is_err());
assert_eq!(
Expand Down
2 changes: 1 addition & 1 deletion datafusion/physical-plan/src/filter.rs
Original file line number Diff line number Diff line change
Expand Up @@ -984,7 +984,7 @@ pub fn batch_filter(
fn filter_and_project(
batch: &RecordBatch,
predicate: &Arc<dyn PhysicalExpr>,
projection: Option<&Vec<usize>>,
projection: Option<&[usize]>,
) -> Result<RecordBatch> {
predicate
.evaluate(batch)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@ impl SortedStreamBatch {
}
}

fn compare_key_values(&self) -> &Vec<ArrayRef> {
fn compare_key_values(&self) -> &[ArrayRef] {
&self.compare_key_values
}
}
Expand Down
2 changes: 1 addition & 1 deletion datafusion/physical-plan/src/memory.rs
Original file line number Diff line number Diff line change
Expand Up @@ -249,7 +249,7 @@ impl LazyMemoryExec {
}

/// Get the batch generators
pub fn generators(&self) -> &Vec<Arc<RwLock<dyn LazyBatchGenerator>>> {
pub fn generators(&self) -> &[Arc<RwLock<dyn LazyBatchGenerator>>] {
&self.batch_generators
}
}
Expand Down
10 changes: 5 additions & 5 deletions datafusion/physical-plan/src/streaming.rs
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ impl StreamingTableExec {
pub fn try_new(
schema: SchemaRef,
partitions: Vec<Arc<dyn PartitionStream>>,
projection: Option<&Vec<usize>>,
projection: Option<&[usize]>,
projected_output_ordering: impl IntoIterator<Item = LexOrdering>,
infinite: bool,
limit: Option<usize>,
Expand Down Expand Up @@ -108,7 +108,7 @@ impl StreamingTableExec {
Ok(Self {
partitions,
projected_schema,
projection: projection.cloned().map(Into::into),
projection: projection.map(Into::into),
projected_output_ordering,
infinite,
limit,
Expand All @@ -117,7 +117,7 @@ impl StreamingTableExec {
})
}

pub fn partitions(&self) -> &Vec<Arc<dyn PartitionStream>> {
pub fn partitions(&self) -> &[Arc<dyn PartitionStream>] {
&self.partitions
}

Expand Down Expand Up @@ -318,7 +318,7 @@ impl ExecutionPlan for StreamingTableExec {

StreamingTableExec::try_new(
Arc::clone(self.partition_schema()),
self.partitions().clone(),
self.partitions().to_vec(),
Some(new_projections.as_ref()),
lex_orderings,
self.is_infinite(),
Expand Down Expand Up @@ -421,7 +421,7 @@ mod test {
StreamingTableExec::try_new(
self.schema.unwrap(),
self.partitions,
self.projection.as_ref(),
self.projection.as_deref(),
self.projected_output_ordering,
self.infinite,
self.limit,
Expand Down
4 changes: 2 additions & 2 deletions datafusion/physical-plan/src/union.rs
Original file line number Diff line number Diff line change
Expand Up @@ -161,7 +161,7 @@ impl UnionExec {
}

/// Get inputs of the execution plan
pub fn inputs(&self) -> &Vec<Arc<dyn ExecutionPlan>> {
pub fn inputs(&self) -> &[Arc<dyn ExecutionPlan>] {
&self.inputs
}

Expand Down Expand Up @@ -526,7 +526,7 @@ impl InterleaveExec {
}

/// Get inputs of the execution plan
pub fn inputs(&self) -> &Vec<Arc<dyn ExecutionPlan>> {
pub fn inputs(&self) -> &[Arc<dyn ExecutionPlan>] {
&self.inputs
}

Expand Down
2 changes: 1 addition & 1 deletion datafusion/proto/src/physical_plan/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -3485,7 +3485,7 @@ impl protobuf::PhysicalPlanNode {
let generators = exec.generators();

// ensure we only have one generator
let [generator] = generators.as_slice() else {
let [generator] = generators else {
return Ok(None);
};

Expand Down
2 changes: 1 addition & 1 deletion datafusion/sql/src/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -504,7 +504,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> {
/// Returns a vector of (column_name, default_expr) pairs
pub(super) fn build_column_defaults(
&self,
columns: &Vec<SQLColumnDef>,
columns: &[SQLColumnDef],
planner_context: &mut PlannerContext,
) -> Result<Vec<(String, Expr)>> {
let mut column_defaults = vec![];
Expand Down
2 changes: 1 addition & 1 deletion datafusion/sql/src/query.rs
Original file line number Diff line number Diff line change
Expand Up @@ -388,7 +388,7 @@ fn to_order_by_exprs(order_by: Option<OrderBy>) -> Result<Vec<OrderByExpr>> {
/// Returns the order by expressions from the query with the select expressions.
pub(crate) fn to_order_by_exprs_with_select(
order_by: Option<OrderBy>,
select_exprs: Option<&Vec<Expr>>,
select_exprs: Option<&[Expr]>,
) -> Result<Vec<OrderByExpr>> {
let Some(OrderBy { kind, interpolate }) = order_by else {
// If no order by, return an empty array.
Expand Down
Loading
Loading