@@ -322,11 +322,11 @@ impl RepartitionExecState {
322322
323323 let mut channels = HashMap :: with_capacity ( txs. len ( ) ) ;
324324 for ( partition, ( tx, rx) ) in txs. into_iter ( ) . zip ( rxs) . enumerate ( ) {
325- let reservation = Arc :: new ( Mutex :: new (
325+ let reservation = Arc :: new (
326326 MemoryConsumer :: new ( format ! ( "{name}[{partition}]" ) )
327327 . with_can_spill ( true )
328328 . register ( context. memory_pool ( ) ) ,
329- ) ) ;
329+ ) ;
330330
331331 // Create spill channels based on mode:
332332 // - preserve_order: one spill channel per (input, output) pair for proper FIFO ordering
@@ -1401,7 +1401,7 @@ impl RepartitionExec {
14011401 // if there is still a receiver, send to it
14021402 if let Some ( channel) = output_channels. get_mut ( & partition) {
14031403 let ( batch_to_send, is_memory_batch) =
1404- match channel. reservation . lock ( ) . try_grow ( size) {
1404+ match channel. reservation . try_grow ( size) {
14051405 Ok ( _) => {
14061406 // Memory available - send in-memory batch
14071407 ( RepartitionBatch :: Memory ( batch) , true )
@@ -1419,7 +1419,7 @@ impl RepartitionExec {
14191419 // If the other end has hung up, it was an early shutdown (e.g. LIMIT)
14201420 // Only shrink memory if it was a memory batch
14211421 if is_memory_batch {
1422- channel. reservation . lock ( ) . shrink ( size) ;
1422+ channel. reservation . shrink ( size) ;
14231423 }
14241424 output_channels. remove ( & partition) ;
14251425 }
@@ -1638,9 +1638,7 @@ impl PerPartitionStream {
16381638 Some ( Some ( v) ) => match v {
16391639 Ok ( RepartitionBatch :: Memory ( batch) ) => {
16401640 // Release memory and return batch
1641- self . reservation
1642- . lock ( )
1643- . shrink ( batch. get_array_memory_size ( ) ) ;
1641+ self . reservation . shrink ( batch. get_array_memory_size ( ) ) ;
16441642 return Poll :: Ready ( Some ( Ok ( batch) ) ) ;
16451643 }
16461644 Ok ( RepartitionBatch :: Spilled ) => {
0 commit comments