@@ -54,6 +54,9 @@ use parquet::{
5454use crate :: execution:: shuffle:: CompressionCodec ;
5555use crate :: parquet:: parquet_support:: write_to_hdfs_with_opendal_async;
5656
57+ #[ cfg( all( test, feature = "hdfs-opendal" ) ) ]
58+ use crate :: parquet:: parquet_support:: write_record_batch_to_hdfs;
59+
5760/// Enum representing different types of Arrow writers based on storage backend
5861enum ParquetWriter {
5962 /// Writer for local file system
@@ -65,7 +68,10 @@ enum ParquetWriter {
6568
6669impl ParquetWriter {
6770 /// Write a RecordBatch to the underlying writer
68- async fn write ( & mut self , batch : & RecordBatch ) -> std:: result:: Result < ( ) , parquet:: errors:: ParquetError > {
71+ async fn write (
72+ & mut self ,
73+ batch : & RecordBatch ,
74+ ) -> std:: result:: Result < ( ) , parquet:: errors:: ParquetError > {
6975 match self {
7076 ParquetWriter :: LocalFile ( writer) => writer. write ( batch) ,
7177 ParquetWriter :: Remote ( writer, output_path) => {
@@ -84,7 +90,7 @@ impl ParquetWriter {
8490 output_path, e
8591 ) )
8692 } ) ?;
87-
93+
8894 write_to_hdfs_with_opendal_async ( & url, Bytes :: from ( buffer) )
8995 . await
9096 . map_err ( |e| {
@@ -99,12 +105,14 @@ impl ParquetWriter {
99105 cursor. set_position ( 0 ) ;
100106
101107 Ok ( ( ) )
102- } ,
108+ }
103109 }
104110 }
105111
106112 /// Close the writer and return the buffer for remote writers
107- fn close ( self ) -> std:: result:: Result < Option < ( Vec < u8 > , String ) > , parquet:: errors:: ParquetError > {
113+ fn close (
114+ self ,
115+ ) -> std:: result:: Result < Option < ( Vec < u8 > , String ) > , parquet:: errors:: ParquetError > {
108116 match self {
109117 ParquetWriter :: LocalFile ( writer) => {
110118 writer. close ( ) ?;
@@ -211,10 +219,12 @@ impl ParquetWriterExec {
211219 // For remote storage (HDFS, S3), write to an in-memory buffer
212220 let buffer = Vec :: new ( ) ;
213221 let cursor = Cursor :: new ( buffer) ;
214- let writer = ArrowWriter :: try_new ( cursor, schema, Some ( props) )
215- . map_err ( |e| DataFusionError :: Execution ( format ! (
216- "Failed to create {} writer: {}" , storage_scheme, e
217- ) ) ) ?;
222+ let writer = ArrowWriter :: try_new ( cursor, schema, Some ( props) ) . map_err ( |e| {
223+ DataFusionError :: Execution ( format ! (
224+ "Failed to create {} writer: {}" ,
225+ storage_scheme, e
226+ ) )
227+ } ) ?;
218228 Ok ( ParquetWriter :: Remote ( writer, output_file_path. to_string ( ) ) )
219229 }
220230 "local" => {
@@ -232,12 +242,14 @@ impl ParquetWriterExec {
232242 ) )
233243 } ) ?;
234244
235- let writer = ArrowWriter :: try_new ( file, schema, Some ( props) )
236- . map_err ( |e| DataFusionError :: Execution ( format ! ( "Failed to create local file writer: {}" , e) ) ) ?;
245+ let writer = ArrowWriter :: try_new ( file, schema, Some ( props) ) . map_err ( |e| {
246+ DataFusionError :: Execution ( format ! ( "Failed to create local file writer: {}" , e) )
247+ } ) ?;
237248 Ok ( ParquetWriter :: LocalFile ( writer) )
238249 }
239250 _ => Err ( DataFusionError :: Execution ( format ! (
240- "Unsupported storage scheme: {}" , storage_scheme
251+ "Unsupported storage scheme: {}" ,
252+ storage_scheme
241253 ) ) ) ,
242254 }
243255 }
@@ -379,7 +391,12 @@ impl ExecutionPlan for ParquetWriterExec {
379391 . set_compression ( compression)
380392 . build ( ) ;
381393
382- let mut writer = Self :: create_arrow_writer ( storage_scheme, & part_file, Arc :: clone ( & output_schema) , props) ?;
394+ let mut writer = Self :: create_arrow_writer (
395+ storage_scheme,
396+ & part_file,
397+ Arc :: clone ( & output_schema) ,
398+ props,
399+ ) ?;
383400
384401 // Clone schema for use in async closure
385402 let schema_for_write = Arc :: clone ( & output_schema) ;
@@ -444,3 +461,67 @@ impl ExecutionPlan for ParquetWriterExec {
444461 ) ) )
445462 }
446463}
464+
465+ #[ cfg( test) ]
466+ mod tests {
467+ use super :: * ;
468+ use arrow:: array:: { Int32Array , StringArray } ;
469+ use arrow:: datatypes:: { DataType , Field , Schema } ;
470+ use std:: sync:: Arc ;
471+
472+ /// Helper function to create a test RecordBatch with 1000 rows of (int, string) data
473+ fn create_test_record_batch ( ) -> Result < RecordBatch > {
474+ let num_rows = 1000 ;
475+
476+ // Create int column with values 0..1000
477+ let int_array = Int32Array :: from_iter_values ( 0 ..num_rows) ;
478+
479+ // Create string column with values "value_0", "value_1", ..., "value_999"
480+ let string_values: Vec < String > = ( 0 ..num_rows)
481+ . map ( |i| format ! ( "value_{}" , i) )
482+ . collect ( ) ;
483+ let string_array = StringArray :: from ( string_values) ;
484+
485+ // Define schema
486+ let schema = Arc :: new ( Schema :: new ( vec ! [
487+ Field :: new( "id" , DataType :: Int32 , false ) ,
488+ Field :: new( "name" , DataType :: Utf8 , false ) ,
489+ ] ) ) ;
490+
491+ // Create RecordBatch
492+ RecordBatch :: try_new (
493+ schema,
494+ vec ! [ Arc :: new( int_array) , Arc :: new( string_array) ] ,
495+ )
496+ . map_err ( |e| DataFusionError :: ArrowError ( Box :: new ( e) , None ) )
497+ }
498+
499+ #[ tokio:: test]
500+ //#[cfg(feature = "hdfs-opendal")]
501+ async fn test_write_to_hdfs ( ) -> Result < ( ) > {
502+ use opendal:: services:: Hdfs ;
503+ use opendal:: Operator ;
504+
505+ // Create test data
506+ let batch = create_test_record_batch ( ) ?;
507+
508+ // Configure HDFS connection
509+ let namenode = "hdfs://namenode:9000" ;
510+ let output_path = "/user/test_write/data.parquet" ;
511+
512+ // Create OpenDAL HDFS operator
513+ let builder = Hdfs :: default ( ) . name_node ( namenode) ;
514+ let op = Operator :: new ( builder)
515+ . map_err ( |e| DataFusionError :: Execution ( format ! ( "Failed to create HDFS operator: {}" , e) ) ) ?
516+ . finish ( ) ;
517+
518+ // Write the batch using write_record_batch_to_hdfs
519+ write_record_batch_to_hdfs ( & op, output_path, batch)
520+ . await
521+ . map_err ( |e| DataFusionError :: Execution ( format ! ( "Failed to write to HDFS: {}" , e) ) ) ?;
522+
523+ println ! ( "Successfully wrote 1000 rows to HDFS at {}{}" , namenode, output_path) ;
524+
525+ Ok ( ( ) )
526+ }
527+ }
0 commit comments