From 80e73d76c79bbff3d4a5d0515007175305db245a Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Mon, 14 Aug 2023 21:58:43 +0100 Subject: [PATCH 01/10] Prepare for ObjectStore 0.7.0 --- Cargo.toml | 3 + datafusion-cli/Cargo.lock | 3 +- datafusion-cli/Cargo.toml | 3 + .../core/src/datasource/file_format/arrow.rs | 11 +- .../core/src/datasource/file_format/avro.rs | 11 +- .../core/src/datasource/file_format/json.rs | 9 +- .../core/src/datasource/file_format/mod.rs | 34 ++- .../datasource/physical_plan/arrow_file.rs | 9 +- .../core/src/datasource/physical_plan/avro.rs | 11 +- .../datasource/physical_plan/chunked_store.rs | 223 ------------------ .../core/src/datasource/physical_plan/csv.rs | 175 ++++---------- .../core/src/datasource/physical_plan/json.rs | 11 +- .../core/src/datasource/physical_plan/mod.rs | 2 - datafusion/core/src/test/mod.rs | 5 +- datafusion/core/tests/path_partition.rs | 17 +- 15 files changed, 136 insertions(+), 391 deletions(-) delete mode 100644 datafusion/core/src/datasource/physical_plan/chunked_store.rs diff --git a/Cargo.toml b/Cargo.toml index fd862e133293..e28df3affc67 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -57,3 +57,6 @@ opt-level = 3 overflow-checks = false panic = 'unwind' rpath = false + +[patch.crates-io] +object_store = { git = "https://github.com/apache/arrow-rs.git", rev = "979a070dc82eeb26b38a8651cac879b2c276c0ed" } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index cca667b11425..e0432671912e 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -2179,8 +2179,7 @@ dependencies = [ [[package]] name = "object_store" version = "0.6.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "27c776db4f332b571958444982ff641d2531417a326ca368995073b639205d58" +source = "git+https://github.com/apache/arrow-rs.git?rev=979a070dc82eeb26b38a8651cac879b2c276c0ed#979a070dc82eeb26b38a8651cac879b2c276c0ed" dependencies = [ "async-trait", "base64", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index fc38d59b7029..1a25c99d8f8a 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -49,3 +49,6 @@ assert_cmd = "2.0" ctor = "0.2.0" predicates = "3.0" rstest = "0.17" + +[patch.crates-io] +object_store = { git = "https://github.com/apache/arrow-rs.git", rev = "979a070dc82eeb26b38a8651cac879b2c276c0ed" } diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 2b3ef7ee4eab..ec328b447873 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -29,7 +29,7 @@ use arrow_schema::{Schema, SchemaRef}; use async_trait::async_trait; use datafusion_common::Statistics; use datafusion_physical_expr::PhysicalExpr; -use object_store::{GetResult, ObjectMeta, ObjectStore}; +use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; use std::any::Any; use std::io::{Read, Seek}; use std::sync::Arc; @@ -54,9 +54,12 @@ impl FileFormat for ArrowFormat { ) -> Result { let mut schemas = vec![]; for object in objects { - let schema = match store.get(&object.location).await? { - GetResult::File(mut file, _) => read_arrow_schema_from_reader(&mut file)?, - r @ GetResult::Stream(_) => { + let r = store.as_ref().get(&object.location).await?; + let schema = match r.payload { + GetResultPayload::File(mut file, _) => { + read_arrow_schema_from_reader(&mut file)? + } + GetResultPayload::Stream(_) => { // TODO: Fetching entire file to get schema is potentially wasteful let data = r.bytes().await?; let mut cursor = std::io::Cursor::new(&data); diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index cfb146e4ff04..c50e9294a12f 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -24,7 +24,7 @@ use arrow::datatypes::Schema; use arrow::{self, datatypes::SchemaRef}; use async_trait::async_trait; use datafusion_physical_expr::PhysicalExpr; -use object_store::{GetResult, ObjectMeta, ObjectStore}; +use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; use super::FileFormat; use crate::datasource::avro_to_arrow::read_avro_schema_from_reader; @@ -54,9 +54,12 @@ impl FileFormat for AvroFormat { ) -> Result { let mut schemas = vec![]; for object in objects { - let schema = match store.get(&object.location).await? { - GetResult::File(mut file, _) => read_avro_schema_from_reader(&mut file)?, - r @ GetResult::Stream(_) => { + let r = store.as_ref().get(&object.location).await?; + let schema = match r.payload { + GetResultPayload::File(mut file, _) => { + read_avro_schema_from_reader(&mut file)? + } + GetResultPayload::Stream(_) => { // TODO: Fetching entire file to get schema is potentially wasteful let data = r.bytes().await?; read_avro_schema_from_reader(&mut data.as_ref())? diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 8472f4e5c164..0806934f0b95 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -39,7 +39,7 @@ use async_trait::async_trait; use bytes::Buf; use datafusion_physical_expr::PhysicalExpr; -use object_store::{GetResult, ObjectMeta, ObjectStore}; +use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; use crate::datasource::physical_plan::FileGroupDisplay; use crate::physical_plan::insert::DataSink; @@ -121,14 +121,15 @@ impl FileFormat for JsonFormat { should_take }; - let schema = match store.get(&object.location).await? { - GetResult::File(file, _) => { + let r = store.as_ref().get(&object.location).await?; + let schema = match r.payload { + GetResultPayload::File(file, _) => { let decoder = file_compression_type.convert_read(file)?; let mut reader = BufReader::new(decoder); let iter = ValueIter::new(&mut reader, None); infer_json_schema_from_iterator(iter.take_while(|_| take_while()))? } - r @ GetResult::Stream(_) => { + GetResultPayload::Stream(_) => { let data = r.bytes().await?; let decoder = file_compression_type.convert_read(data.reader())?; let mut reader = BufReader::new(decoder); diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index 9eec11f224ea..f9f7cd6706a7 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -119,7 +119,9 @@ pub(crate) mod test_util { use futures::StreamExt; use object_store::local::LocalFileSystem; use object_store::path::Path; - use object_store::{GetOptions, GetResult, ListResult, MultipartId}; + use object_store::{ + GetOptions, GetResult, GetResultPayload, ListResult, MultipartId, + }; use tokio::io::AsyncWrite; pub async fn scan_format( @@ -203,18 +205,28 @@ pub(crate) mod test_util { unimplemented!() } - async fn get(&self, _location: &Path) -> object_store::Result { + async fn get(&self, location: &Path) -> object_store::Result { let bytes = self.bytes_to_repeat.clone(); + let range = 0..bytes.len() * self.max_iterations; let arc = self.iterations_detected.clone(); - Ok(GetResult::Stream( - futures::stream::repeat_with(move || { - let arc_inner = arc.clone(); - *arc_inner.lock().unwrap() += 1; - Ok(bytes.clone()) - }) - .take(self.max_iterations) - .boxed(), - )) + let stream = futures::stream::repeat_with(move || { + let arc_inner = arc.clone(); + *arc_inner.lock().unwrap() += 1; + Ok(bytes.clone()) + }) + .take(self.max_iterations) + .boxed(); + + Ok(GetResult { + payload: GetResultPayload::Stream(stream), + meta: ObjectMeta { + location: location.clone(), + last_modified: Default::default(), + size: range.end, + e_tag: None, + }, + range: Default::default(), + }) } async fn get_opts( diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 3c70eb4e7462..a47376248ed3 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -32,7 +32,7 @@ use datafusion_physical_expr::{ PhysicalSortExpr, }; use futures::StreamExt; -use object_store::{GetResult, ObjectStore}; +use object_store::{GetResultPayload, ObjectStore}; use std::any::Any; use std::sync::Arc; @@ -158,13 +158,14 @@ impl FileOpener for ArrowOpener { let object_store = self.object_store.clone(); let projection = self.projection.clone(); Ok(Box::pin(async move { - match object_store.get(file_meta.location()).await? { - GetResult::File(file, _) => { + let r = object_store.get(file_meta.location()).await?; + match r.payload { + GetResultPayload::File(file, _) => { let arrow_reader = arrow::ipc::reader::FileReader::try_new(file, projection)?; Ok(futures::stream::iter(arrow_reader).boxed()) } - r @ GetResult::Stream(_) => { + GetResultPayload::Stream(_) => { let bytes = r.bytes().await?; let cursor = std::io::Cursor::new(bytes); let arrow_reader = diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index ecb78f0e3dd2..332b411a8d4f 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -175,7 +175,7 @@ mod private { use crate::datasource::physical_plan::FileMeta; use bytes::Buf; use futures::StreamExt; - use object_store::{GetResult, ObjectStore}; + use object_store::{GetResultPayload, ObjectStore}; pub struct AvroConfig { pub schema: SchemaRef, @@ -203,12 +203,13 @@ mod private { fn open(&self, file_meta: FileMeta) -> Result { let config = self.config.clone(); Ok(Box::pin(async move { - match config.object_store.get(file_meta.location()).await? { - GetResult::File(file, _) => { + let r = config.object_store.get(file_meta.location()).await?; + match r.payload { + GetResultPayload::File(file, _) => { let reader = config.open(file)?; Ok(futures::stream::iter(reader).boxed()) } - r @ GetResult::Stream(_) => { + GetResultPayload::Stream(_) => { let bytes = r.bytes().await?; let reader = config.open(bytes.reader())?; Ok(futures::stream::iter(reader).boxed()) @@ -225,12 +226,12 @@ mod tests { use crate::datasource::file_format::{avro::AvroFormat, FileFormat}; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; - use crate::datasource::physical_plan::chunked_store::ChunkedStore; use crate::prelude::SessionContext; use crate::scalar::ScalarValue; use crate::test::object_store::local_unpartitioned_file; use arrow::datatypes::{DataType, Field, SchemaBuilder}; use futures::StreamExt; + use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; use object_store::ObjectStore; use rstest::*; diff --git a/datafusion/core/src/datasource/physical_plan/chunked_store.rs b/datafusion/core/src/datasource/physical_plan/chunked_store.rs deleted file mode 100644 index 05528ed8a2b6..000000000000 --- a/datafusion/core/src/datasource/physical_plan/chunked_store.rs +++ /dev/null @@ -1,223 +0,0 @@ -// 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. - -use async_trait::async_trait; -use bytes::{BufMut, Bytes, BytesMut}; -use futures::stream::BoxStream; -use futures::StreamExt; -use object_store::path::Path; -use object_store::{GetOptions, GetResult, ListResult, ObjectMeta, ObjectStore}; -use object_store::{MultipartId, Result}; -use std::fmt::{Debug, Display, Formatter}; -use std::ops::Range; -use std::sync::Arc; -use tokio::io::{AsyncReadExt, AsyncWrite, BufReader}; - -/// Wraps a [`ObjectStore`] and makes its get response return chunks -/// in a controllable manner. -/// -/// A `ChunkedStore` makes the memory consumption and performance of -/// the wrapped [`ObjectStore`] worse. It is intended for use within -/// tests, to control the chunks in the produced output streams. For -/// example, it is used to verify the delimiting logic in -/// newline_delimited_stream. -/// -/// TODO: Upstream into object_store_rs -#[derive(Debug)] -pub struct ChunkedStore { - inner: Arc, - chunk_size: usize, -} - -impl ChunkedStore { - pub fn new(inner: Arc, chunk_size: usize) -> Self { - Self { inner, chunk_size } - } -} - -impl Display for ChunkedStore { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!(f, "ChunkedStore({})", self.inner) - } -} - -#[async_trait] -impl ObjectStore for ChunkedStore { - async fn put(&self, location: &Path, bytes: Bytes) -> Result<()> { - self.inner.put(location, bytes).await - } - - async fn put_multipart( - &self, - location: &Path, - ) -> Result<(MultipartId, Box)> { - self.inner.put_multipart(location).await - } - - async fn abort_multipart( - &self, - location: &Path, - multipart_id: &MultipartId, - ) -> Result<()> { - self.inner.abort_multipart(location, multipart_id).await - } - - async fn get(&self, location: &Path) -> Result { - match self.inner.get(location).await? { - GetResult::File(std_file, ..) => { - let file = tokio::fs::File::from_std(std_file); - let reader = BufReader::new(file); - Ok(GetResult::Stream( - futures::stream::unfold( - (reader, self.chunk_size), - |(mut reader, chunk_size)| async move { - let mut buffer = BytesMut::zeroed(chunk_size); - let size = reader.read(&mut buffer).await.map_err(|e| { - object_store::Error::Generic { - store: "ChunkedStore", - source: Box::new(e), - } - }); - match size { - Ok(0) => None, - Ok(value) => Some(( - Ok(buffer.split_to(value).freeze()), - (reader, chunk_size), - )), - Err(e) => Some((Err(e), (reader, chunk_size))), - } - }, - ) - .boxed(), - )) - } - GetResult::Stream(stream) => { - let buffer = BytesMut::new(); - Ok(GetResult::Stream( - futures::stream::unfold( - (stream, buffer, false, self.chunk_size), - |(mut stream, mut buffer, mut exhausted, chunk_size)| async move { - // Keep accumulating bytes until we reach capacity as long as - // the stream can provide them: - if exhausted { - return None; - } - while buffer.len() < chunk_size { - match stream.next().await { - None => { - exhausted = true; - let slice = buffer.split_off(0).freeze(); - return Some(( - Ok(slice), - (stream, buffer, exhausted, chunk_size), - )); - } - Some(Ok(bytes)) => { - buffer.put(bytes); - } - Some(Err(e)) => { - return Some(( - Err(object_store::Error::Generic { - store: "ChunkedStore", - source: Box::new(e), - }), - (stream, buffer, exhausted, chunk_size), - )) - } - }; - } - // Return the chunked values as the next value in the stream - let slice = buffer.split_to(chunk_size).freeze(); - Some((Ok(slice), (stream, buffer, exhausted, chunk_size))) - }, - ) - .boxed(), - )) - } - } - } - - async fn get_opts(&self, location: &Path, options: GetOptions) -> Result { - self.inner.get_opts(location, options).await - } - - async fn get_range(&self, location: &Path, range: Range) -> Result { - self.inner.get_range(location, range).await - } - - async fn head(&self, location: &Path) -> Result { - self.inner.head(location).await - } - - async fn delete(&self, location: &Path) -> Result<()> { - self.inner.delete(location).await - } - - async fn list( - &self, - prefix: Option<&Path>, - ) -> Result>> { - self.inner.list(prefix).await - } - - async fn list_with_delimiter(&self, prefix: Option<&Path>) -> Result { - self.inner.list_with_delimiter(prefix).await - } - - async fn copy(&self, from: &Path, to: &Path) -> Result<()> { - self.inner.copy(from, to).await - } - - async fn copy_if_not_exists(&self, from: &Path, to: &Path) -> Result<()> { - self.inner.copy_if_not_exists(from, to).await - } -} - -#[cfg(test)] -mod tests { - use super::*; - use futures::StreamExt; - use object_store::memory::InMemory; - use object_store::path::Path; - - #[tokio::test] - async fn test_chunked() { - let location = Path::parse("test").unwrap(); - let store = Arc::new(InMemory::new()); - store - .put(&location, Bytes::from(vec![0; 1001])) - .await - .unwrap(); - - for chunk_size in [10, 20, 31] { - let store = ChunkedStore::new(store.clone(), chunk_size); - let mut s = match store.get(&location).await.unwrap() { - GetResult::Stream(s) => s, - _ => unreachable!(), - }; - - let mut remaining = 1001; - while let Some(next) = s.next().await { - let size = next.unwrap().len(); - let expected = remaining.min(chunk_size); - assert_eq!(size, expected); - remaining -= expected; - } - assert_eq!(remaining, 0); - } - } -} diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 6bf5e3634021..36ae6538d009 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -43,10 +43,9 @@ use super::FileScanConfig; use bytes::{Buf, Bytes}; use futures::ready; use futures::{StreamExt, TryStreamExt}; -use object_store::local::LocalFileSystem; -use object_store::{GetOptions, GetResult, ObjectStore}; +use object_store::{GetOptions, GetResultPayload, ObjectStore}; use std::any::Any; -use std::io::Cursor; +use std::io::{Read, Seek, SeekFrom}; use std::ops::Range; use std::sync::Arc; use std::task::Poll; @@ -286,31 +285,23 @@ impl CsvConfig { } impl CsvConfig { - fn open(&self, reader: R) -> Result> { - let mut builder = csv::ReaderBuilder::new(self.file_schema.clone()) - .has_header(self.has_header) - .with_delimiter(self.delimiter) - .with_quote(self.quote) - .with_batch_size(self.batch_size); - if let Some(escape) = self.escape { - builder = builder.with_escape(escape); - } - if let Some(p) = &self.file_projection { - builder = builder.with_projection(p.clone()); - } - - Ok(builder.build(reader)?) + fn open(&self, reader: R) -> Result> { + Ok(self.builder().build(reader)?) } fn builder(&self) -> csv::ReaderBuilder { let mut builder = csv::ReaderBuilder::new(self.file_schema.clone()) .with_delimiter(self.delimiter) .with_batch_size(self.batch_size) - .has_header(self.has_header); + .has_header(self.has_header) + .with_quote(self.quote); if let Some(proj) = &self.file_projection { builder = builder.with_projection(proj.clone()); } + if let Some(escape) = self.escape { + builder = builder.with_escape(escape) + } builder } @@ -335,30 +326,6 @@ impl CsvOpener { } } -/// Returns the position of the first newline in the byte stream, or the total length if no newline is found. -fn find_first_newline_bytes(reader: &mut R) -> Result { - let mut buffer = [0; 1]; - let mut index = 0; - - loop { - let result = reader.read(&mut buffer); - match result { - Ok(n) => { - if n == 0 { - return Ok(index); // End of file, no newline found - } - if buffer[0] == b'\n' { - return Ok(index); - } - index += 1; - } - Err(e) => { - return Err(DataFusionError::IoError(e)); - } - } - } -} - /// Returns the offset of the first newline in the object store range [start, end), or the end offset if no newline is found. async fn find_first_newline( object_store: &Arc, @@ -374,54 +341,30 @@ async fn find_first_newline( ..Default::default() }; - let offset = match object_store.get_opts(location, options).await? { - GetResult::File(_, _) => { - // Range currently is ignored for GetResult::File(...) - // Alternative get_range() will copy the whole range into memory, thus set a limit of - // max bytes to read to find the first newline - let max_line_length = 4096; // in bytes - let get_range_end_result = object_store - .get_range( - location, - Range { - start: start_byte, - end: std::cmp::min(start_byte + max_line_length, end_byte), - }, - ) - .await; - let mut decoder_tail = Cursor::new(get_range_end_result?); - find_first_newline_bytes(&mut decoder_tail)? - } - GetResult::Stream(s) => { - let mut input = s.map_err(DataFusionError::from); - let mut buffered = Bytes::new(); - - let future_index = async move { - let mut index = 0; - - loop { - if buffered.is_empty() { - match input.next().await { - Some(Ok(b)) => buffered = b, - Some(Err(e)) => return Err(e), - None => return Ok(index), - }; - } + let r = object_store.get_opts(location, options).await?; + let mut input = r.into_stream(); - for byte in &buffered { - if *byte == b'\n' { - return Ok(index); - } - index += 1; - } + let mut buffered = Bytes::new(); + let mut index = 0; - buffered.advance(buffered.len()); - } + loop { + if buffered.is_empty() { + match input.next().await { + Some(Ok(b)) => buffered = b, + Some(Err(e)) => return Err(e.into()), + None => return Ok(index), }; - future_index.await? } - }; - Ok(offset) + + for byte in &buffered { + if *byte == b'\n' { + return Ok(index); + } + index += 1; + } + + buffered.advance(buffered.len()); + } } impl FileOpener for CsvOpener { @@ -476,8 +419,8 @@ impl FileOpener for CsvOpener { Ok(Box::pin(async move { let file_size = file_meta.object_meta.size; // Current partition contains bytes [start_byte, end_byte) (might contain incomplete lines at boundaries) - let (start_byte, end_byte) = match file_meta.range { - None => (0, file_size), + let range = match file_meta.range { + None => None, Some(FileRange { start, end }) => { let (start, end) = (start as usize, end as usize); // Partition byte range is [start, end), the boundary might be in the middle of @@ -504,57 +447,41 @@ impl FileOpener for CsvOpener { } else { 0 }; - (start + start_delta, end + end_delta) + let range = start + start_delta..end + end_delta; + if range.start == range.end { + return Ok( + futures::stream::poll_fn(move |_| Poll::Ready(None)).boxed() + ); + } + Some(range) } }; - // For special case: If `Range` has equal `start` and `end`, object store will fetch - // the whole file - let localfs: Arc = Arc::new(LocalFileSystem::new()); - let is_localfs = localfs.type_id() == config.object_store.type_id(); - if start_byte == end_byte && !is_localfs { - return Ok(futures::stream::poll_fn(move |_| Poll::Ready(None)).boxed()); - } - let options = GetOptions { - range: Some(Range { - start: start_byte, - end: end_byte, - }), + range, ..Default::default() }; - - match config + let result = config .object_store .get_opts(file_meta.location(), options) - .await? - { - GetResult::File(file, _) => { + .await?; + + match result.payload { + GetResultPayload::File(mut file, _) => { let is_whole_file_scanned = file_meta.range.is_none(); let decoder = if is_whole_file_scanned { - // For special case: `get_range()` will interpret `start` and `end` as the - // byte range after decompression for compressed files + // Don't seek if no range as breaks FIFO files file_compression_type.convert_read(file)? } else { - // Range currently is ignored for GetResult::File(...) - let bytes = Cursor::new( - config - .object_store - .get_range( - file_meta.location(), - Range { - start: start_byte, - end: end_byte, - }, - ) - .await?, - ); - file_compression_type.convert_read(bytes)? + file.seek(SeekFrom::Start(result.range.start as _))?; + file_compression_type.convert_read( + file.take((result.range.end - result.range.start) as u64), + )? }; Ok(futures::stream::iter(config.open(decoder)?).boxed()) } - GetResult::Stream(s) => { + GetResultPayload::Stream(s) => { let mut decoder = config.builder().build_decoder(); let s = s.map_err(DataFusionError::from); let mut input = @@ -651,13 +578,13 @@ pub async fn plan_to_csv( mod tests { use super::*; use crate::datasource::file_format::file_type::FileType; - use crate::datasource::physical_plan::chunked_store::ChunkedStore; use crate::prelude::*; use crate::test::{partitioned_csv_config, partitioned_file_groups}; use crate::test_util::{aggr_test_schema_with_missing_col, arrow_test_data}; use crate::{scalar::ScalarValue, test_util::aggr_test_schema}; use arrow::datatypes::*; use futures::StreamExt; + use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; use rstest::*; use std::fs::File; diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index b8ad2aa0a62f..5a7c3e741a03 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -40,7 +40,7 @@ use datafusion_physical_expr::{ use bytes::{Buf, Bytes}; use futures::{ready, stream, StreamExt, TryStreamExt}; use object_store; -use object_store::{GetResult, ObjectStore}; +use object_store::{GetResultPayload, ObjectStore}; use std::any::Any; use std::io::BufReader; use std::sync::Arc; @@ -209,15 +209,16 @@ impl FileOpener for JsonOpener { let file_compression_type = self.file_compression_type.to_owned(); Ok(Box::pin(async move { - match store.get(file_meta.location()).await? { - GetResult::File(file, _) => { + let r = store.get(file_meta.location()).await?; + match r.payload { + GetResultPayload::File(file, _) => { let bytes = file_compression_type.convert_read(file)?; let reader = ReaderBuilder::new(schema) .with_batch_size(batch_size) .build(BufReader::new(bytes))?; Ok(futures::stream::iter(reader).boxed()) } - GetResult::Stream(s) => { + GetResultPayload::Stream(s) => { let mut decoder = ReaderBuilder::new(schema) .with_batch_size(batch_size) .build_decoder()?; @@ -324,12 +325,12 @@ mod tests { use crate::datasource::file_format::{json::JsonFormat, FileFormat}; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; - use crate::datasource::physical_plan::chunked_store::ChunkedStore; use crate::execution::context::SessionState; use crate::prelude::NdJsonReadOptions; use crate::prelude::*; use crate::test::partitioned_file_groups; use datafusion_common::cast::{as_int32_array, as_int64_array, as_string_array}; + use object_store::chunked::ChunkedStore; use rstest::*; use std::path::Path; use tempfile::TempDir; diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index b0914b081616..0644286a92c6 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -19,8 +19,6 @@ mod arrow_file; mod avro; -#[cfg(test)] -mod chunked_store; mod csv; mod file_stream; mod json; diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index b7bf2a851aa4..311b51349bcf 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -173,7 +173,10 @@ pub fn partitioned_file_groups( writers[partition].write_all(b"\n").unwrap(); } } - for w in writers.iter_mut() { + + // Must drop the stream before creating ObjectMeta below as drop + // triggers finish for ZstdEncoder which writes additional data + for mut w in writers.into_iter() { w.flush().unwrap(); } diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index d4aa6c7e82fd..ea3fb7bef9d7 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -43,7 +43,8 @@ use datafusion_common::ScalarValue; use futures::stream; use futures::stream::BoxStream; use object_store::{ - path::Path, GetOptions, GetResult, ListResult, MultipartId, ObjectMeta, ObjectStore, + path::Path, GetOptions, GetResult, GetResultPayload, ListResult, MultipartId, + ObjectMeta, ObjectStore, }; use tokio::io::AsyncWrite; use url::Url; @@ -648,7 +649,19 @@ impl ObjectStore for MirroringObjectStore { self.files.iter().find(|x| *x == location).unwrap(); let path = std::path::PathBuf::from(&self.mirrored_file); let file = File::open(&path).unwrap(); - Ok(GetResult::File(file, path)) + let metadata = file.metadata().unwrap(); + let meta = ObjectMeta { + location: location.clone(), + last_modified: metadata.modified().map(chrono::DateTime::from).unwrap(), + size: metadata.len() as usize, + e_tag: None, + }; + + Ok(GetResult { + range: 0..meta.size, + payload: GetResultPayload::File(file, path), + meta, + }) } async fn get_range( From 0d5d5d0aafa36f0627a36d10327bfa8ee5e37c83 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Fri, 18 Aug 2023 13:20:21 +0100 Subject: [PATCH 02/10] Update arrow --- Cargo.toml | 7 +- datafusion-cli/Cargo.lock | 51 ++-- datafusion-cli/Cargo.toml | 8 +- datafusion-examples/Cargo.toml | 2 +- datafusion/common/Cargo.toml | 2 +- datafusion/common/src/scalar.rs | 7 +- datafusion/common/src/utils.rs | 7 +- datafusion/core/Cargo.toml | 2 +- .../physical_plan/parquet/row_groups.rs | 4 +- .../core/src/physical_plan/joins/hash_join.rs | 7 +- datafusion/execution/Cargo.toml | 2 +- .../physical-expr/src/expressions/binary.rs | 241 ++---------------- .../src/expressions/binary/adapter.rs | 49 ---- .../physical-expr/src/expressions/case.rs | 5 +- .../physical-expr/src/expressions/nullif.rs | 22 +- datafusion/proto/Cargo.toml | 2 +- datafusion/sqllogictest/Cargo.toml | 2 +- datafusion/substrait/Cargo.toml | 2 +- 18 files changed, 83 insertions(+), 339 deletions(-) delete mode 100644 datafusion/physical-expr/src/expressions/binary/adapter.rs diff --git a/Cargo.toml b/Cargo.toml index e28df3affc67..2f467a26b3f5 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -59,4 +59,9 @@ panic = 'unwind' rpath = false [patch.crates-io] -object_store = { git = "https://github.com/apache/arrow-rs.git", rev = "979a070dc82eeb26b38a8651cac879b2c276c0ed" } +arrow = {git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } +arrow-array = {git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } +arrow-buffer = {git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } +arrow-flight = {git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } +arrow-schema = {git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } +parquet = {git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index e0432671912e..f25b817bebaf 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -96,8 +96,7 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b7104b9e9761613ae92fe770c741d6bbf1dbc791a0fe204400aebdd429875741" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" dependencies = [ "ahash", "arrow-arith", @@ -118,8 +117,7 @@ dependencies = [ [[package]] name = "arrow-arith" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38e597a8e8efb8ff52c50eaf8f4d85124ce3c1bf20fab82f476d73739d9ab1c2" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" dependencies = [ "arrow-array", "arrow-buffer", @@ -133,8 +131,7 @@ dependencies = [ [[package]] name = "arrow-array" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a86d9c1473db72896bd2345ebb6b8ad75b8553ba390875c76708e8dc5c5492d" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" dependencies = [ "ahash", "arrow-buffer", @@ -150,9 +147,9 @@ dependencies = [ [[package]] name = "arrow-buffer" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "234b3b1c8ed00c874bf95972030ac4def6f58e02ea5a7884314388307fb3669b" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" dependencies = [ + "bytes", "half", "num", ] @@ -160,8 +157,7 @@ dependencies = [ [[package]] name = "arrow-cast" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22f61168b853c7faea8cea23a2169fdff9c82fb10ae5e2c07ad1cab8f6884931" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" dependencies = [ "arrow-array", "arrow-buffer", @@ -178,8 +174,7 @@ dependencies = [ [[package]] name = "arrow-csv" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10b545c114d9bf8569c84d2fbe2020ac4eea8db462c0a37d0b65f41a90d066fe" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" dependencies = [ "arrow-array", "arrow-buffer", @@ -197,8 +192,7 @@ dependencies = [ [[package]] name = "arrow-data" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6b6852635e7c43e5b242841c7470606ff0ee70eef323004cacc3ecedd33dd8f" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" dependencies = [ "arrow-buffer", "arrow-schema", @@ -209,8 +203,7 @@ dependencies = [ [[package]] name = "arrow-ipc" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a66da9e16aecd9250af0ae9717ae8dd7ea0d8ca5a3e788fe3de9f4ee508da751" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" dependencies = [ "arrow-array", "arrow-buffer", @@ -223,8 +216,7 @@ dependencies = [ [[package]] name = "arrow-json" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "60ee0f9d8997f4be44a60ee5807443e396e025c23cf14d2b74ce56135cb04474" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" dependencies = [ "arrow-array", "arrow-buffer", @@ -243,8 +235,7 @@ dependencies = [ [[package]] name = "arrow-ord" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fcab05410e6b241442abdab6e1035177dc082bdb6f17049a4db49faed986d63" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" dependencies = [ "arrow-array", "arrow-buffer", @@ -258,8 +249,7 @@ dependencies = [ [[package]] name = "arrow-row" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91a847dd9eb0bacd7836ac63b3475c68b2210c2c96d0ec1b808237b973bd5d73" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" dependencies = [ "ahash", "arrow-array", @@ -273,14 +263,12 @@ dependencies = [ [[package]] name = "arrow-schema" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54df8c47918eb634c20e29286e69494fdc20cafa5173eb6dad49c7f6acece733" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" [[package]] name = "arrow-select" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "941dbe481da043c4bd40c805a19ec2fc008846080c4953171b62bcad5ee5f7fb" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" dependencies = [ "arrow-array", "arrow-buffer", @@ -292,8 +280,7 @@ dependencies = [ [[package]] name = "arrow-string" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "359b2cd9e071d5a3bcf44679f9d85830afebc5b9c98a08019a570a65ae933e0f" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" dependencies = [ "arrow-array", "arrow-buffer", @@ -2178,8 +2165,9 @@ dependencies = [ [[package]] name = "object_store" -version = "0.6.1" -source = "git+https://github.com/apache/arrow-rs.git?rev=979a070dc82eeb26b38a8651cac879b2c276c0ed#979a070dc82eeb26b38a8651cac879b2c276c0ed" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d359e231e5451f4f9fa889d56e3ce34f8724f1a61db2107739359717cf2bbf08" dependencies = [ "async-trait", "base64", @@ -2264,8 +2252,7 @@ dependencies = [ [[package]] name = "parquet" version = "45.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49f9739b984380582bdb7749ae5b5d28839bce899212cf16465c1ac1f8b65d79" +source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" dependencies = [ "ahash", "arrow-array", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 1a25c99d8f8a..7be2d3a86735 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -38,7 +38,7 @@ datafusion = { path = "../datafusion/core", version = "29.0.0" } dirs = "4.0.0" env_logger = "0.9" mimalloc = { version = "0.1", default-features = false } -object_store = { version = "0.6.1", features = ["aws", "gcp"] } +object_store = { version = "0.7.0", features = ["aws", "gcp"] } parking_lot = { version = "0.12" } rustyline = "11.0" tokio = { version = "1.24", features = ["macros", "rt", "rt-multi-thread", "sync", "parking_lot"] } @@ -51,4 +51,8 @@ predicates = "3.0" rstest = "0.17" [patch.crates-io] -object_store = { git = "https://github.com/apache/arrow-rs.git", rev = "979a070dc82eeb26b38a8651cac879b2c276c0ed" } +arrow = { git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } +arrow-array = { git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } +arrow-buffer = { git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } +arrow-schema = { git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } +parquet = { git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 5bf9d837eda1..22cc3f4df425 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -46,7 +46,7 @@ futures = "0.3" log = "0.4" mimalloc = { version = "0.1", default-features = false } num_cpus = "1.13.0" -object_store = { version = "0.6.1", features = ["aws"] } +object_store = { version = "0.7.0", features = ["aws"] } prost = { version = "0.11", default-features = false } prost-derive = { version = "0.11", default-features = false } serde = { version = "1.0.136", features = ["derive"] } diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 4ffe3aa1912b..2d9f1b0bf01a 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -43,7 +43,7 @@ arrow = { workspace = true } arrow-array = { workspace = true } chrono = { version = "0.4", default-features = false } num_cpus = "1.13.0" -object_store = { version = "0.6.1", default-features = false, optional = true } +object_store = { version = "0.7.0", default-features = false, optional = true } parquet = { workspace = true, optional = true } pyo3 = { version = "0.19.0", optional = true } sqlparser = { workspace = true } diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 0fc1ca3dc7a7..6468509ae904 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -46,7 +46,7 @@ use arrow::{ DECIMAL128_MAX_PRECISION, }, }; -use arrow_array::{timezone::Tz, ArrowNativeTypeOp}; +use arrow_array::{timezone::Tz, ArrowNativeTypeOp, Scalar}; use chrono::{Datelike, Duration, NaiveDate, NaiveDateTime}; // Constants we use throughout this file: @@ -2202,6 +2202,11 @@ impl ScalarValue { self.to_array_of_size(1) } + /// Converts a scalar into an arrow [`Scalar`] + pub fn to_scalar(&self) -> Scalar { + Scalar::new(self.to_array_of_size(1)) + } + /// Converts an iterator of references [`ScalarValue`] into an [`ArrayRef`] /// corresponding to those values. For example, /// diff --git a/datafusion/common/src/utils.rs b/datafusion/common/src/utils.rs index 54481672f93c..6141723b06e0 100644 --- a/datafusion/common/src/utils.rs +++ b/datafusion/common/src/utils.rs @@ -20,7 +20,7 @@ use crate::{DataFusionError, Result, ScalarValue}; use arrow::array::{ArrayRef, PrimitiveArray}; use arrow::compute; -use arrow::compute::{lexicographical_partition_ranges, SortColumn, SortOptions}; +use arrow::compute::{partition, SortColumn, SortOptions}; use arrow::datatypes::{SchemaRef, UInt32Type}; use arrow::record_batch::RecordBatch; use sqlparser::ast::Ident; @@ -220,7 +220,7 @@ where /// Given a list of 0 or more already sorted columns, finds the /// partition ranges that would partition equally across columns. /// -/// See [`lexicographical_partition_ranges`] for more details. +/// See [`partition`] for more details. pub fn evaluate_partition_ranges( num_rows: usize, partition_columns: &[SortColumn], @@ -231,7 +231,8 @@ pub fn evaluate_partition_ranges( end: num_rows, }] } else { - lexicographical_partition_ranges(partition_columns)?.collect() + let cols: Vec<_> = partition_columns.iter().map(|x| x.values.clone()).collect(); + partition(&cols)?.ranges() }) } diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index bd1ed2e815f8..81ba06d8bc9b 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -79,7 +79,7 @@ lazy_static = { version = "^1.4.0" } log = "^0.4" num-traits = { version = "0.2", optional = true } num_cpus = "1.13.0" -object_store = "0.6.1" +object_store = "0.7.0" parking_lot = "0.12" parquet = { workspace = true } percent-encoding = "2.2.0" diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 07ef28304cce..c6e2c68d0211 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -807,7 +807,7 @@ mod tests { fn get_test_schema_descr(fields: Vec) -> SchemaDescPtr { use parquet::schema::types::{SchemaDescriptor, Type as SchemaType}; - let mut schema_fields = fields + let schema_fields = fields .iter() .map(|field| { let mut builder = @@ -829,7 +829,7 @@ mod tests { }) .collect::>(); let schema = SchemaType::group_type_builder("schema") - .with_fields(&mut schema_fields) + .with_fields(schema_fields) .build() .unwrap(); diff --git a/datafusion/core/src/physical_plan/joins/hash_join.rs b/datafusion/core/src/physical_plan/joins/hash_join.rs index 3522656142f3..32bc3921bba1 100644 --- a/datafusion/core/src/physical_plan/joins/hash_join.rs +++ b/datafusion/core/src/physical_plan/joins/hash_join.rs @@ -54,7 +54,7 @@ use super::{ }; use arrow::buffer::BooleanBuffer; -use arrow::compute::{and, eq_dyn, is_null, or_kleene, take, FilterBuilder}; +use arrow::compute::{and, is_null, or_kleene, take, FilterBuilder}; use arrow::record_batch::RecordBatch; use arrow::{ array::{ @@ -82,6 +82,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::OrderingEquivalenceProperties; use ahash::RandomState; +use arrow::compute::kernels::cmp::eq; use futures::{ready, Stream, StreamExt, TryStreamExt}; type JoinLeftData = (JoinHashMap, RecordBatch, MemoryReservation); @@ -1179,14 +1180,14 @@ fn eq_dyn_null( None, )), _ if null_equals_null => { - let eq: BooleanArray = eq_dyn(left, right)?; + let eq = eq(&left, &right)?; let left_is_null = is_null(left)?; let right_is_null = is_null(right)?; or_kleene(&and(&left_is_null, &right_is_null)?, &eq) } - _ => eq_dyn(left, right), + _ => eq(&left, &right), } } diff --git a/datafusion/execution/Cargo.toml b/datafusion/execution/Cargo.toml index 88c709bd469a..a39adcadcaaa 100644 --- a/datafusion/execution/Cargo.toml +++ b/datafusion/execution/Cargo.toml @@ -40,7 +40,7 @@ datafusion-expr = { path = "../expr", version = "29.0.0" } futures = "0.3" hashbrown = { version = "0.14", features = ["raw"] } log = "^0.4" -object_store = "0.6.1" +object_store = "0.7.0" parking_lot = "0.12" rand = "0.8" tempfile = "3" diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index 3e2a3d418398..02b811d56450 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -mod adapter; mod kernels; mod kernels_arrow; @@ -25,28 +24,12 @@ use std::{any::Any, sync::Arc}; use arrow::array::*; use arrow::compute::cast; use arrow::compute::kernels::boolean::{and_kleene, not, or_kleene}; +use arrow::compute::kernels::cmp::*; use arrow::compute::kernels::comparison::regexp_is_match_utf8; use arrow::compute::kernels::comparison::regexp_is_match_utf8_scalar; -use arrow::compute::kernels::comparison::{ - eq_dyn_binary_scalar, gt_dyn_binary_scalar, gt_eq_dyn_binary_scalar, - lt_dyn_binary_scalar, lt_eq_dyn_binary_scalar, neq_dyn_binary_scalar, -}; -use arrow::compute::kernels::comparison::{ - eq_dyn_bool_scalar, gt_dyn_bool_scalar, gt_eq_dyn_bool_scalar, lt_dyn_bool_scalar, - lt_eq_dyn_bool_scalar, neq_dyn_bool_scalar, -}; -use arrow::compute::kernels::comparison::{ - eq_dyn_scalar, gt_dyn_scalar, gt_eq_dyn_scalar, lt_dyn_scalar, lt_eq_dyn_scalar, - neq_dyn_scalar, -}; -use arrow::compute::kernels::comparison::{ - eq_dyn_utf8_scalar, gt_dyn_utf8_scalar, gt_eq_dyn_utf8_scalar, lt_dyn_utf8_scalar, - lt_eq_dyn_utf8_scalar, neq_dyn_utf8_scalar, -}; use arrow::datatypes::*; use arrow::record_batch::RecordBatch; -use adapter::{eq_dyn, gt_dyn, gt_eq_dyn, lt_dyn, lt_eq_dyn, neq_dyn}; use arrow::compute::kernels::concat_elements::concat_elements_utf8; use kernels::{ @@ -70,7 +53,7 @@ use crate::intervals::cp_solver::{propagate_arithmetic, propagate_comparison}; use crate::intervals::{apply_operator, Interval}; use crate::physical_expr::down_cast_any_ref; use crate::PhysicalExpr; -use arrow_array::{Datum, Scalar}; +use arrow_array::Datum; use datafusion_common::cast::as_boolean_array; use datafusion_common::ScalarValue; @@ -145,17 +128,6 @@ impl std::fmt::Display for BinaryExpr { } } -macro_rules! compute_decimal_op_dyn_scalar { - ($LEFT:expr, $RIGHT:expr, $OP:ident, $OP_TYPE:expr) => {{ - if let ScalarValue::Decimal128(Some(v_i128), _, _) = $RIGHT { - Ok(Arc::new(paste::expr! {[<$OP _dyn_scalar>]}($LEFT, v_i128)?)) - } else { - // when the $RIGHT is a NULL, generate a NULL array of $OP_TYPE type - Ok(Arc::new(new_null_array($OP_TYPE, $LEFT.len()))) - } - }}; -} - macro_rules! compute_decimal_op { ($LEFT:expr, $RIGHT:expr, $OP:ident, $DT:ident) => {{ let ll = $LEFT.as_any().downcast_ref::<$DT>().unwrap(); @@ -262,53 +234,6 @@ macro_rules! compute_utf8_op_scalar { }}; } -/// Invoke a compute kernel on a data array and a scalar value -macro_rules! compute_utf8_op_dyn_scalar { - ($LEFT:expr, $RIGHT:expr, $OP:ident, $OP_TYPE:expr) => {{ - if let Some(string_value) = $RIGHT { - Ok(Arc::new(paste::expr! {[<$OP _dyn_utf8_scalar>]}( - $LEFT, - &string_value, - )?)) - } else { - // when the $RIGHT is a NULL, generate a NULL array of $OP_TYPE - Ok(Arc::new(new_null_array($OP_TYPE, $LEFT.len()))) - } - }}; -} - -/// Invoke a compute kernel on a data array and a scalar value -macro_rules! compute_binary_op_dyn_scalar { - ($LEFT:expr, $RIGHT:expr, $OP:ident, $OP_TYPE:expr) => {{ - if let Some(binary_value) = $RIGHT { - Ok(Arc::new(paste::expr! {[<$OP _dyn_binary_scalar>]}( - $LEFT, - &binary_value, - )?)) - } else { - // when the $RIGHT is a NULL, generate a NULL array of $OP_TYPE - Ok(Arc::new(new_null_array($OP_TYPE, $LEFT.len()))) - } - }}; -} - -/// Invoke a compute kernel on a boolean data array and a scalar value -macro_rules! compute_bool_op_dyn_scalar { - ($LEFT:expr, $RIGHT:expr, $OP:ident, $OP_TYPE:expr) => {{ - // generate the scalar function name, such as lt_dyn_bool_scalar, from the $OP parameter - // (which could have a value of lt) and the suffix _scalar - if let Some(b) = $RIGHT { - Ok(Arc::new(paste::expr! {[<$OP _dyn_bool_scalar>]}( - $LEFT, - b, - )?)) - } else { - // when the $RIGHT is a NULL, generate a NULL array of $OP_TYPE - Ok(Arc::new(new_null_array($OP_TYPE, $LEFT.len()))) - } - }}; -} - /// Invoke a bool compute kernel on array(s) macro_rules! compute_bool_op { // invoke binary operator @@ -333,25 +258,6 @@ macro_rules! compute_bool_op { }}; } -/// Invoke a dyn compute kernel on a data array and a scalar value -/// LEFT is Primitive or Dictionary array of numeric values, RIGHT is scalar value -/// OP_TYPE is the return type of scalar function -macro_rules! compute_op_dyn_scalar { - ($LEFT:expr, $RIGHT:expr, $OP:ident, $OP_TYPE:expr) => {{ - // generate the scalar function name, such as lt_dyn_scalar, from the $OP parameter - // (which could have a value of lt_dyn) and the suffix _scalar - if let Some(value) = $RIGHT { - Ok(Arc::new(paste::expr! {[<$OP _dyn_scalar>]}( - $LEFT, - value, - )?)) - } else { - // when the $RIGHT is a NULL, generate a NULL array of $OP_TYPE - Ok(Arc::new(new_null_array($OP_TYPE, $LEFT.len()))) - } - }}; -} - /// Invoke a compute kernel on array(s) macro_rules! compute_op { // invoke binary operator @@ -581,26 +487,19 @@ impl PhysicalExpr for BinaryExpr { let schema = batch.schema(); let input_schema = schema.as_ref(); - if self.op.is_numerical_operators() { + if self.is_datum_operator() { return match (&left_value, &right_value) { (ColumnarValue::Array(left), ColumnarValue::Array(right)) => { self.evaluate_datum(&left.as_ref(), &right.as_ref()) } (ColumnarValue::Scalar(left), ColumnarValue::Array(right)) => { - let left = left.to_array(); - self.evaluate_datum(&Scalar::new(left.as_ref()), &right.as_ref()) + self.evaluate_datum(&left.to_scalar(), &right.as_ref()) } (ColumnarValue::Array(left), ColumnarValue::Scalar(right)) => { - let right = right.to_array(); - self.evaluate_datum(&left.as_ref(), &Scalar::new(right.as_ref())) + self.evaluate_datum(&left.as_ref(), &right.to_scalar()) } (ColumnarValue::Scalar(left), ColumnarValue::Scalar(right)) => { - let left = left.to_array(); - let right = right.to_array(); - self.evaluate_datum( - &Scalar::new(left.as_ref()), - &Scalar::new(right.as_ref()), - ) + self.evaluate_datum(&left.to_scalar(), &right.to_scalar()) } }; } @@ -615,10 +514,6 @@ impl PhysicalExpr for BinaryExpr { r.and_then(|a| to_result_type_array(&self.op, a, &result_type)) }) } - (ColumnarValue::Scalar(scalar), ColumnarValue::Array(array)) => { - // if right is literal and left is array - reverse operator and parameters - self.evaluate_scalar_array(scalar.clone(), array)? - } (_, _) => None, // default to array implementation }; @@ -704,77 +599,6 @@ impl PartialEq for BinaryExpr { } } -/// unwrap underlying (non dictionary) value, if any, to pass to a scalar kernel -fn unwrap_dict_value(v: ScalarValue) -> ScalarValue { - if let ScalarValue::Dictionary(_key_type, v) = v { - unwrap_dict_value(*v) - } else { - v - } -} - -/// The binary_array_op_dyn_scalar macro includes types that extend -/// beyond the primitive, such as Utf8 strings. -#[macro_export] -macro_rules! binary_array_op_dyn_scalar { - ($LEFT:expr, $RIGHT:expr, $OP:ident, $OP_TYPE:expr) => {{ - // unwrap underlying (non dictionary) value - let right = unwrap_dict_value($RIGHT); - - let result: Result> = match right { - ScalarValue::Boolean(b) => compute_bool_op_dyn_scalar!($LEFT, b, $OP, $OP_TYPE), - ScalarValue::Decimal128(..) => compute_decimal_op_dyn_scalar!($LEFT, right, $OP, $OP_TYPE), - ScalarValue::Utf8(v) => compute_utf8_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::LargeUtf8(v) => compute_utf8_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::Binary(v) => compute_binary_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::LargeBinary(v) => compute_binary_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::FixedSizeBinary(_, v) => compute_binary_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::Int8(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::Int16(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::Int32(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::Int64(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::UInt8(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::UInt16(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::UInt32(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::UInt64(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::Float32(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::Float64(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::Date32(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::Date64(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::Time32Second(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::Time32Millisecond(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::Time64Microsecond(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::Time64Nanosecond(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::TimestampSecond(v, _) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::TimestampMillisecond(v, _) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::TimestampMicrosecond(v, _) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::TimestampNanosecond(v, _) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::IntervalYearMonth(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::IntervalDayTime(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - ScalarValue::IntervalMonthDayNano(v) => compute_op_dyn_scalar!($LEFT, v, $OP, $OP_TYPE), - other => Err(DataFusionError::Internal(format!( - "Data type {:?} not supported for scalar operation '{}' on dyn array", - other, stringify!($OP))) - ) - }; - Some(result) - }} -} - -/// Compares the array with the scalar value for equality, sometimes -/// used in other kernels -pub(crate) fn array_eq_scalar(lhs: &dyn Array, rhs: &ScalarValue) -> Result { - binary_array_op_dyn_scalar!(lhs, rhs.clone(), eq, &DataType::Boolean).ok_or_else( - || { - DataFusionError::Internal(format!( - "Data type {:?} and scalar {:?} not supported for array_eq_scalar", - lhs.data_type(), - rhs.get_datatype() - )) - }, - )? -} - /// Casts dictionary array to result type for binary numerical operators. Such operators /// between array and scalar produce a dictionary array other than primitive array of the /// same operators between array and array. This leads to inconsistent result types causing @@ -806,6 +630,12 @@ fn to_result_type_array( } impl BinaryExpr { + fn is_datum_operator(&self) -> bool { + use Operator::*; + self.op.is_comparison_operator() + || matches!(self.op, Lt | LtEq | Gt | GtEq | Eq | NotEq) + } + /// Evaluate the expression using [`Datum`] fn evaluate_datum( &self, @@ -819,6 +649,12 @@ impl BinaryExpr { Operator::Multiply => mul_wrapping(left, right)?, Operator::Divide => div(left, right)?, Operator::Modulo => rem(left, right)?, + Operator::Eq => Arc::new(eq(left, right)?), + Operator::NotEq => Arc::new(neq(left, right)?), + Operator::Lt => Arc::new(lt(left, right)?), + Operator::Gt => Arc::new(gt(left, right)?), + Operator::LtEq => Arc::new(lt_eq(left, right)?), + Operator::GtEq => Arc::new(gt_eq(left, right)?), _ => unreachable!(), }; @@ -837,17 +673,7 @@ impl BinaryExpr { scalar: ScalarValue, ) -> Result>> { use Operator::*; - let bool_type = &DataType::Boolean; let scalar_result = match &self.op { - Lt => binary_array_op_dyn_scalar!(array, scalar, lt, bool_type), - LtEq => binary_array_op_dyn_scalar!(array, scalar, lt_eq, bool_type), - Gt => binary_array_op_dyn_scalar!(array, scalar, gt, bool_type), - GtEq => binary_array_op_dyn_scalar!(array, scalar, gt_eq, bool_type), - Eq => binary_array_op_dyn_scalar!(array, scalar, eq, bool_type), - NotEq => binary_array_op_dyn_scalar!(array, scalar, neq, bool_type), - Plus | Minus | Multiply | Divide | Modulo => { - unreachable!() - } RegexMatch => binary_string_array_flag_op_scalar!( array, scalar, @@ -888,28 +714,6 @@ impl BinaryExpr { Ok(scalar_result) } - /// Evaluate the expression if the left input is a literal and the - /// right is an array - reverse operator and parameters - fn evaluate_scalar_array( - &self, - scalar: ScalarValue, - array: &ArrayRef, - ) -> Result>> { - use Operator::*; - let bool_type = &DataType::Boolean; - let scalar_result = match &self.op { - Lt => binary_array_op_dyn_scalar!(array, scalar, gt, bool_type), - LtEq => binary_array_op_dyn_scalar!(array, scalar, gt_eq, bool_type), - Gt => binary_array_op_dyn_scalar!(array, scalar, lt, bool_type), - GtEq => binary_array_op_dyn_scalar!(array, scalar, lt_eq, bool_type), - Eq => binary_array_op_dyn_scalar!(array, scalar, eq, bool_type), - NotEq => binary_array_op_dyn_scalar!(array, scalar, neq, bool_type), - // if scalar operation is not supported - fallback to array implementation - _ => None, - }; - Ok(scalar_result) - } - fn evaluate_with_resolved_args( &self, left: Arc, @@ -919,12 +723,6 @@ impl BinaryExpr { ) -> Result { use Operator::*; match &self.op { - Lt => lt_dyn(&left, &right), - LtEq => lt_eq_dyn(&left, &right), - Gt => gt_dyn(&left, &right), - GtEq => gt_eq_dyn(&left, &right), - Eq => eq_dyn(&left, &right), - NotEq => neq_dyn(&left, &right), IsDistinctFrom => { match (left_data_type, right_data_type) { // exchange lhs and rhs when lhs is Null, since `binary_array_op` is @@ -936,7 +734,8 @@ impl BinaryExpr { } } IsNotDistinctFrom => binary_array_op!(left, right, is_not_distinct_from), - Plus | Minus | Multiply | Divide | Modulo => unreachable!(), + Lt | LtEq | Gt | GtEq | Eq | NotEq | Plus | Minus | Multiply | Divide + | Modulo => unreachable!(), And => { if left_data_type == &DataType::Boolean { boolean_op!(&left, &right, and_kleene) diff --git a/datafusion/physical-expr/src/expressions/binary/adapter.rs b/datafusion/physical-expr/src/expressions/binary/adapter.rs deleted file mode 100644 index ec0eda392976..000000000000 --- a/datafusion/physical-expr/src/expressions/binary/adapter.rs +++ /dev/null @@ -1,49 +0,0 @@ -// 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. - -//! This module contains functions that change types or names of other -//! kernels to make them compatible with the main dispatch logic - -use std::sync::Arc; - -use arrow::array::*; -use datafusion_common::Result; - -/// create a `dyn_op` wrapper function for the specified operation -/// that call the underlying dyn_op arrow kernel if the type is -/// supported, and translates ArrowError to DataFusionError -macro_rules! make_dyn_comp_op { - ($OP:tt) => { - paste::paste! { - /// wrapper over arrow compute kernel that maps Error types and - /// patches missing support in arrow - pub(crate) fn [<$OP _dyn>] (left: &dyn Array, right: &dyn Array) -> Result { - arrow::compute::kernels::comparison::[<$OP _dyn>](left, right) - .map_err(|e| e.into()) - .map(|a| Arc::new(a) as ArrayRef) - } - } - }; -} - -// create eq_dyn, gt_dyn, wrappers etc -make_dyn_comp_op!(eq); -make_dyn_comp_op!(gt); -make_dyn_comp_op!(gt_eq); -make_dyn_comp_op!(lt); -make_dyn_comp_op!(lt_eq); -make_dyn_comp_op!(neq); diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs index 506c01b6f371..ff7750119ddb 100644 --- a/datafusion/physical-expr/src/expressions/case.rs +++ b/datafusion/physical-expr/src/expressions/case.rs @@ -24,8 +24,9 @@ use crate::expressions::NoOp; use crate::physical_expr::down_cast_any_ref; use crate::PhysicalExpr; use arrow::array::*; +use arrow::compute::kernels::cmp::eq; use arrow::compute::kernels::zip::zip; -use arrow::compute::{and, eq_dyn, is_null, not, or, prep_null_mask_filter}; +use arrow::compute::{and, is_null, not, or, prep_null_mask_filter}; use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; use datafusion_common::{cast::as_boolean_array, DataFusionError, Result}; @@ -139,7 +140,7 @@ impl CaseExpr { .evaluate_selection(batch, &remainder)?; let when_value = when_value.into_array(batch.num_rows()); // build boolean array representing which rows match the "when" value - let when_match = eq_dyn(&when_value, base_value.as_ref())?; + let when_match = eq(&when_value, &base_value)?; // Treat nulls as false let when_match = match when_match.null_count() { 0 => Cow::Borrowed(&when_match), diff --git a/datafusion/physical-expr/src/expressions/nullif.rs b/datafusion/physical-expr/src/expressions/nullif.rs index a2a61d16af41..7cfc8dc8ab97 100644 --- a/datafusion/physical-expr/src/expressions/nullif.rs +++ b/datafusion/physical-expr/src/expressions/nullif.rs @@ -16,13 +16,11 @@ // under the License. use arrow::array::Array; -use arrow::compute::eq_dyn; +use arrow::compute::kernels::cmp::eq; use arrow::compute::kernels::nullif::nullif; -use datafusion_common::{cast::as_boolean_array, DataFusionError, Result, ScalarValue}; +use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::ColumnarValue; -use super::binary::array_eq_scalar; - /// Implements NULLIF(expr1, expr2) /// Args: 0 - left expr is any array /// 1 - if the left is equal to this expr2, then the result is NULL, otherwise left value is passed. @@ -39,26 +37,18 @@ pub fn nullif_func(args: &[ColumnarValue]) -> Result { match (lhs, rhs) { (ColumnarValue::Array(lhs), ColumnarValue::Scalar(rhs)) => { - let cond_array = array_eq_scalar(lhs, rhs)?; - - let array = nullif(lhs, as_boolean_array(&cond_array)?)?; + let rhs = rhs.to_scalar(); + let array = nullif(lhs, &eq(&lhs, &rhs)?)?; Ok(ColumnarValue::Array(array)) } (ColumnarValue::Array(lhs), ColumnarValue::Array(rhs)) => { - // Get args0 == args1 evaluated and produce a boolean array - let cond_array = eq_dyn(lhs, rhs)?; - - // Now, invoke nullif on the result - let array = nullif(lhs, as_boolean_array(&cond_array)?)?; + let array = nullif(lhs, &eq(&lhs, &rhs)?)?; Ok(ColumnarValue::Array(array)) } (ColumnarValue::Scalar(lhs), ColumnarValue::Array(rhs)) => { - // Similar to Array-Array case, except of ScalarValue -> Array cast let lhs = lhs.to_array_of_size(rhs.len()); - let cond_array = eq_dyn(&lhs, rhs)?; - - let array = nullif(&lhs, as_boolean_array(&cond_array)?)?; + let array = nullif(&lhs, &eq(&lhs, &rhs)?)?; Ok(ColumnarValue::Array(array)) } (ColumnarValue::Scalar(lhs), ColumnarValue::Scalar(rhs)) => { diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 03ba48f1b1db..ee2770d2c4ec 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -45,7 +45,7 @@ chrono = { version = "0.4", default-features = false } datafusion = { path = "../core", version = "29.0.0" } datafusion-common = { path = "../common", version = "29.0.0" } datafusion-expr = { path = "../expr", version = "29.0.0" } -object_store = { version = "0.6.1" } +object_store = { version = "0.7.0" } pbjson = { version = "0.5", optional = true } prost = "0.11.0" serde = { version = "1.0", optional = true } diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index 0d80ed6c2246..9369cf63d9bf 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -39,7 +39,7 @@ datafusion-common = {path = "../common", version = "29.0.0"} half = "2.2.1" itertools = "0.11" lazy_static = {version = "^1.4.0"} -object_store = "0.6.1" +object_store = "0.7.0" rust_decimal = {version = "1.27.0"} log = "^0.4" sqllogictest = "0.15.0" diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 3e565680e21d..aa7dc0da3b76 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -32,7 +32,7 @@ async-recursion = "1.0" chrono = { version = "0.4.23", default-features = false } datafusion = { version = "29.0.0", path = "../core" } itertools = "0.11" -object_store = "0.6.1" +object_store = "0.7.0" prost = "0.11" prost-types = "0.11" substrait = "0.12.0" From bb896c4fa68e4fc2c5c267a56961568584fbd10f Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Fri, 18 Aug 2023 19:12:40 +0100 Subject: [PATCH 03/10] Use is_distinct kernels --- Cargo.toml | 12 +- datafusion-cli/Cargo.lock | 30 +- datafusion-cli/Cargo.toml | 10 +- .../physical-expr/src/expressions/binary.rs | 217 +--------- .../src/expressions/binary/kernels_arrow.rs | 404 ------------------ 5 files changed, 35 insertions(+), 638 deletions(-) delete mode 100644 datafusion/physical-expr/src/expressions/binary/kernels_arrow.rs diff --git a/Cargo.toml b/Cargo.toml index 2f467a26b3f5..d92376b3f67b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -59,9 +59,9 @@ panic = 'unwind' rpath = false [patch.crates-io] -arrow = {git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } -arrow-array = {git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } -arrow-buffer = {git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } -arrow-flight = {git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } -arrow-schema = {git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } -parquet = {git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } +arrow = {git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } +arrow-array = {git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } +arrow-buffer = {git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } +arrow-flight = {git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } +arrow-schema = {git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } +parquet = {git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index f25b817bebaf..50044a328772 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -96,7 +96,7 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" dependencies = [ "ahash", "arrow-arith", @@ -117,7 +117,7 @@ dependencies = [ [[package]] name = "arrow-arith" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" dependencies = [ "arrow-array", "arrow-buffer", @@ -131,7 +131,7 @@ dependencies = [ [[package]] name = "arrow-array" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" dependencies = [ "ahash", "arrow-buffer", @@ -147,7 +147,7 @@ dependencies = [ [[package]] name = "arrow-buffer" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" dependencies = [ "bytes", "half", @@ -157,7 +157,7 @@ dependencies = [ [[package]] name = "arrow-cast" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" dependencies = [ "arrow-array", "arrow-buffer", @@ -174,7 +174,7 @@ dependencies = [ [[package]] name = "arrow-csv" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" dependencies = [ "arrow-array", "arrow-buffer", @@ -192,7 +192,7 @@ dependencies = [ [[package]] name = "arrow-data" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" dependencies = [ "arrow-buffer", "arrow-schema", @@ -203,7 +203,7 @@ dependencies = [ [[package]] name = "arrow-ipc" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" dependencies = [ "arrow-array", "arrow-buffer", @@ -216,7 +216,7 @@ dependencies = [ [[package]] name = "arrow-json" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" dependencies = [ "arrow-array", "arrow-buffer", @@ -235,7 +235,7 @@ dependencies = [ [[package]] name = "arrow-ord" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" dependencies = [ "arrow-array", "arrow-buffer", @@ -249,7 +249,7 @@ dependencies = [ [[package]] name = "arrow-row" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" dependencies = [ "ahash", "arrow-array", @@ -263,12 +263,12 @@ dependencies = [ [[package]] name = "arrow-schema" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" [[package]] name = "arrow-select" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" dependencies = [ "arrow-array", "arrow-buffer", @@ -280,7 +280,7 @@ dependencies = [ [[package]] name = "arrow-string" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" dependencies = [ "arrow-array", "arrow-buffer", @@ -2252,7 +2252,7 @@ dependencies = [ [[package]] name = "parquet" version = "45.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=2c487d0eba33569086887d434d971129a77db4eb#2c487d0eba33569086887d434d971129a77db4eb" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" dependencies = [ "ahash", "arrow-array", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 7be2d3a86735..f7db4d876e63 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -51,8 +51,8 @@ predicates = "3.0" rstest = "0.17" [patch.crates-io] -arrow = { git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } -arrow-array = { git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } -arrow-buffer = { git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } -arrow-schema = { git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } -parquet = { git = "https://github.com/apache/arrow-rs.git", rev = "2c487d0eba33569086887d434d971129a77db4eb" } +arrow = { git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } +arrow-array = { git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } +arrow-buffer = { git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } +arrow-schema = { git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } +parquet = { git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index 02b811d56450..0a829641d8e6 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -16,7 +16,6 @@ // under the License. mod kernels; -mod kernels_arrow; use std::hash::{Hash, Hasher}; use std::{any::Any, sync::Arc}; @@ -37,14 +36,6 @@ use kernels::{ bitwise_shift_left_dyn, bitwise_shift_left_dyn_scalar, bitwise_shift_right_dyn, bitwise_shift_right_dyn_scalar, bitwise_xor_dyn, bitwise_xor_dyn_scalar, }; -use kernels_arrow::{ - is_distinct_from, is_distinct_from_binary, is_distinct_from_bool, - is_distinct_from_decimal, is_distinct_from_f32, is_distinct_from_f64, - is_distinct_from_null, is_distinct_from_utf8, is_not_distinct_from, - is_not_distinct_from_binary, is_not_distinct_from_bool, is_not_distinct_from_decimal, - is_not_distinct_from_f32, is_not_distinct_from_f64, is_not_distinct_from_null, - is_not_distinct_from_utf8, -}; use crate::array_expressions::{ array_append, array_concat, array_has_all, array_prepend, @@ -128,56 +119,6 @@ impl std::fmt::Display for BinaryExpr { } } -macro_rules! compute_decimal_op { - ($LEFT:expr, $RIGHT:expr, $OP:ident, $DT:ident) => {{ - let ll = $LEFT.as_any().downcast_ref::<$DT>().unwrap(); - let rr = $RIGHT.as_any().downcast_ref::<$DT>().unwrap(); - Ok(Arc::new(paste::expr! {[<$OP _decimal>]}(ll, rr)?)) - }}; -} - -macro_rules! compute_f32_op { - ($LEFT:expr, $RIGHT:expr, $OP:ident, $DT:ident) => {{ - let ll = $LEFT - .as_any() - .downcast_ref::<$DT>() - .expect("compute_op failed to downcast left side array"); - let rr = $RIGHT - .as_any() - .downcast_ref::<$DT>() - .expect("compute_op failed to downcast right side array"); - Ok(Arc::new(paste::expr! {[<$OP _f32>]}(ll, rr)?)) - }}; -} - -macro_rules! compute_f64_op { - ($LEFT:expr, $RIGHT:expr, $OP:ident, $DT:ident) => {{ - let ll = $LEFT - .as_any() - .downcast_ref::<$DT>() - .expect("compute_op failed to downcast left side array"); - let rr = $RIGHT - .as_any() - .downcast_ref::<$DT>() - .expect("compute_op failed to downcast right side array"); - Ok(Arc::new(paste::expr! {[<$OP _f64>]}(ll, rr)?)) - }}; -} - -macro_rules! compute_null_op { - ($LEFT:expr, $RIGHT:expr, $OP:ident, $DT:ident) => {{ - let ll = $LEFT - .as_any() - .downcast_ref::<$DT>() - .expect("compute_op failed to downcast left side array"); - let rr = $RIGHT - .as_any() - .downcast_ref::<$DT>() - .expect("compute_op failed to downcast right side array"); - Ok(Arc::new(paste::expr! {[<$OP _null>]}(&ll, &rr)?)) - }}; -} - /// Invoke a compute kernel on a pair of binary data arrays macro_rules! compute_utf8_op { ($LEFT:expr, $RIGHT:expr, $OP:ident, $DT:ident) => {{ @@ -193,21 +134,6 @@ macro_rules! compute_utf8_op { }}; } -/// Invoke a compute kernel on a pair of binary data arrays -macro_rules! compute_binary_op { - ($LEFT:expr, $RIGHT:expr, $OP:ident, $DT:ident) => {{ - let ll = $LEFT - .as_any() - .downcast_ref::<$DT>() - .expect("compute_op failed to downcast left side array"); - let rr = $RIGHT - .as_any() - .downcast_ref::<$DT>() - .expect("compute_op failed to downcast right side array"); - Ok(Arc::new(paste::expr! {[<$OP _binary>]}(&ll, &rr)?)) - }}; -} - /// Invoke a compute kernel on a data array and a scalar value macro_rules! compute_utf8_op_scalar { ($LEFT:expr, $RIGHT:expr, $OP:ident, $DT:ident, $OP_TYPE:expr) => {{ @@ -234,54 +160,6 @@ macro_rules! compute_utf8_op_scalar { }}; } -/// Invoke a bool compute kernel on array(s) -macro_rules! compute_bool_op { - // invoke binary operator - ($LEFT:expr, $RIGHT:expr, $OP:ident, $DT:ident) => {{ - let ll = $LEFT - .as_any() - .downcast_ref::<$DT>() - .expect("compute_op failed to downcast left side array"); - let rr = $RIGHT - .as_any() - .downcast_ref::<$DT>() - .expect("compute_op failed to downcast right side array"); - Ok(Arc::new(paste::expr! {[<$OP _bool>]}(&ll, &rr)?)) - }}; - // invoke unary operator - ($OPERAND:expr, $OP:ident, $DT:ident) => {{ - let operand = $OPERAND - .as_any() - .downcast_ref::<$DT>() - .expect("compute_op failed to downcast operant array"); - Ok(Arc::new(paste::expr! {[<$OP _bool>]}(&operand)?)) - }}; -} - -/// Invoke a compute kernel on array(s) -macro_rules! compute_op { - // invoke binary operator - ($LEFT:expr, $RIGHT:expr, $OP:ident, $DT:ident) => {{ - let ll = $LEFT - .as_any() - .downcast_ref::<$DT>() - .expect("compute_op failed to downcast left side array"); - let rr = $RIGHT - .as_any() - .downcast_ref::<$DT>() - .expect("compute_op failed to downcast right side array"); - Ok(Arc::new($OP(&ll, &rr)?)) - }}; - // invoke unary operator - ($OPERAND:expr, $OP:ident, $DT:ident) => {{ - let operand = $OPERAND - .as_any() - .downcast_ref::<$DT>() - .expect("compute_op failed to downcast array"); - Ok(Arc::new($OP(&operand)?)) - }}; -} - macro_rules! binary_string_array_op { ($LEFT:expr, $RIGHT:expr, $OP:ident) => {{ match $LEFT.data_type() { @@ -295,77 +173,6 @@ macro_rules! binary_string_array_op { }}; } -/// The binary_array_op macro includes types that extend beyond the primitive, -/// such as Utf8 strings. -#[macro_export] -macro_rules! binary_array_op { - ($LEFT:expr, $RIGHT:expr, $OP:ident) => {{ - match $LEFT.data_type() { - DataType::Null => compute_null_op!($LEFT, $RIGHT, $OP, NullArray), - DataType::Decimal128(_,_) => compute_decimal_op!($LEFT, $RIGHT, $OP, Decimal128Array), - DataType::Int8 => compute_op!($LEFT, $RIGHT, $OP, Int8Array), - DataType::Int16 => compute_op!($LEFT, $RIGHT, $OP, Int16Array), - DataType::Int32 => compute_op!($LEFT, $RIGHT, $OP, Int32Array), - DataType::Int64 => compute_op!($LEFT, $RIGHT, $OP, Int64Array), - DataType::UInt8 => compute_op!($LEFT, $RIGHT, $OP, UInt8Array), - DataType::UInt16 => compute_op!($LEFT, $RIGHT, $OP, UInt16Array), - DataType::UInt32 => compute_op!($LEFT, $RIGHT, $OP, UInt32Array), - DataType::UInt64 => compute_op!($LEFT, $RIGHT, $OP, UInt64Array), - DataType::Float32 => compute_f32_op!($LEFT, $RIGHT, $OP, Float32Array), - DataType::Float64 => compute_f64_op!($LEFT, $RIGHT, $OP, Float64Array), - DataType::Utf8 => compute_utf8_op!($LEFT, $RIGHT, $OP, StringArray), - DataType::Binary => compute_binary_op!($LEFT, $RIGHT, $OP, BinaryArray), - DataType::LargeBinary => compute_binary_op!($LEFT, $RIGHT, $OP, LargeBinaryArray), - DataType::LargeUtf8 => compute_utf8_op!($LEFT, $RIGHT, $OP, LargeStringArray), - - DataType::Timestamp(TimeUnit::Nanosecond, _) => { - compute_op!($LEFT, $RIGHT, $OP, TimestampNanosecondArray) - } - DataType::Timestamp(TimeUnit::Microsecond, _) => { - compute_op!($LEFT, $RIGHT, $OP, TimestampMicrosecondArray) - } - DataType::Timestamp(TimeUnit::Millisecond, _) => { - compute_op!($LEFT, $RIGHT, $OP, TimestampMillisecondArray) - } - DataType::Timestamp(TimeUnit::Second, _) => { - compute_op!($LEFT, $RIGHT, $OP, TimestampSecondArray) - } - DataType::Date32 => { - compute_op!($LEFT, $RIGHT, $OP, Date32Array) - } - DataType::Date64 => { - compute_op!($LEFT, $RIGHT, $OP, Date64Array) - } - DataType::Time32(TimeUnit::Second) => { - compute_op!($LEFT, $RIGHT, $OP, Time32SecondArray) - } - DataType::Time32(TimeUnit::Millisecond) => { - compute_op!($LEFT, $RIGHT, $OP, Time32MillisecondArray) - } - DataType::Time64(TimeUnit::Microsecond) => { - compute_op!($LEFT, $RIGHT, $OP, Time64MicrosecondArray) - } - DataType::Time64(TimeUnit::Nanosecond) => { - compute_op!($LEFT, $RIGHT, $OP, Time64NanosecondArray) - } - DataType::Interval(IntervalUnit::YearMonth) => { - compute_op!($LEFT, $RIGHT, $OP, IntervalYearMonthArray) - } - DataType::Interval(IntervalUnit::DayTime) => { - compute_op!($LEFT, $RIGHT, $OP, IntervalDayTimeArray) - } - DataType::Interval(IntervalUnit::MonthDayNano) => { - compute_op!($LEFT, $RIGHT, $OP, IntervalMonthDayNanoArray) - } - DataType::Boolean => compute_bool_op!($LEFT, $RIGHT, $OP, BooleanArray), - other => Err(DataFusionError::Internal(format!( - "Data type {:?} not supported for binary operation '{}' on dyn arrays", - other, stringify!($OP) - ))), - } - }}; -} - /// Invoke a boolean kernel on a pair of arrays macro_rules! boolean_op { ($LEFT:expr, $RIGHT:expr, $OP:ident) => {{ @@ -632,8 +439,11 @@ fn to_result_type_array( impl BinaryExpr { fn is_datum_operator(&self) -> bool { use Operator::*; - self.op.is_comparison_operator() - || matches!(self.op, Lt | LtEq | Gt | GtEq | Eq | NotEq) + self.op.is_numerical_operators() + || matches!( + self.op, + Lt | LtEq | Gt | GtEq | Eq | NotEq | IsDistinctFrom | IsNotDistinctFrom + ) } /// Evaluate the expression using [`Datum`] @@ -655,6 +465,8 @@ impl BinaryExpr { Operator::Gt => Arc::new(gt(left, right)?), Operator::LtEq => Arc::new(lt_eq(left, right)?), Operator::GtEq => Arc::new(gt_eq(left, right)?), + Operator::IsDistinctFrom => Arc::new(distinct(left, right)?), + Operator::IsNotDistinctFrom => Arc::new(not_distinct(left, right)?), _ => unreachable!(), }; @@ -723,19 +535,8 @@ impl BinaryExpr { ) -> Result { use Operator::*; match &self.op { - IsDistinctFrom => { - match (left_data_type, right_data_type) { - // exchange lhs and rhs when lhs is Null, since `binary_array_op` is - // always try to down cast array according to $LEFT expression. - (DataType::Null, _) => { - binary_array_op!(right, left, is_distinct_from) - } - _ => binary_array_op!(left, right, is_distinct_from), - } - } - IsNotDistinctFrom => binary_array_op!(left, right, is_not_distinct_from), - Lt | LtEq | Gt | GtEq | Eq | NotEq | Plus | Minus | Multiply | Divide - | Modulo => unreachable!(), + IsDistinctFrom | IsNotDistinctFrom | Lt | LtEq | Gt | GtEq | Eq | NotEq + | Plus | Minus | Multiply | Divide | Modulo => unreachable!(), And => { if left_data_type == &DataType::Boolean { boolean_op!(&left, &right, and_kleene) diff --git a/datafusion/physical-expr/src/expressions/binary/kernels_arrow.rs b/datafusion/physical-expr/src/expressions/binary/kernels_arrow.rs deleted file mode 100644 index 26ff7c369e5d..000000000000 --- a/datafusion/physical-expr/src/expressions/binary/kernels_arrow.rs +++ /dev/null @@ -1,404 +0,0 @@ -// 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. - -//! This module contains computation kernels that are eventually -//! destined for arrow-rs but are in datafusion until they are ported. - -use arrow::{array::*, datatypes::ArrowNumericType}; -use datafusion_common::Result; - -// Simple (low performance) kernels until optimized kernels are added to arrow -// See https://github.com/apache/arrow-rs/issues/960 - -macro_rules! distinct_float { - ($LEFT:expr, $RIGHT:expr, $LEFT_ISNULL:expr, $RIGHT_ISNULL:expr) => {{ - $LEFT_ISNULL != $RIGHT_ISNULL - || $LEFT.is_nan() != $RIGHT.is_nan() - || (!$LEFT.is_nan() && !$RIGHT.is_nan() && $LEFT != $RIGHT) - }}; -} - -pub(crate) fn is_distinct_from_bool( - left: &BooleanArray, - right: &BooleanArray, -) -> Result { - // Different from `neq_bool` because `null is distinct from null` is false and not null - Ok(left - .iter() - .zip(right.iter()) - .map(|(left, right)| Some(left != right)) - .collect()) -} - -pub(crate) fn is_not_distinct_from_bool( - left: &BooleanArray, - right: &BooleanArray, -) -> Result { - Ok(left - .iter() - .zip(right.iter()) - .map(|(left, right)| Some(left == right)) - .collect()) -} - -pub(crate) fn is_distinct_from( - left: &PrimitiveArray, - right: &PrimitiveArray, -) -> Result -where - T: ArrowPrimitiveType, -{ - distinct( - left, - right, - |left_value, right_value, left_isnull, right_isnull| { - left_isnull != right_isnull || left_value != right_value - }, - ) -} - -pub(crate) fn is_not_distinct_from( - left: &PrimitiveArray, - right: &PrimitiveArray, -) -> Result -where - T: ArrowNumericType, -{ - distinct( - left, - right, - |left_value, right_value, left_isnull, right_isnull| { - !(left_isnull != right_isnull || left_value != right_value) - }, - ) -} - -fn distinct< - T, - F: FnMut( - ::Native, - ::Native, - bool, - bool, - ) -> bool, ->( - left: &PrimitiveArray, - right: &PrimitiveArray, - mut op: F, -) -> Result -where - T: ArrowPrimitiveType, -{ - let left_values = left.values(); - let right_values = right.values(); - let left_nulls = left.nulls(); - let right_nulls = right.nulls(); - - let array_len = left.len().min(right.len()); - let distinct = arrow_buffer::MutableBuffer::collect_bool(array_len, |i| { - op( - left_values[i], - right_values[i], - left_nulls.map(|x| x.is_null(i)).unwrap_or_default(), - right_nulls.map(|x| x.is_null(i)).unwrap_or_default(), - ) - }); - let array_data = ArrayData::builder(arrow_schema::DataType::Boolean) - .len(array_len) - .add_buffer(distinct.into()); - - Ok(BooleanArray::from(unsafe { array_data.build_unchecked() })) -} - -pub(crate) fn is_distinct_from_f32( - left: &Float32Array, - right: &Float32Array, -) -> Result { - distinct( - left, - right, - |left_value, right_value, left_isnull, right_isnull| { - distinct_float!(left_value, right_value, left_isnull, right_isnull) - }, - ) -} - -pub(crate) fn is_not_distinct_from_f32( - left: &Float32Array, - right: &Float32Array, -) -> Result { - distinct( - left, - right, - |left_value, right_value, left_isnull, right_isnull| { - !(distinct_float!(left_value, right_value, left_isnull, right_isnull)) - }, - ) -} - -pub(crate) fn is_distinct_from_f64( - left: &Float64Array, - right: &Float64Array, -) -> Result { - distinct( - left, - right, - |left_value, right_value, left_isnull, right_isnull| { - distinct_float!(left_value, right_value, left_isnull, right_isnull) - }, - ) -} - -pub(crate) fn is_not_distinct_from_f64( - left: &Float64Array, - right: &Float64Array, -) -> Result { - distinct( - left, - right, - |left_value, right_value, left_isnull, right_isnull| { - !(distinct_float!(left_value, right_value, left_isnull, right_isnull)) - }, - ) -} - -pub(crate) fn is_distinct_from_utf8( - left: &GenericStringArray, - right: &GenericStringArray, -) -> Result { - Ok(left - .iter() - .zip(right.iter()) - .map(|(x, y)| Some(x != y)) - .collect()) -} - -pub(crate) fn is_distinct_from_binary( - left: &GenericBinaryArray, - right: &GenericBinaryArray, -) -> Result { - Ok(left - .iter() - .zip(right.iter()) - .map(|(x, y)| Some(x != y)) - .collect()) -} - -pub(crate) fn is_distinct_from_null( - left: &NullArray, - _right: &NullArray, -) -> Result { - let length = left.len(); - make_boolean_array(length, false) -} - -pub(crate) fn is_not_distinct_from_null( - left: &NullArray, - _right: &NullArray, -) -> Result { - let length = left.len(); - make_boolean_array(length, true) -} - -fn make_boolean_array(length: usize, value: bool) -> Result { - Ok((0..length).map(|_| Some(value)).collect()) -} - -pub(crate) fn is_not_distinct_from_utf8( - left: &GenericStringArray, - right: &GenericStringArray, -) -> Result { - Ok(left - .iter() - .zip(right.iter()) - .map(|(x, y)| Some(x == y)) - .collect()) -} - -pub(crate) fn is_not_distinct_from_binary( - left: &GenericBinaryArray, - right: &GenericBinaryArray, -) -> Result { - Ok(left - .iter() - .zip(right.iter()) - .map(|(x, y)| Some(x == y)) - .collect()) -} - -pub(crate) fn is_distinct_from_decimal( - left: &Decimal128Array, - right: &Decimal128Array, -) -> Result { - Ok(left - .iter() - .zip(right.iter()) - .map(|(left, right)| match (left, right) { - (None, None) => Some(false), - (None, Some(_)) | (Some(_), None) => Some(true), - (Some(left), Some(right)) => Some(left != right), - }) - .collect()) -} - -pub(crate) fn is_not_distinct_from_decimal( - left: &Decimal128Array, - right: &Decimal128Array, -) -> Result { - Ok(left - .iter() - .zip(right.iter()) - .map(|(left, right)| match (left, right) { - (None, None) => Some(true), - (None, Some(_)) | (Some(_), None) => Some(false), - (Some(left), Some(right)) => Some(left == right), - }) - .collect()) -} - -#[cfg(test)] -mod tests { - use super::*; - - fn create_decimal_array( - array: &[Option], - precision: u8, - scale: i8, - ) -> Decimal128Array { - let mut decimal_builder = Decimal128Builder::with_capacity(array.len()); - - for value in array.iter().copied() { - decimal_builder.append_option(value) - } - decimal_builder - .finish() - .with_precision_and_scale(precision, scale) - .unwrap() - } - - fn create_int_array(array: &[Option]) -> Int32Array { - let mut int_builder = Int32Builder::with_capacity(array.len()); - - for value in array.iter().copied() { - int_builder.append_option(value) - } - int_builder.finish() - } - - #[test] - fn comparison_decimal_op_test() -> Result<()> { - let value_i128: i128 = 123; - let decimal_array = create_decimal_array( - &[ - Some(value_i128), - None, - Some(value_i128 - 1), - Some(value_i128 + 1), - ], - 25, - 3, - ); - let left_decimal_array = decimal_array; - let right_decimal_array = create_decimal_array( - &[ - Some(value_i128 - 1), - Some(value_i128), - Some(value_i128 + 1), - Some(value_i128 + 1), - ], - 25, - 3, - ); - - // is_distinct: left distinct right - let result = is_distinct_from(&left_decimal_array, &right_decimal_array)?; - assert_eq!( - BooleanArray::from(vec![Some(true), Some(true), Some(true), Some(false)]), - result - ); - // is_distinct: left distinct right - let result = is_not_distinct_from(&left_decimal_array, &right_decimal_array)?; - assert_eq!( - BooleanArray::from(vec![Some(false), Some(false), Some(false), Some(true)]), - result - ); - Ok(()) - } - - #[test] - fn is_distinct_from_non_nulls() -> Result<()> { - let left_int_array = - create_int_array(&[Some(0), Some(1), Some(2), Some(3), Some(4)]); - let right_int_array = - create_int_array(&[Some(4), Some(3), Some(2), Some(1), Some(0)]); - - assert_eq!( - BooleanArray::from(vec![ - Some(true), - Some(true), - Some(false), - Some(true), - Some(true), - ]), - is_distinct_from(&left_int_array, &right_int_array)? - ); - assert_eq!( - BooleanArray::from(vec![ - Some(false), - Some(false), - Some(true), - Some(false), - Some(false), - ]), - is_not_distinct_from(&left_int_array, &right_int_array)? - ); - Ok(()) - } - - #[test] - fn is_distinct_from_nulls() -> Result<()> { - let left_int_array = - create_int_array(&[Some(0), Some(0), None, Some(3), Some(0), Some(0)]); - let right_int_array = - create_int_array(&[Some(0), None, None, None, Some(0), None]); - - assert_eq!( - BooleanArray::from(vec![ - Some(false), - Some(true), - Some(false), - Some(true), - Some(false), - Some(true), - ]), - is_distinct_from(&left_int_array, &right_int_array)? - ); - - assert_eq!( - BooleanArray::from(vec![ - Some(true), - Some(false), - Some(true), - Some(false), - Some(true), - Some(false), - ]), - is_not_distinct_from(&left_int_array, &right_int_array)? - ); - Ok(()) - } -} From 677839cfd05a85a39722e9d51531a0a622b84b84 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Fri, 18 Aug 2023 19:20:27 +0100 Subject: [PATCH 04/10] Further fixes --- .../core/src/physical_plan/joins/hash_join.rs | 13 +++---------- datafusion/physical-expr/src/expressions/nullif.rs | 2 +- 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/datafusion/core/src/physical_plan/joins/hash_join.rs b/datafusion/core/src/physical_plan/joins/hash_join.rs index ba5056473c12..73e063a955df 100644 --- a/datafusion/core/src/physical_plan/joins/hash_join.rs +++ b/datafusion/core/src/physical_plan/joins/hash_join.rs @@ -54,7 +54,7 @@ use super::{ }; use arrow::buffer::BooleanBuffer; -use arrow::compute::{and, is_null, or_kleene, take, FilterBuilder}; +use arrow::compute::{and, take, FilterBuilder}; use arrow::record_batch::RecordBatch; use arrow::{ array::{ @@ -82,7 +82,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::OrderingEquivalenceProperties; use ahash::RandomState; -use arrow::compute::kernels::cmp::eq; +use arrow::compute::kernels::cmp::{eq, not_distinct}; use futures::{ready, Stream, StreamExt, TryStreamExt}; type JoinLeftData = (JoinHashMap, RecordBatch, MemoryReservation); @@ -1179,14 +1179,7 @@ fn eq_dyn_null( BooleanBuffer::collect_bool(left.len(), |_| null_equals_null), None, )), - _ if null_equals_null => { - let eq = eq(&left, &right)?; - - let left_is_null = is_null(left)?; - let right_is_null = is_null(right)?; - - or_kleene(&and(&left_is_null, &right_is_null)?, &eq) - } + _ if null_equals_null => not_distinct(&left, &right), _ => eq(&left, &right), } } diff --git a/datafusion/physical-expr/src/expressions/nullif.rs b/datafusion/physical-expr/src/expressions/nullif.rs index 4366db84bdcf..04a975666525 100644 --- a/datafusion/physical-expr/src/expressions/nullif.rs +++ b/datafusion/physical-expr/src/expressions/nullif.rs @@ -18,7 +18,7 @@ use arrow::array::Array; use arrow::compute::kernels::cmp::eq; use arrow::compute::kernels::nullif::nullif; -use datafusion_common::{internal_err, Result, ScalarValue}; +use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::ColumnarValue; /// Implements NULLIF(expr1, expr2) From 3e0e8c82281e2c783709c46b2de6cf03324c36a4 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Fri, 18 Aug 2023 20:02:58 +0100 Subject: [PATCH 05/10] Update bool cast --- datafusion/core/src/datasource/physical_plan/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index b4522c8ee2f9..679350a172fd 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -1346,9 +1346,9 @@ mod tests { let c2 = mapped_batch.column(1).as_primitive::(); let c4 = mapped_batch.column(2).as_primitive::(); - assert_eq!(c1.value(0), "1"); - assert_eq!(c1.value(1), "0"); - assert_eq!(c1.value(2), "1"); + assert_eq!(c1.value(0), "true"); + assert_eq!(c1.value(1), "false"); + assert_eq!(c1.value(2), "true"); assert_eq!(c2.value(0), 2.0_f64); assert_eq!(c2.value(1), 7.0_f64); From f4fafa9b0e38d9a03e0a887feb981c824374113f Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Fri, 18 Aug 2023 21:22:42 +0100 Subject: [PATCH 06/10] Further fixes --- Cargo.toml | 12 ++++---- datafusion-cli/Cargo.lock | 30 +++++++++---------- datafusion-cli/Cargo.toml | 10 +++---- .../core/src/physical_plan/joins/hash_join.rs | 4 --- datafusion/core/tests/memory_limit.rs | 1 + .../optimizer/tests/optimizer_integration.rs | 4 +-- .../engines/datafusion_engine/normalize.rs | 1 + datafusion/sqllogictest/test_files/ddl.slt | 2 +- .../sqllogictest/test_files/functions.slt | 2 +- 9 files changed, 32 insertions(+), 34 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index d92376b3f67b..29c9e2b64a37 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -59,9 +59,9 @@ panic = 'unwind' rpath = false [patch.crates-io] -arrow = {git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } -arrow-array = {git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } -arrow-buffer = {git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } -arrow-flight = {git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } -arrow-schema = {git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } -parquet = {git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } +arrow = {git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } +arrow-array = {git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } +arrow-buffer = {git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } +arrow-flight = {git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } +arrow-schema = {git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } +parquet = {git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 50044a328772..40b4286c7bba 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -96,7 +96,7 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" dependencies = [ "ahash", "arrow-arith", @@ -117,7 +117,7 @@ dependencies = [ [[package]] name = "arrow-arith" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" dependencies = [ "arrow-array", "arrow-buffer", @@ -131,7 +131,7 @@ dependencies = [ [[package]] name = "arrow-array" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" dependencies = [ "ahash", "arrow-buffer", @@ -147,7 +147,7 @@ dependencies = [ [[package]] name = "arrow-buffer" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" dependencies = [ "bytes", "half", @@ -157,7 +157,7 @@ dependencies = [ [[package]] name = "arrow-cast" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" dependencies = [ "arrow-array", "arrow-buffer", @@ -174,7 +174,7 @@ dependencies = [ [[package]] name = "arrow-csv" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" dependencies = [ "arrow-array", "arrow-buffer", @@ -192,7 +192,7 @@ dependencies = [ [[package]] name = "arrow-data" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" dependencies = [ "arrow-buffer", "arrow-schema", @@ -203,7 +203,7 @@ dependencies = [ [[package]] name = "arrow-ipc" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" dependencies = [ "arrow-array", "arrow-buffer", @@ -216,7 +216,7 @@ dependencies = [ [[package]] name = "arrow-json" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" dependencies = [ "arrow-array", "arrow-buffer", @@ -235,7 +235,7 @@ dependencies = [ [[package]] name = "arrow-ord" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" dependencies = [ "arrow-array", "arrow-buffer", @@ -249,7 +249,7 @@ dependencies = [ [[package]] name = "arrow-row" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" dependencies = [ "ahash", "arrow-array", @@ -263,12 +263,12 @@ dependencies = [ [[package]] name = "arrow-schema" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" [[package]] name = "arrow-select" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" dependencies = [ "arrow-array", "arrow-buffer", @@ -280,7 +280,7 @@ dependencies = [ [[package]] name = "arrow-string" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" dependencies = [ "arrow-array", "arrow-buffer", @@ -2252,7 +2252,7 @@ dependencies = [ [[package]] name = "parquet" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a#d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" dependencies = [ "ahash", "arrow-array", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index f7db4d876e63..e0dc033c7553 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -51,8 +51,8 @@ predicates = "3.0" rstest = "0.17" [patch.crates-io] -arrow = { git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } -arrow-array = { git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } -arrow-buffer = { git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } -arrow-schema = { git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } -parquet = { git = "https://github.com/tustvold/arrow-rs.git", rev = "d55bf0c4dfff8758b517b2a8a29882f1b27f6e8a" } +arrow = { git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } +arrow-array = { git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } +arrow-buffer = { git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } +arrow-schema = { git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } +parquet = { git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } diff --git a/datafusion/core/src/physical_plan/joins/hash_join.rs b/datafusion/core/src/physical_plan/joins/hash_join.rs index 73e063a955df..b1c93dfa26e2 100644 --- a/datafusion/core/src/physical_plan/joins/hash_join.rs +++ b/datafusion/core/src/physical_plan/joins/hash_join.rs @@ -1175,10 +1175,6 @@ fn eq_dyn_null( null_equals_null: bool, ) -> Result { match (left.data_type(), right.data_type()) { - (DataType::Null, DataType::Null) => Ok(BooleanArray::new( - BooleanBuffer::collect_bool(left.len(), |_| null_equals_null), - None, - )), _ if null_equals_null => not_distinct(&left, &right), _ => eq(&left, &right), } diff --git a/datafusion/core/tests/memory_limit.rs b/datafusion/core/tests/memory_limit.rs index 99e6aa5f9fcd..8cfbb08205c7 100644 --- a/datafusion/core/tests/memory_limit.rs +++ b/datafusion/core/tests/memory_limit.rs @@ -244,6 +244,7 @@ async fn sort_preserving_merge() { } #[tokio::test] +#[ignore] // TODO: Fix this async fn sort_spill_reservation() { let partition_size = batches_byte_size(&dict_batches()); diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index 142ae870d4b8..416b90240c71 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -211,7 +211,7 @@ fn concat_literals() -> Result<()> { FROM test"; let plan = test_sql(sql)?; let expected = - "Projection: concat(Utf8(\"1\"), CAST(test.col_int32 AS Utf8), Utf8(\"0hello\"), test.col_utf8, Utf8(\"123.4\")) AS col\ + "Projection: concat(Utf8(\"true\"), CAST(test.col_int32 AS Utf8), Utf8(\"falsehello\"), test.col_utf8, Utf8(\"123.4\")) AS col\ \n TableScan: test projection=[col_int32, col_utf8]"; assert_eq!(expected, format!("{plan:?}")); Ok(()) @@ -224,7 +224,7 @@ fn concat_ws_literals() -> Result<()> { FROM test"; let plan = test_sql(sql)?; let expected = - "Projection: concat_ws(Utf8(\"-\"), Utf8(\"1\"), CAST(test.col_int32 AS Utf8), Utf8(\"0-hello\"), test.col_utf8, Utf8(\"12--3.4\")) AS col\ + "Projection: concat_ws(Utf8(\"-\"), Utf8(\"true\"), CAST(test.col_int32 AS Utf8), Utf8(\"false-hello\"), test.col_utf8, Utf8(\"12--3.4\")) AS col\ \n TableScan: test projection=[col_int32, col_utf8]"; assert_eq!(expected, format!("{plan:?}")); Ok(()) diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs index 85cd6fcd5422..540b7be2f9ac 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs @@ -201,6 +201,7 @@ pub fn cell_to_string(col: &ArrayRef, row: usize) -> Result { Ok(NULL_STR.to_string()) } else { match col.data_type() { + DataType::Null => Ok(NULL_STR.to_string()), DataType::Boolean => { Ok(bool_to_str(get_row_value!(array::BooleanArray, col, row))) } diff --git a/datafusion/sqllogictest/test_files/ddl.slt b/datafusion/sqllogictest/test_files/ddl.slt index b323d1409f20..1f9bfa0aa2e7 100644 --- a/datafusion/sqllogictest/test_files/ddl.slt +++ b/datafusion/sqllogictest/test_files/ddl.slt @@ -302,7 +302,7 @@ CREATE TABLE my_table(c1 float, c2 double, c3 boolean, c4 varchar) AS SELECT *,c query RRBT rowsort SELECT * FROM my_table order by c1 LIMIT 1 ---- -0.00001 0.000000000001 true 1 +0.00001 0.000000000001 true true statement ok DROP TABLE my_table; diff --git a/datafusion/sqllogictest/test_files/functions.slt b/datafusion/sqllogictest/test_files/functions.slt index f8dbf8a00d9d..ec9557a14d38 100644 --- a/datafusion/sqllogictest/test_files/functions.slt +++ b/datafusion/sqllogictest/test_files/functions.slt @@ -516,7 +516,7 @@ from (values query ? SELECT struct(c1,c2,c3,c4,a,b) from simple_struct_test ---- -{c0: 1, c1: 1, c2: 3.1, c3: 3.14, c4: str, c5: text} +{c0: true, c1: 1, c2: 3.1, c3: 3.14, c4: str, c5: text} statement ok drop table simple_struct_test From 43f6a90d4287eb53ab511fd894600c81bb272caa Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Fri, 18 Aug 2023 21:29:33 +0100 Subject: [PATCH 07/10] Lockfile --- datafusion-cli/Cargo.lock | 30 +++++++++++++++--------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 40b4286c7bba..60ba5942efb7 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -96,7 +96,7 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" dependencies = [ "ahash", "arrow-arith", @@ -117,7 +117,7 @@ dependencies = [ [[package]] name = "arrow-arith" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" dependencies = [ "arrow-array", "arrow-buffer", @@ -131,7 +131,7 @@ dependencies = [ [[package]] name = "arrow-array" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" dependencies = [ "ahash", "arrow-buffer", @@ -147,7 +147,7 @@ dependencies = [ [[package]] name = "arrow-buffer" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" dependencies = [ "bytes", "half", @@ -157,7 +157,7 @@ dependencies = [ [[package]] name = "arrow-cast" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" dependencies = [ "arrow-array", "arrow-buffer", @@ -174,7 +174,7 @@ dependencies = [ [[package]] name = "arrow-csv" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" dependencies = [ "arrow-array", "arrow-buffer", @@ -192,7 +192,7 @@ dependencies = [ [[package]] name = "arrow-data" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" dependencies = [ "arrow-buffer", "arrow-schema", @@ -203,7 +203,7 @@ dependencies = [ [[package]] name = "arrow-ipc" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" dependencies = [ "arrow-array", "arrow-buffer", @@ -216,7 +216,7 @@ dependencies = [ [[package]] name = "arrow-json" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" dependencies = [ "arrow-array", "arrow-buffer", @@ -235,7 +235,7 @@ dependencies = [ [[package]] name = "arrow-ord" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" dependencies = [ "arrow-array", "arrow-buffer", @@ -249,7 +249,7 @@ dependencies = [ [[package]] name = "arrow-row" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" dependencies = [ "ahash", "arrow-array", @@ -263,12 +263,12 @@ dependencies = [ [[package]] name = "arrow-schema" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" [[package]] name = "arrow-select" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" dependencies = [ "arrow-array", "arrow-buffer", @@ -280,7 +280,7 @@ dependencies = [ [[package]] name = "arrow-string" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" dependencies = [ "arrow-array", "arrow-buffer", @@ -2252,7 +2252,7 @@ dependencies = [ [[package]] name = "parquet" version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=55f5839dba31bb2f1c0e35fe470331f731253f5d#55f5839dba31bb2f1c0e35fe470331f731253f5d" +source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" dependencies = [ "ahash", "arrow-array", From 33aa0ee72b0a18e561f51125389429a0bbb73aea Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Mon, 21 Aug 2023 16:35:52 +0100 Subject: [PATCH 08/10] Update pin --- Cargo.toml | 24 ++++++++-------- datafusion-cli/Cargo.lock | 60 +++++++++++++++++++-------------------- datafusion-cli/Cargo.toml | 12 ++++---- 3 files changed, 48 insertions(+), 48 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 29c9e2b64a37..8e1ac169d096 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -32,12 +32,12 @@ rust-version = "1.70" version = "29.0.0" [workspace.dependencies] -arrow = { version = "45.0.0", features = ["prettyprint", "dyn_cmp_dict"] } -arrow-array = { version = "45.0.0", default-features = false, features = ["chrono-tz"] } -arrow-buffer = { version = "45.0.0", default-features = false } -arrow-flight = { version = "45.0.0", features = ["flight-sql-experimental"] } -arrow-schema = { version = "45.0.0", default-features = false } -parquet = { version = "45.0.0", features = ["arrow", "async", "object_store"] } +arrow = { version = "46.0.0", features = ["prettyprint", "dyn_cmp_dict"] } +arrow-array = { version = "46.0.0", default-features = false, features = ["chrono-tz"] } +arrow-buffer = { version = "46.0.0", default-features = false } +arrow-flight = { version = "46.0.0", features = ["flight-sql-experimental"] } +arrow-schema = { version = "46.0.0", default-features = false } +parquet = { version = "46.0.0", features = ["arrow", "async", "object_store"] } sqlparser = { version = "0.36.1", features = ["visitor"] } [profile.release] @@ -59,9 +59,9 @@ panic = 'unwind' rpath = false [patch.crates-io] -arrow = {git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } -arrow-array = {git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } -arrow-buffer = {git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } -arrow-flight = {git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } -arrow-schema = {git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } -parquet = {git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } +arrow = {git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } +arrow-array = {git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } +arrow-buffer = {git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } +arrow-flight = {git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } +arrow-schema = {git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } +parquet = {git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 60ba5942efb7..b691825b39b1 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -95,8 +95,8 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" dependencies = [ "ahash", "arrow-arith", @@ -116,8 +116,8 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" dependencies = [ "arrow-array", "arrow-buffer", @@ -130,8 +130,8 @@ dependencies = [ [[package]] name = "arrow-array" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" dependencies = [ "ahash", "arrow-buffer", @@ -146,8 +146,8 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" dependencies = [ "bytes", "half", @@ -156,8 +156,8 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" dependencies = [ "arrow-array", "arrow-buffer", @@ -173,8 +173,8 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" dependencies = [ "arrow-array", "arrow-buffer", @@ -191,8 +191,8 @@ dependencies = [ [[package]] name = "arrow-data" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" dependencies = [ "arrow-buffer", "arrow-schema", @@ -202,8 +202,8 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" dependencies = [ "arrow-array", "arrow-buffer", @@ -215,8 +215,8 @@ dependencies = [ [[package]] name = "arrow-json" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" dependencies = [ "arrow-array", "arrow-buffer", @@ -234,8 +234,8 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" dependencies = [ "arrow-array", "arrow-buffer", @@ -248,8 +248,8 @@ dependencies = [ [[package]] name = "arrow-row" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" dependencies = [ "ahash", "arrow-array", @@ -262,13 +262,13 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" [[package]] name = "arrow-select" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" dependencies = [ "arrow-array", "arrow-buffer", @@ -279,8 +279,8 @@ dependencies = [ [[package]] name = "arrow-string" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" dependencies = [ "arrow-array", "arrow-buffer", @@ -2251,8 +2251,8 @@ dependencies = [ [[package]] name = "parquet" -version = "45.0.0" -source = "git+https://github.com/tustvold/arrow-rs.git?rev=2aff33df7a91e5d46b784ed38cba4263683b5f4e#2aff33df7a91e5d46b784ed38cba4263683b5f4e" +version = "46.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" dependencies = [ "ahash", "arrow-array", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index e0dc033c7553..d5247a31c631 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -29,7 +29,7 @@ rust-version = "1.70" readme = "README.md" [dependencies] -arrow = "45.0.0" +arrow = "46.0.0" async-trait = "0.1.41" aws-config = "0.55" aws-credential-types = "0.55" @@ -51,8 +51,8 @@ predicates = "3.0" rstest = "0.17" [patch.crates-io] -arrow = { git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } -arrow-array = { git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } -arrow-buffer = { git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } -arrow-schema = { git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } -parquet = { git = "https://github.com/tustvold/arrow-rs.git", rev = "2aff33df7a91e5d46b784ed38cba4263683b5f4e" } +arrow = { git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } +arrow-array = { git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } +arrow-buffer = { git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } +arrow-schema = { git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } +parquet = { git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } From f758fca6b7a252d6ad79ac5831a608d5c3658d41 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Thu, 24 Aug 2023 21:30:27 +0100 Subject: [PATCH 09/10] Remove pin --- Cargo.toml | 8 ---- datafusion-cli/Cargo.lock | 45 ++++++++++++------- datafusion-cli/Cargo.toml | 7 --- datafusion/common/src/scalar.rs | 16 ++----- .../core/src/datasource/physical_plan/csv.rs | 1 - .../core/src/physical_plan/joins/hash_join.rs | 3 +- .../physical-expr/src/expressions/binary.rs | 5 ++- 7 files changed, 39 insertions(+), 46 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 79d5acce244a..941161051bea 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -57,11 +57,3 @@ opt-level = 3 overflow-checks = false panic = 'unwind' rpath = false - -[patch.crates-io] -arrow = {git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } -arrow-array = {git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } -arrow-buffer = {git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } -arrow-flight = {git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } -arrow-schema = {git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } -parquet = {git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 714dcff5e066..04e802e4edbb 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -96,7 +96,8 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "04a8801ebb147ad240b2d978d3ab9f73c9ccd4557ba6a03e7800496770ed10e0" dependencies = [ "ahash", "arrow-arith", @@ -117,7 +118,8 @@ dependencies = [ [[package]] name = "arrow-arith" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "895263144bd4a69751cbe6a34a53f26626e19770b313a9fa792c415cd0e78f11" dependencies = [ "arrow-array", "arrow-buffer", @@ -131,7 +133,8 @@ dependencies = [ [[package]] name = "arrow-array" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "226fdc6c3a4ae154a74c24091d36a90b514f0ed7112f5b8322c1d8f354d8e20d" dependencies = [ "ahash", "arrow-buffer", @@ -147,7 +150,8 @@ dependencies = [ [[package]] name = "arrow-buffer" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fc4843af4dd679c2f35b69c572874da8fde33be53eb549a5fb128e7a4b763510" dependencies = [ "bytes", "half", @@ -157,7 +161,8 @@ dependencies = [ [[package]] name = "arrow-cast" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35e8b9990733a9b635f656efda3c9b8308c7a19695c9ec2c7046dd154f9b144b" dependencies = [ "arrow-array", "arrow-buffer", @@ -174,7 +179,8 @@ dependencies = [ [[package]] name = "arrow-csv" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "646fbb4e11dd0afb8083e883f53117713b8caadb4413b3c9e63e3f535da3683c" dependencies = [ "arrow-array", "arrow-buffer", @@ -192,7 +198,8 @@ dependencies = [ [[package]] name = "arrow-data" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "da900f31ff01a0a84da0572209be72b2b6f980f3ea58803635de47913191c188" dependencies = [ "arrow-buffer", "arrow-schema", @@ -203,7 +210,8 @@ dependencies = [ [[package]] name = "arrow-ipc" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2707a8d7ee2d345d045283ece3ae43416175873483e5d96319c929da542a0b1f" dependencies = [ "arrow-array", "arrow-buffer", @@ -216,7 +224,8 @@ dependencies = [ [[package]] name = "arrow-json" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5d1b91a63c356d14eedc778b76d66a88f35ac8498426bb0799a769a49a74a8b4" dependencies = [ "arrow-array", "arrow-buffer", @@ -235,7 +244,8 @@ dependencies = [ [[package]] name = "arrow-ord" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "584325c91293abbca7aaaabf8da9fe303245d641f5f4a18a6058dc68009c7ebf" dependencies = [ "arrow-array", "arrow-buffer", @@ -249,7 +259,8 @@ dependencies = [ [[package]] name = "arrow-row" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0e32afc1329f7b372463b21c6ca502b07cf237e1ed420d87706c1770bb0ebd38" dependencies = [ "ahash", "arrow-array", @@ -263,12 +274,14 @@ dependencies = [ [[package]] name = "arrow-schema" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b104f5daa730f00fde22adc03a12aa5a2ae9ccbbf99cbd53d284119ddc90e03d" [[package]] name = "arrow-select" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "73b3ca55356d1eae07cf48808d8c462cea674393ae6ad1e0b120f40b422eb2b4" dependencies = [ "arrow-array", "arrow-buffer", @@ -280,7 +293,8 @@ dependencies = [ [[package]] name = "arrow-string" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af1433ce02590cae68da0a18ed3a3ed868ffac2c6f24c533ddd2067f7ee04b4a" dependencies = [ "arrow-array", "arrow-buffer", @@ -2260,7 +2274,8 @@ dependencies = [ [[package]] name = "parquet" version = "46.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=90449ffb2ea6ceef43ce8fc97084b3373975f357#90449ffb2ea6ceef43ce8fc97084b3373975f357" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ad2cba786ae07da4d73371a88b9e0f9d3ffac1a9badc83922e0e15814f5c5fa" dependencies = [ "ahash", "arrow-array", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 26c27375311a..82c4a4b71e1d 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -49,10 +49,3 @@ assert_cmd = "2.0" ctor = "0.2.0" predicates = "3.0" rstest = "0.17" - -[patch.crates-io] -arrow = { git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } -arrow-array = { git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } -arrow-buffer = { git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } -arrow-schema = { git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } -parquet = { git = "https://github.com/apache/arrow-rs.git", rev = "90449ffb2ea6ceef43ce8fc97084b3373975f357" } diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 1c9ffb691e4f..3939293aac46 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -1193,9 +1193,7 @@ impl ScalarValue { /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code /// should operate on Arrays directly, using vectorized array kernels pub fn add>(&self, other: T) -> Result { - let s = self.to_array_of_size(1); - let o = other.borrow().to_array_of_size(1); - let r = add_wrapping(&Scalar::new(s.as_ref()), &Scalar::new(o.as_ref()))?; + let r = add_wrapping(&self.to_scalar(), &other.borrow().to_scalar())?; Self::try_from_array(r.as_ref(), 0) } /// Checked addition of `ScalarValue` @@ -1203,9 +1201,7 @@ impl ScalarValue { /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code /// should operate on Arrays directly, using vectorized array kernels pub fn add_checked>(&self, other: T) -> Result { - let s = self.to_array_of_size(1); - let o = other.borrow().to_array_of_size(1); - let r = add(&Scalar::new(s.as_ref()), &Scalar::new(o.as_ref()))?; + let r = add(&self.to_scalar(), &other.borrow().to_scalar())?; Self::try_from_array(r.as_ref(), 0) } @@ -1214,9 +1210,7 @@ impl ScalarValue { /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code /// should operate on Arrays directly, using vectorized array kernels pub fn sub>(&self, other: T) -> Result { - let s = self.to_array_of_size(1); - let o = other.borrow().to_array_of_size(1); - let r = sub_wrapping(&Scalar::new(s.as_ref()), &Scalar::new(o.as_ref()))?; + let r = sub_wrapping(&self.to_scalar(), &other.borrow().to_scalar())?; Self::try_from_array(r.as_ref(), 0) } @@ -1225,9 +1219,7 @@ impl ScalarValue { /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code /// should operate on Arrays directly, using vectorized array kernels pub fn sub_checked>(&self, other: T) -> Result { - let s = self.to_array_of_size(1); - let o = other.borrow().to_array_of_size(1); - let r = sub(&Scalar::new(s.as_ref()), &Scalar::new(o.as_ref()))?; + let r = sub(&self.to_scalar(), &other.borrow().to_scalar())?; Self::try_from_array(r.as_ref(), 0) } diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 1ba5b9c0c9e6..9f670431cb70 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -577,7 +577,6 @@ pub async fn plan_to_csv( #[cfg(test)] mod tests { use super::*; - use crate::datasource::file_format::file_type::FileType; use crate::prelude::*; use crate::test::{partitioned_csv_config, partitioned_file_groups}; use crate::{scalar::ScalarValue, test_util::aggr_test_schema}; diff --git a/datafusion/core/src/physical_plan/joins/hash_join.rs b/datafusion/core/src/physical_plan/joins/hash_join.rs index 28ebcca35629..e0e522b2150a 100644 --- a/datafusion/core/src/physical_plan/joins/hash_join.rs +++ b/datafusion/core/src/physical_plan/joins/hash_join.rs @@ -57,9 +57,8 @@ use arrow::array::{ Array, ArrayRef, BooleanArray, BooleanBufferBuilder, PrimitiveArray, UInt32Array, UInt32BufferBuilder, UInt64Array, UInt64BufferBuilder, }; -use arrow::buffer::BooleanBuffer; use arrow::compute::{and, take, FilterBuilder}; -use arrow::datatypes::{DataType, Schema, SchemaRef}; +use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow::util::bit_util; use arrow_array::cast::downcast_array; diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index 940f5c971516..ac33cdcc4ea8 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -33,9 +33,12 @@ use arrow::array::*; use arrow::compute::cast; use arrow::compute::kernels::boolean::{and_kleene, not, or_kleene}; use arrow::compute::kernels::cmp::*; +use arrow::compute::kernels::comparison::regexp_is_match_utf8; +use arrow::compute::kernels::comparison::regexp_is_match_utf8_scalar; +use arrow::compute::kernels::concat_elements::concat_elements_utf8; use arrow::datatypes::*; use arrow::record_batch::RecordBatch; -use arrow_array::{Datum, Scalar}; +use arrow_array::Datum; use datafusion_common::cast::as_boolean_array; use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::type_coercion::binary::get_result_type; From 5a9be81bdaf99ac32f07c3c8e73d08ab1315aab5 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Fri, 25 Aug 2023 10:01:25 +0100 Subject: [PATCH 10/10] Update sort_spill_reservation --- datafusion/core/tests/memory_limit.rs | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/datafusion/core/tests/memory_limit.rs b/datafusion/core/tests/memory_limit.rs index bcb5ad9906e0..12cde8660038 100644 --- a/datafusion/core/tests/memory_limit.rs +++ b/datafusion/core/tests/memory_limit.rs @@ -237,7 +237,6 @@ async fn sort_preserving_merge() { } #[tokio::test] -#[ignore] // TODO: Fix this async fn sort_spill_reservation() { let partition_size = batches_byte_size(&dict_batches()); @@ -255,7 +254,7 @@ async fn sort_spill_reservation() { // substantial memory .with_query("select * from t ORDER BY a , b DESC") // enough memory to sort if we don't try to merge it all at once - .with_memory_limit((partition_size * 5) / 2) + .with_memory_limit(partition_size) // use a single partiton so only a sort is needed .with_scenario(Scenario::DictionaryStrings(1)) .with_disk_manager_config(DiskManagerConfig::NewOs) @@ -295,7 +294,7 @@ async fn sort_spill_reservation() { // reserve sufficient space up front for merge and this time, // which will force the spills to happen with less buffered // input and thus with enough to merge. - .with_sort_spill_reservation_bytes(2 * partition_size); + .with_sort_spill_reservation_bytes(partition_size / 2); test.with_config(config).with_expected_success().run().await; } @@ -582,7 +581,9 @@ fn make_dict_batches() -> Vec { // ... // 0000000002 - let values: Vec<_> = (i..i + batch_size).map(|x| format!("{x:010}")).collect(); + let values: Vec<_> = (i..i + batch_size) + .map(|x| format!("{:010}", x / 16)) + .collect(); //println!("values: \n{values:?}"); let array: DictionaryArray = values.iter().map(|s| s.as_str()).collect();