neilconway commented on code in PR #21240: URL: https://github.com/apache/datafusion/pull/21240#discussion_r3074840543
########## datafusion/physical-expr/src/scalar_subquery.rs: ########## @@ -0,0 +1,230 @@ +// 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. + +//! Physical expression for uncorrelated scalar subqueries. +//! +//! [`ScalarSubqueryExpr`] reads a cached [`ScalarValue`] that is populated +//! at execution time by `ScalarSubqueryExec`. + +use std::any::Any; +use std::fmt; +use std::hash::Hash; +use std::sync::{Arc, OnceLock}; + +use arrow::datatypes::{DataType, Field, FieldRef, Schema}; +use arrow::record_batch::RecordBatch; +use datafusion_common::{Result, ScalarValue, internal_datafusion_err}; +use datafusion_expr_common::columnar_value::ColumnarValue; +use datafusion_expr_common::sort_properties::{ExprProperties, SortProperties}; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; + +/// A physical expression whose value is provided by a scalar subquery. +/// +/// Subquery execution is handled by `ScalarSubqueryExec`, which stores the +/// result in a shared results container. This expression simply reads from the +/// shared results container at the appropriate index. +/// +/// If the same subquery appears multiple times in a query, there will be +/// multiple `ScalarSubqueryExpr` with the same result index. +#[derive(Debug)] +pub struct ScalarSubqueryExpr { + data_type: DataType, + nullable: bool, + /// Index of this subquery in the shared results container. + index: usize, + /// Shared results container populated by `ScalarSubqueryExec`. + results: Arc<Vec<OnceLock<ScalarValue>>>, Review Comment: Ah, very good catch. The fix is actually a little involved: there's no easy way to reset a `OnceLock`, so we need to switch to a `Mutex`. I added the fix, along with a unit test. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
