diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh
index 8883dd03317a..cb5cdff3603c 100755
--- a/bin/kafka-run-class.sh
+++ b/bin/kafka-run-class.sh
@@ -48,7 +48,7 @@ should_include_file() {
base_dir=$(dirname $0)/..
if [ -z "$SCALA_VERSION" ]; then
- SCALA_VERSION=2.13.6
+ SCALA_VERSION=2.13.8
if [[ -f "$base_dir/gradle.properties" ]]; then
SCALA_VERSION=`grep "^scalaVersion=" "$base_dir/gradle.properties" | cut -d= -f 2`
fi
diff --git a/bin/windows/kafka-run-class.bat b/bin/windows/kafka-run-class.bat
index 58f0921496bd..4855dbfc1c8c 100755
--- a/bin/windows/kafka-run-class.bat
+++ b/bin/windows/kafka-run-class.bat
@@ -27,7 +27,7 @@ set BASE_DIR=%CD%
popd
IF ["%SCALA_VERSION%"] EQU [""] (
- set SCALA_VERSION=2.13.6
+ set SCALA_VERSION=2.13.8
)
IF ["%SCALA_BINARY_VERSION%"] EQU [""] (
diff --git a/build.gradle b/build.gradle
index e1270bf62c43..bb5369f74562 100644
--- a/build.gradle
+++ b/build.gradle
@@ -34,11 +34,11 @@ plugins {
id 'com.github.ben-manes.versions' version '0.42.0'
id 'idea'
id 'java-library'
- id 'org.owasp.dependencycheck' version '7.0.3'
- id 'org.nosphere.apache.rat' version "0.7.0"
+ id 'org.owasp.dependencycheck' version '7.1.1'
+ id 'org.nosphere.apache.rat' version "0.7.1"
- id "com.github.spotbugs" version '5.0.6' apply false
- id 'org.gradle.test-retry' version '1.3.1' apply false
+ id "com.github.spotbugs" version '5.0.9' apply false
+ id 'org.gradle.test-retry' version '1.4.0' apply false
id 'org.scoverage' version '7.0.0' apply false
id 'com.github.johnrengelman.shadow' version '7.1.2' apply false
id "io.swagger.core.v3.swagger-gradle-plugin" version "2.2.0"
@@ -928,6 +928,7 @@ project(':core') {
testImplementation project(':clients').sourceSets.test.output
testImplementation project(':metadata').sourceSets.test.output
testImplementation project(':raft').sourceSets.test.output
+ testImplementation project(':server-common').sourceSets.test.output
testImplementation libs.bcpkix
testImplementation libs.mockitoCore
testImplementation(libs.apacheda) {
@@ -1210,6 +1211,7 @@ project(':metadata') {
testImplementation libs.slf4jlog4j
testImplementation project(':clients').sourceSets.test.output
testImplementation project(':raft').sourceSets.test.output
+ testImplementation project(':server-common').sourceSets.test.output
generator project(':generator')
}
@@ -1648,6 +1650,8 @@ project(':storage') {
testImplementation project(':clients').sourceSets.test.output
testImplementation project(':core')
testImplementation project(':core').sourceSets.test.output
+ testImplementation project(':server-common')
+ testImplementation project(':server-common').sourceSets.test.output
testImplementation libs.junitJupiter
testImplementation libs.mockitoCore
testImplementation libs.bcpkix
diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml
index 28b325b093da..4042cba402fd 100644
--- a/checkstyle/import-control-core.xml
+++ b/checkstyle/import-control-core.xml
@@ -54,6 +54,7 @@
+
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 211d23ff60ae..4b07a26cba5c 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -232,6 +232,7 @@
+
@@ -276,6 +277,9 @@
+
+
+
diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index f6ca0d02fe31..bec3da1637a9 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -39,6 +39,8 @@
+
onStateChange = createStrictMock(Callback.class);
- onStateChange.onCompletion(EasyMock.anyObject(Exception.class), EasyMock.isNull());
- expectLastCall();
-
- replayAll();
-
- WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader);
+ Callback onStateChange = mockCallback();
+ WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, null, null, classLoader);
workerConnector.initialize();
assertFailedMetric(workerConnector);
@@ -167,48 +129,22 @@ public void testFailureIsFinalState() {
workerConnector.doShutdown();
assertStoppedMetric(workerConnector);
- verifyAll();
+ verifyInitialize();
+ verify(listener).onFailure(CONNECTOR, exception);
+ // expect no call to onStartup() after failure
+ verifyCleanShutdown(false);
+
+ verify(onStateChange).onCompletion(any(Exception.class), isNull());
+ verifyNoMoreInteractions(onStateChange);
}
@Test
public void testStartupAndShutdown() {
connector = sourceConnector;
- connector.version();
- expectLastCall().andReturn(VERSION);
-
- offsetStore.start();
- expectLastCall();
-
- connector.initialize(EasyMock.notNull(SourceConnectorContext.class));
- expectLastCall();
-
- connector.start(CONFIG);
- expectLastCall();
-
- listener.onStartup(CONNECTOR);
- expectLastCall();
-
- connector.stop();
- expectLastCall();
-
- listener.onShutdown(CONNECTOR);
- expectLastCall();
-
- ctx.close();
- expectLastCall();
-
- offsetStorageReader.close();
- expectLastCall();
-
- offsetStore.stop();
- expectLastCall();
- Callback onStateChange = createStrictMock(Callback.class);
- onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.STARTED));
- expectLastCall();
-
- replayAll();
+ when(connector.version()).thenReturn(VERSION);
+ Callback onStateChange = mockCallback();
WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader);
workerConnector.initialize();
@@ -219,54 +155,26 @@ public void testStartupAndShutdown() {
workerConnector.doShutdown();
assertStoppedMetric(workerConnector);
- verifyAll();
+ verifyInitialize();
+ verify(connector).start(CONFIG);
+ verify(listener).onStartup(CONNECTOR);
+ verifyCleanShutdown(true);
+
+ verify(onStateChange).onCompletion(isNull(), eq(TargetState.STARTED));
+ verifyNoMoreInteractions(onStateChange);
}
@Test
public void testStartupAndPause() {
connector = sinkConnector;
- connector.version();
- expectLastCall().andReturn(VERSION);
-
- connector.initialize(EasyMock.notNull(SinkConnectorContext.class));
- expectLastCall();
-
- connector.start(CONFIG);
- expectLastCall();
-
- listener.onStartup(CONNECTOR);
- expectLastCall();
-
- connector.stop();
- expectLastCall();
-
- listener.onPause(CONNECTOR);
- expectLastCall();
-
- listener.onShutdown(CONNECTOR);
- expectLastCall();
-
- ctx.close();
- expectLastCall();
+ when(connector.version()).thenReturn(VERSION);
- offsetStorageReader.close();
- expectLastCall();
-
- offsetStore.stop();
- expectLastCall();
-
- Callback onStateChange = createStrictMock(Callback.class);
- onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.STARTED));
- expectLastCall();
- onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.PAUSED));
- expectLastCall();
-
- replayAll();
-
- WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader);
+ Callback onStateChange = mockCallback();
+ WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, null, null, classLoader);
workerConnector.initialize();
assertInitializedSinkMetric(workerConnector);
+
workerConnector.doTransitionTo(TargetState.STARTED, onStateChange);
assertRunningMetric(workerConnector);
workerConnector.doTransitionTo(TargetState.PAUSED, onStateChange);
@@ -275,52 +183,25 @@ public void testStartupAndPause() {
workerConnector.doShutdown();
assertStoppedMetric(workerConnector);
- verifyAll();
+ verifyInitialize();
+ verify(connector).start(CONFIG);
+ verify(listener).onStartup(CONNECTOR);
+ verify(listener).onPause(CONNECTOR);
+ verifyCleanShutdown(true);
+
+ InOrder inOrder = inOrder(onStateChange);
+ inOrder.verify(onStateChange).onCompletion(isNull(), eq(TargetState.STARTED));
+ inOrder.verify(onStateChange).onCompletion(isNull(), eq(TargetState.PAUSED));
+ verifyNoMoreInteractions(onStateChange);
}
@Test
public void testOnResume() {
connector = sourceConnector;
- connector.version();
- expectLastCall().andReturn(VERSION);
-
- connector.initialize(EasyMock.notNull(SourceConnectorContext.class));
- expectLastCall();
- offsetStore.start();
- expectLastCall();
+ when(connector.version()).thenReturn(VERSION);
- listener.onPause(CONNECTOR);
- expectLastCall();
-
- connector.start(CONFIG);
- expectLastCall();
-
- listener.onResume(CONNECTOR);
- expectLastCall();
-
- connector.stop();
- expectLastCall();
-
- listener.onShutdown(CONNECTOR);
- expectLastCall();
-
- ctx.close();
- expectLastCall();
-
- offsetStorageReader.close();
- expectLastCall();
-
- offsetStore.stop();
- expectLastCall();
-
- Callback onStateChange = createStrictMock(Callback.class);
- onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.PAUSED));
- expectLastCall();
- onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.STARTED));
- expectLastCall();
-
- replayAll();
+ Callback onStateChange = mockCallback();
WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader);
@@ -334,42 +215,25 @@ public void testOnResume() {
workerConnector.doShutdown();
assertStoppedMetric(workerConnector);
- verifyAll();
+ verifyInitialize();
+ verify(listener).onPause(CONNECTOR);
+ verify(connector).start(CONFIG);
+ verify(listener).onResume(CONNECTOR);
+ verifyCleanShutdown(true);
+
+ InOrder inOrder = inOrder(onStateChange);
+ inOrder.verify(onStateChange).onCompletion(isNull(), eq(TargetState.PAUSED));
+ inOrder.verify(onStateChange).onCompletion(isNull(), eq(TargetState.STARTED));
+ verifyNoMoreInteractions(onStateChange);
}
@Test
public void testStartupPaused() {
connector = sinkConnector;
- connector.version();
- expectLastCall().andReturn(VERSION);
-
- connector.initialize(EasyMock.notNull(SinkConnectorContext.class));
- expectLastCall();
-
- // connector never gets started
-
- listener.onPause(CONNECTOR);
- expectLastCall();
-
- listener.onShutdown(CONNECTOR);
- expectLastCall();
-
- ctx.close();
- expectLastCall();
+ when(connector.version()).thenReturn(VERSION);
- offsetStorageReader.close();
- expectLastCall();
-
- offsetStore.stop();
- expectLastCall();
-
- Callback onStateChange = createStrictMock(Callback.class);
- onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.PAUSED));
- expectLastCall();
-
- replayAll();
-
- WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader);
+ Callback onStateChange = mockCallback();
+ WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, null, null, classLoader);
workerConnector.initialize();
assertInitializedSinkMetric(workerConnector);
@@ -379,45 +243,25 @@ public void testStartupPaused() {
workerConnector.doShutdown();
assertStoppedMetric(workerConnector);
- verifyAll();
+ verifyInitialize();
+ // connector never gets started
+ verify(listener).onPause(CONNECTOR);
+ verifyCleanShutdown(false);
+
+ verify(onStateChange).onCompletion(isNull(), eq(TargetState.PAUSED));
+ verifyNoMoreInteractions(onStateChange);
}
@Test
public void testStartupFailure() {
RuntimeException exception = new RuntimeException();
-
connector = sinkConnector;
- connector.version();
- expectLastCall().andReturn(VERSION);
-
- connector.initialize(EasyMock.notNull(SinkConnectorContext.class));
- expectLastCall();
- connector.start(CONFIG);
- expectLastCall().andThrow(exception);
+ when(connector.version()).thenReturn(VERSION);
+ doThrow(exception).when(connector).start(CONFIG);
- listener.onFailure(CONNECTOR, exception);
- expectLastCall();
-
- listener.onShutdown(CONNECTOR);
- expectLastCall();
-
- ctx.close();
- expectLastCall();
-
- offsetStorageReader.close();
- expectLastCall();
-
- offsetStore.stop();
- expectLastCall();
-
- Callback onStateChange = createStrictMock(Callback.class);
- onStateChange.onCompletion(EasyMock.anyObject(Exception.class), EasyMock.isNull());
- expectLastCall();
-
- replayAll();
-
- WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader);
+ Callback onStateChange = mockCallback();
+ WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, null, null, classLoader);
workerConnector.initialize();
assertInitializedSinkMetric(workerConnector);
@@ -427,7 +271,13 @@ public void testStartupFailure() {
workerConnector.doShutdown();
assertStoppedMetric(workerConnector);
- verifyAll();
+ verifyInitialize();
+ verify(connector).start(CONFIG);
+ verify(listener).onFailure(CONNECTOR, exception);
+ verifyCleanShutdown(false);
+
+ verify(onStateChange).onCompletion(any(Exception.class), isNull());
+ verifyNoMoreInteractions(onStateChange);
}
@Test
@@ -435,42 +285,11 @@ public void testShutdownFailure() {
RuntimeException exception = new RuntimeException();
connector = sourceConnector;
- connector.version();
- expectLastCall().andReturn(VERSION);
-
- offsetStore.start();
- expectLastCall();
-
- connector.initialize(EasyMock.notNull(SourceConnectorContext.class));
- expectLastCall();
-
- connector.start(CONFIG);
- expectLastCall();
-
- listener.onStartup(CONNECTOR);
- expectLastCall();
-
- connector.stop();
- expectLastCall().andThrow(exception);
-
- Callback onStateChange = createStrictMock(Callback.class);
- onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.STARTED));
- expectLastCall();
-
- listener.onFailure(CONNECTOR, exception);
- expectLastCall();
+ when(connector.version()).thenReturn(VERSION);
- ctx.close();
- expectLastCall();
-
- offsetStorageReader.close();
- expectLastCall();
-
- offsetStore.stop();
- expectLastCall();
-
- replayAll();
+ doThrow(exception).when(connector).stop();
+ Callback onStateChange = mockCallback();
WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader);
workerConnector.initialize();
@@ -481,48 +300,22 @@ public void testShutdownFailure() {
workerConnector.doShutdown();
assertFailedMetric(workerConnector);
- verifyAll();
+ verifyInitialize();
+ verify(connector).start(CONFIG);
+ verify(listener).onStartup(CONNECTOR);
+ verify(onStateChange).onCompletion(isNull(), eq(TargetState.STARTED));
+ verifyNoMoreInteractions(onStateChange);
+ verify(listener).onFailure(CONNECTOR, exception);
+ verifyShutdown(false, true);
}
@Test
public void testTransitionStartedToStarted() {
connector = sourceConnector;
- connector.version();
- expectLastCall().andReturn(VERSION);
-
- offsetStore.start();
- expectLastCall();
-
- connector.initialize(EasyMock.notNull(SourceConnectorContext.class));
- expectLastCall();
-
- connector.start(CONFIG);
- expectLastCall();
-
- // expect only one call to onStartup()
- listener.onStartup(CONNECTOR);
- expectLastCall();
- connector.stop();
- expectLastCall();
+ when(connector.version()).thenReturn(VERSION);
- listener.onShutdown(CONNECTOR);
- expectLastCall();
-
- ctx.close();
- expectLastCall();
-
- offsetStorageReader.close();
- expectLastCall();
-
- offsetStore.stop();
- expectLastCall();
-
- Callback onStateChange = createStrictMock(Callback.class);
- onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.STARTED));
- expectLastCall().times(2);
-
- replayAll();
+ Callback onStateChange = mockCallback();
WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader);
@@ -536,53 +329,21 @@ public void testTransitionStartedToStarted() {
workerConnector.doShutdown();
assertStoppedMetric(workerConnector);
- verifyAll();
+ verifyInitialize();
+ verify(connector).start(CONFIG);
+ // expect only one call to onStartup()
+ verify(listener).onStartup(CONNECTOR);
+ verifyCleanShutdown(true);
+ verify(onStateChange, times(2)).onCompletion(isNull(), eq(TargetState.STARTED));
+ verifyNoMoreInteractions(onStateChange);
}
@Test
public void testTransitionPausedToPaused() {
connector = sourceConnector;
- connector.version();
- expectLastCall().andReturn(VERSION);
-
- offsetStore.start();
- expectLastCall();
-
- connector.initialize(EasyMock.notNull(SourceConnectorContext.class));
- expectLastCall();
-
- connector.start(CONFIG);
- expectLastCall();
-
- listener.onStartup(CONNECTOR);
- expectLastCall();
-
- connector.stop();
- expectLastCall();
-
- listener.onPause(CONNECTOR);
- expectLastCall();
-
- listener.onShutdown(CONNECTOR);
- expectLastCall();
-
- ctx.close();
- expectLastCall();
-
- offsetStorageReader.close();
- expectLastCall();
-
- offsetStore.stop();
- expectLastCall();
-
- Callback onStateChange = createStrictMock(Callback.class);
- onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.STARTED));
- expectLastCall();
- onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.PAUSED));
- expectLastCall().times(2);
-
- replayAll();
+ when(connector.version()).thenReturn(VERSION);
+ Callback onStateChange = mockCallback();
WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader);
workerConnector.initialize();
@@ -597,28 +358,32 @@ public void testTransitionPausedToPaused() {
workerConnector.doShutdown();
assertStoppedMetric(workerConnector);
- verifyAll();
+ verifyInitialize();
+ verify(connector).start(CONFIG);
+ verify(listener).onStartup(CONNECTOR);
+ verify(listener).onPause(CONNECTOR);
+ verifyCleanShutdown(true);
+
+ InOrder inOrder = inOrder(onStateChange);
+ inOrder.verify(onStateChange).onCompletion(isNull(), eq(TargetState.STARTED));
+ inOrder.verify(onStateChange, times(2)).onCompletion(isNull(), eq(TargetState.PAUSED));
+ verifyNoMoreInteractions(onStateChange);
}
@Test
public void testFailConnectorThatIsNeitherSourceNorSink() {
- connector.version();
- expectLastCall().andReturn(VERSION);
-
- Capture exceptionCapture = Capture.newInstance();
- listener.onFailure(EasyMock.eq(CONNECTOR), EasyMock.capture(exceptionCapture));
- expectLastCall();
-
- replayAll();
-
+ connector = mock(Connector.class);
+ when(connector.version()).thenReturn(VERSION);
WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader);
workerConnector.initialize();
+
+ verify(connector).version();
+ ArgumentCaptor exceptionCapture = ArgumentCaptor.forClass(Throwable.class);
+ verify(listener).onFailure(eq(CONNECTOR), exceptionCapture.capture());
Throwable e = exceptionCapture.getValue();
assertTrue(e instanceof ConnectException);
assertTrue(e.getMessage().contains("must be a subclass of"));
-
- verifyAll();
}
protected void assertFailedMetric(WorkerConnector workerConnector) {
@@ -672,6 +437,39 @@ protected void assertInitializedMetric(WorkerConnector workerConnector, String e
assertEquals(VERSION, version);
}
+ @SuppressWarnings("unchecked")
+ private Callback mockCallback() {
+ return mock(Callback.class);
+ }
+
+ private void verifyInitialize() {
+ verify(connector).version();
+ if (connector instanceof SourceConnector) {
+ verify(offsetStore).start();
+ verify(connector).initialize(any(SourceConnectorContext.class));
+ } else {
+ verify(connector).initialize(any(SinkConnectorContext.class));
+ }
+ }
+
+ private void verifyCleanShutdown(boolean started) {
+ verifyShutdown(true, started);
+ }
+
+ private void verifyShutdown(boolean clean, boolean started) {
+ verify(ctx).close();
+ if (connector instanceof SourceConnector) {
+ verify(offsetStorageReader).close();
+ verify(offsetStore).stop();
+ }
+ if (clean) {
+ verify(listener).onShutdown(CONNECTOR);
+ }
+ if (started) {
+ verify(connector).stop();
+ }
+ }
+
private static abstract class TestConnector extends Connector {
}
}
diff --git a/core/src/main/scala/kafka/api/LeaderAndIsr.scala b/core/src/main/scala/kafka/api/LeaderAndIsr.scala
index dd1d381a1445..da68cdb479c1 100644
--- a/core/src/main/scala/kafka/api/LeaderAndIsr.scala
+++ b/core/src/main/scala/kafka/api/LeaderAndIsr.scala
@@ -62,7 +62,7 @@ case class LeaderAndIsr(
if (leader == LeaderAndIsr.NoLeader) None else Some(leader)
}
- def equalsIgnorePartitionEpoch(other: LeaderAndIsr): Boolean = {
+ def equalsAllowStalePartitionEpoch(other: LeaderAndIsr): Boolean = {
if (this == other) {
true
} else if (other == null) {
@@ -71,7 +71,8 @@ case class LeaderAndIsr(
leader == other.leader &&
leaderEpoch == other.leaderEpoch &&
isr.equals(other.isr) &&
- leaderRecoveryState == other.leaderRecoveryState
+ leaderRecoveryState == other.leaderRecoveryState &&
+ partitionEpoch <= other.partitionEpoch
}
}
diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala
index 538c51f9035c..1eab4c4669a0 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -26,7 +26,7 @@ import kafka.log._
import kafka.metrics.KafkaMetricsGroup
import kafka.server._
import kafka.server.checkpoints.OffsetCheckpoints
-import kafka.server.metadata.KRaftMetadataCache
+import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache}
import kafka.utils.CoreUtils.{inReadLock, inWriteLock}
import kafka.utils._
import kafka.zookeeper.ZooKeeperClientException
@@ -881,11 +881,16 @@ class Partition(val topicPartition: TopicPartition,
private def isReplicaIsrEligible(followerReplicaId: Int): Boolean = {
metadataCache match {
// In KRaft mode, only replicas which are not fenced nor in controlled shutdown are
- // allowed to join the ISR. This does not apply to ZK mode.
+ // allowed to join the ISR.
case kRaftMetadataCache: KRaftMetadataCache =>
!kRaftMetadataCache.isBrokerFenced(followerReplicaId) &&
!kRaftMetadataCache.isBrokerShuttingDown(followerReplicaId)
+ // In ZK mode, we just ensure the broker is alive. Although we do not check for shutting down brokers here,
+ // the controller will block them from being added to ISR.
+ case zkMetadataCache: ZkMetadataCache =>
+ zkMetadataCache.hasAliveBroker(followerReplicaId)
+
case _ => true
}
}
diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala
index 8d16eb7e1da8..0154d9cbe54a 100644
--- a/core/src/main/scala/kafka/controller/KafkaController.scala
+++ b/core/src/main/scala/kafka/controller/KafkaController.scala
@@ -2339,14 +2339,15 @@ class KafkaController(val config: KafkaConfig,
if (newLeaderAndIsr.leaderEpoch != currentLeaderAndIsr.leaderEpoch) {
partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH)
None
- } else if (newLeaderAndIsr.partitionEpoch < currentLeaderAndIsr.partitionEpoch) {
- partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION)
- None
- } else if (newLeaderAndIsr.equalsIgnorePartitionEpoch(currentLeaderAndIsr)) {
+ } else if (newLeaderAndIsr.equalsAllowStalePartitionEpoch(currentLeaderAndIsr)) {
// If a partition is already in the desired state, just return it
+ // this check must be done before fencing based on partition epoch to maintain idempotency
partitionResponses(tp) = Right(currentLeaderAndIsr)
None
- } else if (newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) {
+ } else if (newLeaderAndIsr.partitionEpoch != currentLeaderAndIsr.partitionEpoch) {
+ partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION)
+ None
+ } else if (newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) {
partitionResponses(tp) = Left(Errors.INVALID_REQUEST)
info(
s"Rejecting AlterPartition from node $brokerId for $tp because leader is recovering and ISR is greater than 1: " +
@@ -2363,7 +2364,23 @@ class KafkaController(val config: KafkaConfig,
)
None
} else {
- Some(tp -> newLeaderAndIsr)
+ // Pull out replicas being added to ISR and verify they are all online.
+ // If a replica is not online, reject the update as specified in KIP-841.
+ val ineligibleReplicas = newLeaderAndIsr.isr.toSet -- controllerContext.liveBrokerIds
+ if (ineligibleReplicas.nonEmpty) {
+ info(s"Rejecting AlterPartition request from node $brokerId for $tp because " +
+ s"it specified ineligible replicas $ineligibleReplicas in the new ISR ${newLeaderAndIsr.isr}."
+ )
+
+ if (alterPartitionRequestVersion > 1) {
+ partitionResponses(tp) = Left(Errors.INELIGIBLE_REPLICA)
+ } else {
+ partitionResponses(tp) = Left(Errors.OPERATION_NOT_ATTEMPTED)
+ }
+ None
+ } else {
+ Some(tp -> newLeaderAndIsr)
+ }
}
case None =>
diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala
index 0bdd67349756..1008decadb11 100644
--- a/core/src/main/scala/kafka/server/BrokerServer.scala
+++ b/core/src/main/scala/kafka/server/BrokerServer.scala
@@ -50,6 +50,7 @@ import org.apache.kafka.raft.RaftConfig.AddressSpec
import org.apache.kafka.raft.{RaftClient, RaftConfig}
import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.ApiMessageAndVersion
+import org.apache.kafka.server.fault.FaultHandler
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.snapshot.SnapshotWriter
@@ -76,9 +77,13 @@ class BrokerServer(
val raftManager: RaftManager[ApiMessageAndVersion],
val time: Time,
val metrics: Metrics,
+ val brokerMetrics: BrokerServerMetrics,
val threadNamePrefix: Option[String],
val initialOfflineDirs: Seq[String],
- val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]]
+ val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]],
+ val fatalFaultHandler: FaultHandler,
+ val metadataLoadingFaultHandler: FaultHandler,
+ val metadataPublishingFaultHandler: FaultHandler
) extends KafkaBroker {
override def brokerState: BrokerState = Option(lifecycleManager).
@@ -315,8 +320,8 @@ class BrokerServer(
threadNamePrefix,
config.metadataSnapshotMaxNewRecordBytes,
metadataSnapshotter,
- BrokerServerMetrics(metrics)
- )
+ brokerMetrics,
+ metadataLoadingFaultHandler)
val networkListeners = new ListenerCollection()
config.effectiveAdvertisedListeners.foreach { ep =>
@@ -432,7 +437,9 @@ class BrokerServer(
transactionCoordinator,
clientQuotaMetadataManager,
dynamicConfigHandlers.toMap,
- authorizer)
+ authorizer,
+ fatalFaultHandler,
+ metadataPublishingFaultHandler)
// Tell the metadata listener to start publishing its output, and wait for the first
// publish operation to complete. This first operation will initialize logManager,
diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala
index cff88d2b6bee..19a6e307d62b 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -21,7 +21,6 @@ import java.util
import java.util.OptionalLong
import java.util.concurrent.locks.ReentrantLock
import java.util.concurrent.{CompletableFuture, TimeUnit}
-
import kafka.cluster.Broker.ServerInfo
import kafka.metrics.{KafkaMetricsGroup, LinuxIoMetricsCollector}
import kafka.network.{DataPlaneAcceptor, SocketServer}
@@ -38,7 +37,7 @@ import org.apache.kafka.common.security.scram.internals.ScramMechanism
import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache
import org.apache.kafka.common.utils.{LogContext, Time}
import org.apache.kafka.common.{ClusterResource, Endpoint}
-import org.apache.kafka.controller.{BootstrapMetadata, Controller, QuorumController, QuorumControllerMetrics, QuorumFeatures}
+import org.apache.kafka.controller.{BootstrapMetadata, Controller, ControllerMetrics, QuorumController, QuorumFeatures}
import org.apache.kafka.metadata.KafkaConfigSchema
import org.apache.kafka.raft.RaftConfig
import org.apache.kafka.raft.RaftConfig.AddressSpec
@@ -46,6 +45,7 @@ import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.ApiMessageAndVersion
import org.apache.kafka.common.config.ConfigException
import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer
+import org.apache.kafka.server.fault.FaultHandler
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.policy.{AlterConfigPolicy, CreateTopicPolicy}
@@ -61,11 +61,14 @@ class ControllerServer(
val raftManager: RaftManager[ApiMessageAndVersion],
val time: Time,
val metrics: Metrics,
+ val controllerMetrics: ControllerMetrics,
val threadNamePrefix: Option[String],
val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]],
val configSchema: KafkaConfigSchema,
val raftApiVersions: ApiVersions,
- val bootstrapMetadata: BootstrapMetadata
+ val bootstrapMetadata: BootstrapMetadata,
+ val metadataFaultHandler: FaultHandler,
+ val fatalFaultHandler: FaultHandler,
) extends Logging with KafkaMetricsGroup {
import kafka.server.Server._
@@ -199,12 +202,14 @@ class ControllerServer(
setSnapshotMaxNewRecordBytes(config.metadataSnapshotMaxNewRecordBytes).
setLeaderImbalanceCheckIntervalNs(leaderImbalanceCheckIntervalNs).
setMaxIdleIntervalNs(maxIdleIntervalNs).
- setMetrics(new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry(), time)).
+ setMetrics(controllerMetrics).
setCreateTopicPolicy(createTopicPolicy.asJava).
setAlterConfigPolicy(alterConfigPolicy.asJava).
setConfigurationValidator(new ControllerConfigurationValidator()).
setStaticConfig(config.originals).
- setBootstrapMetadata(bootstrapMetadata)
+ setBootstrapMetadata(bootstrapMetadata).
+ setMetadataFaultHandler(metadataFaultHandler).
+ setFatalFaultHandler(fatalFaultHandler)
}
authorizer match {
case Some(a: ClusterMetadataAuthorizer) => controllerBuilder.setAuthorizer(a)
diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala
index 07a311837204..2338ef5e7c41 100644
--- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala
@@ -23,15 +23,17 @@ import kafka.log.{LogConfig, UnifiedLog}
import kafka.metrics.KafkaMetricsReporter
import kafka.raft.KafkaRaftManager
import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole}
+import kafka.server.metadata.BrokerServerMetrics
import kafka.utils.{CoreUtils, Logging, Mx4jLoader, VerifiableProperties}
import org.apache.kafka.common.config.{ConfigDef, ConfigResource}
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.utils.{AppInfoParser, Time}
import org.apache.kafka.common.{KafkaException, Uuid}
-import org.apache.kafka.controller.BootstrapMetadata
+import org.apache.kafka.controller.{BootstrapMetadata, QuorumControllerMetrics}
import org.apache.kafka.metadata.{KafkaConfigSchema, MetadataRecordSerde}
import org.apache.kafka.raft.RaftConfig
import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
+import org.apache.kafka.server.fault.{LoggingFaultHandler, ProcessExitingFaultHandler}
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import java.nio.file.Paths
@@ -81,32 +83,49 @@ class KafkaRaftServer(
)
private val broker: Option[BrokerServer] = if (config.processRoles.contains(BrokerRole)) {
+ val brokerMetrics = BrokerServerMetrics(metrics)
+ val fatalFaultHandler = new ProcessExitingFaultHandler()
+ val metadataLoadingFaultHandler = new LoggingFaultHandler("metadata loading",
+ () => brokerMetrics.metadataLoadErrorCount.getAndIncrement())
+ val metadataApplyingFaultHandler = new LoggingFaultHandler("metadata application",
+ () => brokerMetrics.metadataApplyErrorCount.getAndIncrement())
Some(new BrokerServer(
config,
metaProps,
raftManager,
time,
metrics,
+ brokerMetrics,
threadNamePrefix,
offlineDirs,
- controllerQuorumVotersFuture
+ controllerQuorumVotersFuture,
+ fatalFaultHandler,
+ metadataLoadingFaultHandler,
+ metadataApplyingFaultHandler
))
} else {
None
}
private val controller: Option[ControllerServer] = if (config.processRoles.contains(ControllerRole)) {
+ val controllerMetrics = new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry(), time)
+ val metadataFaultHandler = new LoggingFaultHandler("controller metadata",
+ () => controllerMetrics.incrementMetadataErrorCount())
+ val fatalFaultHandler = new ProcessExitingFaultHandler()
Some(new ControllerServer(
metaProps,
config,
raftManager,
time,
metrics,
+ controllerMetrics,
threadNamePrefix,
controllerQuorumVotersFuture,
KafkaRaftServer.configSchema,
raftManager.apiVersions,
- bootstrapMetadata
+ bootstrapMetadata,
+ metadataFaultHandler,
+ fatalFaultHandler
))
} else {
None
diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala
index 3b79526a954e..3984f467edda 100644
--- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala
+++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala
@@ -19,13 +19,13 @@ package kafka.server.metadata
import java.util
import java.util.concurrent.{CompletableFuture, TimeUnit}
import java.util.function.Consumer
-
import kafka.metrics.KafkaMetricsGroup
import org.apache.kafka.image.{MetadataDelta, MetadataImage}
import org.apache.kafka.common.utils.{LogContext, Time}
import org.apache.kafka.queue.{EventQueue, KafkaEventQueue}
import org.apache.kafka.raft.{Batch, BatchReader, LeaderAndEpoch, RaftClient}
import org.apache.kafka.server.common.ApiMessageAndVersion
+import org.apache.kafka.server.fault.FaultHandler
import org.apache.kafka.snapshot.SnapshotReader
@@ -40,7 +40,8 @@ class BrokerMetadataListener(
threadNamePrefix: Option[String],
val maxBytesBetweenSnapshots: Long,
val snapshotter: Option[MetadataSnapshotter],
- brokerMetrics: BrokerServerMetrics
+ brokerMetrics: BrokerServerMetrics,
+ metadataLoadingFaultHandler: FaultHandler
) extends RaftClient.Listener[ApiMessageAndVersion] with KafkaMetricsGroup {
private val logContext = new LogContext(s"[BrokerMetadataListener id=$brokerId] ")
private val log = logContext.logger(classOf[BrokerMetadataListener])
@@ -109,11 +110,16 @@ class BrokerMetadataListener(
extends EventQueue.FailureLoggingEvent(log) {
override def run(): Unit = {
val results = try {
- val loadResults = loadBatches(_delta, reader, None, None, None)
+ val loadResults = loadBatches(_delta, reader, None, None, None, None)
if (isDebugEnabled) {
debug(s"Loaded new commits: $loadResults")
}
loadResults
+ } catch {
+ case e: Throwable =>
+ metadataLoadingFaultHandler.handleFault(s"Unable to load metadata commits " +
+ s"from the BatchReader starting at base offset ${reader.baseOffset()}", e)
+ return
} finally {
reader.close()
}
@@ -156,19 +162,26 @@ class BrokerMetadataListener(
class HandleSnapshotEvent(reader: SnapshotReader[ApiMessageAndVersion])
extends EventQueue.FailureLoggingEvent(log) {
override def run(): Unit = {
+ val snapshotName = s"${reader.snapshotId().offset}-${reader.snapshotId().epoch}"
try {
- info(s"Loading snapshot ${reader.snapshotId().offset}-${reader.snapshotId().epoch}.")
+ info(s"Loading snapshot ${snapshotName}")
_delta = new MetadataDelta(_image) // Discard any previous deltas.
- val loadResults = loadBatches(
- _delta,
+ val loadResults = loadBatches(_delta,
reader,
Some(reader.lastContainedLogTimestamp),
Some(reader.lastContainedLogOffset),
- Some(reader.lastContainedLogEpoch)
- )
- _delta.finishSnapshot()
- info(s"Loaded snapshot ${reader.snapshotId().offset}-${reader.snapshotId().epoch}: " +
- s"$loadResults")
+ Some(reader.lastContainedLogEpoch),
+ Some(snapshotName))
+ try {
+ _delta.finishSnapshot()
+ } catch {
+ case e: Throwable => metadataLoadingFaultHandler.handleFault(
+ s"Error finishing snapshot ${snapshotName}", e)
+ }
+ info(s"Loaded snapshot ${snapshotName}: ${loadResults}")
+ } catch {
+ case t: Throwable => metadataLoadingFaultHandler.handleFault("Uncaught exception while " +
+ s"loading broker metadata from Metadata snapshot ${snapshotName}", t)
} finally {
reader.close()
}
@@ -201,7 +214,8 @@ class BrokerMetadataListener(
iterator: util.Iterator[Batch[ApiMessageAndVersion]],
lastAppendTimestamp: Option[Long],
lastCommittedOffset: Option[Long],
- lastCommittedEpoch: Option[Int]
+ lastCommittedEpoch: Option[Int],
+ snapshotName: Option[String]
): BatchLoadResults = {
val startTimeNs = time.nanoseconds()
var numBatches = 0
@@ -220,12 +234,20 @@ class BrokerMetadataListener(
trace(s"Metadata batch ${batch.lastOffset}: processing [${index + 1}/${batch.records.size}]:" +
s" ${messageAndVersion.message}")
}
-
_highestOffset = lastCommittedOffset.getOrElse(batch.baseOffset() + index)
-
- delta.replay(highestMetadataOffset, epoch, messageAndVersion.message())
- numRecords += 1
- index += 1
+ try {
+ delta.replay(highestMetadataOffset, epoch, messageAndVersion.message())
+ } catch {
+ case e: Throwable => snapshotName match {
+ case None => metadataLoadingFaultHandler.handleFault(
+ s"Error replaying metadata log record at offset ${_highestOffset}", e)
+ case Some(name) => metadataLoadingFaultHandler.handleFault(
+ s"Error replaying record ${index} from snapshot ${name} at offset ${_highestOffset}", e)
+ }
+ } finally {
+ numRecords += 1
+ index += 1
+ }
}
numBytes = numBytes + batch.sizeInBytes()
metadataBatchSizeHist.update(batch.records().size())
diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
index 212f188504e9..0192bb4afcfd 100644
--- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
+++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
@@ -31,6 +31,7 @@ import org.apache.kafka.common.internals.Topic
import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta, TopicsImage}
import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer
import org.apache.kafka.server.authorizer.Authorizer
+import org.apache.kafka.server.fault.FaultHandler
import scala.collection.mutable
@@ -94,15 +95,19 @@ object BrokerMetadataPublisher extends Logging {
}
}
-class BrokerMetadataPublisher(conf: KafkaConfig,
- metadataCache: KRaftMetadataCache,
- logManager: LogManager,
- replicaManager: ReplicaManager,
- groupCoordinator: GroupCoordinator,
- txnCoordinator: TransactionCoordinator,
- clientQuotaMetadataManager: ClientQuotaMetadataManager,
- dynamicConfigHandlers: Map[String, ConfigHandler],
- private val _authorizer: Option[Authorizer]) extends MetadataPublisher with Logging {
+class BrokerMetadataPublisher(
+ conf: KafkaConfig,
+ metadataCache: KRaftMetadataCache,
+ logManager: LogManager,
+ replicaManager: ReplicaManager,
+ groupCoordinator: GroupCoordinator,
+ txnCoordinator: TransactionCoordinator,
+ clientQuotaMetadataManager: ClientQuotaMetadataManager,
+ dynamicConfigHandlers: Map[String, ConfigHandler],
+ private val _authorizer: Option[Authorizer],
+ fatalFaultHandler: FaultHandler,
+ metadataPublishingFaultHandler: FaultHandler
+) extends MetadataPublisher with Logging {
logIdent = s"[BrokerMetadataPublisher id=${conf.nodeId}] "
import BrokerMetadataPublisher._
@@ -125,8 +130,15 @@ class BrokerMetadataPublisher(conf: KafkaConfig,
override def publish(delta: MetadataDelta, newImage: MetadataImage): Unit = {
val highestOffsetAndEpoch = newImage.highestOffsetAndEpoch()
+ val deltaName = if (_firstPublish) {
+ s"initial MetadataDelta up to ${highestOffsetAndEpoch.offset}"
+ } else {
+ s"MetadataDelta up to ${highestOffsetAndEpoch.offset}"
+ }
try {
- trace(s"Publishing delta $delta with highest offset $highestOffsetAndEpoch")
+ if (isTraceEnabled) {
+ trace(s"Publishing delta $delta with highest offset $highestOffsetAndEpoch")
+ }
// Publish the new metadata image to the metadata cache.
metadataCache.setImage(newImage)
@@ -151,37 +163,50 @@ class BrokerMetadataPublisher(conf: KafkaConfig,
// Apply topic deltas.
Option(delta.topicsDelta()).foreach { topicsDelta =>
- // Notify the replica manager about changes to topics.
- replicaManager.applyDelta(topicsDelta, newImage)
-
- // Update the group coordinator of local changes
- updateCoordinator(
- newImage,
- delta,
- Topic.GROUP_METADATA_TOPIC_NAME,
- groupCoordinator.onElection,
- groupCoordinator.onResignation
- )
-
- // Update the transaction coordinator of local changes
- updateCoordinator(
- newImage,
- delta,
- Topic.TRANSACTION_STATE_TOPIC_NAME,
- txnCoordinator.onElection,
- txnCoordinator.onResignation
- )
-
- // Notify the group coordinator about deleted topics.
- val deletedTopicPartitions = new mutable.ArrayBuffer[TopicPartition]()
- topicsDelta.deletedTopicIds().forEach { id =>
- val topicImage = topicsDelta.image().getTopic(id)
- topicImage.partitions().keySet().forEach {
- id => deletedTopicPartitions += new TopicPartition(topicImage.name(), id)
- }
+ try {
+ // Notify the replica manager about changes to topics.
+ replicaManager.applyDelta(topicsDelta, newImage)
+ } catch {
+ case t: Throwable => metadataPublishingFaultHandler.handleFault("Error applying topics " +
+ s"delta in ${deltaName}", t)
+ }
+ try {
+ // Update the group coordinator of local changes
+ updateCoordinator(newImage,
+ delta,
+ Topic.GROUP_METADATA_TOPIC_NAME,
+ groupCoordinator.onElection,
+ groupCoordinator.onResignation)
+ } catch {
+ case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating group " +
+ s"coordinator with local changes in ${deltaName}", t)
+ }
+ try {
+ // Update the transaction coordinator of local changes
+ updateCoordinator(newImage,
+ delta,
+ Topic.TRANSACTION_STATE_TOPIC_NAME,
+ txnCoordinator.onElection,
+ txnCoordinator.onResignation)
+ } catch {
+ case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating txn " +
+ s"coordinator with local changes in ${deltaName}", t)
}
- if (deletedTopicPartitions.nonEmpty) {
- groupCoordinator.handleDeletedPartitions(deletedTopicPartitions, RequestLocal.NoCaching)
+ try {
+ // Notify the group coordinator about deleted topics.
+ val deletedTopicPartitions = new mutable.ArrayBuffer[TopicPartition]()
+ topicsDelta.deletedTopicIds().forEach { id =>
+ val topicImage = topicsDelta.image().getTopic(id)
+ topicImage.partitions().keySet().forEach {
+ id => deletedTopicPartitions += new TopicPartition(topicImage.name(), id)
+ }
+ }
+ if (deletedTopicPartitions.nonEmpty) {
+ groupCoordinator.handleDeletedPartitions(deletedTopicPartitions, RequestLocal.NoCaching)
+ }
+ } catch {
+ case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating group " +
+ s"coordinator with deleted partitions in ${deltaName}", t)
}
}
@@ -191,39 +216,62 @@ class BrokerMetadataPublisher(conf: KafkaConfig,
val props = newImage.configs().configProperties(resource)
resource.`type`() match {
case TOPIC =>
- // Apply changes to a topic's dynamic configuration.
- info(s"Updating topic ${resource.name()} with new configuration : " +
- toLoggableProps(resource, props).mkString(","))
- dynamicConfigHandlers(ConfigType.Topic).
- processConfigChanges(resource.name(), props)
+ try {
+ // Apply changes to a topic's dynamic configuration.
+ info(s"Updating topic ${resource.name()} with new configuration : " +
+ toLoggableProps(resource, props).mkString(","))
+ dynamicConfigHandlers(ConfigType.Topic).
+ processConfigChanges(resource.name(), props)
+ } catch {
+ case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating topic " +
+ s"${resource.name()} with new configuration: ${toLoggableProps(resource, props).mkString(",")} " +
+ s"in ${deltaName}", t)
+ }
case BROKER =>
if (resource.name().isEmpty) {
- // Apply changes to "cluster configs" (also known as default BROKER configs).
- // These are stored in KRaft with an empty name field.
- info("Updating cluster configuration : " +
- toLoggableProps(resource, props).mkString(","))
- dynamicConfigHandlers(ConfigType.Broker).
- processConfigChanges(ConfigEntityName.Default, props)
+ try {
+ // Apply changes to "cluster configs" (also known as default BROKER configs).
+ // These are stored in KRaft with an empty name field.
+ info("Updating cluster configuration : " +
+ toLoggableProps(resource, props).mkString(","))
+ dynamicConfigHandlers(ConfigType.Broker).
+ processConfigChanges(ConfigEntityName.Default, props)
+ } catch {
+ case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating " +
+ s"cluster with new configuration: ${toLoggableProps(resource, props).mkString(",")} " +
+ s"in ${deltaName}", t)
+ }
} else if (resource.name() == brokerId.toString) {
- // Apply changes to this broker's dynamic configuration.
- info(s"Updating broker $brokerId with new configuration : " +
- toLoggableProps(resource, props).mkString(","))
- dynamicConfigHandlers(ConfigType.Broker).
- processConfigChanges(resource.name(), props)
- // When applying a per broker config (not a cluster config), we also
- // reload any associated file. For example, if the ssl.keystore is still
- // set to /tmp/foo, we still want to reload /tmp/foo in case its contents
- // have changed. This doesn't apply to topic configs or cluster configs.
- reloadUpdatedFilesWithoutConfigChange(props)
+ try {
+ // Apply changes to this broker's dynamic configuration.
+ info(s"Updating broker $brokerId with new configuration : " +
+ toLoggableProps(resource, props).mkString(","))
+ dynamicConfigHandlers(ConfigType.Broker).
+ processConfigChanges(resource.name(), props)
+ // When applying a per broker config (not a cluster config), we also
+ // reload any associated file. For example, if the ssl.keystore is still
+ // set to /tmp/foo, we still want to reload /tmp/foo in case its contents
+ // have changed. This doesn't apply to topic configs or cluster configs.
+ reloadUpdatedFilesWithoutConfigChange(props)
+ } catch {
+ case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating " +
+ s"broker with new configuration: ${toLoggableProps(resource, props).mkString(",")} " +
+ s"in ${deltaName}", t)
+ }
}
case _ => // nothing to do
}
}
}
- // Apply client quotas delta.
- Option(delta.clientQuotasDelta()).foreach { clientQuotasDelta =>
- clientQuotaMetadataManager.update(clientQuotasDelta)
+ try {
+ // Apply client quotas delta.
+ Option(delta.clientQuotasDelta()).foreach { clientQuotasDelta =>
+ clientQuotaMetadataManager.update(clientQuotasDelta)
+ }
+ } catch {
+ case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating client " +
+ s"quotas in ${deltaName}", t)
}
// Apply changes to ACLs. This needs to be handled carefully because while we are
@@ -235,20 +283,30 @@ class BrokerMetadataPublisher(conf: KafkaConfig,
Option(delta.aclsDelta()).foreach( aclsDelta =>
_authorizer match {
case Some(authorizer: ClusterMetadataAuthorizer) => if (aclsDelta.isSnapshotDelta) {
- // If the delta resulted from a snapshot load, we want to apply the new changes
- // all at once using ClusterMetadataAuthorizer#loadSnapshot. If this is the
- // first snapshot load, it will also complete the futures returned by
- // Authorizer#start (which we wait for before processing RPCs).
- authorizer.loadSnapshot(newImage.acls().acls())
+ try {
+ // If the delta resulted from a snapshot load, we want to apply the new changes
+ // all at once using ClusterMetadataAuthorizer#loadSnapshot. If this is the
+ // first snapshot load, it will also complete the futures returned by
+ // Authorizer#start (which we wait for before processing RPCs).
+ authorizer.loadSnapshot(newImage.acls().acls())
+ } catch {
+ case t: Throwable => metadataPublishingFaultHandler.handleFault("Error loading " +
+ s"authorizer snapshot in ${deltaName}", t)
+ }
} else {
- // Because the changes map is a LinkedHashMap, the deltas will be returned in
- // the order they were performed.
- aclsDelta.changes().entrySet().forEach(e =>
- if (e.getValue.isPresent) {
- authorizer.addAcl(e.getKey, e.getValue.get())
- } else {
- authorizer.removeAcl(e.getKey)
- })
+ try {
+ // Because the changes map is a LinkedHashMap, the deltas will be returned in
+ // the order they were performed.
+ aclsDelta.changes().entrySet().forEach(e =>
+ if (e.getValue.isPresent) {
+ authorizer.addAcl(e.getKey, e.getValue.get())
+ } else {
+ authorizer.removeAcl(e.getKey)
+ })
+ } catch {
+ case t: Throwable => metadataPublishingFaultHandler.handleFault("Error loading " +
+ s"authorizer changes in ${deltaName}", t)
+ }
}
case _ => // No ClusterMetadataAuthorizer is configured. There is nothing to do.
})
@@ -258,8 +316,8 @@ class BrokerMetadataPublisher(conf: KafkaConfig,
}
publishedOffsetAtomic.set(newImage.highestOffsetAndEpoch().offset)
} catch {
- case t: Throwable => error(s"Error publishing broker metadata at $highestOffsetAndEpoch", t)
- throw t
+ case t: Throwable => metadataPublishingFaultHandler.handleFault("Uncaught exception while " +
+ s"publishing broker metadata from ${deltaName}", t)
} finally {
_firstPublish = false
}
@@ -282,7 +340,7 @@ class BrokerMetadataPublisher(conf: KafkaConfig,
* @param resignation function to call on resignation; the first parameter is the partition id;
* the second parameter is the leader epoch
*/
- private def updateCoordinator(
+ def updateCoordinator(
image: MetadataImage,
delta: MetadataDelta,
topicName: String,
@@ -317,38 +375,60 @@ class BrokerMetadataPublisher(conf: KafkaConfig,
}
private def initializeManagers(): Unit = {
- // Start log manager, which will perform (potentially lengthy)
- // recovery-from-unclean-shutdown if required.
- logManager.startup(metadataCache.getAllTopics())
-
- // Make the LogCleaner available for reconfiguration. We can't do this prior to this
- // point because LogManager#startup creates the LogCleaner object, if
- // log.cleaner.enable is true. TODO: improve this (see KAFKA-13610)
- Option(logManager.cleaner).foreach(conf.dynamicConfig.addBrokerReconfigurable)
-
- // Start the replica manager.
- replicaManager.startup()
-
- // Start the group coordinator.
- groupCoordinator.startup(() => metadataCache.numPartitions(
- Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(conf.offsetsTopicPartitions))
-
- // Start the transaction coordinator.
- txnCoordinator.startup(() => metadataCache.numPartitions(
- Topic.TRANSACTION_STATE_TOPIC_NAME).getOrElse(conf.transactionTopicPartitions))
+ try {
+ // Start log manager, which will perform (potentially lengthy)
+ // recovery-from-unclean-shutdown if required.
+ logManager.startup(metadataCache.getAllTopics())
+
+ // Make the LogCleaner available for reconfiguration. We can't do this prior to this
+ // point because LogManager#startup creates the LogCleaner object, if
+ // log.cleaner.enable is true. TODO: improve this (see KAFKA-13610)
+ Option(logManager.cleaner).foreach(conf.dynamicConfig.addBrokerReconfigurable)
+ } catch {
+ case t: Throwable => fatalFaultHandler.handleFault("Error starting LogManager", t)
+ }
+ try {
+ // Start the replica manager.
+ replicaManager.startup()
+ } catch {
+ case t: Throwable => fatalFaultHandler.handleFault("Error starting ReplicaManager", t)
+ }
+ try {
+ // Start the group coordinator.
+ groupCoordinator.startup(() => metadataCache.numPartitions(
+ Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(conf.offsetsTopicPartitions))
+ } catch {
+ case t: Throwable => fatalFaultHandler.handleFault("Error starting GroupCoordinator", t)
+ }
+ try {
+ // Start the transaction coordinator.
+ txnCoordinator.startup(() => metadataCache.numPartitions(
+ Topic.TRANSACTION_STATE_TOPIC_NAME).getOrElse(conf.transactionTopicPartitions))
+ } catch {
+ case t: Throwable => fatalFaultHandler.handleFault("Error starting TransactionCoordinator", t)
+ }
}
private def finishInitializingReplicaManager(newImage: MetadataImage): Unit = {
- // Delete log directories which we're not supposed to have, according to the
- // latest metadata. This is only necessary to do when we're first starting up. If
- // we have to load a snapshot later, these topics will appear in deletedTopicIds.
- val strayPartitions = findStrayPartitions(brokerId, newImage.topics, logManager.allLogs)
- if (strayPartitions.nonEmpty) {
- replicaManager.deleteStrayReplicas(strayPartitions)
+ try {
+ // Delete log directories which we're not supposed to have, according to the
+ // latest metadata. This is only necessary to do when we're first starting up. If
+ // we have to load a snapshot later, these topics will appear in deletedTopicIds.
+ val strayPartitions = findStrayPartitions(brokerId, newImage.topics, logManager.allLogs)
+ if (strayPartitions.nonEmpty) {
+ replicaManager.deleteStrayReplicas(strayPartitions)
+ }
+ } catch {
+ case t: Throwable => metadataPublishingFaultHandler.handleFault("Error deleting stray " +
+ "partitions during startup", t)
}
-
- // Make sure that the high water mark checkpoint thread is running for the replica
- // manager.
- replicaManager.startHighWatermarkCheckPointThread()
- }
+ try {
+ // Make sure that the high water mark checkpoint thread is running for the replica
+ // manager.
+ replicaManager.startHighWatermarkCheckPointThread()
+ } catch {
+ case t: Throwable => metadataPublishingFaultHandler.handleFault("Error starting high " +
+ "watermark checkpoint thread during startup", t)
+ }
+}
}
diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala
index dd77b277c8b6..2a236ca74970 100644
--- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala
+++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala
@@ -24,18 +24,50 @@ import org.apache.kafka.queue.{EventQueue, KafkaEventQueue}
import org.apache.kafka.server.common.ApiMessageAndVersion
import org.apache.kafka.snapshot.SnapshotWriter
+import java.util.function.Consumer
+
trait SnapshotWriterBuilder {
def build(committedOffset: Long,
committedEpoch: Int,
lastContainedLogTime: Long): Option[SnapshotWriter[ApiMessageAndVersion]]
}
+/**
+ * The RecordListConsumer takes as input a potentially long list of records, and feeds the
+ * SnapshotWriter a series of smaller lists of records.
+ *
+ * Note: from the perspective of Kafka, the snapshot file is really just a list of records,
+ * and we don't care about batches. Batching is irrelevant to the meaning of the snapshot.
+ */
+class RecordListConsumer(
+ val maxRecordsInBatch: Int,
+ val writer: SnapshotWriter[ApiMessageAndVersion]
+) extends Consumer[java.util.List[ApiMessageAndVersion]] {
+ override def accept(messages: java.util.List[ApiMessageAndVersion]): Unit = {
+ var i = 0
+ while (i < messages.size()) {
+ writer.append(messages.subList(i, Math.min(i + maxRecordsInBatch, messages.size())));
+ i += maxRecordsInBatch
+ }
+ }
+}
+
class BrokerMetadataSnapshotter(
brokerId: Int,
val time: Time,
threadNamePrefix: Option[String],
writerBuilder: SnapshotWriterBuilder
) extends Logging with MetadataSnapshotter {
+ /**
+ * The maximum number of records we will put in each batch.
+ *
+ * From the perspective of the Raft layer, the limit on batch size is specified in terms of
+ * bytes, not number of records. @See {@link KafkaRaftClient#MAX_BATCH_SIZE_BYTES} for details.
+ * However, it's more convenient to limit the batch size here in terms of number of records.
+ * So we chose a low number that will not cause problems.
+ */
+ private val maxRecordsInBatch = 1024
+
private val logContext = new LogContext(s"[BrokerMetadataSnapshotter id=$brokerId] ")
logIdent = logContext.logPrefix()
@@ -77,9 +109,11 @@ class BrokerMetadataSnapshotter(
class CreateSnapshotEvent(image: MetadataImage,
writer: SnapshotWriter[ApiMessageAndVersion])
extends EventQueue.Event {
+
override def run(): Unit = {
try {
- image.write(writer.append(_))
+ val consumer = new RecordListConsumer(maxRecordsInBatch, writer)
+ image.write(consumer)
writer.freeze()
} finally {
try {
diff --git a/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala b/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala
index 0db6f0071c48..3e68ae85f923 100644
--- a/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala
+++ b/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala
@@ -28,6 +28,8 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable
val lastAppliedRecordOffset: AtomicLong = new AtomicLong(0)
val lastAppliedRecordTimestamp: AtomicLong = new AtomicLong(0)
+ val metadataLoadErrorCount: AtomicLong = new AtomicLong(0)
+ val metadataApplyErrorCount: AtomicLong = new AtomicLong(0)
val lastAppliedRecordOffsetName = metrics.metricName(
"last-applied-record-offset",
@@ -47,6 +49,18 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable
"The difference between now and the timestamp of the last record from the cluster metadata partition that was applied by the broker"
)
+ val metadataLoadErrorCountName = metrics.metricName(
+ "metadata-load-error-count",
+ metricGroupName,
+ "The number of errors encountered by the BrokerMetadataListener while loading the metadata log and generating a new MetadataDelta based on it."
+ )
+
+ val metadataApplyErrorCountName = metrics.metricName(
+ "metadata-apply-error-count",
+ metricGroupName,
+ "The number of errors encountered by the BrokerMetadataPublisher while applying a new MetadataImage based on the latest MetadataDelta."
+ )
+
addMetric(metrics, lastAppliedRecordOffsetName) { _ =>
lastAppliedRecordOffset.get
}
@@ -59,11 +73,21 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable
now - lastAppliedRecordTimestamp.get
}
+ addMetric(metrics, metadataLoadErrorCountName) { _ =>
+ metadataLoadErrorCount.get
+ }
+
+ addMetric(metrics, metadataApplyErrorCountName) { _ =>
+ metadataApplyErrorCount.get
+ }
+
override def close(): Unit = {
List(
lastAppliedRecordOffsetName,
lastAppliedRecordTimestampName,
- lastAppliedRecordLagMsName
+ lastAppliedRecordLagMsName,
+ metadataLoadErrorCountName,
+ metadataApplyErrorCountName
).foreach(metrics.removeMetric)
}
}
diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
index c961d71bbe58..ecee13c49824 100644
--- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
+++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
@@ -24,6 +24,7 @@
import kafka.server.KafkaConfig$;
import kafka.server.KafkaRaftServer;
import kafka.server.MetaProperties;
+import kafka.server.metadata.BrokerServerMetrics$;
import kafka.tools.StorageTool;
import kafka.utils.Logging;
import org.apache.kafka.clients.CommonClientConfigs;
@@ -36,10 +37,12 @@
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.controller.BootstrapMetadata;
import org.apache.kafka.controller.Controller;
+import org.apache.kafka.controller.MockControllerMetrics;
import org.apache.kafka.metadata.MetadataRecordSerde;
import org.apache.kafka.raft.RaftConfig;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
+import org.apache.kafka.server.fault.MockFaultHandler;
import org.apache.kafka.test.TestUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -115,6 +118,8 @@ public void close() {
public static class Builder {
private TestKitNodes nodes;
private Map configProps = new HashMap<>();
+ private MockFaultHandler metadataFaultHandler = new MockFaultHandler("metadataFaultHandler");
+ private MockFaultHandler fatalFaultHandler = new MockFaultHandler("fatalFaultHandler");
public Builder(TestKitNodes nodes) {
this.nodes = nodes;
@@ -125,6 +130,11 @@ public Builder setConfigProp(String key, String value) {
return this;
}
+ public Builder setMetadataFaultHandler(MockFaultHandler metadataFaultHandler) {
+ this.metadataFaultHandler = metadataFaultHandler;
+ return this;
+ }
+
public KafkaClusterTestKit build() throws Exception {
Map controllers = new HashMap<>();
Map brokers = new HashMap<>();
@@ -186,11 +196,14 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS
raftManager,
Time.SYSTEM,
new Metrics(),
+ new MockControllerMetrics(),
Option.apply(threadNamePrefix),
connectFutureManager.future,
KafkaRaftServer.configSchema(),
raftManager.apiVersions(),
- bootstrapMetadata
+ bootstrapMetadata,
+ metadataFaultHandler,
+ fatalFaultHandler
);
controllers.put(node.id(), controller);
controller.socketServerFirstBoundPortFuture().whenComplete((port, e) -> {
@@ -240,15 +253,20 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS
Time.SYSTEM, new Metrics(), Option.apply(threadNamePrefix), connectFutureManager.future);
raftManagers.put(node.id(), raftManager);
}
+ Metrics metrics = new Metrics();
BrokerServer broker = new BrokerServer(
config,
nodes.brokerProperties(node.id()),
raftManager,
Time.SYSTEM,
- new Metrics(),
+ metrics,
+ BrokerServerMetrics$.MODULE$.apply(metrics),
Option.apply(threadNamePrefix),
JavaConverters.asScalaBuffer(Collections.emptyList()).toSeq(),
- connectFutureManager.future
+ connectFutureManager.future,
+ fatalFaultHandler,
+ metadataFaultHandler,
+ metadataFaultHandler
);
brokers.put(node.id(), broker);
}
@@ -273,7 +291,8 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS
throw e;
}
return new KafkaClusterTestKit(executorService, nodes, controllers,
- brokers, raftManagers, connectFutureManager, baseDirectory);
+ brokers, raftManagers, connectFutureManager, baseDirectory,
+ metadataFaultHandler, fatalFaultHandler);
}
private String listeners(int node) {
@@ -314,14 +333,20 @@ static private void setupNodeDirectories(File baseDirectory,
private final Map> raftManagers;
private final ControllerQuorumVotersFutureManager controllerQuorumVotersFutureManager;
private final File baseDirectory;
-
- private KafkaClusterTestKit(ExecutorService executorService,
- TestKitNodes nodes,
- Map controllers,
- Map brokers,
- Map> raftManagers,
- ControllerQuorumVotersFutureManager controllerQuorumVotersFutureManager,
- File baseDirectory) {
+ private final MockFaultHandler metadataFaultHandler;
+ private final MockFaultHandler fatalFaultHandler;
+
+ private KafkaClusterTestKit(
+ ExecutorService executorService,
+ TestKitNodes nodes,
+ Map controllers,
+ Map brokers,
+ Map> raftManagers,
+ ControllerQuorumVotersFutureManager controllerQuorumVotersFutureManager,
+ File baseDirectory,
+ MockFaultHandler metadataFaultHandler,
+ MockFaultHandler fatalFaultHandler
+ ) {
this.executorService = executorService;
this.nodes = nodes;
this.controllers = controllers;
@@ -329,6 +354,8 @@ private KafkaClusterTestKit(ExecutorService executorService,
this.raftManagers = raftManagers;
this.controllerQuorumVotersFutureManager = controllerQuorumVotersFutureManager;
this.baseDirectory = baseDirectory;
+ this.metadataFaultHandler = metadataFaultHandler;
+ this.fatalFaultHandler = fatalFaultHandler;
}
public void format() throws Exception {
@@ -520,6 +547,8 @@ public void close() throws Exception {
executorService.shutdownNow();
executorService.awaitTermination(5, TimeUnit.MINUTES);
}
+ metadataFaultHandler.maybeRethrowFirstException();
+ fatalFaultHandler.maybeRethrowFirstException();
}
private void waitForAllFutures(List>> futureEntries)
diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
index a2393cdccbcf..c4ca966f9abc 100755
--- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
+++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
@@ -24,6 +24,7 @@ import java.util.{Collections, Properties}
import java.util.concurrent.CompletableFuture
import javax.security.auth.login.Configuration
import kafka.raft.KafkaRaftManager
+import kafka.server.metadata.BrokerServerMetrics
import kafka.tools.StorageTool
import kafka.utils.{CoreUtils, Logging, TestInfoUtils, TestUtils}
import kafka.zk.{AdminZkClient, EmbeddedZookeeper, KafkaZkClient}
@@ -32,10 +33,12 @@ import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.common.security.JaasUtils
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.{Exit, Time}
-import org.apache.kafka.controller.BootstrapMetadata
+import org.apache.kafka.controller.{BootstrapMetadata, QuorumControllerMetrics}
import org.apache.kafka.metadata.MetadataRecordSerde
import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec}
import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
+import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler}
+import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.zookeeper.client.ZKClientConfig
import org.apache.zookeeper.{WatchedEvent, Watcher, ZooKeeper}
import org.junit.jupiter.api.Assertions._
@@ -81,26 +84,34 @@ class ZooKeeperQuorumImplementation(
}
}
-class KRaftQuorumImplementation(val raftManager: KafkaRaftManager[ApiMessageAndVersion],
- val controllerServer: ControllerServer,
- val metadataDir: File,
- val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]],
- val clusterId: String,
- val log: Logging) extends QuorumImplementation {
+class KRaftQuorumImplementation(
+ val raftManager: KafkaRaftManager[ApiMessageAndVersion],
+ val controllerServer: ControllerServer,
+ val metadataDir: File,
+ val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]],
+ val clusterId: String,
+ val log: Logging,
+ val faultHandler: FaultHandler
+) extends QuorumImplementation {
override def createBroker(
config: KafkaConfig,
time: Time,
startup: Boolean,
threadNamePrefix: Option[String],
): KafkaBroker = {
+ val metrics = new Metrics()
val broker = new BrokerServer(config = config,
metaProps = new MetaProperties(clusterId, config.nodeId),
raftManager = raftManager,
time = time,
- metrics = new Metrics(),
+ metrics = metrics,
+ brokerMetrics = BrokerServerMetrics(metrics),
threadNamePrefix = Some("Broker%02d_".format(config.nodeId)),
initialOfflineDirs = Seq(),
- controllerQuorumVotersFuture = controllerQuorumVotersFuture)
+ controllerQuorumVotersFuture = controllerQuorumVotersFuture,
+ fatalFaultHandler = faultHandler,
+ metadataLoadingFaultHandler = faultHandler,
+ metadataPublishingFaultHandler = faultHandler)
if (startup) broker.startup()
broker
}
@@ -188,6 +199,8 @@ abstract class QuorumTestHarness extends Logging {
}
}
+ val faultHandler = new MockFaultHandler("quorumTestHarnessFaultHandler")
+
// Note: according to the junit documentation: "JUnit Jupiter does not guarantee the execution
// order of multiple @BeforeEach methods that are declared within a single test class or test
// interface." Therefore, if you have things you would like to do before each test case runs, it
@@ -303,11 +316,14 @@ abstract class QuorumTestHarness extends Logging {
raftManager = raftManager,
time = Time.SYSTEM,
metrics = controllerMetrics,
+ controllerMetrics = new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry(), Time.SYSTEM),
threadNamePrefix = Option(threadNamePrefix),
controllerQuorumVotersFuture = controllerQuorumVotersFuture,
configSchema = KafkaRaftServer.configSchema,
raftApiVersions = raftManager.apiVersions,
bootstrapMetadata = BootstrapMetadata.create(metadataVersion, bootstrapRecords.asJava),
+ metadataFaultHandler = faultHandler,
+ fatalFaultHandler = faultHandler,
)
controllerServer.socketServerFirstBoundPortFuture.whenComplete((port, e) => {
if (e != null) {
@@ -331,7 +347,8 @@ abstract class QuorumTestHarness extends Logging {
metadataDir,
controllerQuorumVotersFuture,
metaProperties.clusterId,
- this)
+ this,
+ faultHandler)
}
private def newZooKeeperQuorum(): ZooKeeperQuorumImplementation = {
@@ -374,6 +391,7 @@ abstract class QuorumTestHarness extends Logging {
}
System.clearProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)
Configuration.setConfiguration(null)
+ faultHandler.maybeRethrowFirstException()
}
// Trigger session expiry by reusing the session id in another client
diff --git a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala
index 503ce7d2beef..f8dccd17d0d8 100644
--- a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala
+++ b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala
@@ -78,7 +78,8 @@ class RaftClusterSnapshotTest {
raftManager.replicatedLog.latestSnapshot.get(),
new MetadataRecordSerde(),
BufferSupplier.create(),
- 1
+ 1,
+ true
)
) { snapshot =>
// Check that the snapshot is non-empty
diff --git a/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala b/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala
index df114ef59e58..ea2b439c166a 100644
--- a/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala
+++ b/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala
@@ -37,12 +37,14 @@ final class BrokerServerMetricsTest {
val expectedMetrics = Set(
new MetricName("last-applied-record-offset", expectedGroup, "", Collections.emptyMap()),
new MetricName("last-applied-record-timestamp", expectedGroup, "", Collections.emptyMap()),
- new MetricName("last-applied-record-lag-ms", expectedGroup, "", Collections.emptyMap())
+ new MetricName("last-applied-record-lag-ms", expectedGroup, "", Collections.emptyMap()),
+ new MetricName("metadata-load-error-count", expectedGroup, "", Collections.emptyMap()),
+ new MetricName("metadata-apply-error-count", expectedGroup, "", Collections.emptyMap())
)
TestUtils.resource(BrokerServerMetrics(metrics)) { brokerMetrics =>
val metricsMap = metrics.metrics().asScala.filter{ case (name, _) => name.group == expectedGroup }
- assertEquals(3, metricsMap.size)
+ assertEquals(expectedMetrics.size, metricsMap.size)
metricsMap.foreach { case (name, metric) =>
assertTrue(expectedMetrics.contains(name))
}
@@ -85,4 +87,36 @@ final class BrokerServerMetricsTest {
assertEquals(time.milliseconds - timestamp, lagMetric.metricValue.asInstanceOf[Long])
}
}
+
+ @Test
+ def testMetadataLoadErrorCount(): Unit = {
+ val time = new MockTime()
+ val metrics = new Metrics(time)
+ TestUtils.resource(BrokerServerMetrics(metrics)) { brokerMetrics =>
+ val metadataLoadErrorCountMetric = metrics.metrics().get(brokerMetrics.metadataLoadErrorCountName)
+
+ assertEquals(0L, metadataLoadErrorCountMetric.metricValue.asInstanceOf[Long])
+
+ // Update metric value and check
+ val errorCount = 100
+ brokerMetrics.metadataLoadErrorCount.set(errorCount)
+ assertEquals(errorCount, metadataLoadErrorCountMetric.metricValue.asInstanceOf[Long])
+ }
+ }
+
+ @Test
+ def testMetadataApplyErrorCount(): Unit = {
+ val time = new MockTime()
+ val metrics = new Metrics(time)
+ TestUtils.resource(BrokerServerMetrics(metrics)) { brokerMetrics =>
+ val metadataApplyErrorCountMetric = metrics.metrics().get(brokerMetrics.metadataApplyErrorCountName)
+
+ assertEquals(0L, metadataApplyErrorCountMetric.metricValue.asInstanceOf[Long])
+
+ // Update metric value and check
+ val errorCount = 100
+ brokerMetrics.metadataApplyErrorCount.set(errorCount)
+ assertEquals(errorCount, metadataApplyErrorCountMetric.metricValue.asInstanceOf[Long])
+ }
+ }
}
diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
index 503821957944..948abc6c3b87 100644
--- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
+++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
@@ -46,7 +46,7 @@ import java.nio.ByteBuffer
import java.util.Optional
import java.util.concurrent.{CountDownLatch, Semaphore}
import kafka.server.epoch.LeaderEpochFileCache
-import kafka.server.metadata.KRaftMetadataCache
+import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache}
import org.apache.kafka.clients.ClientResponse
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.replica.ClientMetadata
@@ -55,6 +55,8 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0
import org.apache.kafka.server.metrics.KafkaYammerMetrics
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.ValueSource
import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._
@@ -1375,8 +1377,11 @@ class PartitionTest extends AbstractPartitionTest {
assertEquals(alterPartitionListener.failures.get, 1)
}
- @Test
- def testIsrNotExpandedIfReplicaIsFenced(): Unit = {
+ @ParameterizedTest
+ @ValueSource(strings = Array("zk", "kraft"))
+ def testIsrNotExpandedIfReplicaIsFencedOrShutdown(quorum: String): Unit = {
+ val kraft = quorum == "kraft"
+
val log = logManager.getOrCreateLog(topicPartition, topicId = None)
seedLogData(log, numRecords = 10, leaderEpoch = 4)
@@ -1386,7 +1391,19 @@ class PartitionTest extends AbstractPartitionTest {
val replicas = List(brokerId, remoteBrokerId)
val isr = Set(brokerId)
- val metadataCache = mock(classOf[KRaftMetadataCache])
+ val metadataCache: MetadataCache = if (kraft) mock(classOf[KRaftMetadataCache]) else mock(classOf[ZkMetadataCache])
+
+ // Mark the remote broker as eligible or ineligible in the metadata cache of the leader.
+ // When using kraft, we can make the broker ineligible by fencing it.
+ // In ZK mode, we must mark the broker as alive for it to be eligible.
+ def markRemoteReplicaEligible(eligible: Boolean): Unit = {
+ if (kraft) {
+ when(metadataCache.asInstanceOf[KRaftMetadataCache].isBrokerFenced(remoteBrokerId)).thenReturn(!eligible)
+ } else {
+ when(metadataCache.hasAliveBroker(remoteBrokerId)).thenReturn(eligible)
+ }
+ }
+
val partition = new Partition(
topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs,
@@ -1414,6 +1431,8 @@ class PartitionTest extends AbstractPartitionTest {
assertEquals(isr, partition.partitionState.isr)
assertEquals(isr, partition.partitionState.maximalIsr)
+ markRemoteReplicaEligible(true)
+
// Fetch to let the follower catch up to the log end offset and
// to check if an expansion is possible.
fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset)
@@ -1430,7 +1449,7 @@ class PartitionTest extends AbstractPartitionTest {
assertEquals(replicas.toSet, partition.partitionState.maximalIsr)
assertEquals(1, alterPartitionManager.isrUpdates.size)
- // Controller rejects the expansion because the broker is fenced.
+ // Controller rejects the expansion because the broker is fenced or offline.
alterPartitionManager.failIsrUpdate(Errors.INELIGIBLE_REPLICA)
// The leader reverts back to the previous ISR.
@@ -1439,8 +1458,8 @@ class PartitionTest extends AbstractPartitionTest {
assertFalse(partition.partitionState.isInflight)
assertEquals(0, alterPartitionManager.isrUpdates.size)
- // The leader eventually learns about the fenced broker.
- when(metadataCache.isBrokerFenced(remoteBrokerId)).thenReturn(true)
+ // The leader eventually learns about the fenced or offline broker.
+ markRemoteReplicaEligible(false)
// The follower fetches again.
fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset)
@@ -1451,8 +1470,8 @@ class PartitionTest extends AbstractPartitionTest {
assertFalse(partition.partitionState.isInflight)
assertEquals(0, alterPartitionManager.isrUpdates.size)
- // The broker is eventually unfenced.
- when(metadataCache.isBrokerFenced(remoteBrokerId)).thenReturn(false)
+ // The broker is eventually unfenced or brought back online.
+ markRemoteReplicaEligible(true)
// The follower fetches again.
fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset)
diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
index 57cbeafd4d03..0c8d000656a4 100644
--- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
@@ -26,11 +26,11 @@ import kafka.server.{KafkaConfig, KafkaServer, QuorumTestHarness}
import kafka.utils.{LogCaptureAppender, TestUtils}
import kafka.zk.{FeatureZNodeStatus, _}
import org.apache.kafka.common.errors.{ControllerMovedException, StaleBrokerEpochException}
-import org.apache.kafka.common.message.AlterPartitionRequestData
-import org.apache.kafka.common.message.AlterPartitionResponseData
+import org.apache.kafka.common.message.{AlterPartitionRequestData, AlterPartitionResponseData}
import org.apache.kafka.common.metrics.KafkaMetric
import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource
import org.apache.kafka.common.{ElectionType, TopicPartition, Uuid}
import org.apache.kafka.metadata.LeaderRecoveryState
import org.apache.kafka.server.common.MetadataVersion
@@ -40,8 +40,7 @@ import org.apache.log4j.Level
import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertTrue}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
-import org.junit.jupiter.params.provider.Arguments
-import org.junit.jupiter.params.provider.MethodSource
+import org.junit.jupiter.params.provider.{Arguments, MethodSource}
import org.mockito.Mockito.{doAnswer, spy, verify}
import org.mockito.invocation.InvocationOnMock
@@ -904,12 +903,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
).asJava)
).asJava)
- val future = new CompletableFuture[AlterPartitionResponseData]()
- controller.eventManager.put(AlterPartitionReceived(
- alterPartitionRequest,
- alterPartitionVersion,
- future.complete
- ))
+ val future = alterPartitionFuture(alterPartitionRequest, alterPartitionVersion)
val expectedAlterPartitionResponse = new AlterPartitionResponseData()
.setTopics(Seq(new AlterPartitionResponseData.TopicData()
@@ -968,12 +962,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
).asJava)
).asJava)
- val future = new CompletableFuture[AlterPartitionResponseData]()
- controller.eventManager.put(AlterPartitionReceived(
- alterPartitionRequest,
- ApiKeys.ALTER_PARTITION.latestVersion,
- future.complete
- ))
+ val future = alterPartitionFuture(alterPartitionRequest, ApiKeys.ALTER_PARTITION.latestVersion)
val expectedAlterPartitionResponse = new AlterPartitionResponseData()
.setTopics(Seq(new AlterPartitionResponseData.TopicData()
@@ -1002,47 +991,120 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val controller = getController().kafkaController
val leaderIsrAndControllerEpochMap = zkClient.getTopicPartitionStates(Seq(tp))
- val newLeaderAndIsr = leaderIsrAndControllerEpochMap(tp).leaderAndIsr
+ val oldLeaderAndIsr = leaderIsrAndControllerEpochMap(tp).leaderAndIsr
+ val newIsr = List(oldLeaderAndIsr.leader)
+ val newPartitionEpoch = oldLeaderAndIsr.partitionEpoch + 1
val topicId = controller.controllerContext.topicIds(tp.topic)
val brokerId = otherBroker.config.brokerId
val brokerEpoch = controller.controllerContext.liveBrokerIdAndEpochs(otherBroker.config.brokerId)
- // When re-sending the current ISR, we should not get and error or any ISR changes
- val alterPartitionRequest = new AlterPartitionRequestData()
- .setBrokerId(brokerId)
- .setBrokerEpoch(brokerEpoch)
- .setTopics(Seq(new AlterPartitionRequestData.TopicData()
- .setTopicId(topicId)
- .setPartitions(Seq(new AlterPartitionRequestData.PartitionData()
- .setPartitionIndex(tp.partition)
- .setLeaderEpoch(newLeaderAndIsr.leaderEpoch)
- .setPartitionEpoch(newLeaderAndIsr.partitionEpoch)
- .setNewIsr(newLeaderAndIsr.isr.map(Int.box).asJava)
- .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value)
+ def sendAndVerifyAlterPartitionResponse(requestPartitionEpoch: Int): Unit = {
+ val alterPartitionRequest = new AlterPartitionRequestData()
+ .setBrokerId(brokerId)
+ .setBrokerEpoch(brokerEpoch)
+ .setTopics(Seq(new AlterPartitionRequestData.TopicData()
+ .setTopicId(topicId)
+ .setPartitions(Seq(new AlterPartitionRequestData.PartitionData()
+ .setPartitionIndex(tp.partition)
+ .setLeaderEpoch(oldLeaderAndIsr.leaderEpoch)
+ .setPartitionEpoch(requestPartitionEpoch)
+ .setNewIsr(newIsr.map(Int.box).asJava)
+ .setLeaderRecoveryState(oldLeaderAndIsr.leaderRecoveryState.value)
+ ).asJava)
).asJava)
- ).asJava)
- val future = new CompletableFuture[AlterPartitionResponseData]()
- controller.eventManager.put(AlterPartitionReceived(
- alterPartitionRequest,
- AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION,
- future.complete
- ))
+ val future = alterPartitionFuture(alterPartitionRequest, AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION)
- val expectedAlterPartitionResponse = new AlterPartitionResponseData()
- .setTopics(Seq(new AlterPartitionResponseData.TopicData()
- .setTopicId(topicId)
- .setPartitions(Seq(new AlterPartitionResponseData.PartitionData()
- .setPartitionIndex(tp.partition)
- .setLeaderId(brokerId)
- .setLeaderEpoch(newLeaderAndIsr.leaderEpoch)
- .setPartitionEpoch(newLeaderAndIsr.partitionEpoch)
- .setIsr(newLeaderAndIsr.isr.map(Int.box).asJava)
- .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value)
+ // When re-sending an ISR update, we should not get and error or any ISR changes
+ val expectedAlterPartitionResponse = new AlterPartitionResponseData()
+ .setTopics(Seq(new AlterPartitionResponseData.TopicData()
+ .setTopicId(topicId)
+ .setPartitions(Seq(new AlterPartitionResponseData.PartitionData()
+ .setPartitionIndex(tp.partition)
+ .setLeaderId(brokerId)
+ .setLeaderEpoch(oldLeaderAndIsr.leaderEpoch)
+ .setPartitionEpoch(newPartitionEpoch)
+ .setIsr(newIsr.map(Int.box).asJava)
+ .setLeaderRecoveryState(oldLeaderAndIsr.leaderRecoveryState.value)
+ ).asJava)
).asJava)
+ assertEquals(expectedAlterPartitionResponse, future.get(10, TimeUnit.SECONDS))
+ }
+
+ // send a request, expect the partition epoch to be incremented
+ sendAndVerifyAlterPartitionResponse(oldLeaderAndIsr.partitionEpoch)
+
+ // re-send the same request with various partition epochs (less/equal/greater than the current
+ // epoch), expect it to succeed while the partition epoch remains the same
+ sendAndVerifyAlterPartitionResponse(oldLeaderAndIsr.partitionEpoch)
+ sendAndVerifyAlterPartitionResponse(newPartitionEpoch)
+ }
+
+ @ParameterizedTest
+ @ApiKeyVersionsSource(apiKey = ApiKeys.ALTER_PARTITION)
+ def testShutdownBrokerNotAddedToIsr(alterPartitionVersion: Short): Unit = {
+ servers = makeServers(2)
+ val controllerId = TestUtils.waitUntilControllerElected(zkClient)
+ val otherBroker = servers.find(_.config.brokerId != controllerId).get
+ val brokerId = otherBroker.config.brokerId
+ val tp = new TopicPartition("t", 0)
+ val assignment = Map(tp.partition -> Seq(controllerId, brokerId))
+ val fullIsr = List(controllerId, brokerId)
+ TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
+
+ // Shut down follower.
+ servers(brokerId).shutdown()
+ servers(brokerId).awaitShutdown()
+
+ val controller = getController().kafkaController
+ val leaderIsrAndControllerEpochMap = controller.controllerContext.partitionsLeadershipInfo
+ val leaderAndIsr = leaderIsrAndControllerEpochMap(tp).leaderAndIsr
+ val topicId = controller.controllerContext.topicIds(tp.topic)
+ val controllerEpoch = controller.controllerContext.liveBrokerIdAndEpochs(controllerId)
+
+ // We expect only the controller (online broker) to be in ISR
+ assertEquals(List(controllerId), leaderAndIsr.isr)
+
+ val requestTopic = new AlterPartitionRequestData.TopicData()
+ .setPartitions(Seq(new AlterPartitionRequestData.PartitionData()
+ .setPartitionIndex(tp.partition)
+ .setLeaderEpoch(leaderAndIsr.leaderEpoch)
+ .setPartitionEpoch(leaderAndIsr.partitionEpoch)
+ .setNewIsr(fullIsr.map(Int.box).asJava)
+ .setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value)).asJava)
+ if (alterPartitionVersion > 1) requestTopic.setTopicId(topicId) else requestTopic.setTopicName(tp.topic)
+
+ // Try to update ISR to contain the offline broker.
+ val alterPartitionRequest = new AlterPartitionRequestData()
+ .setBrokerId(controllerId)
+ .setBrokerEpoch(controllerEpoch)
+ .setTopics(Seq(requestTopic).asJava)
+
+ val future = alterPartitionFuture(alterPartitionRequest, alterPartitionVersion)
+
+ val expectedError = if (alterPartitionVersion > 1) Errors.INELIGIBLE_REPLICA else Errors.OPERATION_NOT_ATTEMPTED
+ val expectedResponseTopic = new AlterPartitionResponseData.TopicData()
+ .setPartitions(Seq(new AlterPartitionResponseData.PartitionData()
+ .setPartitionIndex(tp.partition)
+ .setErrorCode(expectedError.code())
+ .setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value)
).asJava)
+ if (alterPartitionVersion > 1) expectedResponseTopic.setTopicId(topicId) else expectedResponseTopic.setTopicName(tp.topic)
+
+ // We expect an ineligble replica error response for the partition.
+ val expectedAlterPartitionResponse = new AlterPartitionResponseData()
+ .setTopics(Seq(expectedResponseTopic).asJava)
+ val newLeaderIsrAndControllerEpochMap = controller.controllerContext.partitionsLeadershipInfo
+ val newLeaderAndIsr = newLeaderIsrAndControllerEpochMap(tp).leaderAndIsr
assertEquals(expectedAlterPartitionResponse, future.get(10, TimeUnit.SECONDS))
+ assertEquals(List(controllerId), newLeaderAndIsr.isr)
+
+ // Bring replica back online.
+ servers(brokerId).startup()
+
+ // Wait for broker to rejoin ISR.
+ TestUtils.waitUntilTrue(() => fullIsr == zkClient.getTopicPartitionState(tp).get.leaderAndIsr.isr, "Replica did not rejoin ISR.")
}
@Test
@@ -1100,7 +1162,6 @@ class ControllerIntegrationTest extends QuorumTestHarness {
assertAlterPartition(
partitionError = Errors.UNKNOWN_TOPIC_ID,
- topicPartition = tp,
topicIdOpt = Some(Uuid.randomUuid())
)
@@ -1118,9 +1179,15 @@ class ControllerIntegrationTest extends QuorumTestHarness {
assertAlterPartition(
partitionError = Errors.INVALID_UPDATE_VERSION,
+ isr = Set(leaderId),
partitionEpoch = partitionEpoch - 1
)
+ assertAlterPartition(
+ partitionError = Errors.INVALID_UPDATE_VERSION,
+ partitionEpoch = partitionEpoch + 1
+ )
+
assertAlterPartition(
partitionError = Errors.FENCED_LEADER_EPOCH,
leaderEpoch = leaderEpoch - 1
@@ -1322,12 +1389,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
.setNewIsr(isr.toList.map(Int.box).asJava)
.setLeaderRecoveryState(leaderRecoveryState)).asJava)).asJava)
- val future = new CompletableFuture[AlterPartitionResponseData]()
- getController().kafkaController.eventManager.put(AlterPartitionReceived(
- alterPartitionRequest,
- if (topicIdOpt.isDefined) AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION else 1,
- future.complete
- ))
+ val future = alterPartitionFuture(alterPartitionRequest, if (topicIdOpt.isDefined) AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION else 1)
val expectedAlterPartitionResponse = if (topLevelError != Errors.NONE) {
new AlterPartitionResponseData().setErrorCode(topLevelError.code)
@@ -1802,4 +1864,15 @@ class ControllerIntegrationTest extends QuorumTestHarness {
servers.filter(s => s.config.brokerId == controllerId).head
}
+ private def alterPartitionFuture(alterPartitionRequest: AlterPartitionRequestData,
+ alterPartitionVersion: Short): CompletableFuture[AlterPartitionResponseData] = {
+ val future = new CompletableFuture[AlterPartitionResponseData]()
+ getController().kafkaController.eventManager.put(AlterPartitionReceived(
+ alterPartitionRequest,
+ alterPartitionVersion,
+ future.complete
+ ))
+ future
+ }
+
}
diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala
index b21fe877f208..29de3c0f2428 100644
--- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala
+++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala
@@ -233,16 +233,21 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
@ValueSource(strings = Array("kraft"))
def testKRaftControllerMetrics(quorum: String): Unit = {
val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics
-
- assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=ActiveControllerCount"), 1)
- assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=OfflinePartitionsCount"), 1)
- assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=PreferredReplicaImbalanceCount"), 1)
- assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=GlobalTopicCount"), 1)
- assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=GlobalPartitionCount"), 1)
- assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=LastCommittedRecordOffset"), 1)
- assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=LastAppliedRecordOffset"), 1)
- assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=LastAppliedRecordTimestamp"), 1)
- assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=LastAppliedRecordLagMs"), 1)
+ Set(
+ "kafka.controller:type=KafkaController,name=ActiveControllerCount",
+ "kafka.controller:type=KafkaController,name=GlobalPartitionCount",
+ "kafka.controller:type=KafkaController,name=GlobalTopicCount",
+ "kafka.controller:type=KafkaController,name=LastAppliedRecordLagMs",
+ "kafka.controller:type=KafkaController,name=LastAppliedRecordOffset",
+ "kafka.controller:type=KafkaController,name=LastAppliedRecordTimestamp",
+ "kafka.controller:type=KafkaController,name=LastCommittedRecordOffset",
+ "kafka.controller:type=KafkaController,name=MetadataErrorCount",
+ "kafka.controller:type=KafkaController,name=OfflinePartitionsCount",
+ "kafka.controller:type=KafkaController,name=PreferredReplicaImbalanceCount",
+ ).foreach(expected => {
+ assertEquals(1, metrics.keySet.asScala.count(_.getMBeanName.equals(expected)),
+ s"Unable to find ${expected}")
+ })
}
/**
diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala
index 6de448f28029..6c8c2599d296 100644
--- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala
@@ -27,12 +27,20 @@ import org.apache.kafka.common.{Endpoint, Uuid}
import org.apache.kafka.image.{MetadataDelta, MetadataImage}
import org.apache.kafka.metadata.{BrokerRegistration, RecordTestUtils, VersionRange}
import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
+import org.apache.kafka.server.fault.MockFaultHandler
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
-import org.junit.jupiter.api.Test
+import org.junit.jupiter.api.{AfterEach, Test}
import scala.jdk.CollectionConverters._
class BrokerMetadataListenerTest {
+ private val metadataLoadingFaultHandler = new MockFaultHandler("metadata loading")
+
+ @AfterEach
+ def verifyNoFaults(): Unit = {
+ metadataLoadingFaultHandler.maybeRethrowFirstException()
+ }
+
private def newBrokerMetadataListener(
metrics: BrokerServerMetrics = BrokerServerMetrics(new Metrics()),
snapshotter: Option[MetadataSnapshotter] = None,
@@ -44,7 +52,8 @@ class BrokerMetadataListenerTest {
threadNamePrefix = None,
maxBytesBetweenSnapshots = maxBytesBetweenSnapshots,
snapshotter = snapshotter,
- brokerMetrics = metrics
+ brokerMetrics = metrics,
+ metadataLoadingFaultHandler = metadataLoadingFaultHandler
)
}
@@ -77,6 +86,8 @@ class BrokerMetadataListenerTest {
assertEquals(100L, listener.highestMetadataOffset)
assertEquals(0L, metrics.lastAppliedRecordOffset.get)
assertEquals(0L, metrics.lastAppliedRecordTimestamp.get)
+ assertEquals(0L, metrics.metadataLoadErrorCount.get)
+ assertEquals(0L, metrics.metadataApplyErrorCount.get)
val fencedTimestamp = 500L
val fencedLastOffset = 200L
@@ -110,6 +121,8 @@ class BrokerMetadataListenerTest {
assertEquals(fencedLastOffset, metrics.lastAppliedRecordOffset.get)
assertEquals(fencedTimestamp, metrics.lastAppliedRecordTimestamp.get)
+ assertEquals(0L, metrics.metadataLoadErrorCount.get)
+ assertEquals(0L, metrics.metadataApplyErrorCount.get)
} finally {
listener.close()
}
diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala
index 6742530ef514..652b8b3a0c29 100644
--- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala
+++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala
@@ -17,17 +17,16 @@
package unit.kafka.server.metadata
-import java.util.Collections.{singleton, singletonMap}
+import java.util.Collections.{singleton, singletonList, singletonMap}
import java.util.Properties
import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
-
import kafka.log.UnifiedLog
-import kafka.server.KafkaConfig
+import kafka.server.{BrokerServer, KafkaConfig}
import kafka.server.metadata.BrokerMetadataPublisher
import kafka.testkit.{KafkaClusterTestKit, TestKitNodes}
import kafka.utils.TestUtils
import org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET
-import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ConfigEntry}
+import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ConfigEntry, NewTopic}
import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.config.ConfigResource.Type.BROKER
import org.apache.kafka.common.utils.Exit
@@ -35,10 +34,12 @@ import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.image.{MetadataImageTest, TopicImage, TopicsImage}
import org.apache.kafka.metadata.LeaderRecoveryState
import org.apache.kafka.metadata.PartitionRegistration
-import org.junit.jupiter.api.Assertions.assertEquals
+import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler}
+import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertTrue}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito
+import org.mockito.Mockito.doThrow
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
@@ -176,6 +177,25 @@ class BrokerMetadataPublisherTest {
new TopicsImage(idsMap.asJava, namesMap.asJava)
}
+ private def newMockPublisher(
+ broker: BrokerServer,
+ errorHandler: FaultHandler = new MockFaultHandler("publisher")
+ ): BrokerMetadataPublisher = {
+ Mockito.spy(new BrokerMetadataPublisher(
+ conf = broker.config,
+ metadataCache = broker.metadataCache,
+ logManager = broker.logManager,
+ replicaManager = broker.replicaManager,
+ groupCoordinator = broker.groupCoordinator,
+ txnCoordinator = broker.transactionCoordinator,
+ clientQuotaMetadataManager = broker.clientQuotaMetadataManager,
+ dynamicConfigHandlers = broker.dynamicConfigHandlers.toMap,
+ _authorizer = Option.empty,
+ errorHandler,
+ errorHandler
+ ))
+ }
+
@Test
def testReloadUpdatedFilesWithoutConfigChange(): Unit = {
val cluster = new KafkaClusterTestKit.Builder(
@@ -187,17 +207,7 @@ class BrokerMetadataPublisherTest {
cluster.startup()
cluster.waitForReadyBrokers()
val broker = cluster.brokers().values().iterator().next()
- val publisher = Mockito.spy(new BrokerMetadataPublisher(
- conf = broker.config,
- metadataCache = broker.metadataCache,
- logManager = broker.logManager,
- replicaManager = broker.replicaManager,
- groupCoordinator = broker.groupCoordinator,
- txnCoordinator = broker.transactionCoordinator,
- clientQuotaMetadataManager = broker.clientQuotaMetadataManager,
- dynamicConfigHandlers = broker.dynamicConfigHandlers.toMap,
- _authorizer = Option.empty
- ))
+ val publisher = newMockPublisher(broker)
val numTimesReloadCalled = new AtomicInteger(0)
Mockito.when(publisher.reloadUpdatedFilesWithoutConfigChange(any[Properties]())).
thenAnswer(new Answer[Unit]() {
@@ -227,4 +237,39 @@ class BrokerMetadataPublisherTest {
cluster.close()
}
}
+
+ @Test
+ def testExceptionInUpdateCoordinator(): Unit = {
+ val errorHandler = new MockFaultHandler("publisher")
+ val cluster = new KafkaClusterTestKit.Builder(
+ new TestKitNodes.Builder().
+ setNumBrokerNodes(1).
+ setNumControllerNodes(1).build()).
+ setMetadataFaultHandler(errorHandler).build()
+ try {
+ cluster.format()
+ cluster.startup()
+ cluster.waitForReadyBrokers()
+ val broker = cluster.brokers().values().iterator().next()
+ TestUtils.retry(60000) {
+ assertNotNull(broker.metadataPublisher)
+ }
+ val publisher = Mockito.spy(broker.metadataPublisher)
+ doThrow(new RuntimeException("injected failure")).when(publisher).updateCoordinator(any(), any(), any(), any(), any())
+ broker.metadataListener.alterPublisher(publisher).get()
+ val admin = Admin.create(cluster.clientProperties())
+ try {
+ admin.createTopics(singletonList(new NewTopic("foo", 1, 1.toShort))).all().get()
+ } finally {
+ admin.close()
+ }
+ TestUtils.retry(60000) {
+ assertTrue(Option(errorHandler.firstException()).
+ flatMap(e => Option(e.getMessage())).getOrElse("(none)").contains("injected failure"))
+ }
+ } finally {
+ errorHandler.setIgnore(true)
+ cluster.close()
+ }
+ }
}
diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala
index e6702ee287fe..ff2326c92fa3 100644
--- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala
+++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer
import java.util.Optional
import java.util.concurrent.{CompletableFuture, CountDownLatch}
import org.apache.kafka.common.memory.MemoryPool
+import org.apache.kafka.common.metadata.FenceBrokerRecord
import org.apache.kafka.common.protocol.ByteBufferAccessor
import org.apache.kafka.common.record.{CompressionType, MemoryRecords}
import org.apache.kafka.common.utils.Time
@@ -33,6 +34,8 @@ import org.apache.kafka.snapshot.{MockRawSnapshotWriter, RecordsSnapshotWriter,
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
import org.junit.jupiter.api.Test
+import java.util
+import java.util.Arrays.asList
import scala.compat.java8.OptionConverters._
class BrokerMetadataSnapshotterTest {
@@ -104,4 +107,31 @@ class BrokerMetadataSnapshotterTest {
snapshotter.close()
}
}
+
+ class MockSnapshotWriter extends SnapshotWriter[ApiMessageAndVersion] {
+ val batches = new util.ArrayList[util.List[ApiMessageAndVersion]]
+ override def snapshotId(): OffsetAndEpoch = new OffsetAndEpoch(0, 0)
+ override def lastContainedLogOffset(): Long = 0
+ override def lastContainedLogEpoch(): Int = 0
+ override def isFrozen: Boolean = false
+ override def append(batch: util.List[ApiMessageAndVersion]): Unit = batches.add(batch)
+ override def freeze(): Unit = {}
+ override def close(): Unit = {}
+ }
+
+ @Test
+ def testRecordListConsumer(): Unit = {
+ val writer = new MockSnapshotWriter()
+ val consumer = new RecordListConsumer(3, writer)
+ val m = new ApiMessageAndVersion(new FenceBrokerRecord().setId(1).setEpoch(1), 0.toShort)
+ consumer.accept(asList(m, m))
+ assertEquals(asList(asList(m, m)), writer.batches)
+ consumer.accept(asList(m))
+ assertEquals(asList(asList(m, m), asList(m)), writer.batches)
+ consumer.accept(asList(m, m, m, m))
+ assertEquals(asList(asList(m, m), asList(m), asList(m, m, m), asList(m)), writer.batches)
+ consumer.accept(asList(m, m, m, m, m, m, m, m))
+ assertEquals(asList(asList(m, m), asList(m), asList(m, m, m), asList(m), asList(m, m, m), asList(m, m, m), asList(m, m)),
+ writer.batches)
+ }
}
diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle
index 849197112531..bfca97c49ea1 100644
--- a/gradle/dependencies.gradle
+++ b/gradle/dependencies.gradle
@@ -62,7 +62,7 @@ versions += [
checkstyle: "8.36.2",
commonsCli: "1.4",
dropwizardMetrics: "4.1.12.1",
- gradle: "7.5",
+ gradle: "7.5.1",
grgit: "4.1.1",
httpclient: "4.5.13",
easymock: "4.3",
@@ -70,7 +70,7 @@ versions += [
jacksonDatabind: "2.13.3",
jacoco: "0.8.7",
javassist: "3.27.0-GA",
- jetty: "9.4.44.v20210927",
+ jetty: "9.4.48.v20220622",
jersey: "2.34",
jline: "3.21.0",
jmh: "1.35",
@@ -81,7 +81,7 @@ versions += [
jfreechart: "1.0.0",
jopt: "5.0.4",
jose4j: "0.7.9",
- junit: "5.8.2",
+ junit: "5.9.0",
jqwik: "1.6.5",
kafka_0100: "0.10.0.1",
kafka_0101: "0.10.1.1",
@@ -104,7 +104,7 @@ versions += [
lz4: "1.8.0",
mavenArtifact: "3.8.4",
metrics: "2.2.0",
- mockito: "4.4.0",
+ mockito: "4.6.1",
netty: "4.1.79.Final",
powermock: "2.0.9",
reflections: "0.9.12",
diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties
index ef80eb5051f2..593bdcca0f33 100644
--- a/gradle/wrapper/gradle-wrapper.properties
+++ b/gradle/wrapper/gradle-wrapper.properties
@@ -1,6 +1,6 @@
distributionBase=GRADLE_USER_HOME
distributionPath=wrapper/dists
-distributionSha256Sum=97a52d145762adc241bad7fd18289bf7f6801e08ece6badf80402fe2b9f250b1
-distributionUrl=https\://services.gradle.org/distributions/gradle-7.5-all.zip
+distributionSha=db9c8211ed63f61f60292c69e80d89196f9eb36665e369e7f00ac4cc841c2219
+distributionUrl=https\://services.gradle.org/distributions/gradle-7.5.1-all.zip
zipStoreBase=GRADLE_USER_HOME
zipStorePath=wrapper/dists
diff --git a/gradlew b/gradlew
index 0c1e473f0ced..3c53cb3fbb88 100755
--- a/gradlew
+++ b/gradlew
@@ -118,7 +118,7 @@ esac
# Loop in case we encounter an error.
for attempt in 1 2 3; do
if [ ! -e "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" ]; then
- if ! curl -s -S --retry 3 -L -o "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" "https://raw.githubusercontent.com/gradle/gradle/v7.5.0/gradle/wrapper/gradle-wrapper.jar"; then
+ if ! curl -s -S --retry 3 -L -o "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" "https://raw.githubusercontent.com/gradle/gradle/v7.5.1/gradle/wrapper/gradle-wrapper.jar"; then
rm -f "$APP_HOME/gradle/wrapper/gradle-wrapper.jar"
# Pause for a bit before looping in case the server throttled us.
sleep 5
@@ -218,6 +218,12 @@ set -- \
org.gradle.wrapper.GradleWrapperMain \
"$@"
+# Stop when "xargs" is not available.
+if ! command -v xargs >/dev/null 2>&1
+then
+ die "xargs is not available"
+fi
+
# Use "xargs" to parse quoted args.
#
# With -n1 it outputs one arg per line, with the quotes and backslashes removed.
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java b/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java
index 6b470664d6e9..ff243aebfcb6 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java
@@ -51,6 +51,10 @@ public interface ControllerMetrics extends AutoCloseable {
int preferredReplicaImbalanceCount();
+ void incrementMetadataErrorCount();
+
+ int metadataErrorCount();
+
void setLastAppliedRecordOffset(long offset);
long lastAppliedRecordOffset();
diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
index 0290e0040c2d..ef87248f1342 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
@@ -91,6 +91,7 @@
import org.apache.kafka.server.authorizer.AclDeleteResult;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
+import org.apache.kafka.server.fault.FaultHandler;
import org.apache.kafka.server.policy.AlterConfigPolicy;
import org.apache.kafka.server.policy.CreateTopicPolicy;
import org.apache.kafka.snapshot.SnapshotReader;
@@ -149,6 +150,8 @@ public final class QuorumController implements Controller {
static public class Builder {
private final int nodeId;
private final String clusterId;
+ private FaultHandler fatalFaultHandler = null;
+ private FaultHandler metadataFaultHandler = null;
private Time time = Time.SYSTEM;
private String threadNamePrefix = null;
private LogContext logContext = null;
@@ -175,6 +178,16 @@ public Builder(int nodeId, String clusterId) {
this.clusterId = clusterId;
}
+ public Builder setFatalFaultHandler(FaultHandler fatalFaultHandler) {
+ this.fatalFaultHandler = fatalFaultHandler;
+ return this;
+ }
+
+ public Builder setMetadataFaultHandler(FaultHandler metadataFaultHandler) {
+ this.metadataFaultHandler = metadataFaultHandler;
+ return this;
+ }
+
public int nodeId() {
return nodeId;
}
@@ -287,6 +300,10 @@ public QuorumController build() throws Exception {
throw new IllegalStateException("You must specify an initial metadata.version using the kafka-storage tool.");
} else if (quorumFeatures == null) {
throw new IllegalStateException("You must specify the quorum features");
+ } else if (fatalFaultHandler == null) {
+ throw new IllegalStateException("You must specify a fatal fault handler.");
+ } else if (metadataFaultHandler == null) {
+ throw new IllegalStateException("You must specify a metadata fault handler.");
}
if (threadNamePrefix == null) {
@@ -304,6 +321,8 @@ public QuorumController build() throws Exception {
try {
queue = new KafkaEventQueue(time, logContext, threadNamePrefix + "QuorumController");
return new QuorumController(
+ fatalFaultHandler,
+ metadataFaultHandler,
logContext,
nodeId,
clusterId,
@@ -426,12 +445,18 @@ private Throwable handleEventException(String name,
exception.getClass().getSimpleName(), deltaUs);
return exception;
}
- log.warn("{}: failed with unknown server exception {} at epoch {} in {} us. " +
- "Renouncing leadership and reverting to the last committed offset {}.",
- name, exception.getClass().getSimpleName(), curClaimEpoch, deltaUs,
- lastCommittedOffset, exception);
- raftClient.resign(curClaimEpoch);
- renounce();
+ if (isActiveController()) {
+ log.warn("{}: failed with unknown server exception {} at epoch {} in {} us. " +
+ "Renouncing leadership and reverting to the last committed offset {}.",
+ name, exception.getClass().getSimpleName(), curClaimEpoch, deltaUs,
+ lastCommittedOffset, exception);
+ renounce();
+ } else {
+ log.warn("{}: failed with unknown server exception {} in {} us. " +
+ "The controller is already in standby mode.",
+ name, exception.getClass().getSimpleName(), deltaUs,
+ exception);
+ }
return new UnknownServerException(exception);
}
@@ -702,7 +727,7 @@ public void run() throws Exception {
long now = time.nanoseconds();
controllerMetrics.updateEventQueueTime(NANOSECONDS.toMillis(now - eventCreatedTimeNs));
int controllerEpoch = curClaimEpoch;
- if (controllerEpoch == -1) {
+ if (!isActiveController()) {
throw newNotControllerException();
}
startProcessingTimeNs = OptionalLong.of(now);
@@ -728,9 +753,26 @@ public void run() throws Exception {
"reaches offset {}", this, resultAndOffset.offset());
}
} else {
- // If the operation returned a batch of records, those records need to be
- // written before we can return our result to the user. Here, we hand off
- // the batch of records to the raft client. They will be written out
+ // 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;
+ for (ApiMessageAndVersion message : result.records()) {
+ try {
+ replay(message.message(), Optional.empty());
+ } catch (Throwable e) {
+ String failureMessage = String.format("Unable to apply %s record, which was " +
+ "%d of %d record(s) in the batch following last writeOffset %d.",
+ message.message().getClass().getSimpleName(), i, result.records().size(),
+ writeOffset);
+ throw fatalFaultHandler.handleFault(failureMessage, e);
+ }
+ i++;
+ }
+
+ // If the operation returned a batch of records, and those records could be applied,
+ // they need to be written before we can return our result to the user. Here, we
+ // hand off the batch of records to the raft client. They will be written out
// asynchronously.
final long offset;
if (result.isAtomic()) {
@@ -741,9 +783,6 @@ public void run() throws Exception {
op.processBatchEndOffset(offset);
updateWriteOffset(offset);
resultAndOffset = ControllerResultAndOffset.of(offset, result);
- for (ApiMessageAndVersion message : result.records()) {
- replay(message.message(), Optional.empty(), offset);
- }
snapshotRegistry.getOrCreateSnapshot(offset);
log.debug("Read-write operation {} will be completed when the log " +
@@ -789,9 +828,9 @@ private CompletableFuture prependWriteEvent(String name,
return event.future();
}
- private CompletableFuture appendWriteEvent(String name,
- OptionalLong deadlineNs,
- ControllerWriteOperation op) {
+ CompletableFuture appendWriteEvent(String name,
+ OptionalLong deadlineNs,
+ ControllerWriteOperation op) {
ControllerWriteEvent event = new ControllerWriteEvent<>(name, op);
if (deadlineNs.isPresent()) {
queue.appendWithDeadline(deadlineNs.getAsLong(), event);
@@ -841,11 +880,20 @@ public void handleCommit(BatchReader reader) {
"offset {} and epoch {}.", offset, epoch);
}
}
- for (ApiMessageAndVersion messageAndVersion : messages) {
- replay(messageAndVersion.message(), Optional.empty(), offset);
+ int i = 1;
+ for (ApiMessageAndVersion message : messages) {
+ try {
+ replay(message.message(), Optional.empty());
+ } 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());
+ throw metadataFaultHandler.handleFault(failureMessage, e);
+ }
+ i++;
}
}
-
updateLastCommittedState(offset, epoch, batch.appendTimestamp());
processedRecordsSize += batch.sizeInBytes();
}
@@ -862,13 +910,9 @@ public void handleSnapshot(SnapshotReader reader) {
appendRaftEvent(String.format("handleSnapshot[snapshotId=%s]", reader.snapshotId()), () -> {
try {
if (isActiveController()) {
- throw new IllegalStateException(
- String.format(
- "Asked to load snapshot (%s) when it is the active controller (%d)",
- reader.snapshotId(),
- curClaimEpoch
- )
- );
+ throw fatalFaultHandler.handleFault(String.format("Asked to load snapshot " +
+ "(%s) when it is the active controller (%d)", reader.snapshotId(),
+ curClaimEpoch));
}
log.info("Starting to replay snapshot ({}), from last commit offset ({}) and epoch ({})",
reader.snapshotId(), lastCommittedOffset, lastCommittedEpoch);
@@ -882,26 +926,28 @@ public void handleSnapshot(SnapshotReader reader) {
if (log.isDebugEnabled()) {
if (log.isTraceEnabled()) {
- log.trace(
- "Replaying snapshot ({}) batch with last offset of {}: {}",
- reader.snapshotId(),
- offset,
- messages
- .stream()
- .map(ApiMessageAndVersion::toString)
- .collect(Collectors.joining(", "))
- );
+ log.trace("Replaying snapshot ({}) batch with last offset of {}: {}",
+ reader.snapshotId(), offset, messages.stream().map(ApiMessageAndVersion::toString).
+ collect(Collectors.joining(", ")));
} else {
- log.debug(
- "Replaying snapshot ({}) batch with last offset of {}",
- reader.snapshotId(),
- offset
- );
+ log.debug("Replaying snapshot ({}) batch with last offset of {}",
+ reader.snapshotId(), offset);
}
}
- for (ApiMessageAndVersion messageAndVersion : messages) {
- replay(messageAndVersion.message(), Optional.of(reader.snapshotId()), offset);
+ int i = 1;
+ for (ApiMessageAndVersion message : messages) {
+ try {
+ replay(message.message(), Optional.of(reader.snapshotId()));
+ } catch (Throwable e) {
+ String failureMessage = String.format("Unable to apply %s record " +
+ "from snapshot %s on standby controller, which was %d of " +
+ "%d record(s) in the batch with baseOffset %d.",
+ message.message().getClass().getSimpleName(), reader.snapshotId(),
+ i, messages.size(), batch.baseOffset());
+ throw metadataFaultHandler.handleFault(failureMessage, e);
+ }
+ i++;
}
}
updateLastCommittedState(
@@ -968,10 +1014,14 @@ public void handleLeaderChange(LeaderAndEpoch newLeader) {
if (exception != null) {
log.error("Failed to bootstrap metadata.", exception);
appendRaftEvent("bootstrapMetadata[" + curClaimEpoch + "]", () -> {
- log.warn("Renouncing the leadership at oldEpoch {} since we could not bootstrap " +
- "metadata. Reverting to last committed offset {}.",
- curClaimEpoch, lastCommittedOffset);
- renounce();
+ if (isActiveController()) {
+ log.warn("Renouncing the leadership at oldEpoch {} since we could not bootstrap " +
+ "metadata. Reverting to last committed offset {}.",
+ curClaimEpoch, lastCommittedOffset);
+ renounce();
+ } else {
+ log.warn("Unable to bootstrap metadata on standby controller.");
+ }
});
}
});
@@ -998,10 +1048,12 @@ public void handleLeaderChange(LeaderAndEpoch newLeader) {
});
} else if (isActiveController()) {
appendRaftEvent("handleRenounce[" + curClaimEpoch + "]", () -> {
- log.warn("Renouncing the leadership at oldEpoch {} due to a metadata " +
- "log event. Reverting to last committed offset {}.", curClaimEpoch,
- lastCommittedOffset);
- renounce();
+ if (isActiveController()) {
+ log.warn("Renouncing the leadership at oldEpoch {} due to a metadata " +
+ "log event. Reverting to last committed offset {}.", curClaimEpoch,
+ lastCommittedOffset);
+ renounce();
+ }
});
}
}
@@ -1078,26 +1130,34 @@ private void updateLastCommittedState(long offset, int epoch, long timestamp) {
}
private void renounce() {
- curClaimEpoch = -1;
- controllerMetrics.setActive(false);
- purgatory.failAll(newNotControllerException());
-
- if (snapshotRegistry.hasSnapshot(lastCommittedOffset)) {
- newBytesSinceLastSnapshot = 0;
- snapshotRegistry.revertToSnapshot(lastCommittedOffset);
- authorizer.ifPresent(a -> a.loadSnapshot(aclControlManager.idToAcl()));
- } else {
- resetState();
- raftClient.unregister(metaLogListener);
- metaLogListener = new QuorumMetaLogListener();
- raftClient.register(metaLogListener);
+ try {
+ if (curClaimEpoch == -1) {
+ throw new RuntimeException("Cannot renounce leadership because we are not the " +
+ "current leader.");
+ }
+ raftClient.resign(curClaimEpoch);
+ curClaimEpoch = -1;
+ controllerMetrics.setActive(false);
+ purgatory.failAll(newNotControllerException());
+
+ if (snapshotRegistry.hasSnapshot(lastCommittedOffset)) {
+ newBytesSinceLastSnapshot = 0;
+ snapshotRegistry.revertToSnapshot(lastCommittedOffset);
+ authorizer.ifPresent(a -> a.loadSnapshot(aclControlManager.idToAcl()));
+ } else {
+ resetState();
+ raftClient.unregister(metaLogListener);
+ metaLogListener = new QuorumMetaLogListener();
+ raftClient.register(metaLogListener);
+ }
+ updateWriteOffset(-1);
+ clusterControl.deactivate();
+ cancelMaybeFenceReplicas();
+ cancelMaybeBalancePartitionLeaders();
+ cancelNextWriteNoOpRecord();
+ } catch (Throwable e) {
+ fatalFaultHandler.handleFault("exception while renouncing leadership", e);
}
-
- updateWriteOffset(-1);
- clusterControl.deactivate();
- cancelMaybeFenceReplicas();
- cancelMaybeBalancePartitionLeaders();
- cancelNextWriteNoOpRecord();
}
private void scheduleDeferredWriteEvent(String name, long deadlineNs,
@@ -1246,70 +1306,60 @@ private void handleFeatureControlChange() {
}
@SuppressWarnings("unchecked")
- private void replay(ApiMessage message, Optional snapshotId, long offset) {
- try {
- MetadataRecordType type = MetadataRecordType.fromId(message.apiKey());
- switch (type) {
- case REGISTER_BROKER_RECORD:
- clusterControl.replay((RegisterBrokerRecord) message);
- break;
- case UNREGISTER_BROKER_RECORD:
- clusterControl.replay((UnregisterBrokerRecord) message);
- break;
- case TOPIC_RECORD:
- replicationControl.replay((TopicRecord) message);
- break;
- case PARTITION_RECORD:
- replicationControl.replay((PartitionRecord) message);
- break;
- case CONFIG_RECORD:
- configurationControl.replay((ConfigRecord) message);
- break;
- case PARTITION_CHANGE_RECORD:
- replicationControl.replay((PartitionChangeRecord) message);
- break;
- case FENCE_BROKER_RECORD:
- clusterControl.replay((FenceBrokerRecord) message);
- break;
- case UNFENCE_BROKER_RECORD:
- clusterControl.replay((UnfenceBrokerRecord) message);
- break;
- case REMOVE_TOPIC_RECORD:
- replicationControl.replay((RemoveTopicRecord) message);
- break;
- case FEATURE_LEVEL_RECORD:
- featureControl.replay((FeatureLevelRecord) message);
- handleFeatureControlChange();
- break;
- case CLIENT_QUOTA_RECORD:
- clientQuotaControlManager.replay((ClientQuotaRecord) message);
- break;
- case PRODUCER_IDS_RECORD:
- producerIdControlManager.replay((ProducerIdsRecord) message);
- break;
- case BROKER_REGISTRATION_CHANGE_RECORD:
- clusterControl.replay((BrokerRegistrationChangeRecord) message);
- break;
- case ACCESS_CONTROL_ENTRY_RECORD:
- aclControlManager.replay((AccessControlEntryRecord) message, snapshotId);
- break;
- case REMOVE_ACCESS_CONTROL_ENTRY_RECORD:
- aclControlManager.replay((RemoveAccessControlEntryRecord) message, snapshotId);
- break;
- case NO_OP_RECORD:
- // NoOpRecord is an empty record and doesn't need to be replayed
- break;
- default:
- throw new RuntimeException("Unhandled record type " + type);
- }
- } catch (Exception e) {
- if (snapshotId.isPresent()) {
- log.error("Error replaying record {} from snapshot {} at last offset {}.",
- message.toString(), snapshotId.get(), offset, e);
- } else {
- log.error("Error replaying record {} at last offset {}.",
- message.toString(), offset, e);
- }
+ private void replay(ApiMessage message, Optional snapshotId) {
+ MetadataRecordType type = MetadataRecordType.fromId(message.apiKey());
+ switch (type) {
+ case REGISTER_BROKER_RECORD:
+ clusterControl.replay((RegisterBrokerRecord) message);
+ break;
+ case UNREGISTER_BROKER_RECORD:
+ clusterControl.replay((UnregisterBrokerRecord) message);
+ break;
+ case TOPIC_RECORD:
+ replicationControl.replay((TopicRecord) message);
+ break;
+ case PARTITION_RECORD:
+ replicationControl.replay((PartitionRecord) message);
+ break;
+ case CONFIG_RECORD:
+ configurationControl.replay((ConfigRecord) message);
+ break;
+ case PARTITION_CHANGE_RECORD:
+ replicationControl.replay((PartitionChangeRecord) message);
+ break;
+ case FENCE_BROKER_RECORD:
+ clusterControl.replay((FenceBrokerRecord) message);
+ break;
+ case UNFENCE_BROKER_RECORD:
+ clusterControl.replay((UnfenceBrokerRecord) message);
+ break;
+ case REMOVE_TOPIC_RECORD:
+ replicationControl.replay((RemoveTopicRecord) message);
+ break;
+ case FEATURE_LEVEL_RECORD:
+ featureControl.replay((FeatureLevelRecord) message);
+ handleFeatureControlChange();
+ break;
+ case CLIENT_QUOTA_RECORD:
+ clientQuotaControlManager.replay((ClientQuotaRecord) message);
+ break;
+ case PRODUCER_IDS_RECORD:
+ producerIdControlManager.replay((ProducerIdsRecord) message);
+ break;
+ case BROKER_REGISTRATION_CHANGE_RECORD:
+ clusterControl.replay((BrokerRegistrationChangeRecord) message);
+ break;
+ case ACCESS_CONTROL_ENTRY_RECORD:
+ aclControlManager.replay((AccessControlEntryRecord) message, snapshotId);
+ break;
+ case REMOVE_ACCESS_CONTROL_ENTRY_RECORD:
+ aclControlManager.replay((RemoveAccessControlEntryRecord) message, snapshotId);
+ break;
+ case NO_OP_RECORD:
+ // NoOpRecord is an empty record and doesn't need to be replayed
+ break;
+ default:
+ throw new RuntimeException("Unhandled record type " + type);
}
}
@@ -1344,8 +1394,24 @@ private void resetState() {
updateLastCommittedState(-1, -1, -1);
}
+ /**
+ * Handles faults that should normally be fatal to the process.
+ */
+ private final FaultHandler fatalFaultHandler;
+
+ /**
+ * Handles faults in metadata handling that are normally not fatal.
+ */
+ private final FaultHandler metadataFaultHandler;
+
+ /**
+ * The slf4j log context, used to create new loggers.
+ */
private final LogContext logContext;
+ /**
+ * The slf4j logger.
+ */
private final Logger log;
/**
@@ -1530,28 +1596,34 @@ private enum ImbalanceSchedule {
private final BootstrapMetadata bootstrapMetadata;
- private QuorumController(LogContext logContext,
- int nodeId,
- String clusterId,
- KafkaEventQueue queue,
- Time time,
- KafkaConfigSchema configSchema,
- RaftClient raftClient,
- QuorumFeatures quorumFeatures,
- short defaultReplicationFactor,
- int defaultNumPartitions,
- ReplicaPlacer replicaPlacer,
- long snapshotMaxNewRecordBytes,
- OptionalLong leaderImbalanceCheckIntervalNs,
- OptionalLong maxIdleIntervalNs,
- long sessionTimeoutNs,
- ControllerMetrics controllerMetrics,
- Optional createTopicPolicy,
- Optional alterConfigPolicy,
- ConfigurationValidator configurationValidator,
- Optional authorizer,
- Map staticConfig,
- BootstrapMetadata bootstrapMetadata) {
+ private QuorumController(
+ FaultHandler fatalFaultHandler,
+ FaultHandler metadataFaultHandler,
+ LogContext logContext,
+ int nodeId,
+ String clusterId,
+ KafkaEventQueue queue,
+ Time time,
+ KafkaConfigSchema configSchema,
+ RaftClient raftClient,
+ QuorumFeatures quorumFeatures,
+ short defaultReplicationFactor,
+ int defaultNumPartitions,
+ ReplicaPlacer replicaPlacer,
+ long snapshotMaxNewRecordBytes,
+ OptionalLong leaderImbalanceCheckIntervalNs,
+ OptionalLong maxIdleIntervalNs,
+ long sessionTimeoutNs,
+ ControllerMetrics controllerMetrics,
+ Optional createTopicPolicy,
+ Optional alterConfigPolicy,
+ ConfigurationValidator configurationValidator,
+ Optional authorizer,
+ Map staticConfig,
+ BootstrapMetadata bootstrapMetadata
+ ) {
+ this.fatalFaultHandler = fatalFaultHandler;
+ this.metadataFaultHandler = metadataFaultHandler;
this.logContext = logContext;
this.log = logContext.logger(QuorumController.class);
this.nodeId = nodeId;
diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java
index 5abf0d977069..b96a687b0f34 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java
@@ -26,6 +26,7 @@
import java.util.Arrays;
import java.util.Objects;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
public final class QuorumControllerMetrics implements ControllerMetrics {
@@ -47,6 +48,8 @@ public final class QuorumControllerMetrics implements ControllerMetrics {
"KafkaController", "OfflinePartitionsCount");
private final static MetricName PREFERRED_REPLICA_IMBALANCE_COUNT = getMetricName(
"KafkaController", "PreferredReplicaImbalanceCount");
+ private final static MetricName METADATA_ERROR_COUNT = getMetricName(
+ "KafkaController", "MetadataErrorCount");
private final static MetricName LAST_APPLIED_RECORD_OFFSET = getMetricName(
"KafkaController", "LastAppliedRecordOffset");
private final static MetricName LAST_COMMITTED_RECORD_OFFSET = getMetricName(
@@ -64,6 +67,7 @@ public final class QuorumControllerMetrics implements ControllerMetrics {
private volatile int globalPartitionCount;
private volatile int offlinePartitionCount;
private volatile int preferredReplicaImbalanceCount;
+ private volatile AtomicInteger metadataErrorCount;
private final AtomicLong lastAppliedRecordOffset = new AtomicLong(0);
private final AtomicLong lastCommittedRecordOffset = new AtomicLong(0);
private final AtomicLong lastAppliedRecordTimestamp = new AtomicLong(0);
@@ -74,6 +78,7 @@ public final class QuorumControllerMetrics implements ControllerMetrics {
private final Gauge globalTopicCountGauge;
private final Gauge offlinePartitionCountGauge;
private final Gauge preferredReplicaImbalanceCountGauge;
+ private final Gauge metadataErrorCountGauge;
private final Gauge lastAppliedRecordOffsetGauge;
private final Gauge lastCommittedRecordOffsetGauge;
private final Gauge lastAppliedRecordTimestampGauge;
@@ -93,6 +98,7 @@ public QuorumControllerMetrics(
this.globalPartitionCount = 0;
this.offlinePartitionCount = 0;
this.preferredReplicaImbalanceCount = 0;
+ this.metadataErrorCount = new AtomicInteger(0);
this.activeControllerCount = registry.newGauge(ACTIVE_CONTROLLER_COUNT, new Gauge() {
@Override
public Integer value() {
@@ -137,6 +143,12 @@ public Integer value() {
return preferredReplicaImbalanceCount;
}
});
+ this.metadataErrorCountGauge = registry.newGauge(METADATA_ERROR_COUNT, new Gauge() {
+ @Override
+ public Integer value() {
+ return metadataErrorCount.get();
+ }
+ });
lastAppliedRecordOffsetGauge = registry.newGauge(LAST_APPLIED_RECORD_OFFSET, new Gauge() {
@Override
public Long value() {
@@ -242,6 +254,15 @@ public int preferredReplicaImbalanceCount() {
return this.preferredReplicaImbalanceCount;
}
+ @Override
+ public void incrementMetadataErrorCount() {
+ this.metadataErrorCount.getAndIncrement();
+ }
+
+ @Override
+ public int metadataErrorCount() {
+ return this.metadataErrorCount.get();
+ }
@Override
public void setLastAppliedRecordOffset(long offset) {
lastAppliedRecordOffset.set(offset);
@@ -276,12 +297,15 @@ public long lastAppliedRecordTimestamp() {
public void close() {
Arrays.asList(
ACTIVE_CONTROLLER_COUNT,
+ FENCED_BROKER_COUNT,
+ ACTIVE_BROKER_COUNT,
EVENT_QUEUE_TIME_MS,
EVENT_QUEUE_PROCESSING_TIME_MS,
GLOBAL_TOPIC_COUNT,
GLOBAL_PARTITION_COUNT,
OFFLINE_PARTITION_COUNT,
PREFERRED_REPLICA_IMBALANCE_COUNT,
+ METADATA_ERROR_COUNT,
LAST_APPLIED_RECORD_OFFSET,
LAST_COMMITTED_RECORD_OFFSET,
LAST_APPLIED_RECORD_TIMESTAMP,
diff --git a/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java b/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java
index 5991fcc34f31..ca13d90ddeae 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java
@@ -17,6 +17,8 @@
package org.apache.kafka.controller;
+import java.util.concurrent.atomic.AtomicInteger;
+
public final class MockControllerMetrics implements ControllerMetrics {
private volatile boolean active = false;
private volatile int fencedBrokers = 0;
@@ -25,6 +27,7 @@ public final class MockControllerMetrics implements ControllerMetrics {
private volatile int partitions = 0;
private volatile int offlinePartitions = 0;
private volatile int preferredReplicaImbalances = 0;
+ private volatile AtomicInteger metadataErrors = new AtomicInteger(0);
private volatile long lastAppliedRecordOffset = 0;
private volatile long lastCommittedRecordOffset = 0;
private volatile long lastAppliedRecordTimestamp = 0;
@@ -111,6 +114,16 @@ public int preferredReplicaImbalanceCount() {
return this.preferredReplicaImbalances;
}
+ @Override
+ public void incrementMetadataErrorCount() {
+ this.metadataErrors.getAndIncrement();
+ }
+
+ @Override
+ public int metadataErrorCount() {
+ return this.metadataErrors.get();
+ }
+
@Override
public void setLastAppliedRecordOffset(long offset) {
lastAppliedRecordOffset = offset;
diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsTest.java
index 2ab999559435..400b860197e5 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsTest.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsTest.java
@@ -36,10 +36,13 @@ public void testKafkaControllerMetricNames() {
String expectedType = "KafkaController";
Set expectedMetricNames = Utils.mkSet(
"ActiveControllerCount",
+ "FencedBrokerCount",
+ "ActiveBrokerCount",
"GlobalTopicCount",
"GlobalPartitionCount",
"OfflinePartitionsCount",
"PreferredReplicaImbalanceCount",
+ "MetadataErrorCount",
"LastAppliedRecordLagMs",
"LastAppliedRecordOffset",
"LastAppliedRecordTimestamp",
@@ -125,6 +128,25 @@ public void testLastAppliedRecordMetrics() {
}
}
+ @Test
+ public void testMetadataErrorCount() {
+ MetricsRegistry registry = new MetricsRegistry();
+ MockTime time = new MockTime();
+ try {
+ try (QuorumControllerMetrics quorumControllerMetrics = new QuorumControllerMetrics(registry, time)) {
+ @SuppressWarnings("unchecked")
+ Gauge metadataErrorCount = (Gauge) registry
+ .allMetrics()
+ .get(metricName("KafkaController", "MetadataErrorCount"));
+ assertEquals(0, metadataErrorCount.value());
+ quorumControllerMetrics.incrementMetadataErrorCount();
+ assertEquals(1, metadataErrorCount.value());
+ }
+ } finally {
+ registry.shutdown();
+ }
+ }
+
private static void assertMetricsCreatedAndRemovedUponClose(String expectedType, Set expectedMetricNames) {
MetricsRegistry registry = new MetricsRegistry();
MockTime time = new MockTime();
@@ -151,10 +173,18 @@ private static MetricName metricName(String type, String name) {
}
private static void assertMetricsCreated(MetricsRegistry registry, Set expectedMetricNames, String expectedType) {
+ assertEquals(registry.allMetrics().keySet().stream()
+ .filter(k -> k.getType() == expectedType).count(),
+ expectedMetricNames.size());
expectedMetricNames.forEach(expectedName -> {
MetricName expectMetricName = metricName(expectedType, expectedName);
assertTrue(registry.allMetrics().containsKey(expectMetricName), "Missing metric: " + expectMetricName);
});
+ registry.allMetrics().forEach((actualMetricName, actualMetric) -> {
+ if (actualMetricName.getType() == expectedType) {
+ assertTrue(expectedMetricNames.contains(actualMetricName.getName()), "Unexpected metric: " + actualMetricName);
+ }
+ });
}
private static void assertMetricsRemoved(MetricsRegistry registry, Set expectedMetricNames, String expectedType) {
diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
index a62b1f682f06..e83928956268 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
@@ -45,6 +45,7 @@
import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
import org.apache.kafka.common.message.RequestHeaderData;
+import org.apache.kafka.common.metadata.ConfigRecord;
import org.apache.kafka.common.security.auth.KafkaPrincipal;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.common.config.ConfigResource;
@@ -738,7 +739,8 @@ private SnapshotReader createSnapshotReader(RawSnapshotRea
reader,
new MetadataRecordSerde(),
BufferSupplier.create(),
- Integer.MAX_VALUE
+ Integer.MAX_VALUE,
+ true
);
}
@@ -1181,6 +1183,30 @@ public void testQuorumControllerCompletesAuthorizerInitialLoad() throws Throwabl
}
}
+ @Test
+ public void testFatalMetadataReplayErrorOnActive() throws Throwable {
+ try (LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(3, Optional.empty())) {
+ try (QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv(logEnv, b -> {
+ })) {
+ QuorumController active = controlEnv.activeController();
+ CompletableFuture future = active.appendWriteEvent("errorEvent",
+ OptionalLong.empty(), () -> {
+ return ControllerResult.of(Collections.singletonList(new ApiMessageAndVersion(
+ new ConfigRecord().
+ setName(null).
+ setResourceName(null).
+ setResourceType((byte) 255).
+ setValue(null), (short) 0)), null);
+ });
+ assertThrows(ExecutionException.class, () -> future.get());
+ assertEquals(NullPointerException.class,
+ controlEnv.fatalFaultHandler().firstException().getCause().getClass());
+ controlEnv.fatalFaultHandler().setIgnore(true);
+ controlEnv.metadataFaultHandler().setIgnore(true);
+ }
+ }
+ }
+
private static void assertInitialLoadFuturesNotComplete(List authorizers) {
for (int i = 0; i < authorizers.size(); i++) {
assertFalse(authorizers.get(i).initialLoadFuture().isDone(),
diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java
index 4cc45a9774b3..40dd21c88d33 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java
@@ -34,6 +34,7 @@
import org.apache.kafka.metalog.LocalLogManagerTestEnv;
import org.apache.kafka.raft.LeaderAndEpoch;
import org.apache.kafka.server.common.MetadataVersion;
+import org.apache.kafka.server.fault.MockFaultHandler;
import org.apache.kafka.test.TestUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -44,6 +45,8 @@ public class QuorumControllerTestEnv implements AutoCloseable {
private final List controllers;
private final LocalLogManagerTestEnv logEnv;
+ private final MockFaultHandler fatalFaultHandler = new MockFaultHandler("fatalFaultHandler");
+ private final MockFaultHandler metadataFaultHandler = new MockFaultHandler("metadataFaultHandler");
public QuorumControllerTestEnv(
LocalLogManagerTestEnv logEnv,
@@ -84,6 +87,8 @@ public QuorumControllerTestEnv(
sessionTimeoutMillis.ifPresent(timeout -> {
builder.setSessionTimeoutNs(NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS));
});
+ builder.setFatalFaultHandler(fatalFaultHandler);
+ builder.setMetadataFaultHandler(metadataFaultHandler);
builderConsumer.accept(builder);
this.controllers.add(builder.build());
}
@@ -117,6 +122,14 @@ public List controllers() {
return controllers;
}
+ public MockFaultHandler fatalFaultHandler() {
+ return fatalFaultHandler;
+ }
+
+ public MockFaultHandler metadataFaultHandler() {
+ return metadataFaultHandler;
+ }
+
@Override
public void close() throws InterruptedException {
for (QuorumController controller : controllers) {
@@ -125,5 +138,7 @@ public void close() throws InterruptedException {
for (QuorumController controller : controllers) {
controller.close();
}
+ fatalFaultHandler.maybeRethrowFirstException();
+ metadataFaultHandler.maybeRethrowFirstException();
}
}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java b/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java
index 2c61dbcdc74d..f7fa18f20a4c 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java
@@ -41,6 +41,7 @@
import java.util.OptionalLong;
import java.util.Optional;
+import static org.apache.kafka.raft.KafkaRaftClient.MAX_BATCH_SIZE_BYTES;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -94,7 +95,7 @@ private SnapshotWriter createSnapshotWriter(
) {
return RecordsSnapshotWriter.createWithHeader(
() -> createNewSnapshot(new OffsetAndEpoch(committedOffset + 1, 1)),
- 1024,
+ MAX_BATCH_SIZE_BYTES,
MemoryPool.NONE,
new MockTime(),
lastContainedLogTime,
diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
index c8e39ae32896..e24d86bd873f 100644
--- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
+++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
@@ -496,7 +496,8 @@ private void scheduleLogCheck() {
snapshot.get(),
new MetadataRecordSerde(),
BufferSupplier.create(),
- Integer.MAX_VALUE
+ Integer.MAX_VALUE,
+ true
)
);
}
diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
index 53372728aab4..cac7a8a3cb99 100644
--- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
+++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
@@ -333,7 +333,12 @@ private void updateListenersProgress(long highWatermark) {
private Optional> latestSnapshot() {
return log.latestSnapshot().map(reader ->
- RecordsSnapshotReader.of(reader, serde, BufferSupplier.create(), MAX_BATCH_SIZE_BYTES)
+ RecordsSnapshotReader.of(reader,
+ serde,
+ BufferSupplier.create(),
+ MAX_BATCH_SIZE_BYTES,
+ true /* Validate batch CRC*/
+ )
);
}
@@ -2519,7 +2524,8 @@ private void fireHandleCommit(long baseOffset, Records records) {
serde,
BufferSupplier.create(),
MAX_BATCH_SIZE_BYTES,
- this
+ this,
+ true /* Validate batch CRC*/
)
);
}
diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java
index e95206100a30..61819a9dccae 100644
--- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java
+++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java
@@ -100,11 +100,12 @@ public static RecordsBatchReader of(
RecordSerde serde,
BufferSupplier bufferSupplier,
int maxBatchSize,
- CloseListener> closeListener
+ CloseListener> closeListener,
+ boolean doCrcValidation
) {
return new RecordsBatchReader<>(
baseOffset,
- new RecordsIterator<>(records, serde, bufferSupplier, maxBatchSize),
+ new RecordsIterator<>(records, serde, bufferSupplier, maxBatchSize, doCrcValidation),
closeListener
);
}
diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java
index 866f541fb248..ff415aa72ada 100644
--- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java
+++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java
@@ -41,6 +41,9 @@ public final class RecordsIterator implements Iterator>, AutoCloseab
private final RecordSerde serde;
private final BufferSupplier bufferSupplier;
private final int batchSize;
+ // Setting to true will make the RecordsIterator perform a CRC Validation
+ // on the batch header when iterating over them
+ private final boolean doCrcValidation;
private Iterator nextBatches = Collections.emptyIterator();
private Optional> nextBatch = Optional.empty();
@@ -54,12 +57,14 @@ public RecordsIterator(
Records records,
RecordSerde serde,
BufferSupplier bufferSupplier,
- int batchSize
+ int batchSize,
+ boolean doCrcValidation
) {
this.records = records;
this.serde = serde;
this.bufferSupplier = bufferSupplier;
this.batchSize = Math.max(batchSize, Records.HEADER_SIZE_UP_TO_MAGIC);
+ this.doCrcValidation = doCrcValidation;
}
@Override
@@ -163,7 +168,6 @@ private Optional> nextBatch() {
if (nextBatches.hasNext()) {
MutableRecordBatch nextBatch = nextBatches.next();
-
// Update the buffer position to reflect the read batch
allocatedBuffer.ifPresent(buffer -> buffer.position(buffer.position() + nextBatch.sizeInBytes()));
@@ -180,6 +184,11 @@ private Optional> nextBatch() {
}
private Batch readBatch(DefaultRecordBatch batch) {
+ if (doCrcValidation) {
+ // Perform a CRC validity check on this batch
+ batch.ensureValid();
+ }
+
final Batch result;
if (batch.isControlBatch()) {
result = Batch.control(
diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java
index 89ad26322299..92b695146c39 100644
--- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java
+++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java
@@ -104,11 +104,12 @@ public static RecordsSnapshotReader of(
RawSnapshotReader snapshot,
RecordSerde serde,
BufferSupplier bufferSupplier,
- int maxBatchSize
+ int maxBatchSize,
+ boolean doCrcValidation
) {
return new RecordsSnapshotReader<>(
snapshot.snapshotId(),
- new RecordsIterator<>(snapshot.records(), serde, bufferSupplier, maxBatchSize)
+ new RecordsIterator<>(snapshot.records(), serde, bufferSupplier, maxBatchSize, doCrcValidation)
);
}
diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java
index 4f79dc18cc6e..a6117a33ca0e 100644
--- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java
+++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java
@@ -1112,7 +1112,7 @@ private void assertCommittedData(RaftNode node) {
startOffset.set(snapshotId.offset);
try (SnapshotReader snapshot =
- RecordsSnapshotReader.of(log.readSnapshot(snapshotId).get(), node.intSerde, BufferSupplier.create(), Integer.MAX_VALUE)) {
+ RecordsSnapshotReader.of(log.readSnapshot(snapshotId).get(), node.intSerde, BufferSupplier.create(), Integer.MAX_VALUE, true)) {
// Expect only one batch with only one record
assertTrue(snapshot.hasNext());
Batch batch = snapshot.next();
diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java
index 6fe540711c26..ae8b1dfb8e2c 100644
--- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java
+++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java
@@ -100,7 +100,8 @@ private void testBatchReader(
serde,
bufferSupplier,
MAX_BATCH_BYTES,
- closeListener
+ closeListener,
+ true
);
for (TestBatch batch : expectedBatches) {
diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java
index 7d9848931205..9dfbfd62fbfe 100644
--- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java
+++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java
@@ -30,7 +30,9 @@
import java.util.stream.Stream;
import net.jqwik.api.ForAll;
import net.jqwik.api.Property;
+import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.record.CompressionType;
+import org.apache.kafka.common.record.DefaultRecordBatch;
import org.apache.kafka.common.record.FileRecords;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.Records;
@@ -42,6 +44,7 @@
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import org.mockito.Mockito;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -60,7 +63,7 @@ private static Stream emptyRecords() throws IOException {
@ParameterizedTest
@MethodSource("emptyRecords")
void testEmptyRecords(Records records) {
- testIterator(Collections.emptyList(), records);
+ testIterator(Collections.emptyList(), records, true);
}
@Property
@@ -71,7 +74,7 @@ public void testMemoryRecords(
List> batches = createBatches(seed);
MemoryRecords memRecords = buildRecords(compressionType, batches);
- testIterator(batches, memRecords);
+ testIterator(batches, memRecords, true);
}
@Property
@@ -85,18 +88,58 @@ public void testFileRecords(
FileRecords fileRecords = FileRecords.open(TestUtils.tempFile());
fileRecords.append(memRecords);
- testIterator(batches, fileRecords);
+ testIterator(batches, fileRecords, true);
+ fileRecords.close();
+ }
+
+ @Property
+ public void testCrcValidation(
+ @ForAll CompressionType compressionType,
+ @ForAll long seed
+ ) throws IOException {
+ List> batches = createBatches(seed);
+ MemoryRecords memRecords = buildRecords(compressionType, batches);
+ // Read the Batch CRC for the first batch from the buffer
+ ByteBuffer readBuf = memRecords.buffer();
+ readBuf.position(DefaultRecordBatch.CRC_OFFSET);
+ int actualCrc = readBuf.getInt();
+ // Corrupt the CRC on the first batch
+ memRecords.buffer().putInt(DefaultRecordBatch.CRC_OFFSET, actualCrc + 1);
+
+ assertThrows(CorruptRecordException.class, () -> testIterator(batches, memRecords, true));
+
+ FileRecords fileRecords = FileRecords.open(TestUtils.tempFile());
+ fileRecords.append(memRecords);
+ assertThrows(CorruptRecordException.class, () -> testIterator(batches, fileRecords, true));
+
+ // Verify check does not trigger when doCrcValidation is false
+ assertDoesNotThrow(() -> testIterator(batches, memRecords, false));
+ assertDoesNotThrow(() -> testIterator(batches, fileRecords, false));
+
+ // Fix the corruption
+ memRecords.buffer().putInt(DefaultRecordBatch.CRC_OFFSET, actualCrc);
+
+ // Verify check does not trigger when the corruption is fixed
+ assertDoesNotThrow(() -> testIterator(batches, memRecords, true));
+ FileRecords moreFileRecords = FileRecords.open(TestUtils.tempFile());
+ moreFileRecords.append(memRecords);
+ assertDoesNotThrow(() -> testIterator(batches, moreFileRecords, true));
+
+ fileRecords.close();
+ moreFileRecords.close();
}
private void testIterator(
List> expectedBatches,
- Records records
+ Records records,
+ boolean validateCrc
) {
Set allocatedBuffers = Collections.newSetFromMap(new IdentityHashMap<>());
RecordsIterator iterator = createIterator(
records,
- mockBufferSupplier(allocatedBuffers)
+ mockBufferSupplier(allocatedBuffers),
+ validateCrc
);
for (TestBatch batch : expectedBatches) {
@@ -111,8 +154,12 @@ private void testIterator(
assertEquals(Collections.emptySet(), allocatedBuffers);
}
- static RecordsIterator createIterator(Records records, BufferSupplier bufferSupplier) {
- return new RecordsIterator<>(records, STRING_SERDE, bufferSupplier, Records.HEADER_SIZE_UP_TO_MAGIC);
+ static RecordsIterator createIterator(
+ Records records,
+ BufferSupplier bufferSupplier,
+ boolean validateCrc
+ ) {
+ return new RecordsIterator<>(records, STRING_SERDE, bufferSupplier, Records.HEADER_SIZE_UP_TO_MAGIC, validateCrc);
}
static BufferSupplier mockBufferSupplier(Set buffers) {
diff --git a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java
index 05d1929f2715..cd86c709ff98 100644
--- a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java
+++ b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java
@@ -192,7 +192,8 @@ private SnapshotReader readSnapshot(
context.log.readSnapshot(snapshotId).get(),
context.serde,
BufferSupplier.create(),
- maxBatchSize
+ maxBatchSize,
+ true
);
}
@@ -246,7 +247,7 @@ record = records.next();
public static void assertSnapshot(List> batches, RawSnapshotReader reader) {
assertSnapshot(
batches,
- RecordsSnapshotReader.of(reader, new StringSerde(), BufferSupplier.create(), Integer.MAX_VALUE)
+ RecordsSnapshotReader.of(reader, new StringSerde(), BufferSupplier.create(), Integer.MAX_VALUE, true)
);
}
diff --git a/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandler.java b/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandler.java
new file mode 100644
index 000000000000..5efc145ea944
--- /dev/null
+++ b/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandler.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.server.fault;
+
+
+/**
+ * Handle a server fault.
+ */
+public interface FaultHandler {
+ /**
+ * Handle a fault.
+ *
+ * @param failureMessage The failure message to log.
+ *
+ * @return The fault exception.
+ */
+ default RuntimeException handleFault(String failureMessage) {
+ return handleFault(failureMessage, null);
+ }
+
+ /**
+ * Handle a fault.
+ *
+ * @param failureMessage The failure message to log.
+ * @param cause The exception that caused the problem, or null.
+ *
+ * @return The fault exception.
+ */
+ RuntimeException handleFault(String failureMessage, Throwable cause);
+}
diff --git a/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandlerException.java b/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandlerException.java
new file mode 100644
index 000000000000..ec3b7dc4b0c1
--- /dev/null
+++ b/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandlerException.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.server.fault;
+
+
+/**
+ * An exception thrown by a fault handler.
+ */
+public class FaultHandlerException extends RuntimeException {
+ public FaultHandlerException(String failureMessage, Throwable cause) {
+ super(failureMessage, cause);
+ // If a cause exception was provided, set our the stack trace its stack trace. This is
+ // useful in junit tests where a limited number of stack frames are printed, and usually
+ // the stack frames of cause exceptions get trimmed.
+ if (cause != null) {
+ setStackTrace(cause.getStackTrace());
+ }
+ }
+
+ public FaultHandlerException(String failureMessage) {
+ this(failureMessage, null);
+ }
+}
diff --git a/server-common/src/main/java/org/apache/kafka/server/fault/LoggingFaultHandler.java b/server-common/src/main/java/org/apache/kafka/server/fault/LoggingFaultHandler.java
new file mode 100644
index 000000000000..9242cef4eb9d
--- /dev/null
+++ b/server-common/src/main/java/org/apache/kafka/server/fault/LoggingFaultHandler.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.server.fault;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A fault handler which logs an error message and executes a runnable.
+ */
+public class LoggingFaultHandler implements FaultHandler {
+ private static final Logger log = LoggerFactory.getLogger(LoggingFaultHandler.class);
+ private final String type;
+ private final Runnable action;
+
+ public LoggingFaultHandler(
+ String type,
+ Runnable action
+ ) {
+ this.type = type;
+ this.action = action;
+ }
+
+ @Override
+ public RuntimeException handleFault(String failureMessage, Throwable cause) {
+ if (cause == null) {
+ log.error("Encountered {} fault: {}", type, failureMessage);
+ } else {
+ log.error("Encountered {} fault: {}", type, failureMessage, cause);
+ }
+ try {
+ action.run();
+ } catch (Throwable e) {
+ log.error("Failed to run LoggingFaultHandler action.", e);
+ }
+ return new FaultHandlerException(failureMessage, cause);
+ }
+}
diff --git a/server-common/src/main/java/org/apache/kafka/server/fault/ProcessExitingFaultHandler.java b/server-common/src/main/java/org/apache/kafka/server/fault/ProcessExitingFaultHandler.java
new file mode 100644
index 000000000000..b7c0d241a2ad
--- /dev/null
+++ b/server-common/src/main/java/org/apache/kafka/server/fault/ProcessExitingFaultHandler.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.server.fault;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.kafka.common.utils.Exit;
+
+
+/**
+ * This is a fault handler which exits the Java process.
+ */
+public class ProcessExitingFaultHandler implements FaultHandler {
+ private static final Logger log = LoggerFactory.getLogger(ProcessExitingFaultHandler.class);
+
+ @Override
+ public RuntimeException handleFault(String failureMessage, Throwable cause) {
+ if (cause == null) {
+ log.error("Encountered fatal fault: {}", failureMessage);
+ } else {
+ log.error("Encountered fatal fault: {}", failureMessage, cause);
+ }
+ Exit.exit(1);
+ return null;
+ }
+}
diff --git a/server-common/src/test/java/org/apache/kafka/server/fault/LoggingFaultHandlerTest.java b/server-common/src/test/java/org/apache/kafka/server/fault/LoggingFaultHandlerTest.java
new file mode 100644
index 000000000000..1a11098a21b4
--- /dev/null
+++ b/server-common/src/test/java/org/apache/kafka/server/fault/LoggingFaultHandlerTest.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.server.fault;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+
+/**
+ * Tests LoggingFaultHandler
+ */
+public class LoggingFaultHandlerTest {
+ /**
+ * Test handling faults with and without exceptions.
+ */
+ @Test
+ public void testHandleFault() {
+ AtomicInteger counter = new AtomicInteger(0);
+ LoggingFaultHandler handler = new LoggingFaultHandler("test", () -> {
+ counter.incrementAndGet();
+ });
+ handler.handleFault("uh oh");
+ assertEquals(1, counter.get());
+ handler.handleFault("uh oh", new RuntimeException("yikes"));
+ assertEquals(2, counter.get());
+ }
+
+ /**
+ * Test handling an exception in the action callback.
+ */
+ @Test
+ public void testHandleExceptionInAction() {
+ LoggingFaultHandler handler = new LoggingFaultHandler("test", () -> {
+ throw new RuntimeException("action failed");
+ });
+ handler.handleFault("uh oh"); // should not throw
+ handler.handleFault("uh oh", new RuntimeException("yikes")); // should not throw
+ }
+}
diff --git a/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandler.java b/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandler.java
new file mode 100644
index 000000000000..e49f2bdc6c25
--- /dev/null
+++ b/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandler.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.server.fault;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a fault handler suitable for use in JUnit tests. It will store the result of the first
+ * call to handleFault that was made.
+ */
+public class MockFaultHandler implements FaultHandler {
+ private static final Logger log = LoggerFactory.getLogger(MockFaultHandler.class);
+
+ private final String name;
+ private FaultHandlerException firstException = null;
+ private boolean ignore = false;
+
+ public MockFaultHandler(String name) {
+ this.name = name;
+ }
+
+ @Override
+ public synchronized RuntimeException handleFault(String failureMessage, Throwable cause) {
+ if (cause == null) {
+ log.error("Encountered {} fault: {}", name, failureMessage);
+ } else {
+ log.error("Encountered {} fault: {}", name, failureMessage, cause);
+ }
+ FaultHandlerException e = (cause == null) ?
+ new FaultHandlerException(name + ": " + failureMessage) :
+ new FaultHandlerException(name + ": " + failureMessage +
+ ": " + cause.getMessage(), cause);
+ if (firstException == null) {
+ firstException = e;
+ }
+ return firstException;
+ }
+
+ public synchronized void maybeRethrowFirstException() {
+ if (firstException != null && !ignore) {
+ throw firstException;
+ }
+ }
+
+ public synchronized FaultHandlerException firstException() {
+ return firstException;
+ }
+
+ public synchronized void setIgnore(boolean ignore) {
+ this.ignore = ignore;
+ }
+}