|
16 | 16 | import org.elasticsearch.Version; |
17 | 17 | import org.elasticsearch.action.ActionListener; |
18 | 18 | import org.elasticsearch.action.ActionListenerResponseHandler; |
| 19 | +import org.elasticsearch.action.ActionRunnable; |
19 | 20 | import org.elasticsearch.action.support.RetryableAction; |
20 | 21 | import org.elasticsearch.cluster.node.DiscoveryNode; |
21 | 22 | import org.elasticsearch.common.breaker.CircuitBreakingException; |
@@ -202,12 +203,17 @@ public void writeFileChunk(StoreFileMetadata fileMetadata, long position, Releas |
202 | 203 | final RecoveryFileChunkRequest request = new RecoveryFileChunkRequest( |
203 | 204 | recoveryId, requestSeqNo, shardId, fileMetadata, position, content, lastChunk, totalTranslogOps, throttleTimeInNanos); |
204 | 205 | final Writeable.Reader<TransportResponse.Empty> reader = in -> TransportResponse.Empty.INSTANCE; |
205 | | - executeRetryableAction( |
206 | | - action, |
207 | | - request, |
208 | | - fileChunkRequestOptions, |
209 | | - ActionListener.runBefore(listener.map(r -> null), request::decRef), |
210 | | - reader); |
| 206 | + |
| 207 | + // Fork the actual sending onto a separate thread so we can send them concurrently even if CPU-bound (e.g. using compression). |
| 208 | + // The AsyncIOProcessor and MultiFileWriter both concentrate their work onto fewer threads if possible, but once we have |
| 209 | + // chunks to send we want to increase parallelism again. |
| 210 | + threadPool.generic().execute(ActionRunnable.wrap(listener, l -> |
| 211 | + executeRetryableAction( |
| 212 | + action, |
| 213 | + request, |
| 214 | + fileChunkRequestOptions, |
| 215 | + ActionListener.runBefore(l.map(r -> null), request::decRef), |
| 216 | + reader))); |
211 | 217 | } |
212 | 218 |
|
213 | 219 | @Override |
|
0 commit comments