|
| 1 | +use std::sync::Arc; |
| 2 | + |
| 3 | +use dashmap::{mapref::one::Ref, DashMap}; |
| 4 | +use datafusion::execution::{ |
| 5 | + object_store::{ObjectStoreRegistry, ObjectStoreUrl}, |
| 6 | + TaskContext, |
| 7 | +}; |
| 8 | +use delta_kernel::engine::parse_json as arrow_parse_json; |
| 9 | +use delta_kernel::{ |
| 10 | + engine::default::{ |
| 11 | + executor::tokio::{TokioBackgroundExecutor, TokioMultiThreadExecutor}, |
| 12 | + json::DefaultJsonHandler, |
| 13 | + parquet::DefaultParquetHandler, |
| 14 | + }, |
| 15 | + error::DeltaResult as KernelResult, |
| 16 | + schema::SchemaRef, |
| 17 | + EngineData, FileDataReadResultIterator, FileMeta, JsonHandler, ParquetHandler, PredicateRef, |
| 18 | +}; |
| 19 | +use itertools::Itertools; |
| 20 | +use tokio::runtime::{Handle, RuntimeFlavor}; |
| 21 | + |
| 22 | +use super::storage::{group_by_store, AsObjectStoreUrl}; |
| 23 | + |
| 24 | +#[derive(Clone)] |
| 25 | +pub struct DataFusionFileFormatHandler { |
| 26 | + ctx: Arc<TaskContext>, |
| 27 | + pq_registry: Arc<DashMap<ObjectStoreUrl, Arc<dyn ParquetHandler>>>, |
| 28 | + json_registry: Arc<DashMap<ObjectStoreUrl, Arc<dyn JsonHandler>>>, |
| 29 | + handle: Handle, |
| 30 | +} |
| 31 | + |
| 32 | +impl DataFusionFileFormatHandler { |
| 33 | + /// Create a new [`DatafusionParquetHandler`] instance. |
| 34 | + pub fn new(ctx: Arc<TaskContext>, handle: Handle) -> Self { |
| 35 | + Self { |
| 36 | + ctx, |
| 37 | + pq_registry: DashMap::new().into(), |
| 38 | + json_registry: DashMap::new().into(), |
| 39 | + handle, |
| 40 | + } |
| 41 | + } |
| 42 | + |
| 43 | + fn registry(&self) -> Arc<dyn ObjectStoreRegistry> { |
| 44 | + self.ctx.runtime_env().object_store_registry.clone() |
| 45 | + } |
| 46 | + |
| 47 | + fn get_or_create_pq( |
| 48 | + &self, |
| 49 | + url: ObjectStoreUrl, |
| 50 | + ) -> KernelResult<Ref<'_, ObjectStoreUrl, Arc<dyn ParquetHandler>>> { |
| 51 | + if let Some(handler) = self.pq_registry.get(&url) { |
| 52 | + return Ok(handler); |
| 53 | + } |
| 54 | + let store = self |
| 55 | + .registry() |
| 56 | + .get_store(url.as_ref()) |
| 57 | + .map_err(delta_kernel::Error::generic_err)?; |
| 58 | + |
| 59 | + let handler: Arc<dyn ParquetHandler> = match self.handle.runtime_flavor() { |
| 60 | + RuntimeFlavor::MultiThread => Arc::new(DefaultParquetHandler::new( |
| 61 | + store, |
| 62 | + Arc::new(TokioMultiThreadExecutor::new(self.handle.clone())), |
| 63 | + )), |
| 64 | + RuntimeFlavor::CurrentThread => Arc::new(DefaultParquetHandler::new( |
| 65 | + store, |
| 66 | + Arc::new(TokioBackgroundExecutor::new()), |
| 67 | + )), |
| 68 | + _ => panic!("unsupported runtime flavor"), |
| 69 | + }; |
| 70 | + |
| 71 | + self.pq_registry.insert(url.clone(), handler); |
| 72 | + Ok(self.pq_registry.get(&url).unwrap()) |
| 73 | + } |
| 74 | + |
| 75 | + fn get_or_create_json( |
| 76 | + &self, |
| 77 | + url: ObjectStoreUrl, |
| 78 | + ) -> KernelResult<Ref<'_, ObjectStoreUrl, Arc<dyn JsonHandler>>> { |
| 79 | + if let Some(handler) = self.json_registry.get(&url) { |
| 80 | + return Ok(handler); |
| 81 | + } |
| 82 | + let store = self |
| 83 | + .registry() |
| 84 | + .get_store(url.as_ref()) |
| 85 | + .map_err(delta_kernel::Error::generic_err)?; |
| 86 | + |
| 87 | + let handler: Arc<dyn JsonHandler> = match self.handle.runtime_flavor() { |
| 88 | + RuntimeFlavor::MultiThread => Arc::new(DefaultJsonHandler::new( |
| 89 | + store, |
| 90 | + Arc::new(TokioMultiThreadExecutor::new(self.handle.clone())), |
| 91 | + )), |
| 92 | + RuntimeFlavor::CurrentThread => Arc::new(DefaultJsonHandler::new( |
| 93 | + store, |
| 94 | + Arc::new(TokioBackgroundExecutor::new()), |
| 95 | + )), |
| 96 | + _ => panic!("unsupported runtime flavor"), |
| 97 | + }; |
| 98 | + |
| 99 | + self.json_registry.insert(url.clone(), handler); |
| 100 | + Ok(self.json_registry.get(&url).unwrap()) |
| 101 | + } |
| 102 | +} |
| 103 | + |
| 104 | +impl ParquetHandler for DataFusionFileFormatHandler { |
| 105 | + fn read_parquet_files( |
| 106 | + &self, |
| 107 | + files: &[FileMeta], |
| 108 | + physical_schema: SchemaRef, |
| 109 | + predicate: Option<PredicateRef>, |
| 110 | + ) -> KernelResult<FileDataReadResultIterator> { |
| 111 | + let grouped_files = group_by_store(files.to_vec()); |
| 112 | + Ok(Box::new( |
| 113 | + grouped_files |
| 114 | + .into_iter() |
| 115 | + .map(|(url, files)| { |
| 116 | + self.get_or_create_pq(url)?.read_parquet_files( |
| 117 | + &files.to_vec(), |
| 118 | + physical_schema.clone(), |
| 119 | + predicate.clone(), |
| 120 | + ) |
| 121 | + }) |
| 122 | + // TODO: this should not do any blocking operations, since this should |
| 123 | + // happen when the iterators are polled and we are just creating a vec of iterators. |
| 124 | + // Is this correct? |
| 125 | + .try_collect::<_, Vec<_>, _>()? |
| 126 | + .into_iter() |
| 127 | + .flatten(), |
| 128 | + )) |
| 129 | + } |
| 130 | +} |
| 131 | + |
| 132 | +impl JsonHandler for DataFusionFileFormatHandler { |
| 133 | + fn parse_json( |
| 134 | + &self, |
| 135 | + json_strings: Box<dyn EngineData>, |
| 136 | + output_schema: SchemaRef, |
| 137 | + ) -> KernelResult<Box<dyn EngineData>> { |
| 138 | + arrow_parse_json(json_strings, output_schema) |
| 139 | + } |
| 140 | + |
| 141 | + fn read_json_files( |
| 142 | + &self, |
| 143 | + files: &[FileMeta], |
| 144 | + physical_schema: SchemaRef, |
| 145 | + predicate: Option<PredicateRef>, |
| 146 | + ) -> KernelResult<FileDataReadResultIterator> { |
| 147 | + let grouped_files = group_by_store(files.to_vec()); |
| 148 | + Ok(Box::new( |
| 149 | + grouped_files |
| 150 | + .into_iter() |
| 151 | + .map(|(url, files)| { |
| 152 | + self.get_or_create_json(url)?.read_json_files( |
| 153 | + &files.to_vec(), |
| 154 | + physical_schema.clone(), |
| 155 | + predicate.clone(), |
| 156 | + ) |
| 157 | + }) |
| 158 | + // TODO: this should not do any blocking operations, since this should |
| 159 | + // happen when the iterators are polled and we are just creating a vec of iterators. |
| 160 | + // Is this correct? |
| 161 | + .try_collect::<_, Vec<_>, _>()? |
| 162 | + .into_iter() |
| 163 | + .flatten(), |
| 164 | + )) |
| 165 | + } |
| 166 | + |
| 167 | + fn write_json_file( |
| 168 | + &self, |
| 169 | + path: &url::Url, |
| 170 | + data: Box<dyn Iterator<Item = KernelResult<Box<dyn EngineData>>> + Send + '_>, |
| 171 | + overwrite: bool, |
| 172 | + ) -> KernelResult<()> { |
| 173 | + self.get_or_create_json(path.as_object_store_url())? |
| 174 | + .write_json_file(path, data, overwrite) |
| 175 | + } |
| 176 | +} |
0 commit comments