15
15
// specific language governing permissions and limitations
16
16
// under the License.
17
17
18
+ use std:: fmt:: Formatter ;
18
19
use std:: sync:: Arc ;
19
20
use std:: { any:: Any , pin:: Pin } ;
20
21
21
22
use crate :: client:: BallistaClient ;
22
23
use crate :: memory_stream:: MemoryStream ;
23
24
use crate :: serde:: scheduler:: PartitionLocation ;
24
25
26
+ use crate :: utils:: WrappedStream ;
25
27
use async_trait:: async_trait;
26
28
use datafusion:: arrow:: datatypes:: SchemaRef ;
29
+ use datafusion:: arrow:: error:: Result as ArrowResult ;
30
+ use datafusion:: arrow:: record_batch:: RecordBatch ;
27
31
use datafusion:: physical_plan:: { DisplayFormatType , ExecutionPlan , Partitioning } ;
28
32
use datafusion:: {
29
33
error:: { DataFusionError , Result } ,
30
34
physical_plan:: RecordBatchStream ,
31
35
} ;
36
+ use futures:: { future, Stream , StreamExt } ;
32
37
use log:: info;
33
- use std:: fmt:: Formatter ;
34
38
35
- /// ShuffleReaderExec reads partitions that have already been materialized by an executor.
39
+ /// ShuffleReaderExec reads partitions that have already been materialized by a query stage
40
+ /// being executed by an executor
36
41
#[ derive( Debug , Clone ) ]
37
42
pub struct ShuffleReaderExec {
38
- // The query stage that is responsible for producing the shuffle partitions that
39
- // this operator will read
40
- pub ( crate ) partition_location : Vec < PartitionLocation > ,
43
+ /// Each partition of a shuffle can read data from multiple locations
44
+ pub ( crate ) partition : Vec < Vec < PartitionLocation > > ,
41
45
pub ( crate ) schema : SchemaRef ,
42
46
}
43
47
44
48
impl ShuffleReaderExec {
45
49
/// Create a new ShuffleReaderExec
46
50
pub fn try_new (
47
- partition_meta : Vec < PartitionLocation > ,
51
+ partition : Vec < Vec < PartitionLocation > > ,
48
52
schema : SchemaRef ,
49
53
) -> Result < Self > {
50
- Ok ( Self {
51
- partition_location : partition_meta,
52
- schema,
53
- } )
54
+ Ok ( Self { partition, schema } )
54
55
}
55
56
}
56
57
@@ -65,7 +66,7 @@ impl ExecutionPlan for ShuffleReaderExec {
65
66
}
66
67
67
68
fn output_partitioning ( & self ) -> Partitioning {
68
- Partitioning :: UnknownPartitioning ( self . partition_location . len ( ) )
69
+ Partitioning :: UnknownPartitioning ( self . partition . len ( ) )
69
70
}
70
71
71
72
fn children ( & self ) -> Vec < Arc < dyn ExecutionPlan > > {
@@ -86,23 +87,18 @@ impl ExecutionPlan for ShuffleReaderExec {
86
87
partition : usize ,
87
88
) -> Result < Pin < Box < dyn RecordBatchStream + Send + Sync > > > {
88
89
info ! ( "ShuffleReaderExec::execute({})" , partition) ;
89
- let partition_location = & self . partition_location [ partition] ;
90
-
91
- let mut client = BallistaClient :: try_new (
92
- & partition_location. executor_meta . host ,
93
- partition_location. executor_meta . port ,
94
- )
95
- . await
96
- . map_err ( |e| DataFusionError :: Execution ( format ! ( "Ballista Error: {:?}" , e) ) ) ?;
97
90
98
- client
99
- . fetch_partition (
100
- & partition_location. partition_id . job_id ,
101
- partition_location. partition_id . stage_id ,
102
- partition,
103
- )
91
+ let partition_locations = & self . partition [ partition] ;
92
+ let result = future:: join_all ( partition_locations. iter ( ) . map ( fetch_partition) )
104
93
. await
105
- . map_err ( |e| DataFusionError :: Execution ( format ! ( "Ballista Error: {:?}" , e) ) )
94
+ . into_iter ( )
95
+ . collect :: < Result < Vec < _ > > > ( ) ?;
96
+
97
+ let result = WrappedStream :: new (
98
+ Box :: pin ( futures:: stream:: iter ( result) . flatten ( ) ) ,
99
+ Arc :: new ( self . schema . as_ref ( ) . clone ( ) ) ,
100
+ ) ;
101
+ Ok ( Box :: pin ( result) )
106
102
}
107
103
108
104
fn fmt_as (
@@ -113,22 +109,46 @@ impl ExecutionPlan for ShuffleReaderExec {
113
109
match t {
114
110
DisplayFormatType :: Default => {
115
111
let loc_str = self
116
- . partition_location
112
+ . partition
117
113
. iter ( )
118
- . map ( |l| {
119
- format ! (
120
- "[executor={} part={}:{}:{} stats={:?}]" ,
121
- l. executor_meta. id,
122
- l. partition_id. job_id,
123
- l. partition_id. stage_id,
124
- l. partition_id. partition_id,
125
- l. partition_stats
126
- )
114
+ . map ( |x| {
115
+ x. iter ( )
116
+ . map ( |l| {
117
+ format ! (
118
+ "[executor={} part={}:{}:{} stats={:?}]" ,
119
+ l. executor_meta. id,
120
+ l. partition_id. job_id,
121
+ l. partition_id. stage_id,
122
+ l. partition_id. partition_id,
123
+ l. partition_stats
124
+ )
125
+ } )
126
+ . collect :: < Vec < String > > ( )
127
+ . join ( "," )
127
128
} )
128
129
. collect :: < Vec < String > > ( )
129
- . join ( ", " ) ;
130
+ . join ( "\n " ) ;
130
131
write ! ( f, "ShuffleReaderExec: partition_locations={}" , loc_str)
131
132
}
132
133
}
133
134
}
134
135
}
136
+
137
+ async fn fetch_partition (
138
+ location : & PartitionLocation ,
139
+ ) -> Result < Pin < Box < dyn RecordBatchStream + Send + Sync > > > {
140
+ let metadata = & location. executor_meta ;
141
+ let partition_id = & location. partition_id ;
142
+ let mut ballista_client =
143
+ BallistaClient :: try_new ( metadata. host . as_str ( ) , metadata. port as u16 )
144
+ . await
145
+ . map_err ( |e| DataFusionError :: Execution ( format ! ( "{:?}" , e) ) ) ?;
146
+ Ok ( ballista_client
147
+ . fetch_partition (
148
+ & partition_id. job_id ,
149
+ partition_id. stage_id as usize ,
150
+ partition_id. partition_id as usize ,
151
+ )
152
+ . await
153
+ . map_err ( |e| DataFusionError :: Execution ( format ! ( "{:?}" , e) ) ) ?)
154
+ }
0 commit comments