19
19
20
20
package org .elasticsearch .action .bulk ;
21
21
22
+ import org .elasticsearch .ElasticSearchIllegalStateException ;
22
23
import org .elasticsearch .action .ActionListener ;
23
24
import org .elasticsearch .action .ActionRequest ;
24
25
import org .elasticsearch .action .delete .DeleteRequest ;
25
26
import org .elasticsearch .action .index .IndexRequest ;
26
27
import org .elasticsearch .client .Client ;
28
+ import org .elasticsearch .client .internal .InternalClient ;
27
29
import org .elasticsearch .common .Nullable ;
28
30
import org .elasticsearch .common .bytes .BytesReference ;
29
31
import org .elasticsearch .common .unit .ByteSizeUnit ;
30
32
import org .elasticsearch .common .unit .ByteSizeValue ;
33
+ import org .elasticsearch .common .unit .TimeValue ;
34
+ import org .elasticsearch .common .util .concurrent .EsExecutors ;
31
35
32
- import java .util .concurrent .Semaphore ;
36
+ import java .util .concurrent .*;
37
+ import java .util .concurrent .atomic .AtomicLong ;
33
38
34
39
/**
35
40
* A bulk processor is a thread safe bulk processing class, allowing to easily set when to "flush" a new bulk request
36
- * (either based on number of actions, or based on the size), and to easily control the number of concurrent bulk
41
+ * (either based on number of actions, based on the size, or time ), and to easily control the number of concurrent bulk
37
42
* requests allowed to be executed in parallel.
38
43
* <p/>
39
44
* In order to create a new bulk processor, use the {@link Builder}.
40
45
*/
41
46
public class BulkProcessor {
42
47
48
+ /**
49
+ * A listener for the execution.
50
+ */
51
+ public static interface Listener {
52
+
53
+ /**
54
+ * Callback before the bulk is executed.
55
+ */
56
+ void beforeBulk (long executionId , BulkRequest request );
57
+
58
+ /**
59
+ * Callback after a successful execution of bulk request.
60
+ */
61
+ void afterBulk (long executionId , BulkRequest request , BulkResponse response );
62
+
63
+ /**
64
+ * Callback after a failed execution of bulk request.
65
+ */
66
+ void afterBulk (long executionId , BulkRequest request , Throwable failure );
67
+ }
68
+
43
69
/**
44
70
* A builder used to create a build an instance of a bulk processor.
45
71
*/
46
72
public static class Builder {
47
73
48
74
private final Client client ;
49
- private final ActionListener < BulkResponse > listener ;
75
+ private final Listener listener ;
50
76
77
+ private String name ;
51
78
private int concurrentRequests = 1 ;
52
79
private int bulkActions = 1000 ;
53
80
private ByteSizeValue bulkSize = new ByteSizeValue (5 , ByteSizeUnit .MB );
81
+ private TimeValue flushInterval = null ;
54
82
55
83
/**
56
84
* Creates a builder of bulk processor with the client to use and the listener that will be used
57
85
* to be notified on the completion of bulk requests.
58
86
*/
59
- public Builder (Client client , ActionListener < BulkResponse > listener ) {
87
+ public Builder (Client client , Listener listener ) {
60
88
this .client = client ;
61
89
this .listener = listener ;
62
90
}
63
91
92
+ /**
93
+ * Sets an optional name to identify this bulk processor.
94
+ */
95
+ public Builder setName (String name ) {
96
+ this .name = name ;
97
+ return this ;
98
+ }
99
+
64
100
/**
65
101
* Sets the number of concurrent requests allowed to be executed. A value of 0 means that only a single
66
102
* request will be allowed to be executed. A value of 1 means 1 concurrent request is allowed to be executed
@@ -89,38 +125,87 @@ public Builder setBulkSize(ByteSizeValue bulkSize) {
89
125
return this ;
90
126
}
91
127
128
+ /**
129
+ * Sets a flush interval flushing *any* bulk actions pending if the interval passes. Defaults to not set.
130
+ * <p/>
131
+ * Note, both {@link #setBulkActions(int)} and {@link #setBulkSize(org.elasticsearch.common.unit.ByteSizeValue)}
132
+ * can be set to <tt>-1</tt> with the flush interval set allowing for complete async processing of bulk actions.
133
+ */
134
+ public Builder setFlushInterval (TimeValue flushInterval ) {
135
+ this .flushInterval = flushInterval ;
136
+ return this ;
137
+ }
138
+
92
139
/**
93
140
* Builds a new bulk processor.
94
141
*/
95
142
public BulkProcessor build () {
96
- return new BulkProcessor (client , listener , concurrentRequests , bulkActions , bulkSize );
143
+ return new BulkProcessor (client , listener , name , concurrentRequests , bulkActions , bulkSize , flushInterval );
97
144
}
98
145
}
99
146
100
- public static Builder builder (Client client , ActionListener < BulkResponse > listener ) {
147
+ public static Builder builder (Client client , Listener listener ) {
101
148
return new Builder (client , listener );
102
149
}
103
150
104
151
private final Client client ;
105
- private final ActionListener <BulkResponse > listener ;
152
+ private final Listener listener ;
153
+
154
+ private final String name ;
106
155
107
- private int concurrentRequests ;
156
+ private final int concurrentRequests ;
108
157
private final int bulkActions ;
109
158
private final int bulkSize ;
159
+ private final TimeValue flushInterval ;
110
160
111
161
private final Semaphore semaphore ;
162
+ private final ScheduledThreadPoolExecutor scheduler ;
163
+ private final ScheduledFuture scheduledFuture ;
164
+
165
+ private final AtomicLong executionIdGen = new AtomicLong ();
112
166
113
167
private BulkRequest bulkRequest ;
114
168
115
- BulkProcessor (Client client , ActionListener <BulkResponse > listener , int concurrentRequests , int bulkActions , ByteSizeValue bulkSize ) {
169
+ private volatile boolean closed = false ;
170
+
171
+ BulkProcessor (Client client , Listener listener , @ Nullable String name , int concurrentRequests , int bulkActions , ByteSizeValue bulkSize , @ Nullable TimeValue flushInterval ) {
116
172
this .client = client ;
117
173
this .listener = listener ;
174
+ this .name = name ;
118
175
this .concurrentRequests = concurrentRequests ;
119
176
this .bulkActions = bulkActions ;
120
177
this .bulkSize = bulkSize .bytesAsInt ();
121
178
122
179
this .semaphore = new Semaphore (concurrentRequests );
123
180
this .bulkRequest = new BulkRequest ();
181
+
182
+ this .flushInterval = flushInterval ;
183
+ if (flushInterval != null ) {
184
+ this .scheduler = (ScheduledThreadPoolExecutor ) Executors .newScheduledThreadPool (1 , EsExecutors .daemonThreadFactory (((InternalClient ) client ).settings (), (name != null ? "[" + name + "]" : "" ) + "bulk_processor" ));
185
+ this .scheduler .setExecuteExistingDelayedTasksAfterShutdownPolicy (false );
186
+ this .scheduler .setContinueExistingPeriodicTasksAfterShutdownPolicy (false );
187
+ this .scheduledFuture = this .scheduler .scheduleWithFixedDelay (new Flush (), flushInterval .millis (), flushInterval .millis (), TimeUnit .MILLISECONDS );
188
+ } else {
189
+ this .scheduler = null ;
190
+ this .scheduledFuture = null ;
191
+ }
192
+ }
193
+
194
+ /**
195
+ * Closes the processor. If flushing by time is enabled, then its shutdown. Any remaining bulk actions are flushed.
196
+ */
197
+ public synchronized void close () {
198
+ if (closed ) {
199
+ return ;
200
+ }
201
+ closed = true ;
202
+ if (this .scheduledFuture != null ) {
203
+ this .scheduledFuture .cancel (false );
204
+ this .scheduler .shutdown ();
205
+ }
206
+ if (bulkRequest .numberOfActions () > 0 ) {
207
+ execute ();
208
+ }
124
209
}
125
210
126
211
/**
@@ -155,28 +240,44 @@ public synchronized BulkProcessor add(BytesReference data, boolean contentUnsafe
155
240
}
156
241
157
242
private void executeIfNeeded () {
243
+ if (closed ) {
244
+ throw new ElasticSearchIllegalStateException ("bulk process already closed" );
245
+ }
246
+ this .closed = true ;
158
247
if (!isOverTheLimit ()) {
159
248
return ;
160
249
}
250
+ execute ();
251
+ }
252
+
253
+ // (currently) needs to be executed under a lock
254
+ private void execute () {
255
+ final BulkRequest bulkRequest = this .bulkRequest ;
256
+ final long executionId = executionIdGen .incrementAndGet ();
257
+
258
+ this .bulkRequest = new BulkRequest ();
259
+
161
260
if (concurrentRequests == 0 ) {
162
261
// execute in a blocking fashion...
163
262
try {
164
- listener .onResponse (client .bulk (bulkRequest ).actionGet ());
263
+ listener .beforeBulk (executionId , bulkRequest );
264
+ listener .afterBulk (executionId , bulkRequest , client .bulk (bulkRequest ).actionGet ());
165
265
} catch (Exception e ) {
166
- listener .onFailure ( e );
266
+ listener .afterBulk ( executionId , bulkRequest , e );
167
267
}
168
268
} else {
169
269
try {
170
270
semaphore .acquire ();
171
271
} catch (InterruptedException e ) {
172
- listener .onFailure ( e );
272
+ listener .afterBulk ( executionId , bulkRequest , e );
173
273
return ;
174
274
}
275
+ listener .beforeBulk (executionId , bulkRequest );
175
276
client .bulk (bulkRequest , new ActionListener <BulkResponse >() {
176
277
@ Override
177
278
public void onResponse (BulkResponse response ) {
178
279
try {
179
- listener .onResponse ( response );
280
+ listener .afterBulk ( executionId , bulkRequest , response );
180
281
} finally {
181
282
semaphore .release ();
182
283
}
@@ -185,14 +286,13 @@ public void onResponse(BulkResponse response) {
185
286
@ Override
186
287
public void onFailure (Throwable e ) {
187
288
try {
188
- listener .onFailure ( e );
289
+ listener .afterBulk ( executionId , bulkRequest , e );
189
290
} finally {
190
291
semaphore .release ();
191
292
}
192
293
}
193
294
});
194
295
}
195
- bulkRequest = new BulkRequest ();
196
296
}
197
297
198
298
private boolean isOverTheLimit () {
@@ -204,4 +304,20 @@ private boolean isOverTheLimit() {
204
304
}
205
305
return false ;
206
306
}
307
+
308
+ class Flush implements Runnable {
309
+
310
+ @ Override
311
+ public void run () {
312
+ synchronized (BulkProcessor .this ) {
313
+ if (closed ) {
314
+ return ;
315
+ }
316
+ if (bulkRequest .numberOfActions () == 0 ) {
317
+ return ;
318
+ }
319
+ execute ();
320
+ }
321
+ }
322
+ }
207
323
}
0 commit comments