1919
2020package org .elasticsearch .action .index ;
2121
22+ import org .elasticsearch .ElasticsearchIllegalStateException ;
2223import org .elasticsearch .ExceptionsHelper ;
2324import org .elasticsearch .action .ActionListener ;
2425import org .elasticsearch .action .RoutingMissingException ;
4243import org .elasticsearch .common .settings .Settings ;
4344import org .elasticsearch .index .IndexService ;
4445import org .elasticsearch .index .engine .Engine ;
46+ import org .elasticsearch .index .mapper .MapperService ;
4547import org .elasticsearch .index .mapper .Mapping ;
4648import org .elasticsearch .index .mapper .SourceToParse ;
4749import org .elasticsearch .index .shard .IndexShard ;
5153import org .elasticsearch .threadpool .ThreadPool ;
5254import org .elasticsearch .transport .TransportService ;
5355
56+ import java .io .IOException ;
57+
5458/**
5559 * Performs the index operation.
5660 * <p/>
@@ -167,23 +171,6 @@ protected ShardIterator shards(ClusterState clusterState, InternalRequest reques
167171 .indexShards (clusterService .state (), request .concreteIndex (), request .request ().type (), request .request ().id (), request .request ().routing ());
168172 }
169173
170- private void applyMappingUpdate (IndexService indexService , String type , Mapping update ) throws Throwable {
171- // HACK: Rivers seem to have something specific that triggers potential
172- // deadlocks when doing concurrent indexing. So for now they keep the
173- // old behaviour of updating mappings locally first and then
174- // asynchronously notifying the master
175- // this can go away when rivers are removed
176- final String indexName = indexService .index ().name ();
177- final String indexUUID = indexService .indexUUID ();
178- if (indexName .equals (RiverIndexName .Conf .indexName (settings ))) {
179- indexService .mapperService ().merge (type , new CompressedString (update .toBytes ()), true );
180- mappingUpdatedAction .updateMappingOnMaster (indexName , indexUUID , type , update , null );
181- } else {
182- mappingUpdatedAction .updateMappingOnMasterSynchronously (indexName , indexUUID , type , update );
183- indexService .mapperService ().merge (type , new CompressedString (update .toBytes ()), true );
184- }
185- }
186-
187174 @ Override
188175 protected Tuple <IndexResponse , IndexRequest > shardOperationOnPrimary (ClusterState clusterState , PrimaryOperationRequest shardRequest ) throws Throwable {
189176 final IndexRequest request = shardRequest .request ;
@@ -206,19 +193,53 @@ protected Tuple<IndexResponse, IndexRequest> shardOperationOnPrimary(ClusterStat
206193
207194 if (request .opType () == IndexRequest .OpType .INDEX ) {
208195 Engine .Index index = indexShard .prepareIndex (sourceToParse , request .version (), request .versionType (), Engine .Operation .Origin .PRIMARY , request .canHaveDuplicates ());
209- if (index .parsedDoc ().dynamicMappingsUpdate () != null ) {
210- applyMappingUpdate (indexService , request .type (), index .parsedDoc ().dynamicMappingsUpdate ());
196+ Mapping update = index .parsedDoc ().dynamicMappingsUpdate ();
197+ if (update != null ) {
198+ final String indexName = indexService .index ().name ();
199+ if (indexName .equals (RiverIndexName .Conf .indexName (settings ))) {
200+ // With rivers, we have a chicken and egg problem if indexing
201+ // the _meta document triggers a mapping update. Because we would
202+ // like to validate the mapping update first, but on the other
203+ // hand putting the mapping would start the river, which expects
204+ // to find a _meta document
205+ // So we have no choice but to index first and send mappings afterwards
206+ MapperService mapperService = indexService .mapperService ();
207+ mapperService .merge (request .type (), new CompressedString (update .toBytes ()), true );
208+ indexShard .index (index );
209+ mappingUpdatedAction .updateMappingOnMasterAsynchronously (indexName , request .type (), update );
210+ } else {
211+ mappingUpdatedAction .updateMappingOnMasterSynchronously (indexName , request .type (), update );
212+ indexShard .index (index );
213+ }
214+ } else {
215+ indexShard .index (index );
211216 }
212- indexShard .index (index );
213217 version = index .version ();
214218 created = index .created ();
215219 } else {
216220 Engine .Create create = indexShard .prepareCreate (sourceToParse ,
217221 request .version (), request .versionType (), Engine .Operation .Origin .PRIMARY , request .canHaveDuplicates (), request .autoGeneratedId ());
218- if (create .parsedDoc ().dynamicMappingsUpdate () != null ) {
219- applyMappingUpdate (indexService , request .type (), create .parsedDoc ().dynamicMappingsUpdate ());
222+ Mapping update = create .parsedDoc ().dynamicMappingsUpdate ();
223+ if (update != null ) {
224+ final String indexName = indexService .index ().name ();
225+ if (indexName .equals (RiverIndexName .Conf .indexName (settings ))) {
226+ // With rivers, we have a chicken and egg problem if indexing
227+ // the _meta document triggers a mapping update. Because we would
228+ // like to validate the mapping update first, but on the other
229+ // hand putting the mapping would start the river, which expects
230+ // to find a _meta document
231+ // So we have no choice but to index first and send mappings afterwards
232+ MapperService mapperService = indexService .mapperService ();
233+ mapperService .merge (request .type (), new CompressedString (update .toBytes ()), true );
234+ indexShard .create (create );
235+ mappingUpdatedAction .updateMappingOnMasterAsynchronously (indexName , request .type (), update );
236+ } else {
237+ mappingUpdatedAction .updateMappingOnMasterSynchronously (indexName , request .type (), update );
238+ indexShard .create (create );
239+ }
240+ } else {
241+ indexShard .create (create );
220242 }
221- indexShard .create (create );
222243 version = create .version ();
223244 created = true ;
224245 }
@@ -239,17 +260,36 @@ protected Tuple<IndexResponse, IndexRequest> shardOperationOnPrimary(ClusterStat
239260 }
240261
241262 @ Override
242- protected void shardOperationOnReplica (ReplicaOperationRequest shardRequest ) {
243- IndexShard indexShard = indicesService .indexServiceSafe (shardRequest .shardId .getIndex ()).shardSafe (shardRequest .shardId .id ());
263+ protected void shardOperationOnReplica (ReplicaOperationRequest shardRequest ) throws IOException {
264+ IndexService indexService = indicesService .indexServiceSafe (shardRequest .shardId .getIndex ());
265+ IndexShard indexShard = indexService .shardSafe (shardRequest .shardId .id ());
244266 IndexRequest request = shardRequest .request ;
245267 SourceToParse sourceToParse = SourceToParse .source (SourceToParse .Origin .REPLICA , request .source ()).type (request .type ()).id (request .id ())
246268 .routing (request .routing ()).parent (request .parent ()).timestamp (request .timestamp ()).ttl (request .ttl ());
247269 if (request .opType () == IndexRequest .OpType .INDEX ) {
248270 Engine .Index index = indexShard .prepareIndex (sourceToParse , request .version (), request .versionType (), Engine .Operation .Origin .REPLICA , request .canHaveDuplicates ());
271+ if (index .parsedDoc ().dynamicMappingsUpdate () != null ) {
272+ if (indexService .index ().name ().equals (RiverIndexName .Conf .indexName (settings ))) {
273+ // mappings updates on the _river are not validated synchronously so we can't
274+ // assume they are here when indexing on a replica
275+ indexService .mapperService ().merge (request .type (), new CompressedString (index .parsedDoc ().dynamicMappingsUpdate ().toBytes ()), true );
276+ } else {
277+ throw new ElasticsearchIllegalStateException ("Index operations on replicas should not trigger dynamic mappings updates: [" + index .parsedDoc ().dynamicMappingsUpdate () + "]" );
278+ }
279+ }
249280 indexShard .index (index );
250281 } else {
251282 Engine .Create create = indexShard .prepareCreate (sourceToParse ,
252283 request .version (), request .versionType (), Engine .Operation .Origin .REPLICA , request .canHaveDuplicates (), request .autoGeneratedId ());
284+ if (create .parsedDoc ().dynamicMappingsUpdate () != null ) {
285+ if (indexService .index ().name ().equals (RiverIndexName .Conf .indexName (settings ))) {
286+ // mappings updates on the _river are not validated synchronously so we can't
287+ // assume they are here when indexing on a replica
288+ indexService .mapperService ().merge (request .type (), new CompressedString (create .parsedDoc ().dynamicMappingsUpdate ().toBytes ()), true );
289+ } else {
290+ throw new ElasticsearchIllegalStateException ("Index operations on replicas should not trigger dynamic mappings updates: [" + create .parsedDoc ().dynamicMappingsUpdate () + "]" );
291+ }
292+ }
253293 indexShard .create (create );
254294 }
255295 if (request .refresh ()) {
0 commit comments