Skip to content

Commit 473ceff

Browse files
committed
cleanups
1 parent 7440c1c commit 473ceff

File tree

3 files changed

+15
-11
lines changed

3 files changed

+15
-11
lines changed

datafusion-examples/examples/thread_pools.rs

+4-3
Original file line numberDiff line numberDiff line change
@@ -25,9 +25,10 @@
2525
//! due to congestion control and increased latencies for processing network
2626
//! messages.
2727
use arrow::util::pretty::pretty_format_batches;
28+
use datafusion::common::runtime::dedicated_executor;
2829
use datafusion::error::Result;
2930
use datafusion::execution::SendableRecordBatchStream;
30-
use datafusion::physical_plan::{dedicated_executor, DedicatedExecutor};
31+
use datafusion::physical_plan::DedicatedExecutor;
3132
use datafusion::prelude::*;
3233
use futures::stream::StreamExt;
3334
use object_store::http::HttpBuilder;
@@ -167,13 +168,13 @@ async fn different_runtime_advanced() -> Result<()> {
167168
// ctx.register_object_store(&base_url, http_store);
168169
// A Tokio 1.x context was found, but timers are disabled. Call `enable_time` on the runtime builder to enable timers.
169170

170-
let http_store = DedicatedExecutor::wrap_object_store(http_store);
171+
let http_store = dedicated_executor.wrap_object_store(http_store);
171172

172173
// Tell datafusion about processing http:// urls with this wrapped object store
173174
ctx.register_object_store(&base_url, http_store);
174175

175176
// Plan (and execute) the query on the dedicated runtime
176-
let mut stream = dedicated_executor
177+
let stream = dedicated_executor
177178
.spawn(async move {
178179
// Plan / execute the query
179180
let url = "https://github.com/apache/arrow-testing/raw/master/data/csv/aggregate_test_100.csv";

datafusion/physical-plan/src/dedicated_executor.rs

+8-2
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,10 @@ use crate::io_object_store::IoObjectStore;
2525
use crate::stream::RecordBatchStreamAdapter;
2626
use crate::SendableRecordBatchStream;
2727
use datafusion_common::DataFusionError;
28-
use futures::{future::{BoxFuture, Shared}, Future, FutureExt, Stream, TryFutureExt};
28+
use futures::{
29+
future::{BoxFuture, Shared},
30+
Future, FutureExt, Stream, TryFutureExt,
31+
};
2932
use log::{info, warn};
3033
use object_store::ObjectStore;
3134
use parking_lot::RwLock;
@@ -221,7 +224,10 @@ impl DedicatedExecutor {
221224
/// Note that this object store will only work correctly if run on this
222225
/// dedicated executor. If you try and use it on another executor, it will
223226
/// panic with "no IO runtime registered" type error.
224-
pub fn wrap_object_store(&self, object_store: Arc<dyn ObjectStore>) -> Arc<IoObjectStore> {
227+
pub fn wrap_object_store(
228+
&self,
229+
object_store: Arc<dyn ObjectStore>,
230+
) -> Arc<IoObjectStore> {
225231
Arc::new(IoObjectStore::new(self.clone(), object_store))
226232
}
227233

datafusion/physical-plan/src/io_object_store.rs

+3-6
Original file line numberDiff line numberDiff line change
@@ -17,15 +17,14 @@
1717

1818
use std::sync::Arc;
1919

20+
use crate::dedicated_executor::JobError;
2021
use crate::DedicatedExecutor;
2122
use async_trait::async_trait;
2223
use futures::stream::BoxStream;
23-
use futures::StreamExt;
2424
use object_store::{
2525
path::Path, GetOptions, GetResult, ListResult, MultipartUpload, ObjectMeta,
2626
ObjectStore, PutMultipartOpts, PutOptions, PutPayload, PutResult, Result,
2727
};
28-
use crate::dedicated_executor::JobError;
2928

3029
/// 'ObjectStore' that wraps an inner `ObjectStore` and wraps all the underlying
3130
/// methods with [`DedicatedExecutor::spawn_io`] so that they are run on the Tokio Runtime
@@ -57,7 +56,7 @@ impl std::fmt::Display for IoObjectStore {
5756
fn convert_error(e: JobError) -> object_store::Error {
5857
object_store::Error::Generic {
5958
store: "IoObjectStore",
60-
source: Box::new(e)
59+
source: Box::new(e),
6160
}
6261
}
6362

@@ -98,7 +97,7 @@ impl ObjectStore for IoObjectStore {
9897

9998
inner_stream
10099
//self.executor.run_stream(inner_stream, convert_error)
101-
// .boxed()
100+
// .boxed()
102101
}
103102

104103
async fn list_with_delimiter(&self, prefix: Option<&Path>) -> Result<ListResult> {
@@ -136,6 +135,4 @@ impl ObjectStore for IoObjectStore {
136135
})
137136
.await
138137
}
139-
140-
141138
}

0 commit comments

Comments
 (0)