@@ -23,16 +23,18 @@ use std::{
2323 fmt,
2424 fmt:: { Debug , Formatter } ,
2525 fs:: File ,
26- io:: Cursor ,
2726 sync:: Arc ,
2827} ;
2928
29+ #[ cfg( feature = "hdfs-opendal" ) ]
3030use opendal:: Operator ;
31+ #[ cfg( feature = "hdfs-opendal" ) ]
32+ use std:: io:: Cursor ;
3133
3234use crate :: execution:: shuffle:: CompressionCodec ;
33- use crate :: parquet:: parquet_support:: {
34- create_hdfs_operator , is_hdfs_scheme , prepare_object_store_with_configs ,
35- } ;
35+ use crate :: parquet:: parquet_support:: is_hdfs_scheme ;
36+ # [ cfg ( feature = "hdfs-opendal" ) ]
37+ use crate :: parquet :: parquet_support :: { create_hdfs_operator , prepare_object_store_with_configs } ;
3638use arrow:: datatypes:: { Schema , SchemaRef } ;
3739use arrow:: record_batch:: RecordBatch ;
3840use async_trait:: async_trait;
@@ -45,7 +47,7 @@ use datafusion::{
4547 metrics:: { ExecutionPlanMetricsSet , MetricsSet } ,
4648 stream:: RecordBatchStreamAdapter ,
4749 DisplayAs , DisplayFormatType , ExecutionPlan , ExecutionPlanProperties , PlanProperties ,
48- SendableRecordBatchStream , Statistics ,
50+ SendableRecordBatchStream ,
4951 } ,
5052} ;
5153use futures:: TryStreamExt ;
@@ -64,6 +66,7 @@ enum ParquetWriter {
6466 /// Contains the arrow writer, HDFS operator, and destination path
6567 /// an Arrow writer writes to in-memory buffer the data converted to Parquet format
6668 /// The opendal::Writer is created lazily on first write
69+ #[ cfg( feature = "hdfs-opendal" ) ]
6770 Remote (
6871 ArrowWriter < Cursor < Vec < u8 > > > ,
6972 Option < opendal:: Writer > ,
@@ -80,6 +83,7 @@ impl ParquetWriter {
8083 ) -> std:: result:: Result < ( ) , parquet:: errors:: ParquetError > {
8184 match self {
8285 ParquetWriter :: LocalFile ( writer) => writer. write ( batch) ,
86+ #[ cfg( feature = "hdfs-opendal" ) ]
8387 ParquetWriter :: Remote (
8488 arrow_parquet_buffer_writer,
8589 hdfs_writer_opt,
@@ -134,6 +138,7 @@ impl ParquetWriter {
134138 writer. close ( ) ?;
135139 Ok ( ( ) )
136140 }
141+ #[ cfg( feature = "hdfs-opendal" ) ]
137142 ParquetWriter :: Remote (
138143 arrow_parquet_buffer_writer,
139144 mut hdfs_writer_opt,
@@ -208,7 +213,7 @@ pub struct ParquetWriterExec {
208213 /// Metrics
209214 metrics : ExecutionPlanMetricsSet ,
210215 /// Cache for plan properties
211- cache : PlanProperties ,
216+ cache : Arc < PlanProperties > ,
212217}
213218
214219impl ParquetWriterExec {
@@ -228,12 +233,12 @@ impl ParquetWriterExec {
228233 // Preserve the input's partitioning so each partition writes its own file
229234 let input_partitioning = input. output_partitioning ( ) . clone ( ) ;
230235
231- let cache = PlanProperties :: new (
236+ let cache = Arc :: new ( PlanProperties :: new (
232237 EquivalenceProperties :: new ( Arc :: clone ( & input. schema ( ) ) ) ,
233238 input_partitioning,
234239 EmissionType :: Final ,
235240 Boundedness :: Bounded ,
236- ) ;
241+ ) ) ;
237242
238243 Ok ( ParquetWriterExec {
239244 input,
@@ -275,7 +280,7 @@ impl ParquetWriterExec {
275280 output_file_path : & str ,
276281 schema : SchemaRef ,
277282 props : WriterProperties ,
278- runtime_env : Arc < datafusion:: execution:: runtime_env:: RuntimeEnv > ,
283+ _runtime_env : Arc < datafusion:: execution:: runtime_env:: RuntimeEnv > ,
279284 object_store_options : & HashMap < String , String > ,
280285 ) -> Result < ParquetWriter > {
281286 // Parse URL and match on storage scheme directly
@@ -284,11 +289,11 @@ impl ParquetWriterExec {
284289 } ) ?;
285290
286291 if is_hdfs_scheme ( & url, object_store_options) {
287- // HDFS storage
292+ # [ cfg ( feature = "hdfs-opendal" ) ]
288293 {
289294 // Use prepare_object_store_with_configs to create and register the object store
290295 let ( _object_store_url, object_store_path) = prepare_object_store_with_configs (
291- runtime_env ,
296+ _runtime_env ,
292297 output_file_path. to_string ( ) ,
293298 object_store_options,
294299 )
@@ -324,6 +329,12 @@ impl ParquetWriterExec {
324329 object_store_path. to_string ( ) ,
325330 ) )
326331 }
332+ #[ cfg( not( feature = "hdfs-opendal" ) ) ]
333+ {
334+ Err ( DataFusionError :: Execution (
335+ "HDFS support is not enabled. Rebuild with the 'hdfs-opendal' feature." . into ( ) ,
336+ ) )
337+ }
327338 } else if output_file_path. starts_with ( "file://" )
328339 || output_file_path. starts_with ( "file:" )
329340 || !output_file_path. contains ( "://" )
@@ -405,11 +416,7 @@ impl ExecutionPlan for ParquetWriterExec {
405416 Some ( self . metrics . clone_inner ( ) )
406417 }
407418
408- fn statistics ( & self ) -> Result < Statistics > {
409- self . input . partition_statistics ( None )
410- }
411-
412- fn properties ( & self ) -> & PlanProperties {
419+ fn properties ( & self ) -> & Arc < PlanProperties > {
413420 & self . cache
414421 }
415422
@@ -576,6 +583,7 @@ mod tests {
576583
577584 /// Helper function to create a test RecordBatch with 1000 rows of (int, string) data
578585 /// Example batch_id 1 -> 0..1000, 2 -> 1001..2000
586+ #[ allow( dead_code) ]
579587 fn create_test_record_batch ( batch_id : i32 ) -> Result < RecordBatch > {
580588 assert ! ( batch_id > 0 , "batch_id must be greater than 0" ) ;
581589 let num_rows = batch_id * 1000 ;
0 commit comments