|
27 | 27 | import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
|
28 | 28 | import org.elasticsearch.threadpool.ThreadPool;
|
29 | 29 |
|
| 30 | +import java.util.concurrent.CountDownLatch; |
30 | 31 | import java.util.concurrent.Semaphore;
|
31 | 32 | import java.util.concurrent.TimeUnit;
|
32 | 33 | import java.util.function.BiConsumer;
|
33 | 34 |
|
34 | 35 | /**
|
35 |
| - * Abstracts the low-level details of bulk request handling |
| 36 | + * Implements the low-level details of bulk request handling |
36 | 37 | */
|
37 |
| -abstract class BulkRequestHandler { |
38 |
| - protected final Logger logger; |
39 |
| - protected final BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer; |
40 |
| - protected final ThreadPool threadPool; |
41 |
| - |
42 |
| - protected BulkRequestHandler(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer, ThreadPool threadPool) { |
| 38 | +public final class BulkRequestHandler { |
| 39 | + private final Logger logger; |
| 40 | + private final BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer; |
| 41 | + private final BulkProcessor.Listener listener; |
| 42 | + private final Semaphore semaphore; |
| 43 | + private final Retry retry; |
| 44 | + private final int concurrentRequests; |
| 45 | + |
| 46 | + BulkRequestHandler(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer, BackoffPolicy backoffPolicy, |
| 47 | + BulkProcessor.Listener listener, ThreadPool threadPool, |
| 48 | + int concurrentRequests) { |
| 49 | + assert concurrentRequests >= 0; |
43 | 50 | this.logger = Loggers.getLogger(getClass());
|
44 | 51 | this.consumer = consumer;
|
45 |
| - this.threadPool = threadPool; |
46 |
| - } |
47 |
| - |
48 |
| - |
49 |
| - public abstract void execute(BulkRequest bulkRequest, long executionId); |
50 |
| - |
51 |
| - public abstract boolean awaitClose(long timeout, TimeUnit unit) throws InterruptedException; |
52 |
| - |
53 |
| - |
54 |
| - public static BulkRequestHandler syncHandler(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer, |
55 |
| - BackoffPolicy backoffPolicy, BulkProcessor.Listener listener, |
56 |
| - ThreadPool threadPool) { |
57 |
| - return new SyncBulkRequestHandler(consumer, backoffPolicy, listener, threadPool); |
| 52 | + this.listener = listener; |
| 53 | + this.concurrentRequests = concurrentRequests; |
| 54 | + this.retry = new Retry(EsRejectedExecutionException.class, backoffPolicy, threadPool); |
| 55 | + this.semaphore = new Semaphore(concurrentRequests > 0 ? concurrentRequests : 1); |
58 | 56 | }
|
59 | 57 |
|
60 |
| - public static BulkRequestHandler asyncHandler(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer, |
61 |
| - BackoffPolicy backoffPolicy, BulkProcessor.Listener listener, |
62 |
| - ThreadPool threadPool, int concurrentRequests) { |
63 |
| - return new AsyncBulkRequestHandler(consumer, backoffPolicy, listener, threadPool, concurrentRequests); |
64 |
| - } |
65 |
| - |
66 |
| - private static class SyncBulkRequestHandler extends BulkRequestHandler { |
67 |
| - private final BulkProcessor.Listener listener; |
68 |
| - private final BackoffPolicy backoffPolicy; |
69 |
| - |
70 |
| - SyncBulkRequestHandler(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer, BackoffPolicy backoffPolicy, |
71 |
| - BulkProcessor.Listener listener, ThreadPool threadPool) { |
72 |
| - super(consumer, threadPool); |
73 |
| - this.backoffPolicy = backoffPolicy; |
74 |
| - this.listener = listener; |
75 |
| - } |
76 |
| - |
77 |
| - @Override |
78 |
| - public void execute(BulkRequest bulkRequest, long executionId) { |
79 |
| - boolean afterCalled = false; |
80 |
| - try { |
81 |
| - listener.beforeBulk(executionId, bulkRequest); |
82 |
| - BulkResponse bulkResponse = Retry |
83 |
| - .on(EsRejectedExecutionException.class) |
84 |
| - .policy(backoffPolicy) |
85 |
| - .using(threadPool) |
86 |
| - .withSyncBackoff(consumer, bulkRequest, Settings.EMPTY); |
87 |
| - afterCalled = true; |
88 |
| - listener.afterBulk(executionId, bulkRequest, bulkResponse); |
89 |
| - } catch (InterruptedException e) { |
90 |
| - Thread.currentThread().interrupt(); |
91 |
| - logger.info((Supplier<?>) () -> new ParameterizedMessage("Bulk request {} has been cancelled.", executionId), e); |
92 |
| - if (!afterCalled) { |
93 |
| - listener.afterBulk(executionId, bulkRequest, e); |
94 |
| - } |
95 |
| - } catch (Exception e) { |
96 |
| - logger.warn((Supplier<?>) () -> new ParameterizedMessage("Failed to execute bulk request {}.", executionId), e); |
97 |
| - if (!afterCalled) { |
98 |
| - listener.afterBulk(executionId, bulkRequest, e); |
| 58 | + public void execute(BulkRequest bulkRequest, long executionId) { |
| 59 | + Runnable toRelease = () -> {}; |
| 60 | + boolean bulkRequestSetupSuccessful = false; |
| 61 | + try { |
| 62 | + listener.beforeBulk(executionId, bulkRequest); |
| 63 | + semaphore.acquire(); |
| 64 | + toRelease = semaphore::release; |
| 65 | + CountDownLatch latch = new CountDownLatch(1); |
| 66 | + retry.withBackoff(consumer, bulkRequest, new ActionListener<BulkResponse>() { |
| 67 | + @Override |
| 68 | + public void onResponse(BulkResponse response) { |
| 69 | + try { |
| 70 | + listener.afterBulk(executionId, bulkRequest, response); |
| 71 | + } finally { |
| 72 | + semaphore.release(); |
| 73 | + latch.countDown(); |
| 74 | + } |
99 | 75 | }
|
100 |
| - } |
101 |
| - } |
102 | 76 |
|
103 |
| - @Override |
104 |
| - public boolean awaitClose(long timeout, TimeUnit unit) throws InterruptedException { |
105 |
| - // we are "closed" immediately as there is no request in flight |
106 |
| - return true; |
107 |
| - } |
108 |
| - } |
109 |
| - |
110 |
| - private static class AsyncBulkRequestHandler extends BulkRequestHandler { |
111 |
| - private final BackoffPolicy backoffPolicy; |
112 |
| - private final BulkProcessor.Listener listener; |
113 |
| - private final Semaphore semaphore; |
114 |
| - private final int concurrentRequests; |
115 |
| - |
116 |
| - private AsyncBulkRequestHandler(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer, BackoffPolicy backoffPolicy, |
117 |
| - BulkProcessor.Listener listener, ThreadPool threadPool, |
118 |
| - int concurrentRequests) { |
119 |
| - super(consumer, threadPool); |
120 |
| - this.backoffPolicy = backoffPolicy; |
121 |
| - assert concurrentRequests > 0; |
122 |
| - this.listener = listener; |
123 |
| - this.concurrentRequests = concurrentRequests; |
124 |
| - this.semaphore = new Semaphore(concurrentRequests); |
125 |
| - } |
126 |
| - |
127 |
| - @Override |
128 |
| - public void execute(BulkRequest bulkRequest, long executionId) { |
129 |
| - boolean bulkRequestSetupSuccessful = false; |
130 |
| - boolean acquired = false; |
131 |
| - try { |
132 |
| - listener.beforeBulk(executionId, bulkRequest); |
133 |
| - semaphore.acquire(); |
134 |
| - acquired = true; |
135 |
| - Retry.on(EsRejectedExecutionException.class) |
136 |
| - .policy(backoffPolicy) |
137 |
| - .using(threadPool) |
138 |
| - .withAsyncBackoff(consumer, bulkRequest, new ActionListener<BulkResponse>() { |
139 |
| - @Override |
140 |
| - public void onResponse(BulkResponse response) { |
141 |
| - try { |
142 |
| - listener.afterBulk(executionId, bulkRequest, response); |
143 |
| - } finally { |
144 |
| - semaphore.release(); |
145 |
| - } |
146 |
| - } |
147 |
| - |
148 |
| - @Override |
149 |
| - public void onFailure(Exception e) { |
150 |
| - try { |
151 |
| - listener.afterBulk(executionId, bulkRequest, e); |
152 |
| - } finally { |
153 |
| - semaphore.release(); |
154 |
| - } |
155 |
| - } |
156 |
| - }, Settings.EMPTY); |
157 |
| - bulkRequestSetupSuccessful = true; |
158 |
| - } catch (InterruptedException e) { |
159 |
| - Thread.currentThread().interrupt(); |
160 |
| - logger.info((Supplier<?>) () -> new ParameterizedMessage("Bulk request {} has been cancelled.", executionId), e); |
161 |
| - listener.afterBulk(executionId, bulkRequest, e); |
162 |
| - } catch (Exception e) { |
163 |
| - logger.warn((Supplier<?>) () -> new ParameterizedMessage("Failed to execute bulk request {}.", executionId), e); |
164 |
| - listener.afterBulk(executionId, bulkRequest, e); |
165 |
| - } finally { |
166 |
| - if (!bulkRequestSetupSuccessful && acquired) { // if we fail on client.bulk() release the semaphore |
167 |
| - semaphore.release(); |
| 77 | + @Override |
| 78 | + public void onFailure(Exception e) { |
| 79 | + try { |
| 80 | + listener.afterBulk(executionId, bulkRequest, e); |
| 81 | + } finally { |
| 82 | + semaphore.release(); |
| 83 | + latch.countDown(); |
| 84 | + } |
168 | 85 | }
|
| 86 | + }, Settings.EMPTY); |
| 87 | + bulkRequestSetupSuccessful = true; |
| 88 | + if (concurrentRequests == 0) { |
| 89 | + latch.await(); |
| 90 | + } |
| 91 | + } catch (InterruptedException e) { |
| 92 | + Thread.currentThread().interrupt(); |
| 93 | + logger.info((Supplier<?>) () -> new ParameterizedMessage("Bulk request {} has been cancelled.", executionId), e); |
| 94 | + listener.afterBulk(executionId, bulkRequest, e); |
| 95 | + } catch (Exception e) { |
| 96 | + logger.warn((Supplier<?>) () -> new ParameterizedMessage("Failed to execute bulk request {}.", executionId), e); |
| 97 | + listener.afterBulk(executionId, bulkRequest, e); |
| 98 | + } finally { |
| 99 | + if (bulkRequestSetupSuccessful == false) { // if we fail on client.bulk() release the semaphore |
| 100 | + toRelease.run(); |
169 | 101 | }
|
170 | 102 | }
|
| 103 | + } |
171 | 104 |
|
172 |
| - @Override |
173 |
| - public boolean awaitClose(long timeout, TimeUnit unit) throws InterruptedException { |
174 |
| - if (semaphore.tryAcquire(this.concurrentRequests, timeout, unit)) { |
175 |
| - semaphore.release(this.concurrentRequests); |
176 |
| - return true; |
177 |
| - } |
178 |
| - return false; |
| 105 | + boolean awaitClose(long timeout, TimeUnit unit) throws InterruptedException { |
| 106 | + if (semaphore.tryAcquire(this.concurrentRequests, timeout, unit)) { |
| 107 | + semaphore.release(this.concurrentRequests); |
| 108 | + return true; |
179 | 109 | }
|
| 110 | + return false; |
180 | 111 | }
|
181 | 112 | }
|
0 commit comments