-
Notifications
You must be signed in to change notification settings - Fork 24.7k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Push primary term to replication tracker #38044
Merged
jasontedor
merged 4 commits into
elastic:master
from
jasontedor:replication-tracker-primary-term
Jan 31, 2019
Merged
Changes from 1 commit
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
5d11aab
Push primary term to replication tracker
jasontedor 0895a5c
Merge branch 'master' into replication-tracker-primary-term
jasontedor 72e2e7b
Feedback
jasontedor 65060ed
Merge remote-tracking branch 'elastic/master' into replication-tracke…
jasontedor File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -199,7 +199,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl | |
protected volatile ShardRouting shardRouting; | ||
protected volatile IndexShardState state; | ||
private volatile long pendingPrimaryTerm; // see JavaDocs for getPendingPrimaryTerm | ||
private volatile long operationPrimaryTerm; | ||
protected final AtomicReference<Engine> currentEngineReference = new AtomicReference<>(); | ||
final EngineFactory engineFactory; | ||
|
||
|
@@ -308,7 +307,7 @@ public IndexShard( | |
final String aId = shardRouting.allocationId().getId(); | ||
this.globalCheckpointListeners = | ||
new GlobalCheckpointListeners(shardId, threadPool.executor(ThreadPool.Names.LISTENER), threadPool.scheduler(), logger); | ||
this.replicationTracker = | ||
final ReplicationTracker replicationTracker = | ||
new ReplicationTracker( | ||
shardId, | ||
aId, | ||
|
@@ -317,6 +316,7 @@ public IndexShard( | |
globalCheckpointListeners::globalCheckpointUpdated, | ||
threadPool::absoluteTimeInMillis, | ||
retentionLeaseSyncer); | ||
this.replicationTracker = replicationTracker; | ||
|
||
// the query cache is a node-level thing, however we want the most popular filters | ||
// to be computed on a per-shard basis | ||
|
@@ -336,8 +336,9 @@ public boolean shouldCache(Query query) { | |
} | ||
indexShardOperationPermits = new IndexShardOperationPermits(shardId, threadPool); | ||
searcherWrapper = indexSearcherWrapper; | ||
pendingPrimaryTerm = indexSettings.getIndexMetaData().primaryTerm(shardId.id()); | ||
operationPrimaryTerm = pendingPrimaryTerm; | ||
final long primaryTerm = indexSettings.getIndexMetaData().primaryTerm(shardId.id()); | ||
this.pendingPrimaryTerm = primaryTerm; | ||
replicationTracker.setOperationPrimaryTerm(primaryTerm); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can we set this in the constructor of ReplicationTracker? Then we would always have a proper operation term in ReplicationTracker. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I pushed 72e2e7b. |
||
refreshListeners = buildRefreshListeners(); | ||
lastSearcherAccess.set(threadPool.relativeTimeInMillis()); | ||
persistMetadata(path, indexSettings, shardRouting, null, logger); | ||
|
@@ -399,7 +400,7 @@ public long getPendingPrimaryTerm() { | |
|
||
/** Returns the primary term that is currently being used to assign to operations */ | ||
public long getOperationPrimaryTerm() { | ||
return this.operationPrimaryTerm; | ||
return replicationTracker.getOperationPrimaryTerm(); | ||
} | ||
|
||
/** | ||
|
@@ -508,7 +509,7 @@ public void updateShardState(final ShardRouting newRouting, | |
assert pendingPrimaryTerm == newPrimaryTerm : | ||
"shard term changed on primary. expected [" + newPrimaryTerm + "] but was [" + pendingPrimaryTerm + "]" + | ||
", current routing: " + currentRouting + ", new routing: " + newRouting; | ||
assert operationPrimaryTerm == newPrimaryTerm; | ||
assert getOperationPrimaryTerm() == newPrimaryTerm; | ||
try { | ||
replicationTracker.activatePrimaryMode(getLocalCheckpoint()); | ||
/* | ||
|
@@ -704,23 +705,23 @@ public Engine.IndexResult applyIndexOperationOnPrimary(long version, VersionType | |
boolean isRetry) | ||
throws IOException { | ||
assert versionType.validateVersionForWrites(version); | ||
return applyIndexOperation(getEngine(), UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, versionType, ifSeqNo, | ||
return applyIndexOperation(getEngine(), UNASSIGNED_SEQ_NO, getOperationPrimaryTerm(), version, versionType, ifSeqNo, | ||
ifPrimaryTerm, autoGeneratedTimestamp, isRetry, Engine.Operation.Origin.PRIMARY, sourceToParse); | ||
} | ||
|
||
public Engine.IndexResult applyIndexOperationOnReplica(long seqNo, long version, long autoGeneratedTimeStamp, | ||
boolean isRetry, SourceToParse sourceToParse) | ||
throws IOException { | ||
return applyIndexOperation(getEngine(), seqNo, operationPrimaryTerm, version, null, UNASSIGNED_SEQ_NO, 0, | ||
return applyIndexOperation(getEngine(), seqNo, getOperationPrimaryTerm(), version, null, UNASSIGNED_SEQ_NO, 0, | ||
autoGeneratedTimeStamp, isRetry, Engine.Operation.Origin.REPLICA, sourceToParse); | ||
} | ||
|
||
private Engine.IndexResult applyIndexOperation(Engine engine, long seqNo, long opPrimaryTerm, long version, | ||
@Nullable VersionType versionType, long ifSeqNo, long ifPrimaryTerm, | ||
long autoGeneratedTimeStamp, boolean isRetry, Engine.Operation.Origin origin, | ||
SourceToParse sourceToParse) throws IOException { | ||
assert opPrimaryTerm <= this.operationPrimaryTerm: "op term [ " + opPrimaryTerm + " ] > shard term [" + this.operationPrimaryTerm | ||
+ "]"; | ||
assert opPrimaryTerm <= getOperationPrimaryTerm() | ||
: "op term [ " + opPrimaryTerm + " ] > shard term [" + getOperationPrimaryTerm() + "]"; | ||
ensureWriteAllowed(origin); | ||
Engine.Index operation; | ||
try { | ||
|
@@ -783,13 +784,13 @@ private Engine.IndexResult index(Engine engine, Engine.Index index) throws IOExc | |
} | ||
|
||
public Engine.NoOpResult markSeqNoAsNoop(long seqNo, String reason) throws IOException { | ||
return markSeqNoAsNoop(getEngine(), seqNo, operationPrimaryTerm, reason, Engine.Operation.Origin.REPLICA); | ||
return markSeqNoAsNoop(getEngine(), seqNo, getOperationPrimaryTerm(), reason, Engine.Operation.Origin.REPLICA); | ||
} | ||
|
||
private Engine.NoOpResult markSeqNoAsNoop(Engine engine, long seqNo, long opPrimaryTerm, String reason, | ||
Engine.Operation.Origin origin) throws IOException { | ||
assert opPrimaryTerm <= this.operationPrimaryTerm : "op term [ " + opPrimaryTerm + " ] > shard term [" + this.operationPrimaryTerm | ||
+ "]"; | ||
assert opPrimaryTerm <= getOperationPrimaryTerm() | ||
: "op term [ " + opPrimaryTerm + " ] > shard term [" + getOperationPrimaryTerm() + "]"; | ||
long startTime = System.nanoTime(); | ||
ensureWriteAllowed(origin); | ||
final Engine.NoOp noOp = new Engine.NoOp(seqNo, opPrimaryTerm, origin, startTime, reason); | ||
|
@@ -805,31 +806,31 @@ private Engine.NoOpResult noOp(Engine engine, Engine.NoOp noOp) { | |
} | ||
|
||
public Engine.IndexResult getFailedIndexResult(Exception e, long version) { | ||
return new Engine.IndexResult(e, version, operationPrimaryTerm); | ||
return new Engine.IndexResult(e, version, getOperationPrimaryTerm()); | ||
} | ||
|
||
public Engine.DeleteResult getFailedDeleteResult(Exception e, long version) { | ||
return new Engine.DeleteResult(e, version, operationPrimaryTerm); | ||
return new Engine.DeleteResult(e, version, getOperationPrimaryTerm()); | ||
} | ||
|
||
public Engine.DeleteResult applyDeleteOperationOnPrimary(long version, String type, String id, VersionType versionType, | ||
long ifSeqNo, long ifPrimaryTerm) | ||
throws IOException { | ||
assert versionType.validateVersionForWrites(version); | ||
return applyDeleteOperation(getEngine(), UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, type, id, versionType, | ||
return applyDeleteOperation(getEngine(), UNASSIGNED_SEQ_NO, getOperationPrimaryTerm(), version, type, id, versionType, | ||
ifSeqNo, ifPrimaryTerm, Engine.Operation.Origin.PRIMARY); | ||
} | ||
|
||
public Engine.DeleteResult applyDeleteOperationOnReplica(long seqNo, long version, String type, String id) throws IOException { | ||
return applyDeleteOperation( | ||
getEngine(), seqNo, operationPrimaryTerm, version, type, id, null, UNASSIGNED_SEQ_NO, 0, Engine.Operation.Origin.REPLICA); | ||
getEngine(), seqNo, getOperationPrimaryTerm(), version, type, id, null, UNASSIGNED_SEQ_NO, 0, Engine.Operation.Origin.REPLICA); | ||
} | ||
|
||
private Engine.DeleteResult applyDeleteOperation(Engine engine, long seqNo, long opPrimaryTerm, long version, String type, String id, | ||
@Nullable VersionType versionType, long ifSeqNo, long ifPrimaryTerm, | ||
Engine.Operation.Origin origin) throws IOException { | ||
assert opPrimaryTerm <= this.operationPrimaryTerm : "op term [ " + opPrimaryTerm + " ] > shard term [" + this.operationPrimaryTerm | ||
+ "]"; | ||
assert opPrimaryTerm <= getOperationPrimaryTerm() | ||
: "op term [ " + opPrimaryTerm + " ] > shard term [" + getOperationPrimaryTerm() + "]"; | ||
ensureWriteAllowed(origin); | ||
// When there is a single type, the unique identifier is only composed of the _id, | ||
// so there is no way to differentiate foo#1 from bar#1. This is especially an issue | ||
|
@@ -845,7 +846,7 @@ private Engine.DeleteResult applyDeleteOperation(Engine engine, long seqNo, long | |
return new Engine.DeleteResult(update); | ||
} | ||
} catch (MapperParsingException | IllegalArgumentException | TypeMissingException e) { | ||
return new Engine.DeleteResult(e, version, operationPrimaryTerm, seqNo, false); | ||
return new Engine.DeleteResult(e, version, getOperationPrimaryTerm(), seqNo, false); | ||
} | ||
if (mapperService.resolveDocumentType(type).equals(mapperService.documentMapper().type()) == false) { | ||
// We should never get there due to the fact that we generate mapping updates on deletes, | ||
|
@@ -1272,7 +1273,7 @@ public void prepareForIndexRecovery() { | |
} | ||
|
||
public void trimOperationOfPreviousPrimaryTerms(long aboveSeqNo) { | ||
getEngine().trimOperationsFromTranslog(operationPrimaryTerm, aboveSeqNo); | ||
getEngine().trimOperationsFromTranslog(getOperationPrimaryTerm(), aboveSeqNo); | ||
} | ||
|
||
/** | ||
|
@@ -2382,7 +2383,7 @@ private EngineConfig newEngineConfig() { | |
Collections.singletonList(refreshListeners), | ||
Collections.singletonList(new RefreshMetricUpdater(refreshMetric)), | ||
indexSort, circuitBreakerService, replicationTracker, replicationTracker::getRetentionLeases, | ||
() -> operationPrimaryTerm, tombstoneDocSupplier()); | ||
() -> getOperationPrimaryTerm(), tombstoneDocSupplier()); | ||
} | ||
|
||
/** | ||
|
@@ -2462,7 +2463,7 @@ private <E extends Exception> void bumpPrimaryTerm(final long newPrimaryTerm, | |
@Nullable ActionListener<Releasable> combineWithAction) { | ||
assert Thread.holdsLock(mutex); | ||
assert newPrimaryTerm > pendingPrimaryTerm || (newPrimaryTerm >= pendingPrimaryTerm && combineWithAction != null); | ||
assert operationPrimaryTerm <= pendingPrimaryTerm; | ||
assert getOperationPrimaryTerm() <= pendingPrimaryTerm; | ||
final CountDownLatch termUpdated = new CountDownLatch(1); | ||
asyncBlockOperations(new ActionListener<Releasable>() { | ||
@Override | ||
|
@@ -2488,12 +2489,12 @@ private void innerFail(final Exception e) { | |
public void onResponse(final Releasable releasable) { | ||
final RunOnce releaseOnce = new RunOnce(releasable::close); | ||
try { | ||
assert operationPrimaryTerm <= pendingPrimaryTerm; | ||
assert getOperationPrimaryTerm() <= pendingPrimaryTerm; | ||
termUpdated.await(); | ||
// indexShardOperationPermits doesn't guarantee that async submissions are executed | ||
// in the order submitted. We need to guard against another term bump | ||
if (operationPrimaryTerm < newPrimaryTerm) { | ||
operationPrimaryTerm = newPrimaryTerm; | ||
if (getOperationPrimaryTerm() < newPrimaryTerm) { | ||
replicationTracker.setOperationPrimaryTerm(newPrimaryTerm); | ||
onBlocked.run(); | ||
} | ||
} catch (final Exception e) { | ||
|
@@ -2579,14 +2580,14 @@ private void innerAcquireReplicaOperationPermit(final long opPrimaryTerm, | |
final ActionListener<Releasable> operationListener = new ActionListener<Releasable>() { | ||
@Override | ||
public void onResponse(final Releasable releasable) { | ||
if (opPrimaryTerm < operationPrimaryTerm) { | ||
if (opPrimaryTerm < getOperationPrimaryTerm()) { | ||
releasable.close(); | ||
final String message = String.format( | ||
Locale.ROOT, | ||
"%s operation primary term [%d] is too old (current [%d])", | ||
shardId, | ||
opPrimaryTerm, | ||
operationPrimaryTerm); | ||
getOperationPrimaryTerm()); | ||
onPermitAcquired.onFailure(new IllegalStateException(message)); | ||
} else { | ||
assert assertReplicationTarget(); | ||
|
@@ -2647,7 +2648,7 @@ public void onFailure(final Exception e) { | |
} | ||
|
||
private boolean requirePrimaryTermUpdate(final long opPrimaryTerm, final boolean allPermits) { | ||
return (opPrimaryTerm > pendingPrimaryTerm) || (allPermits && opPrimaryTerm > operationPrimaryTerm); | ||
return (opPrimaryTerm > pendingPrimaryTerm) || (allPermits && opPrimaryTerm > getOperationPrimaryTerm()); | ||
} | ||
|
||
public int getActiveOperationsCount() { | ||
|
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
please move getter and setter further down. I understand this is your preferred coding style. However, it conflicts with the coding style we have adapted for all these classes and don't think this is the right time and place to switch things up.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I pushed 72e2e7b.