19
19
20
20
package org .elasticsearch .cluster .action .index ;
21
21
22
+ import com .google .common .collect .Lists ;
23
+ import com .google .common .collect .Sets ;
22
24
import org .elasticsearch .ElasticsearchException ;
23
25
import org .elasticsearch .action .ActionListener ;
24
26
import org .elasticsearch .action .ActionRequestValidationException ;
31
33
import org .elasticsearch .cluster .metadata .IndexMetaData ;
32
34
import org .elasticsearch .cluster .metadata .MetaDataMappingService ;
33
35
import org .elasticsearch .cluster .node .DiscoveryNode ;
36
+ import org .elasticsearch .common .collect .Tuple ;
34
37
import org .elasticsearch .common .compress .CompressedString ;
35
38
import org .elasticsearch .common .inject .Inject ;
36
39
import org .elasticsearch .common .io .stream .StreamInput ;
37
40
import org .elasticsearch .common .io .stream .StreamOutput ;
38
41
import org .elasticsearch .common .settings .Settings ;
42
+ import org .elasticsearch .common .unit .TimeValue ;
43
+ import org .elasticsearch .common .util .concurrent .ConcurrentCollections ;
44
+ import org .elasticsearch .common .util .concurrent .EsExecutors ;
39
45
import org .elasticsearch .index .mapper .DocumentMapper ;
40
- import org .elasticsearch .index .mapper .MapperService ;
41
- import org .elasticsearch .indices .IndicesService ;
46
+ import org .elasticsearch .node .settings .NodeSettingsService ;
42
47
import org .elasticsearch .threadpool .ThreadPool ;
43
48
import org .elasticsearch .transport .TransportService ;
44
49
45
50
import java .io .IOException ;
46
- import java .util .concurrent .CountDownLatch ;
51
+ import java .util .Collections ;
52
+ import java .util .List ;
53
+ import java .util .Set ;
54
+ import java .util .concurrent .BlockingQueue ;
47
55
import java .util .concurrent .TimeUnit ;
48
56
import java .util .concurrent .atomic .AtomicLong ;
49
57
53
61
*/
54
62
public class MappingUpdatedAction extends TransportMasterNodeOperationAction <MappingUpdatedAction .MappingUpdatedRequest , MappingUpdatedAction .MappingUpdatedResponse > {
55
63
64
+ public static final String INDICES_MAPPING_ADDITIONAL_MAPPING_CHANGE_TIME = "indices.mapping.additional_mapping_change_time" ;
65
+
56
66
private final AtomicLong mappingUpdateOrderGen = new AtomicLong ();
57
67
private final MetaDataMappingService metaDataMappingService ;
58
- private final IndicesService indicesService ;
59
68
60
- private final boolean waitForMappingChange ;
69
+ private volatile MasterMappingUpdater masterMappingUpdater ;
70
+
71
+ private volatile TimeValue additionalMappingChangeTime ;
72
+
73
+ class ApplySettings implements NodeSettingsService .Listener {
74
+ @ Override
75
+ public void onRefreshSettings (Settings settings ) {
76
+ final TimeValue current = MappingUpdatedAction .this .additionalMappingChangeTime ;
77
+ final TimeValue newValue = settings .getAsTime (INDICES_MAPPING_ADDITIONAL_MAPPING_CHANGE_TIME , current );
78
+ if (!current .equals (newValue )) {
79
+ logger .info ("updating " + INDICES_MAPPING_ADDITIONAL_MAPPING_CHANGE_TIME + " from [{}] to [{}]" , current , newValue );
80
+ MappingUpdatedAction .this .additionalMappingChangeTime = newValue ;
81
+ }
82
+ }
83
+ }
61
84
62
85
@ Inject
63
86
public MappingUpdatedAction (Settings settings , TransportService transportService , ClusterService clusterService , ThreadPool threadPool ,
64
- MetaDataMappingService metaDataMappingService , IndicesService indicesService ) {
87
+ MetaDataMappingService metaDataMappingService , NodeSettingsService nodeSettingsService ) {
65
88
super (settings , transportService , clusterService , threadPool );
66
89
this .metaDataMappingService = metaDataMappingService ;
67
- this .indicesService = indicesService ;
68
- this .waitForMappingChange = settings .getAsBoolean ("action.wait_on_mapping_change" , false );
90
+ // this setting should probably always be 0, just add the option to wait for more changes within a time window
91
+ this .additionalMappingChangeTime = settings .getAsTime (INDICES_MAPPING_ADDITIONAL_MAPPING_CHANGE_TIME , TimeValue .timeValueMillis (0 ));
92
+ nodeSettingsService .addListener (new ApplySettings ());
69
93
}
70
94
71
- public void updateMappingOnMaster (String index , String type , boolean neverWaitForMappingChange ) {
72
- IndexMetaData metaData = clusterService .state ().metaData ().index (index );
73
- if (metaData != null ) {
74
- updateMappingOnMaster (index , type , metaData .getUUID (), neverWaitForMappingChange );
75
- }
95
+ public void start () {
96
+ this .masterMappingUpdater = new MasterMappingUpdater (EsExecutors .threadName (settings , "master_mapping_updater" ));
97
+ this .masterMappingUpdater .start ();
76
98
}
77
99
78
- public void updateMappingOnMaster (String index , String type , String indexUUID , boolean neverWaitForMappingChange ) {
79
- final MapperService mapperService = indicesService .indexServiceSafe (index ).mapperService ();
80
- final DocumentMapper documentMapper = mapperService .documentMapper (type );
81
- if (documentMapper != null ) { // should not happen
82
- updateMappingOnMaster (documentMapper , index , type , indexUUID , neverWaitForMappingChange );
83
- }
100
+ public void stop () {
101
+ this .masterMappingUpdater .close ();
102
+ this .masterMappingUpdater = null ;
84
103
}
85
104
86
- public void updateMappingOnMaster (DocumentMapper documentMapper , String index , String type , String indexUUID , boolean neverWaitForMappingChange ) {
87
- final CountDownLatch latch = new CountDownLatch (1 );
88
- final MappingUpdatedAction .MappingUpdatedRequest mappingRequest ;
89
- try {
90
- // we generate the order id before we get the mapping to send and refresh the source, so
91
- // if 2 happen concurrently, we know that the later order will include the previous one
92
- long orderId = mappingUpdateOrderGen .incrementAndGet ();
93
- documentMapper .refreshSource ();
94
- DiscoveryNode node = clusterService .localNode ();
95
- mappingRequest = new MappingUpdatedAction .MappingUpdatedRequest (
96
- index , indexUUID , type , documentMapper .mappingSource (), orderId , node != null ? node .id () : null
97
- );
98
- } catch (Throwable t ) {
99
- logger .warn ("Failed to update master on updated mapping for index [" + index + "], type [" + type + "]" , t );
100
- latch .countDown ();
101
- throw t ;
102
- }
103
- logger .trace ("Sending mapping updated to master: {}" , mappingRequest );
104
- execute (mappingRequest , new ActionListener <MappingUpdatedAction .MappingUpdatedResponse >() {
105
- @ Override
106
- public void onResponse (MappingUpdatedAction .MappingUpdatedResponse mappingUpdatedResponse ) {
107
- // all is well
108
- latch .countDown ();
109
- logger .debug ("Successfully updated master with mapping update: {}" , mappingRequest );
110
- }
111
-
112
- @ Override
113
- public void onFailure (Throwable e ) {
114
- latch .countDown ();
115
- logger .warn ("Failed to update master on updated mapping for {}" , e , mappingRequest );
116
- }
117
- });
118
- if (waitForMappingChange && !neverWaitForMappingChange ) {
119
- try {
120
- latch .await (5 , TimeUnit .SECONDS );
121
- } catch (InterruptedException e ) {
122
- Thread .currentThread ().interrupt ();
123
- }
124
- }
105
+ public void updateMappingOnMaster (String index , DocumentMapper documentMapper , String indexUUID ) {
106
+ masterMappingUpdater .add (new MappingChange (documentMapper , index , indexUUID ));
125
107
}
126
108
127
109
@ Override
@@ -256,4 +238,105 @@ public String toString() {
256
238
return "index [" + index + "], indexUUID [" + indexUUID + "], type [" + type + "] and source [" + mappingSource + "]" ;
257
239
}
258
240
}
241
+
242
+ private static class MappingChange {
243
+ public final DocumentMapper documentMapper ;
244
+ public final String index ;
245
+ public final String indexUUID ;
246
+
247
+ MappingChange (DocumentMapper documentMapper , String index , String indexUUID ) {
248
+ this .documentMapper = documentMapper ;
249
+ this .index = index ;
250
+ this .indexUUID = indexUUID ;
251
+ }
252
+ }
253
+
254
+ /**
255
+ * The master mapping updater removes the overhead of refreshing the mapping (refreshSource) on the
256
+ * indexing thread.
257
+ * <p/>
258
+ * It also allows to reduce multiple mapping updates on the same index(UUID) and type into one update
259
+ * (refreshSource + sending to master), which allows to offload the number of times mappings are updated
260
+ * and sent to master for heavy single index requests that each introduce a new mapping, and when
261
+ * multiple shards exists on the same nodes, allowing to work on the index level in this case.
262
+ */
263
+ private class MasterMappingUpdater extends Thread {
264
+
265
+ private volatile boolean running = true ;
266
+ private final BlockingQueue <MappingChange > queue = ConcurrentCollections .newBlockingQueue ();
267
+
268
+ public MasterMappingUpdater (String name ) {
269
+ super (name );
270
+ }
271
+
272
+ public void add (MappingChange change ) {
273
+ queue .add (change );
274
+ }
275
+
276
+ public void close () {
277
+ running = false ;
278
+ this .interrupt ();
279
+ }
280
+
281
+ @ Override
282
+ public void run () {
283
+ while (running ) {
284
+ try {
285
+ MappingChange polledChange = queue .poll (10 , TimeUnit .MINUTES );
286
+ if (polledChange == null ) {
287
+ continue ;
288
+ }
289
+ List <MappingChange > changes = Lists .newArrayList (polledChange );
290
+ if (additionalMappingChangeTime .millis () > 0 ) {
291
+ Thread .sleep (additionalMappingChangeTime .millis ());
292
+ }
293
+ queue .drainTo (changes );
294
+ Collections .reverse (changes ); // process then in newest one to oldest
295
+ Set <Tuple <String , String >> seenIndexAndTypes = Sets .newHashSet ();
296
+ for (MappingChange change : changes ) {
297
+ Tuple <String , String > checked = Tuple .tuple (change .indexUUID , change .documentMapper .type ());
298
+ if (seenIndexAndTypes .contains (checked )) {
299
+ continue ;
300
+ }
301
+ seenIndexAndTypes .add (checked );
302
+
303
+ final MappingUpdatedAction .MappingUpdatedRequest mappingRequest ;
304
+ try {
305
+ // we generate the order id before we get the mapping to send and refresh the source, so
306
+ // if 2 happen concurrently, we know that the later order will include the previous one
307
+ long orderId = mappingUpdateOrderGen .incrementAndGet ();
308
+ change .documentMapper .refreshSource ();
309
+ DiscoveryNode node = clusterService .localNode ();
310
+ mappingRequest = new MappingUpdatedAction .MappingUpdatedRequest (
311
+ change .index , change .indexUUID , change .documentMapper .type (), change .documentMapper .mappingSource (), orderId , node != null ? node .id () : null
312
+ );
313
+ } catch (Throwable t ) {
314
+ logger .warn ("Failed to update master on updated mapping for index [" + change .index + "], type [" + change .documentMapper .type () + "]" , t );
315
+ continue ;
316
+ }
317
+ logger .trace ("sending mapping updated to master: {}" , mappingRequest );
318
+ execute (mappingRequest , new ActionListener <MappingUpdatedAction .MappingUpdatedResponse >() {
319
+ @ Override
320
+ public void onResponse (MappingUpdatedAction .MappingUpdatedResponse mappingUpdatedResponse ) {
321
+ logger .debug ("successfully updated master with mapping update: {}" , mappingRequest );
322
+ }
323
+
324
+ @ Override
325
+ public void onFailure (Throwable e ) {
326
+ logger .warn ("failed to update master on updated mapping for {}" , e , mappingRequest );
327
+ }
328
+ });
329
+
330
+ }
331
+ } catch (InterruptedException e ) {
332
+ // are we shutting down? continue and check
333
+ if (running ) {
334
+ logger .warn ("failed to process mapping updates" , e );
335
+ }
336
+ } catch (Throwable t ) {
337
+ logger .warn ("failed to process mapping updates" , t );
338
+ }
339
+ }
340
+ }
341
+ }
259
342
}
0 commit comments