20
20
package org .elasticsearch .index .shard ;
21
21
22
22
import org .apache .logging .log4j .Logger ;
23
+ import org .elasticsearch .Assertions ;
23
24
import org .elasticsearch .action .ActionListener ;
24
25
import org .elasticsearch .action .support .ContextPreservingActionListener ;
25
26
import org .elasticsearch .action .support .ThreadedActionListener ;
26
27
import org .elasticsearch .common .CheckedRunnable ;
27
28
import org .elasticsearch .common .Nullable ;
28
29
import org .elasticsearch .common .lease .Releasable ;
30
+ import org .elasticsearch .common .util .concurrent .AbstractRunnable ;
29
31
import org .elasticsearch .common .util .concurrent .ThreadContext .StoredContext ;
30
32
import org .elasticsearch .threadpool .ThreadPool ;
31
33
36
38
import java .util .concurrent .TimeUnit ;
37
39
import java .util .concurrent .TimeoutException ;
38
40
import java .util .concurrent .atomic .AtomicBoolean ;
41
+ import java .util .function .Consumer ;
39
42
import java .util .function .Supplier ;
40
43
44
+ /**
45
+ * Tracks shard operation permits. Each operation on the shard obtains a permit. When we need to block operations (e.g., to transition
46
+ * between terms) we immediately delay all operations to a queue, obtain all available permits, and wait for outstanding operations to drain
47
+ * and return their permits. Delayed operations will acquire permits and be completed after the operation that blocked all operations has
48
+ * completed.
49
+ */
41
50
final class IndexShardOperationPermits implements Closeable {
51
+
42
52
private final ShardId shardId ;
43
53
private final Logger logger ;
44
54
private final ThreadPool threadPool ;
45
55
46
56
private static final int TOTAL_PERMITS = Integer .MAX_VALUE ;
47
- // fair semaphore to ensure that blockOperations() does not starve under thread contention
48
- final Semaphore semaphore = new Semaphore (TOTAL_PERMITS , true );
49
- @ Nullable private List <ActionListener <Releasable >> delayedOperations ; // operations that are delayed
57
+ final Semaphore semaphore = new Semaphore (TOTAL_PERMITS , true ); // fair to ensure a blocking thread is not starved
58
+ private final List <ActionListener <Releasable >> delayedOperations = new ArrayList <>(); // operations that are delayed
50
59
private volatile boolean closed ;
60
+ private boolean delayed ; // does not need to be volatile as all accesses are done under a lock on this
51
61
52
- IndexShardOperationPermits (ShardId shardId , Logger logger , ThreadPool threadPool ) {
62
+ /**
63
+ * Construct operation permits for the specified shards.
64
+ *
65
+ * @param shardId the shard
66
+ * @param logger the logger for the shard
67
+ * @param threadPool the thread pool (used to execute delayed operations)
68
+ */
69
+ IndexShardOperationPermits (final ShardId shardId , final Logger logger , final ThreadPool threadPool ) {
53
70
this .shardId = shardId ;
54
71
this .logger = logger ;
55
72
this .threadPool = threadPool ;
@@ -61,99 +78,170 @@ public void close() {
61
78
}
62
79
63
80
/**
64
- * Wait for in-flight operations to finish and executes onBlocked under the guarantee that no new operations are started. Queues
65
- * operations that are occurring in the meanwhile and runs them once onBlocked has executed.
81
+ * Wait for in-flight operations to finish and executes {@code onBlocked} under the guarantee that no new operations are started. Queues
82
+ * operations that are occurring in the meanwhile and runs them once {@code onBlocked} has executed.
66
83
*
67
- * @param timeout the maximum time to wait for the in-flight operations block
68
- * @param timeUnit the time unit of the {@code timeout} argument
84
+ * @param timeout the maximum time to wait for the in-flight operations block
85
+ * @param timeUnit the time unit of the {@code timeout} argument
69
86
* @param onBlocked the action to run once the block has been acquired
70
- * @throws InterruptedException if calling thread is interrupted
71
- * @throws TimeoutException if timed out waiting for in-flight operations to finish
87
+ * @param <E> the type of checked exception thrown by {@code onBlocked}
88
+ * @throws InterruptedException if calling thread is interrupted
89
+ * @throws TimeoutException if timed out waiting for in-flight operations to finish
72
90
* @throws IndexShardClosedException if operation permit has been closed
73
91
*/
74
- public <E extends Exception > void blockOperations (long timeout , TimeUnit timeUnit , CheckedRunnable <E > onBlocked ) throws
75
- InterruptedException , TimeoutException , E {
92
+ <E extends Exception > void blockOperations (
93
+ final long timeout ,
94
+ final TimeUnit timeUnit ,
95
+ final CheckedRunnable <E > onBlocked ) throws InterruptedException , TimeoutException , E {
76
96
if (closed ) {
77
97
throw new IndexShardClosedException (shardId );
78
98
}
99
+ delayOperations ();
79
100
try {
80
- if (semaphore .tryAcquire (TOTAL_PERMITS , timeout , timeUnit )) {
81
- assert semaphore .availablePermits () == 0 ;
82
- try {
83
- onBlocked .run ();
84
- } finally {
85
- semaphore .release (TOTAL_PERMITS );
86
- }
101
+ doBlockOperations (timeout , timeUnit , onBlocked );
102
+ } finally {
103
+ releaseDelayedOperations ();
104
+ }
105
+ }
106
+
107
+ /**
108
+ * Immediately delays operations and on another thread waits for in-flight operations to finish and then executes {@code onBlocked}
109
+ * under the guarantee that no new operations are started. Delayed operations are run after {@code onBlocked} has executed. After
110
+ * operations are delayed and the blocking is forked to another thread, returns to the caller. If a failure occurs while blocking
111
+ * operations or executing {@code onBlocked} then the {@code onFailure} handler will be invoked.
112
+ *
113
+ * @param timeout the maximum time to wait for the in-flight operations block
114
+ * @param timeUnit the time unit of the {@code timeout} argument
115
+ * @param onBlocked the action to run once the block has been acquired
116
+ * @param onFailure the action to run if a failure occurs while blocking operations
117
+ * @param <E> the type of checked exception thrown by {@code onBlocked} (not thrown on the calling thread)
118
+ */
119
+ <E extends Exception > void asyncBlockOperations (
120
+ final long timeout , final TimeUnit timeUnit , final CheckedRunnable <E > onBlocked , final Consumer <Exception > onFailure ) {
121
+ delayOperations ();
122
+ threadPool .executor (ThreadPool .Names .GENERIC ).execute (new AbstractRunnable () {
123
+ @ Override
124
+ public void onFailure (final Exception e ) {
125
+ onFailure .accept (e );
126
+ }
127
+
128
+ @ Override
129
+ protected void doRun () throws Exception {
130
+ doBlockOperations (timeout , timeUnit , onBlocked );
131
+ }
132
+
133
+ @ Override
134
+ public void onAfter () {
135
+ releaseDelayedOperations ();
136
+ }
137
+ });
138
+ }
139
+
140
+ private void delayOperations () {
141
+ synchronized (this ) {
142
+ if (delayed ) {
143
+ throw new IllegalStateException ("operations are already delayed" );
87
144
} else {
88
- throw new TimeoutException ("timed out during blockOperations" );
145
+ assert delayedOperations .isEmpty ();
146
+ delayed = true ;
89
147
}
90
- } finally {
91
- final List <ActionListener <Releasable >> queuedActions ;
148
+ }
149
+ }
150
+
151
+ private <E extends Exception > void doBlockOperations (
152
+ final long timeout ,
153
+ final TimeUnit timeUnit ,
154
+ final CheckedRunnable <E > onBlocked ) throws InterruptedException , TimeoutException , E {
155
+ if (Assertions .ENABLED ) {
156
+ // since delayed is not volatile, we have to synchronize even here for visibility
92
157
synchronized (this ) {
93
- queuedActions = delayedOperations ;
94
- delayedOperations = null ;
158
+ assert delayed ;
95
159
}
96
- if (queuedActions != null ) {
97
- // Try acquiring permits on fresh thread (for two reasons):
98
- // - blockOperations can be called on recovery thread which can be expected to be interrupted when recovery is cancelled.
99
- // Interruptions are bad here as permit acquisition will throw an InterruptedException which will be swallowed by
100
- // ThreadedActionListener if the queue of the thread pool on which it submits is full.
101
- // - if permit is acquired and queue of the thread pool which the ThreadedActionListener uses is full, the onFailure
102
- // handler is executed on the calling thread. This should not be the recovery thread as it would delay the recovery.
103
- threadPool .executor (ThreadPool .Names .GENERIC ).execute (() -> {
104
- for (ActionListener <Releasable > queuedAction : queuedActions ) {
105
- acquire (queuedAction , null , false );
106
- }
107
- });
160
+ }
161
+ if (semaphore .tryAcquire (TOTAL_PERMITS , timeout , timeUnit )) {
162
+ assert semaphore .availablePermits () == 0 ;
163
+ try {
164
+ onBlocked .run ();
165
+ } finally {
166
+ semaphore .release (TOTAL_PERMITS );
108
167
}
168
+ } else {
169
+ throw new TimeoutException ("timeout while blocking operations" );
170
+ }
171
+ }
172
+
173
+ private void releaseDelayedOperations () {
174
+ final List <ActionListener <Releasable >> queuedActions ;
175
+ synchronized (this ) {
176
+ assert delayed ;
177
+ queuedActions = new ArrayList <>(delayedOperations );
178
+ delayedOperations .clear ();
179
+ delayed = false ;
180
+ }
181
+ if (!queuedActions .isEmpty ()) {
182
+ /*
183
+ * Try acquiring permits on fresh thread (for two reasons):
184
+ * - blockOperations can be called on a recovery thread which can be expected to be interrupted when recovery is cancelled;
185
+ * interruptions are bad here as permit acquisition will throw an interrupted exception which will be swallowed by
186
+ * the threaded action listener if the queue of the thread pool on which it submits is full
187
+ * - if a permit is acquired and the queue of the thread pool which the the threaded action listener uses is full, the
188
+ * onFailure handler is executed on the calling thread; this should not be the recovery thread as it would delay the
189
+ * recovery
190
+ */
191
+ threadPool .executor (ThreadPool .Names .GENERIC ).execute (() -> {
192
+ for (ActionListener <Releasable > queuedAction : queuedActions ) {
193
+ acquire (queuedAction , null , false );
194
+ }
195
+ });
109
196
}
110
197
}
111
198
112
199
/**
113
200
* Acquires a permit whenever permit acquisition is not blocked. If the permit is directly available, the provided
114
201
* {@link ActionListener} will be called on the calling thread. During calls of
115
- * {@link #blockOperations(long, TimeUnit, CheckedRunnable)}, permit acquisition can be delayed. The provided ActionListener will
116
- * then be called using the provided executor once operations are no longer blocked.
202
+ * {@link #blockOperations(long, TimeUnit, CheckedRunnable)}, permit acquisition can be delayed. The provided {@link ActionListener}
203
+ * will then be called using the provided executor once operations are no longer blocked.
117
204
*
118
205
* @param onAcquired {@link ActionListener} that is invoked once acquisition is successful or failed
119
206
* @param executorOnDelay executor to use for delayed call
120
207
* @param forceExecution whether the runnable should force its execution in case it gets rejected
121
208
*/
122
- public void acquire (ActionListener <Releasable > onAcquired , String executorOnDelay , boolean forceExecution ) {
209
+ public void acquire (final ActionListener <Releasable > onAcquired , final String executorOnDelay , final boolean forceExecution ) {
123
210
if (closed ) {
124
211
onAcquired .onFailure (new IndexShardClosedException (shardId ));
125
212
return ;
126
213
}
127
- Releasable releasable ;
214
+ final Releasable releasable ;
128
215
try {
129
216
synchronized (this ) {
130
- releasable = tryAcquire ();
131
- if (releasable == null ) {
132
- // blockOperations is executing, this operation will be retried by blockOperations once it finishes
133
- if (delayedOperations == null ) {
134
- delayedOperations = new ArrayList <>();
135
- }
217
+ if (delayed ) {
136
218
final Supplier <StoredContext > contextSupplier = threadPool .getThreadContext ().newRestorableContext (false );
137
219
if (executorOnDelay != null ) {
138
220
delayedOperations .add (
139
- new ThreadedActionListener <>(logger , threadPool , executorOnDelay ,
140
- new ContextPreservingActionListener <>(contextSupplier , onAcquired ), forceExecution ));
221
+ new ThreadedActionListener <>(logger , threadPool , executorOnDelay ,
222
+ new ContextPreservingActionListener <>(contextSupplier , onAcquired ), forceExecution ));
141
223
} else {
142
224
delayedOperations .add (new ContextPreservingActionListener <>(contextSupplier , onAcquired ));
143
225
}
144
226
return ;
227
+ } else {
228
+ releasable = tryAcquire ();
229
+ assert releasable != null ;
145
230
}
146
231
}
147
- } catch (InterruptedException e ) {
232
+ } catch (final InterruptedException e ) {
148
233
onAcquired .onFailure (e );
149
234
return ;
150
235
}
236
+ // execute this outside the synchronized block!
151
237
onAcquired .onResponse (releasable );
152
238
}
153
239
154
- @ Nullable private Releasable tryAcquire () throws InterruptedException {
155
- if (semaphore .tryAcquire (1 , 0 , TimeUnit .SECONDS )) { // the untimed tryAcquire methods do not honor the fairness setting
156
- AtomicBoolean closed = new AtomicBoolean ();
240
+ @ Nullable
241
+ private Releasable tryAcquire () throws InterruptedException {
242
+ assert Thread .holdsLock (this );
243
+ if (semaphore .tryAcquire (1 , 0 , TimeUnit .SECONDS )) { // the un-timed tryAcquire methods do not honor the fairness setting
244
+ final AtomicBoolean closed = new AtomicBoolean ();
157
245
return () -> {
158
246
if (closed .compareAndSet (false , true )) {
159
247
semaphore .release (1 );
@@ -163,13 +251,23 @@ public void acquire(ActionListener<Releasable> onAcquired, String executorOnDela
163
251
return null ;
164
252
}
165
253
166
- public int getActiveOperationsCount () {
254
+ /**
255
+ * Obtain the active operation count, or zero if all permits are held (even if there are outstanding operations in flight).
256
+ *
257
+ * @return the active operation count, or zero when all permits ar eheld
258
+ */
259
+ int getActiveOperationsCount () {
167
260
int availablePermits = semaphore .availablePermits ();
168
261
if (availablePermits == 0 ) {
169
- // when blockOperations is holding all permits
262
+ /*
263
+ * This occurs when either doBlockOperations is holding all the permits or there are outstanding operations in flight and the
264
+ * remainder of the permits are held by doBlockOperations. We do not distinguish between these two cases and simply say that
265
+ * the active operations count is zero.
266
+ */
170
267
return 0 ;
171
268
} else {
172
269
return TOTAL_PERMITS - availablePermits ;
173
270
}
174
271
}
272
+
175
273
}
0 commit comments