-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-15213: provide the exact offset to QuorumController.replay #13643
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
Changes from all commits
3019459
395216e
9cc3d9b
fb681d3
1e6437a
226a197
9996d3a
0b56700
c660f5f
e6c8fcd
ca999fb
fb67f9d
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -77,6 +77,7 @@ | |
| import org.apache.kafka.common.utils.Time; | ||
| import org.apache.kafka.common.utils.Utils; | ||
| import org.apache.kafka.controller.errors.ControllerExceptions; | ||
| import org.apache.kafka.controller.errors.EventHandlerExceptionInfo; | ||
| import org.apache.kafka.controller.metrics.QuorumControllerMetrics; | ||
| import org.apache.kafka.metadata.BrokerHeartbeatReply; | ||
| import org.apache.kafka.metadata.BrokerRegistrationReply; | ||
|
|
@@ -457,44 +458,39 @@ private void handleEventEnd(String name, long startProcessingTimeNs) { | |
| controllerMetrics.updateEventQueueProcessingTime(NANOSECONDS.toMillis(deltaNs)); | ||
| } | ||
|
|
||
| private Throwable handleEventException(String name, | ||
| OptionalLong startProcessingTimeNs, | ||
| Throwable exception) { | ||
| if (!startProcessingTimeNs.isPresent() && | ||
| ControllerExceptions.isTimeoutException(exception)) { | ||
| // If the event never started, and the exception is a timeout, increment the timed | ||
| // out metric. | ||
| controllerMetrics.incrementOperationsTimedOut(); | ||
| private Throwable handleEventException( | ||
| String name, | ||
| OptionalLong startProcessingTimeNs, | ||
| Throwable exception | ||
| ) { | ||
| OptionalLong deltaUs; | ||
| if (startProcessingTimeNs.isPresent()) { | ||
| long endProcessingTime = time.nanoseconds(); | ||
| long deltaNs = endProcessingTime - startProcessingTimeNs.getAsLong(); | ||
| deltaUs = OptionalLong.of(MICROSECONDS.convert(deltaNs, NANOSECONDS)); | ||
| } else { | ||
| deltaUs = OptionalLong.empty(); | ||
| } | ||
| Throwable externalException = | ||
| ControllerExceptions.toExternalException(exception, () -> latestController()); | ||
| if (!startProcessingTimeNs.isPresent()) { | ||
| log.error("{}: unable to start processing because of {}. Reason: {}", name, | ||
| exception.getClass().getSimpleName(), exception.getMessage()); | ||
| return externalException; | ||
| EventHandlerExceptionInfo info = EventHandlerExceptionInfo. | ||
| fromInternal(exception, () -> latestController()); | ||
| int epoch = curClaimEpoch; | ||
| if (epoch == -1) { | ||
| epoch = lastCommittedEpoch; | ||
| } | ||
| long endProcessingTime = time.nanoseconds(); | ||
| long deltaNs = endProcessingTime - startProcessingTimeNs.getAsLong(); | ||
| long deltaUs = MICROSECONDS.convert(deltaNs, NANOSECONDS); | ||
| if (ControllerExceptions.isExpected(exception)) { | ||
| log.info("{}: failed with {} in {} us. Reason: {}", name, | ||
| exception.getClass().getSimpleName(), deltaUs, exception.getMessage()); | ||
| return externalException; | ||
| String failureMessage = info.failureMessage(epoch, deltaUs, | ||
| isActiveController(), lastCommittedOffset); | ||
| if (info.isTimeoutException() && (!deltaUs.isPresent())) { | ||
| controllerMetrics.incrementOperationsTimedOut(); | ||
| } | ||
| if (isActiveController()) { | ||
| nonFatalFaultHandler.handleFault(String.format("%s: failed with unexpected server " + | ||
| "exception %s at epoch %d in %d us. Renouncing leadership and reverting " + | ||
| "to the last committed offset %d.", | ||
| name, exception.getClass().getSimpleName(), curClaimEpoch, deltaUs, | ||
| lastCommittedOffset), exception); | ||
| renounce(); | ||
| if (info.isFault()) { | ||
| nonFatalFaultHandler.handleFault(name + ": " + failureMessage, exception); | ||
| } else { | ||
| nonFatalFaultHandler.handleFault(String.format("%s: failed with unexpected server " + | ||
| "exception %s in %d us. The controller is already in standby mode.", | ||
| name, exception.getClass().getSimpleName(), deltaUs), | ||
| exception); | ||
| log.info("{}: {}", name, failureMessage); | ||
| } | ||
| if (info.causesFailover() && isActiveController()) { | ||
| renounce(); | ||
| } | ||
| return externalException; | ||
| return info.effectiveExternalException(); | ||
| } | ||
|
|
||
| private long updateEventStartMetricsAndGetTime(OptionalLong eventCreatedTimeNs) { | ||
|
|
@@ -755,22 +751,28 @@ public Long apply(List<ApiMessageAndVersion> records) { | |
| // Start by trying to apply the record to our in-memory state. This should always | ||
| // succeed; if it does not, that's a fatal error. It is important to do this before | ||
| // scheduling the record for Raft replication. | ||
| int i = 1; | ||
| int recordIndex = 0; | ||
| for (ApiMessageAndVersion message : records) { | ||
| long recordOffset = prevEndOffset + 1 + recordIndex; | ||
| try { | ||
| replay(message.message(), Optional.empty(), prevEndOffset + records.size()); | ||
| replay(message.message(), Optional.empty(), recordOffset); | ||
| } catch (Throwable e) { | ||
| String failureMessage = String.format("Unable to apply %s record, which was " + | ||
| "%d of %d record(s) in the batch following last write offset %d.", | ||
| message.message().getClass().getSimpleName(), i, records.size(), | ||
| prevEndOffset); | ||
| String failureMessage = String.format("Unable to apply %s " + | ||
| "record at offset %d on active controller, from the " + | ||
| "batch with baseOffset %d", | ||
| message.message().getClass().getSimpleName(), | ||
| recordOffset, prevEndOffset + 1); | ||
| throw fatalFaultHandler.handleFault(failureMessage, e); | ||
| } | ||
| i++; | ||
| recordIndex++; | ||
| } | ||
| prevEndOffset = raftClient.scheduleAtomicAppend(controllerEpoch, records); | ||
| snapshotRegistry.getOrCreateSnapshot(prevEndOffset); | ||
| return prevEndOffset; | ||
| long nextEndOffset = prevEndOffset + recordIndex; | ||
| raftClient.scheduleAtomicAppend(controllerEpoch, | ||
| OptionalLong.of(prevEndOffset + 1), | ||
| records); | ||
| snapshotRegistry.getOrCreateSnapshot(nextEndOffset); | ||
|
mumrah marked this conversation as resolved.
|
||
| prevEndOffset = nextEndOffset; | ||
| return nextEndOffset; | ||
| } | ||
| }); | ||
| op.processBatchEndOffset(offset); | ||
|
|
@@ -988,18 +990,20 @@ public void handleCommit(BatchReader<ApiMessageAndVersion> reader) { | |
| log.debug("Replaying commits from the active node up to " + | ||
| "offset {} and epoch {}.", offset, epoch); | ||
| } | ||
| int i = 1; | ||
| int recordIndex = 0; | ||
| for (ApiMessageAndVersion message : messages) { | ||
| long recordOffset = batch.baseOffset() + recordIndex; | ||
| try { | ||
| replay(message.message(), Optional.empty(), offset); | ||
| replay(message.message(), Optional.empty(), recordOffset); | ||
| } catch (Throwable e) { | ||
| String failureMessage = String.format("Unable to apply %s record on standby " + | ||
| "controller, which was %d of %d record(s) in the batch with baseOffset %d.", | ||
| message.message().getClass().getSimpleName(), i, messages.size(), | ||
| batch.baseOffset()); | ||
| String failureMessage = String.format("Unable to apply %s " + | ||
| "record at offset %d on standby controller, from the " + | ||
| "batch with baseOffset %d", | ||
| message.message().getClass().getSimpleName(), | ||
| recordOffset, batch.baseOffset()); | ||
| throw fatalFaultHandler.handleFault(failureMessage, e); | ||
| } | ||
| i++; | ||
| recordIndex++; | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -1008,13 +1012,6 @@ public void handleCommit(BatchReader<ApiMessageAndVersion> reader) { | |
| epoch, | ||
| batch.appendTimestamp() | ||
| ); | ||
|
|
||
| if (offset >= raftClient.latestSnapshotId().map(OffsetAndEpoch::offset).orElse(0L)) { | ||
| oldestNonSnapshottedTimestamp = Math.min( | ||
| oldestNonSnapshottedTimestamp, | ||
| batch.appendTimestamp() | ||
| ); | ||
| } | ||
| } | ||
| } finally { | ||
| reader.close(); | ||
|
|
@@ -1094,10 +1091,10 @@ public void handleLeaderChange(LeaderAndEpoch newLeader) { | |
| renounce(); | ||
| } | ||
| } else if (newLeader.isLeader(nodeId)) { | ||
| log.info("Becoming the active controller at epoch {}, committed offset {}, " + | ||
| "committed epoch {}", newLeader.epoch(), lastCommittedOffset, | ||
| lastCommittedEpoch); | ||
| claim(newLeader.epoch()); | ||
| long newLastWriteOffset = raftClient.logEndOffset() - 1; | ||
| log.info("Becoming the active controller at epoch {}, last write offset {}.", | ||
| newLeader.epoch(), newLastWriteOffset); | ||
| claim(newLeader.epoch(), newLastWriteOffset); | ||
| } else { | ||
| log.info("In the new epoch {}, the leader is {}.", | ||
| newLeader.epoch(), newLeaderName); | ||
|
|
@@ -1168,15 +1165,15 @@ private void updateWriteOffset(long offset) { | |
| } | ||
| } | ||
|
|
||
| private void claim(int epoch) { | ||
| private void claim(int epoch, long newLastWriteOffset) { | ||
| try { | ||
| if (curClaimEpoch != -1) { | ||
| throw new RuntimeException("Cannot claim leadership because we are already the " + | ||
| "active controller."); | ||
| } | ||
| curClaimEpoch = epoch; | ||
| controllerMetrics.setActive(true); | ||
| updateWriteOffset(lastCommittedOffset); | ||
| updateWriteOffset(newLastWriteOffset); | ||
|
Comment on lines
-1179
to
+1176
Member
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. Previously, we would update Now, while we're processing a leader change, we ask RaftClient for its end offset. Is there any possibility that commits could be made that would make this end offset greater than we expect? Basically, can we be sure that the end offset doesn't change between the time Raft becomes the leader and this event is processed?
Contributor
Author
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. Well, there is not really any difference between the previous iterations of this PR and the current one in this regard. If some other component that isn't the controller is adding messages, our supplied
Member
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. Understood, I just wanted to make sure I understood the behavior here regarding the new RaftClient API. It sounds like it's fine for us as long as we're single writer. |
||
| clusterControl.activate(); | ||
|
|
||
| // Before switching to active, create an in-memory snapshot at the last committed | ||
|
|
@@ -1516,25 +1513,24 @@ private void handleFeatureControlChange() { | |
| * | ||
| * @param message The metadata record | ||
| * @param snapshotId The snapshotId if this record is from a snapshot | ||
| * @param batchLastOffset The offset of the last record in the log batch, or the lastContainedLogOffset | ||
| * if this record is from a snapshot, this is used along with RegisterBrokerRecord | ||
| * @param offset The offset of the record | ||
| */ | ||
| private void replay(ApiMessage message, Optional<OffsetAndEpoch> snapshotId, long batchLastOffset) { | ||
| private void replay(ApiMessage message, Optional<OffsetAndEpoch> snapshotId, long offset) { | ||
| if (log.isTraceEnabled()) { | ||
| if (snapshotId.isPresent()) { | ||
| log.trace("Replaying snapshot {} record {}", | ||
| Snapshots.filenameFromSnapshotId(snapshotId.get()), | ||
| recordRedactor.toLoggableString(message)); | ||
| } else { | ||
| log.trace("Replaying log record {} with batchLastOffset {}", | ||
| recordRedactor.toLoggableString(message), batchLastOffset); | ||
| log.trace("Replaying log record {} with offset {}", | ||
| recordRedactor.toLoggableString(message), offset); | ||
| } | ||
| } | ||
| logReplayTracker.replay(message); | ||
| MetadataRecordType type = MetadataRecordType.fromId(message.apiKey()); | ||
| switch (type) { | ||
| case REGISTER_BROKER_RECORD: | ||
| clusterControl.replay((RegisterBrokerRecord) message, batchLastOffset); | ||
| clusterControl.replay((RegisterBrokerRecord) message, offset); | ||
| break; | ||
| case UNREGISTER_BROKER_RECORD: | ||
| clusterControl.replay((UnregisterBrokerRecord) message); | ||
|
|
@@ -1767,11 +1763,6 @@ private void resetToEmptyState() { | |
| */ | ||
| private long writeOffset; | ||
|
|
||
| /** | ||
| * Timestamp for the oldest record that was committed but not included in a snapshot. | ||
| */ | ||
| private long oldestNonSnapshottedTimestamp = Long.MAX_VALUE; | ||
|
|
||
| /** | ||
| * How long to delay partition leader balancing operations. | ||
| */ | ||
|
|
@@ -2328,4 +2319,11 @@ Time time() { | |
| QuorumControllerMetrics controllerMetrics() { | ||
| return controllerMetrics; | ||
| } | ||
|
|
||
| // VisibleForTesting | ||
| void setWriteOffset(long newWriteOffset) { | ||
| appendControlEvent("setWriteOffset", () -> { | ||
| this.writeOffset = newWriteOffset; | ||
| }); | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.