Skip to content

Commit 493cd16

Browse files
committed
Rename methods
1 parent c43177e commit 493cd16

File tree

4 files changed

+14
-14
lines changed

4 files changed

+14
-14
lines changed

server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -355,12 +355,12 @@ protected void doRun() throws Exception {
355355
}
356356

357357
acquirePrimaryOperationPermit(indexShard, request, ActionListener.wrap(
358-
releasable -> runWithReleasable(new PrimaryShardReference(indexShard, releasable)),
358+
releasable -> runWithPrimaryShardReference(new PrimaryShardReference(indexShard, releasable)),
359359
this::onFailure
360360
));
361361
}
362362

363-
void runWithReleasable(final PrimaryShardReference primaryShardReference) {
363+
void runWithPrimaryShardReference(final PrimaryShardReference primaryShardReference) {
364364
try {
365365
final ClusterState clusterState = clusterService.state();
366366
final IndexMetaData indexMetaData = clusterState.metaData().getIndexSafe(primaryShardReference.routingEntry().index());

server/src/main/java/org/elasticsearch/index/shard/IndexShard.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -2306,9 +2306,9 @@ public void acquirePrimaryOperationPermit(ActionListener<Releasable> onPermitAcq
23062306
* Acquire all primary operation permits. Once all permits are acquired, the provided ActionListener is called.
23072307
* It is the responsibility of the caller to close the {@link Releasable}.
23082308
*/
2309-
public void acquirePrimaryAllOperationsPermits(final ActionListener<Releasable> onPermitAcquired, final TimeValue timeout) {
2309+
public void acquireAllPrimaryOperationsPermits(final ActionListener<Releasable> onPermitAcquired, final TimeValue timeout) {
23102310
verifyNotClosed();
2311-
assert shardRouting.primary() : "acquirePrimaryAllOperationsPermits should only be called on primary shard: " + shardRouting;
2311+
assert shardRouting.primary() : "acquireAllPrimaryOperationsPermits should only be called on primary shard: " + shardRouting;
23122312

23132313
indexShardOperationPermits.asyncBlockOperations(onPermitAcquired, timeout.duration(), timeout.timeUnit());
23142314
}
@@ -2458,7 +2458,7 @@ public void acquireReplicaOperationPermit(final long opPrimaryTerm,
24582458

24592459
/**
24602460
* Acquire all replica operation permits whenever the shard is ready for indexing (see
2461-
* {@link #acquirePrimaryAllOperationsPermits(ActionListener, TimeValue)}. If the given primary term is lower than then one in
2461+
* {@link #acquireAllPrimaryOperationsPermits(ActionListener, TimeValue)}. If the given primary term is lower than then one in
24622462
* {@link #shardRouting}, the {@link ActionListener#onFailure(Exception)} method of the provided listener is invoked with an
24632463
* {@link IllegalStateException}.
24642464
*

server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationAllPermitsAcquisitionTests.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -208,11 +208,11 @@ protected void doRun() throws Exception {
208208
}
209209

210210
@Override
211-
void runWithReleasable(final TransportReplicationAction.PrimaryShardReference reference) {
211+
void runWithPrimaryShardReference(final TransportReplicationAction.PrimaryShardReference reference) {
212212
assertThat(reference.indexShard.getActiveOperationsCount(), greaterThan(0));
213213
assertSame(primary, reference.indexShard);
214214
assertBlockIsPresentForDelayedOp();
215-
super.runWithReleasable(reference);
215+
super.runWithPrimaryShardReference(reference);
216216
}
217217

218218
@Override
@@ -250,7 +250,7 @@ private void assertBlockIsPresentForDelayedOp() {
250250
TransportReplicationAction.AsyncPrimaryAction asyncPrimaryAction =
251251
allPermitsAction.new AsyncPrimaryAction(request(), allocationId(), primaryTerm(), transportChannel(allPermitFuture), null) {
252252
@Override
253-
void runWithReleasable(final TransportReplicationAction.PrimaryShardReference reference) {
253+
void runWithPrimaryShardReference(final TransportReplicationAction.PrimaryShardReference reference) {
254254
assertEquals("All permits must be acquired", 0, reference.indexShard.getActiveOperationsCount());
255255
assertSame(primary, reference.indexShard);
256256

@@ -274,7 +274,7 @@ void runWithReleasable(final TransportReplicationAction.PrimaryShardReference re
274274
} catch (InterruptedException | BrokenBarrierException e) {
275275
onFailure(e);
276276
}
277-
super.runWithReleasable(reference);
277+
super.runWithPrimaryShardReference(reference);
278278
}
279279
};
280280
asyncPrimaryAction.run();
@@ -434,7 +434,7 @@ protected void acquirePrimaryOperationPermit(IndexShard shard, Request request,
434434
assertTrue(shard.routingEntry().primary());
435435
assertSame(primary, shard);
436436
if (acquireAllPermits) {
437-
shard.acquirePrimaryAllOperationsPermits(onAcquired, timeout);
437+
shard.acquireAllPrimaryOperationsPermits(onAcquired, timeout);
438438
} else {
439439
super.acquirePrimaryOperationPermit(shard, request, onAcquired);
440440
}

server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -313,7 +313,7 @@ public void testClosesPreventsNewOperations() throws Exception {
313313
expectThrows(IndexShardClosedException.class,
314314
() -> indexShard.acquirePrimaryOperationPermit(null, ThreadPool.Names.WRITE, ""));
315315
expectThrows(IndexShardClosedException.class,
316-
() -> indexShard.acquirePrimaryAllOperationsPermits(null, TimeValue.timeValueSeconds(30L)));
316+
() -> indexShard.acquireAllPrimaryOperationsPermits(null, TimeValue.timeValueSeconds(30L)));
317317
expectThrows(IndexShardClosedException.class,
318318
() -> indexShard.acquireReplicaOperationPermit(indexShard.getPendingPrimaryTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO,
319319
randomNonNegativeLong(), null, ThreadPool.Names.WRITE, ""));
@@ -665,7 +665,7 @@ public void onFailure(final Exception e) {
665665
if (singlePermit) {
666666
indexShard.acquirePrimaryOperationPermit(future, ThreadPool.Names.WRITE, "");
667667
} else {
668-
indexShard.acquirePrimaryAllOperationsPermits(future, TimeValue.timeValueHours(1L));
668+
indexShard.acquireAllPrimaryOperationsPermits(future, TimeValue.timeValueHours(1L));
669669
}
670670
assertEquals(0, indexShard.getActiveOperationsCount());
671671
});
@@ -688,7 +688,7 @@ public void onResponse(final Releasable releasable) {
688688
}
689689
}
690690
};
691-
indexShard.acquirePrimaryAllOperationsPermits(futureAllPermits, TimeValue.timeValueSeconds(30L));
691+
indexShard.acquireAllPrimaryOperationsPermits(futureAllPermits, TimeValue.timeValueSeconds(30L));
692692
allPermitsAcquired.await();
693693
assertTrue(blocked.get());
694694
assertEquals(0, indexShard.getActiveOperationsCount());
@@ -781,7 +781,7 @@ public void testOperationPermitOnReplicaShards() throws Exception {
781781
assertThat(e, hasToString(containsString("acquirePrimaryOperationPermit should only be called on primary shard")));
782782

783783
e = expectThrows(AssertionError.class,
784-
() -> indexShard.acquirePrimaryAllOperationsPermits(null, TimeValue.timeValueSeconds(30L)));
784+
() -> indexShard.acquireAllPrimaryOperationsPermits(null, TimeValue.timeValueSeconds(30L)));
785785
assertThat(e, hasToString(containsString("acquirePrimaryAllOperationsPermits should only be called on primary shard")));
786786
}
787787

0 commit comments

Comments
 (0)