66import org .elasticsearch .action .admin .indices .mapping .get .GetMappingsResponse ;
77import org .elasticsearch .action .admin .indices .mapping .put .PutMappingResponse ;
88import org .elasticsearch .action .admin .indices .refresh .RefreshResponse ;
9+ import org .elasticsearch .action .bulk .BulkRequestBuilder ;
10+ import org .elasticsearch .action .bulk .BulkResponse ;
911import org .elasticsearch .action .count .CountResponse ;
1012import org .elasticsearch .action .get .GetResponse ;
13+ import org .elasticsearch .action .index .IndexRequest ;
14+ import org .elasticsearch .action .index .IndexRequestBuilder ;
1115import org .elasticsearch .action .index .IndexResponse ;
1216import org .elasticsearch .client .Client ;
1317import org .elasticsearch .cluster .metadata .MappingMetaData ;
1923import org .elasticsearch .index .mapper .MapperService ;
2024import org .elasticsearch .index .mapper .MergeMappingException ;
2125import org .elasticsearch .test .ElasticsearchIntegrationTest ;
26+ import org .elasticsearch .threadpool .ThreadPool ;
2227import org .hamcrest .Matchers ;
2328import org .junit .Test ;
2429
3641public class UpdateMappingTests extends ElasticsearchIntegrationTest {
3742
3843 @ Test
39- public void dynamicUpdates () throws Exception {
44+ public void dynamicUpdates_Index () throws Exception {
4045 client ().admin ().indices ().prepareCreate ("test" )
4146 .setSettings (
4247 ImmutableSettings .settingsBuilder ()
@@ -45,7 +50,8 @@ public void dynamicUpdates() throws Exception {
4550 ).execute ().actionGet ();
4651 client ().admin ().cluster ().prepareHealth ().setWaitForEvents (Priority .LANGUID ).setWaitForGreenStatus ().execute ().actionGet ();
4752
48- int recCount = 200 ;
53+ ThreadPool .Info info = cluster ().getInstance (ThreadPool .class ).info (ThreadPool .Names .INDEX );
54+ int recCount = info .getMax () + (int ) info .getQueueSize ().getSingles ();
4955 final CountDownLatch latch = new CountDownLatch (recCount );
5056 for (int rec = 0 ; rec < recCount ; rec ++) {
5157 client ().prepareIndex ("test" , "type" , "rec" + rec ).setSource ("field" + rec , "some_value" ).execute (new ActionListener <IndexResponse >() {
@@ -84,6 +90,73 @@ public boolean apply(Object input) {
8490 }
8591 }
8692
93+ @ Test
94+ public void dynamicUpdates_Bulk () throws Exception {
95+ client ().admin ().indices ().prepareCreate ("test" )
96+ .setSettings (
97+ ImmutableSettings .settingsBuilder ()
98+ .put ("index.number_of_shards" , 1 )
99+ .put ("index.number_of_replicas" , 0 )
100+ ).execute ().actionGet ();
101+ client ().admin ().cluster ().prepareHealth ().setWaitForEvents (Priority .LANGUID ).setWaitForGreenStatus ().execute ().actionGet ();
102+
103+ ThreadPool .Info info = cluster ().getInstance (ThreadPool .class ).info (ThreadPool .Names .BULK );
104+ int bulkCount = info .getMax () + (int ) info .getQueueSize ().getSingles ();
105+ int bulkSize = between (4 , 10 );
106+ int recCount = bulkCount * bulkSize ;
107+ int idCounter = 0 ;
108+ final CountDownLatch latch = new CountDownLatch (bulkCount );
109+ for (int i = 0 ; i < bulkCount ; i ++) {
110+ BulkRequestBuilder bulk = client ().prepareBulk ();
111+ for (int rec = 0 ; rec < bulkSize ; rec ++) {
112+ int id = idCounter ++;
113+ bulk .add (new IndexRequestBuilder (client ())
114+ .setOpType (IndexRequest .OpType .INDEX )
115+ .setIndex ("test" )
116+ .setType ("type" )
117+ .setId ("rec" + id )
118+ .setSource ("field" + id , "some_value" ));
119+ }
120+ bulk .execute (new ActionListener <BulkResponse >() {
121+ @ Override
122+ public void onResponse (BulkResponse bulkItemResponses ) {
123+ if (bulkItemResponses .hasFailures ()) {
124+ System .out .println ("failed to index in test: " + bulkItemResponses .buildFailureMessage ());
125+ }
126+ latch .countDown ();
127+ }
128+
129+ @ Override
130+ public void onFailure (Throwable e ) {
131+ logger .error ("failed to index in test" , e );
132+ latch .countDown ();
133+ }
134+ });
135+ }
136+ latch .await ();
137+
138+ logger .info ("wait till the mappings have been processed..." );
139+ awaitBusy (new Predicate <Object >() {
140+ @ Override
141+ public boolean apply (Object input ) {
142+ PendingClusterTasksResponse pendingTasks = client ().admin ().cluster ().preparePendingClusterTasks ().get ();
143+ return pendingTasks .pendingTasks ().isEmpty ();
144+ }
145+ });
146+
147+ logger .info ("checking all the documents are there" );
148+ RefreshResponse refreshResponse = client ().admin ().indices ().prepareRefresh ().execute ().actionGet ();
149+ assertThat (refreshResponse .getFailedShards (), equalTo (0 ));
150+ CountResponse response = client ().prepareCount ("test" ).execute ().actionGet ();
151+ assertThat (response .getCount (), equalTo ((long ) recCount ));
152+
153+ logger .info ("checking all the fields are in the mappings" );
154+ String source = client ().admin ().cluster ().prepareState ().get ().getState ().getMetaData ().getIndices ().get ("test" ).getMappings ().get ("type" ).source ().string ();
155+ for (int rec = 0 ; rec < recCount ; rec ++) {
156+ assertThat (source , containsString ("\" field" + rec + "\" " ));
157+ }
158+ }
159+
87160 @ Test (expected = MergeMappingException .class )
88161 public void updateMappingWithConflicts () throws Exception {
89162
0 commit comments