19
19
package org .elasticsearch .index .shard ;
20
20
21
21
import org .apache .lucene .store .LockObtainFailedException ;
22
- import org .elasticsearch .core .internal .io .IOUtils ;
23
22
import org .elasticsearch .ExceptionsHelper ;
24
23
import org .elasticsearch .Version ;
25
24
import org .elasticsearch .action .ActionListener ;
42
41
import org .elasticsearch .cluster .routing .UnassignedInfo ;
43
42
import org .elasticsearch .cluster .service .ClusterService ;
44
43
import org .elasticsearch .common .CheckedRunnable ;
44
+ import org .elasticsearch .common .UUIDs ;
45
45
import org .elasticsearch .common .breaker .CircuitBreaker ;
46
46
import org .elasticsearch .common .bytes .BytesArray ;
47
47
import org .elasticsearch .common .lucene .uid .Versions ;
50
50
import org .elasticsearch .common .unit .ByteSizeValue ;
51
51
import org .elasticsearch .common .unit .TimeValue ;
52
52
import org .elasticsearch .common .xcontent .XContentType ;
53
+ import org .elasticsearch .core .internal .io .IOUtils ;
53
54
import org .elasticsearch .env .Environment ;
54
55
import org .elasticsearch .env .NodeEnvironment ;
55
56
import org .elasticsearch .env .ShardLock ;
102
103
import static org .elasticsearch .test .hamcrest .ElasticsearchAssertions .assertHitCount ;
103
104
import static org .elasticsearch .test .hamcrest .ElasticsearchAssertions .assertNoFailures ;
104
105
import static org .elasticsearch .test .hamcrest .ElasticsearchAssertions .assertNoSearchHits ;
106
+ import static org .hamcrest .Matchers .allOf ;
105
107
import static org .hamcrest .Matchers .containsString ;
106
108
import static org .hamcrest .Matchers .equalTo ;
107
109
import static org .hamcrest .Matchers .greaterThan ;
@@ -347,6 +349,7 @@ public void testMaybeFlush() throws Exception {
347
349
.setRefreshPolicy (randomBoolean () ? IMMEDIATE : NONE ).get ();
348
350
assertBusy (() -> { // this is async
349
351
assertFalse (shard .shouldPeriodicallyFlush ());
352
+ assertThat (shard .flushStats ().getPeriodic (), greaterThan (0L ));
350
353
});
351
354
assertEquals (0 , translog .stats ().getUncommittedOperations ());
352
355
translog .sync ();
@@ -444,8 +447,12 @@ public void testStressMaybeFlushOrRollTranslogGeneration() throws Exception {
444
447
if (flush ) {
445
448
final FlushStats flushStats = shard .flushStats ();
446
449
final long total = flushStats .getTotal ();
450
+ final long periodic = flushStats .getPeriodic ();
447
451
client ().prepareIndex ("test" , "test" , "1" ).setSource ("{}" , XContentType .JSON ).get ();
448
- check = () -> assertEquals (total + 1 , shard .flushStats ().getTotal ());
452
+ check = () -> {
453
+ assertThat (shard .flushStats ().getTotal (), equalTo (total + 1 ));
454
+ assertThat (shard .flushStats ().getPeriodic (), equalTo (periodic + 1 ));
455
+ };
449
456
} else {
450
457
final long generation = shard .getEngine ().getTranslog ().currentFileGeneration ();
451
458
client ().prepareIndex ("test" , "test" , "1" ).setSource ("{}" , XContentType .JSON ).get ();
@@ -461,6 +468,30 @@ public void testStressMaybeFlushOrRollTranslogGeneration() throws Exception {
461
468
check .run ();
462
469
}
463
470
471
+ public void testFlushStats () throws Exception {
472
+ final IndexService indexService = createIndex ("test" );
473
+ ensureGreen ();
474
+ Settings settings = Settings .builder ().put ("index.translog.flush_threshold_size" , "" + between (200 , 300 ) + "b" ).build ();
475
+ client ().admin ().indices ().prepareUpdateSettings ("test" ).setSettings (settings ).get ();
476
+ final int numDocs = between (10 , 100 );
477
+ for (int i = 0 ; i < numDocs ; i ++) {
478
+ client ().prepareIndex ("test" , "doc" , Integer .toString (i )).setSource ("{}" , XContentType .JSON ).get ();
479
+ }
480
+ // A flush stats may include the new total count but the old period count - assert eventually.
481
+ assertBusy (() -> {
482
+ final FlushStats flushStats = client ().admin ().indices ().prepareStats ("test" ).clear ().setFlush (true ).get ().getTotal ().flush ;
483
+ assertThat (flushStats .getPeriodic (), allOf (equalTo (flushStats .getTotal ()), greaterThan (0L )));
484
+ });
485
+ assertBusy (() -> assertThat (indexService .getShard (0 ).shouldPeriodicallyFlush (), equalTo (false )));
486
+ settings = Settings .builder ().put ("index.translog.flush_threshold_size" , (String ) null ).build ();
487
+ client ().admin ().indices ().prepareUpdateSettings ("test" ).setSettings (settings ).get ();
488
+
489
+ client ().prepareIndex ("test" , "doc" , UUIDs .randomBase64UUID ()).setSource ("{}" , XContentType .JSON ).get ();
490
+ client ().admin ().indices ().prepareFlush ("test" ).setForce (randomBoolean ()).setWaitIfOngoing (true ).get ();
491
+ final FlushStats flushStats = client ().admin ().indices ().prepareStats ("test" ).clear ().setFlush (true ).get ().getTotal ().flush ;
492
+ assertThat (flushStats .getTotal (), greaterThan (flushStats .getPeriodic ()));
493
+ }
494
+
464
495
public void testShardHasMemoryBufferOnTranslogRecover () throws Throwable {
465
496
createIndex ("test" );
466
497
ensureGreen ();
0 commit comments