From 7f284497cb46366cc266ec6abac4661e6e3a839d Mon Sep 17 00:00:00 2001 From: Liam Clarke-Hutchinson Date: Tue, 15 Mar 2022 01:03:41 +1300 Subject: [PATCH 001/447] KAFKA-13438: Replace EasyMock and PowerMock with Mockito in WorkerTest (#11817) Reviewers: Mickael Maison --- build.gradle | 4 +- ...nnectorClientConfigOverridePolicyTest.java | 4 +- .../kafka/connect/runtime/WorkerTest.java | 1274 +++++++---------- 3 files changed, 484 insertions(+), 798 deletions(-) diff --git a/build.gradle b/build.gradle index 2b91533c1602..8e1ae634e253 100644 --- a/build.gradle +++ b/build.gradle @@ -406,7 +406,7 @@ subprojects { "**/KafkaBasedLogTest.*", "**/OffsetStorageWriterTest.*", "**/StandaloneHerderTest.*", "**/SourceTaskOffsetCommitterTest.*", "**/WorkerConfigTransformerTest.*", "**/WorkerGroupMemberTest.*", "**/WorkerSinkTaskTest.*", "**/WorkerSinkTaskThreadedTest.*", "**/WorkerSourceTaskTest.*", - "**/WorkerTaskTest.*", "**/WorkerTest.*", + "**/WorkerTaskTest.*", // streams tests "**/KafkaStreamsTest.*" ]) @@ -2439,7 +2439,7 @@ project(':connect:runtime') { testImplementation libs.junitVintageEngine testImplementation libs.powermockJunit4 testImplementation libs.powermockEasymock - testImplementation libs.mockitoCore + testImplementation libs.mockitoInline testImplementation libs.httpclient testRuntimeOnly libs.slf4jlog4j diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/connector/policy/BaseConnectorClientConfigOverridePolicyTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/connector/policy/BaseConnectorClientConfigOverridePolicyTest.java index 28fee73a9396..719de7ed7b70 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/connector/policy/BaseConnectorClientConfigOverridePolicyTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/connector/policy/BaseConnectorClientConfigOverridePolicyTest.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.connect.health.ConnectorType; -import org.apache.kafka.connect.runtime.WorkerTest; +import org.apache.kafka.connect.runtime.SampleSourceConnector; import org.junit.Assert; import java.util.List; @@ -43,7 +43,7 @@ private List configValues(Map clientConfig) { ConnectorClientConfigRequest connectorClientConfigRequest = new ConnectorClientConfigRequest( "test", ConnectorType.SOURCE, - WorkerTest.WorkerTestConnector.class, + SampleSourceConnector.class, clientConfig, ConnectorClientConfigRequest.ClientType.PRODUCER); return policyToTest().validate(connectorClientConfigRequest); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index 94306ff68418..2b210794207a 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -19,12 +19,9 @@ import java.util.Collection; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.Configurable; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.config.AbstractConfig; -import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.provider.MockFileConfigProvider; import org.apache.kafka.common.metrics.MetricsReporter; @@ -32,19 +29,14 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.connector.ConnectorContext; -import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy; import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; import org.apache.kafka.connect.connector.policy.NoneConnectorClientConfigOverridePolicy; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.json.JsonConverter; -import org.apache.kafka.connect.json.JsonConverterConfig; import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; import org.apache.kafka.connect.runtime.MockConnectMetrics.MockMetricsReporter; import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; -import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; import org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader; import org.apache.kafka.connect.runtime.isolation.PluginClassLoader; import org.apache.kafka.connect.runtime.isolation.Plugins; @@ -59,56 +51,44 @@ import org.apache.kafka.connect.storage.Converter; import org.apache.kafka.connect.storage.HeaderConverter; import org.apache.kafka.connect.storage.OffsetBackingStore; -import org.apache.kafka.connect.storage.OffsetStorageReader; -import org.apache.kafka.connect.storage.OffsetStorageWriter; import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.util.ConnectUtils; import org.apache.kafka.connect.util.ConnectorTaskId; import org.apache.kafka.connect.util.FutureCallback; import org.apache.kafka.connect.util.ParameterizedTest; import org.apache.kafka.connect.util.ThreadedTest; -import org.apache.kafka.connect.util.TopicAdmin; -import org.apache.kafka.connect.util.TopicCreationGroup; -import org.easymock.EasyMock; +import org.junit.After; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; -import org.powermock.api.easymock.PowerMock; -import org.powermock.api.easymock.annotation.Mock; -import org.powermock.api.easymock.annotation.MockNice; -import org.powermock.api.easymock.annotation.MockStrict; -import org.powermock.core.classloader.annotations.PowerMockIgnore; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; +import org.junit.runners.Parameterized; +import org.mockito.Mock; +import org.mockito.MockedConstruction; +import org.mockito.MockedStatic; +import org.mockito.Mockito; +import org.mockito.MockitoSession; +import org.mockito.internal.stubbing.answers.CallsRealMethods; +import org.mockito.quality.Strictness; import javax.management.MBeanServer; -import javax.management.ObjectInstance; import javax.management.ObjectName; import java.lang.management.ManagementFactory; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; -import org.powermock.modules.junit4.PowerMockRunnerDelegate; import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; -import static org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest.NOOP_OPERATOR; -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.eq; -import static org.easymock.EasyMock.expectLastCall; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -117,11 +97,21 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; import static org.junit.Assert.fail; - -@RunWith(PowerMockRunner.class) -@PowerMockRunnerDelegate(ParameterizedTest.class) -@PrepareForTest({Worker.class, Plugins.class, ConnectUtils.class}) -@PowerMockIgnore("javax.management.*") +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyMap; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockConstructionWithAnswer; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(Parameterized.class) public class WorkerTest extends ThreadedTest { private static final String CONNECTOR_ID = "test-connector"; @@ -131,44 +121,61 @@ public class WorkerTest extends ThreadedTest { private final ConnectorClientConfigOverridePolicy noneConnectorClientConfigOverridePolicy = new NoneConnectorClientConfigOverridePolicy(); private final ConnectorClientConfigOverridePolicy allConnectorClientConfigOverridePolicy = new AllConnectorClientConfigOverridePolicy(); - private Map workerProps = new HashMap<>(); + private final Map workerProps = new HashMap<>(); private WorkerConfig config; private Worker worker; - private Map defaultProducerConfigs = new HashMap<>(); - private Map defaultConsumerConfigs = new HashMap<>(); + private final Map defaultProducerConfigs = new HashMap<>(); + private final Map defaultConsumerConfigs = new HashMap<>(); @Mock private Plugins plugins; + @Mock private PluginClassLoader pluginLoader; + @Mock private DelegatingClassLoader delegatingLoader; + @Mock private OffsetBackingStore offsetBackingStore; - @MockStrict + + @Mock private TaskStatus.Listener taskStatusListener; - @MockStrict + + @Mock private ConnectorStatus.Listener connectorStatusListener; - @Mock private Herder herder; - @Mock private StatusBackingStore statusBackingStore; - @Mock private SourceConnector sourceConnector; - @Mock private SinkConnector sinkConnector; - @Mock private CloseableConnectorContext ctx; + @Mock + private Herder herder; + + @Mock + private StatusBackingStore statusBackingStore; + + @Mock + private SourceConnector sourceConnector; + + @Mock + private SinkConnector sinkConnector; + + @Mock + private CloseableConnectorContext ctx; + @Mock private TestSourceTask task; - @Mock private WorkerSourceTask workerTask; - @Mock private Converter keyConverter; - @Mock private Converter valueConverter; @Mock private Converter taskKeyConverter; @Mock private Converter taskValueConverter; @Mock private HeaderConverter taskHeaderConverter; @Mock private ExecutorService executorService; - @MockNice private ConnectorConfig connectorConfig; + @Mock private ConnectorConfig connectorConfig; private String mockFileProviderTestId; private Map connectorProps; - private boolean enableTopicCreation; + private final boolean enableTopicCreation; + + private MockedStatic pluginsMockedStatic; + private MockedStatic connectUtilsMockedStatic; + private MockedConstruction sourceTaskMockedConstruction; + private MockitoSession mockitoSession; @ParameterizedTest.Parameters public static Collection parameters() { @@ -182,6 +189,13 @@ public WorkerTest(boolean enableTopicCreation) { @Before public void setup() { super.setup(); + + // Use strict mode to detect unused mocks + mockitoSession = Mockito.mockitoSession() + .initMocks(this) + .strictness(Strictness.STRICT_STUBS) + .startMocking(); + workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); @@ -213,69 +227,73 @@ public void setup() { // Some common defaults. They might change on individual tests connectorProps = anyConnectorConfigMap(); - PowerMock.mockStatic(Plugins.class); + + pluginsMockedStatic = mockStatic(Plugins.class); + + // pass through things that aren't explicitly mocked out + connectUtilsMockedStatic = mockStatic(ConnectUtils.class, new CallsRealMethods()); + connectUtilsMockedStatic.when(() -> ConnectUtils.lookupKafkaClusterId(any(WorkerConfig.class))).thenReturn(CLUSTER_ID); + + // Make calls to new WorkerSourceTask() return a mock to avoid the source task trying to connect to a broker. + sourceTaskMockedConstruction = mockConstructionWithAnswer(WorkerSourceTask.class, invocation -> { + + // provide implementations of three methods used during testing + switch (invocation.getMethod().getName()) { + case "id": + return TASK_ID; + case "loader": + return pluginLoader; + case "awaitStop": + return true; + default: + return null; + } + }); } - @Test - public void testStartAndStopConnector() throws Throwable { - expectConverters(); - expectStartStorage(); + @After + public void teardown() { + // Critical to always close MockedStatics + // Ideal would be to use try-with-resources in an individual test, but it introduced a rather large level of + // indentation of most test bodies, hence sticking with setup() / teardown() + pluginsMockedStatic.close(); + connectUtilsMockedStatic.close(); + sourceTaskMockedConstruction.close(); - final String connectorClass = WorkerTestConnector.class.getName(); + mockitoSession.finishMocking(); + } - // Create - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(connectorClass)).andReturn(pluginLoader); - EasyMock.expect(plugins.newConnector(connectorClass)) - .andReturn(sourceConnector); - EasyMock.expect(sourceConnector.version()).andReturn("1.0"); + @Test + public void testStartAndStopConnector() throws Throwable { + final String connectorClass = SampleSourceConnector.class.getName(); connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass); - EasyMock.expect(sourceConnector.version()).andReturn("1.0"); - - expectFileConfigProvider(); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)) - .andReturn(delegatingLoader) - .times(3); - sourceConnector.initialize(anyObject(ConnectorContext.class)); - EasyMock.expectLastCall(); - sourceConnector.start(connectorProps); - EasyMock.expectLastCall(); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) - .andReturn(pluginLoader).times(3); - - connectorStatusListener.onStartup(CONNECTOR_ID); - EasyMock.expectLastCall(); - - // Remove - sourceConnector.stop(); - EasyMock.expectLastCall(); - - connectorStatusListener.onShutdown(CONNECTOR_ID); - EasyMock.expectLastCall(); - - ctx.close(); - expectLastCall(); - - expectStopStorage(); - expectClusterId(); + // Create + when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); + when(plugins.delegatingLoader()).thenReturn(delegatingLoader); + when(delegatingLoader.connectorLoader(connectorClass)).thenReturn(pluginLoader); + when(plugins.newConnector(connectorClass)).thenReturn(sourceConnector); + when(sourceConnector.version()).thenReturn("1.0"); - PowerMock.replayAll(); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(pluginLoader)).thenReturn(delegatingLoader); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader); + connectUtilsMockedStatic.when(() -> ConnectUtils.lookupKafkaClusterId(any(WorkerConfig.class))) + .thenReturn(CLUSTER_ID); worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); - worker.herder = herder; worker.start(); assertEquals(Collections.emptySet(), worker.connectorNames()); FutureCallback onFirstStart = new FutureCallback<>(); + worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onFirstStart); + // Wait for the connector to actually start assertEquals(TargetState.STARTED, onFirstStart.get(1000, TimeUnit.MILLISECONDS)); - assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); + assertEquals(Collections.singleton(CONNECTOR_ID), worker.connectorNames()); + FutureCallback onSecondStart = new FutureCallback<>(); worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onSecondStart); @@ -289,54 +307,60 @@ public void testStartAndStopConnector() throws Throwable { assertStatistics(worker, 1, 0); assertStartupStatistics(worker, 1, 0, 0, 0); worker.stopAndAwaitConnector(CONNECTOR_ID); + assertStatistics(worker, 0, 0); assertStartupStatistics(worker, 1, 0, 0, 0); assertEquals(Collections.emptySet(), worker.connectorNames()); + // Nothing should be left, so this should effectively be a nop worker.stop(); assertStatistics(worker, 0, 0); - PowerMock.verifyAll(); + + verify(plugins, times(2)).currentThreadLoader(); + verify(plugins).delegatingLoader(); + verify(delegatingLoader).connectorLoader(connectorClass); + verify(plugins).newConnector(connectorClass); + verify(sourceConnector, times(2)).version(); + verify(sourceConnector).initialize(any(ConnectorContext.class)); + verify(sourceConnector).start(connectorProps); + verify(connectorStatusListener).onStartup(CONNECTOR_ID); + + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(pluginLoader), times(2)); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader), times(2)); + connectUtilsMockedStatic.verify(() -> ConnectUtils.lookupKafkaClusterId(any(WorkerConfig.class))); + + verify(sourceConnector).stop(); + verify(connectorStatusListener).onShutdown(CONNECTOR_ID); + verify(ctx).close(); MockFileConfigProvider.assertClosed(mockFileProviderTestId); } - private void expectFileConfigProvider() { - EasyMock.expect(plugins.newConfigProvider(EasyMock.anyObject(), - EasyMock.eq("config.providers.file"), EasyMock.anyObject())) - .andAnswer(() -> { - MockFileConfigProvider mockFileConfigProvider = new MockFileConfigProvider(); - mockFileConfigProvider.configure(Collections.singletonMap("testId", mockFileProviderTestId)); - return mockFileConfigProvider; - }); + private void mockFileConfigProvider() { + MockFileConfigProvider mockFileConfigProvider = new MockFileConfigProvider(); + mockFileConfigProvider.configure(Collections.singletonMap("testId", mockFileProviderTestId)); + when(plugins.newConfigProvider(any(AbstractConfig.class), + eq("config.providers.file"), + any(ClassLoaderUsage.class))) + .thenReturn(mockFileConfigProvider); } @Test public void testStartConnectorFailure() throws Exception { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); final String nonConnectorClass = "java.util.HashMap"; connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, nonConnectorClass); // Bad connector class name Exception exception = new ConnectException("Failed to find Connector"); - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader); - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(nonConnectorClass)).andReturn(delegatingLoader); - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) - .andReturn(delegatingLoader).times(2); - EasyMock.expect(plugins.newConnector(EasyMock.anyString())) - .andThrow(exception); - - connectorStatusListener.onFailure( - EasyMock.eq(CONNECTOR_ID), - EasyMock.anyObject() - ); - EasyMock.expectLastCall(); - expectClusterId(); + when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); + when(plugins.delegatingLoader()).thenReturn(delegatingLoader); + when(delegatingLoader.connectorLoader(nonConnectorClass)).thenReturn(delegatingLoader); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(delegatingLoader); + connectUtilsMockedStatic.when(() -> ConnectUtils.lookupKafkaClusterId(any(WorkerConfig.class))) + .thenReturn("test-cluster"); - PowerMock.replayAll(); + when(plugins.newConnector(anyString())).thenThrow(exception); worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); worker.herder = herder; @@ -361,57 +385,35 @@ public void testStartConnectorFailure() throws Exception { assertStatistics(worker, 0, 0); assertStartupStatistics(worker, 1, 1, 0, 0); - PowerMock.verifyAll(); + verify(plugins).currentThreadLoader(); + verify(plugins).delegatingLoader(); + verify(plugins).delegatingLoader(); + verify(delegatingLoader).connectorLoader(nonConnectorClass); + verify(plugins).newConnector(anyString()); + verify(connectorStatusListener).onFailure(eq(CONNECTOR_ID), any(ConnectException.class)); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader), times(2)); + connectUtilsMockedStatic.verify(() -> ConnectUtils.lookupKafkaClusterId(any(WorkerConfig.class))); } @Test public void testAddConnectorByAlias() throws Throwable { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - final String connectorAlias = "WorkerTestConnector"; + final String connectorAlias = "SampleSourceConnector"; - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(connectorAlias)).andReturn(pluginLoader); - EasyMock.expect(plugins.newConnector(connectorAlias)).andReturn(sinkConnector); - EasyMock.expect(sinkConnector.version()).andReturn("1.0"); + when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); + when(plugins.delegatingLoader()).thenReturn(delegatingLoader); + when(plugins.newConnector(connectorAlias)).thenReturn(sinkConnector); + when(delegatingLoader.connectorLoader(connectorAlias)).thenReturn(pluginLoader); + when(sinkConnector.version()).thenReturn("1.0"); + + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(pluginLoader)).thenReturn(delegatingLoader); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader); + connectUtilsMockedStatic.when(() -> ConnectUtils.lookupKafkaClusterId(any(WorkerConfig.class))) + .thenReturn("test-cluster"); connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorAlias); connectorProps.put(SinkConnectorConfig.TOPICS_CONFIG, "gfieyls, wfru"); - EasyMock.expect(sinkConnector.version()).andReturn("1.0"); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)) - .andReturn(delegatingLoader) - .times(3); - sinkConnector.initialize(anyObject(ConnectorContext.class)); - EasyMock.expectLastCall(); - sinkConnector.start(connectorProps); - EasyMock.expectLastCall(); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) - .andReturn(pluginLoader) - .times(3); - - connectorStatusListener.onStartup(CONNECTOR_ID); - EasyMock.expectLastCall(); - - // Remove - sinkConnector.stop(); - EasyMock.expectLastCall(); - - connectorStatusListener.onShutdown(CONNECTOR_ID); - EasyMock.expectLastCall(); - - ctx.close(); - expectLastCall(); - - expectStopStorage(); - expectClusterId(); - - PowerMock.replayAll(); - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); worker.herder = herder; worker.start(); @@ -422,7 +424,7 @@ public void testAddConnectorByAlias() throws Throwable { worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onStart); // Wait for the connector to actually start assertEquals(TargetState.STARTED, onStart.get(1000, TimeUnit.MILLISECONDS)); - assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); + assertEquals(Collections.singleton(CONNECTOR_ID), worker.connectorNames()); assertStatistics(worker, 1, 0); assertStartupStatistics(worker, 1, 0, 0, 0); @@ -435,57 +437,37 @@ public void testAddConnectorByAlias() throws Throwable { assertStatistics(worker, 0, 0); assertStartupStatistics(worker, 1, 0, 0, 0); - PowerMock.verifyAll(); + verify(plugins, times(2)).currentThreadLoader(); + verify(plugins).delegatingLoader(); + verify(plugins).newConnector(connectorAlias); + verify(delegatingLoader).connectorLoader(connectorAlias); + verify(sinkConnector, times(2)).version(); + verify(sinkConnector).initialize(any(ConnectorContext.class)); + verify(sinkConnector).start(connectorProps); + verify(sinkConnector).stop(); + verify(connectorStatusListener).onStartup(CONNECTOR_ID); + verify(ctx).close(); + + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(pluginLoader), times(2)); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader), times(2)); + connectUtilsMockedStatic.verify(() -> ConnectUtils.lookupKafkaClusterId(any(WorkerConfig.class))); } @Test public void testAddConnectorByShortAlias() throws Throwable { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); final String shortConnectorAlias = "WorkerTest"; - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(shortConnectorAlias)).andReturn(pluginLoader); - EasyMock.expect(plugins.newConnector(shortConnectorAlias)).andReturn(sinkConnector); - EasyMock.expect(sinkConnector.version()).andReturn("1.0"); - + when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); + when(plugins.delegatingLoader()).thenReturn(delegatingLoader); + when(plugins.newConnector(shortConnectorAlias)).thenReturn(sinkConnector); + when(delegatingLoader.connectorLoader(shortConnectorAlias)).thenReturn(pluginLoader); + when(sinkConnector.version()).thenReturn("1.0"); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(pluginLoader)).thenReturn(delegatingLoader); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader); connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, shortConnectorAlias); connectorProps.put(SinkConnectorConfig.TOPICS_CONFIG, "gfieyls, wfru"); - EasyMock.expect(sinkConnector.version()).andReturn("1.0"); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)) - .andReturn(delegatingLoader) - .times(3); - sinkConnector.initialize(anyObject(ConnectorContext.class)); - EasyMock.expectLastCall(); - sinkConnector.start(connectorProps); - EasyMock.expectLastCall(); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) - .andReturn(pluginLoader) - .times(3); - - connectorStatusListener.onStartup(CONNECTOR_ID); - EasyMock.expectLastCall(); - - // Remove - sinkConnector.stop(); - EasyMock.expectLastCall(); - - connectorStatusListener.onShutdown(CONNECTOR_ID); - EasyMock.expectLastCall(); - - ctx.close(); - expectLastCall(); - - expectStopStorage(); - expectClusterId(); - - PowerMock.replayAll(); - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); worker.herder = herder; worker.start(); @@ -496,7 +478,7 @@ public void testAddConnectorByShortAlias() throws Throwable { worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onStart); // Wait for the connector to actually start assertEquals(TargetState.STARTED, onStart.get(1000, TimeUnit.MILLISECONDS)); - assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); + assertEquals(Collections.singleton(CONNECTOR_ID), worker.connectorNames()); assertStatistics(worker, 1, 0); worker.stopAndAwaitConnector(CONNECTOR_ID); @@ -506,81 +488,54 @@ public void testAddConnectorByShortAlias() throws Throwable { worker.stop(); assertStatistics(worker, 0, 0); - PowerMock.verifyAll(); + verify(plugins, times(2)).currentThreadLoader(); + verify(plugins).delegatingLoader(); + verify(plugins).newConnector(shortConnectorAlias); + verify(sinkConnector, times(2)).version(); + verify(sinkConnector).initialize(any(ConnectorContext.class)); + verify(sinkConnector).start(connectorProps); + verify(connectorStatusListener).onStartup(CONNECTOR_ID); + verify(sinkConnector).stop(); + verify(connectorStatusListener).onShutdown(CONNECTOR_ID); + verify(ctx).close(); + + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader), times(2)); + connectUtilsMockedStatic.verify(() -> ConnectUtils.lookupKafkaClusterId(any(WorkerConfig.class))); } @Test public void testStopInvalidConnector() { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - expectClusterId(); - - PowerMock.replayAll(); - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); worker.herder = herder; worker.start(); worker.stopAndAwaitConnector(CONNECTOR_ID); - PowerMock.verifyAll(); + verifyConverters(); } @Test public void testReconfigureConnectorTasks() throws Throwable { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); + final String connectorClass = SampleSourceConnector.class.getName(); - final String connectorClass = WorkerTestConnector.class.getName(); + when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); + when(plugins.delegatingLoader()).thenReturn(delegatingLoader); + when(delegatingLoader.connectorLoader(connectorClass)).thenReturn(pluginLoader); + when(plugins.newConnector(connectorClass)).thenReturn(sinkConnector); + when(sinkConnector.version()).thenReturn("1.0"); - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(3); - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader).times(1); - EasyMock.expect(delegatingLoader.connectorLoader(connectorClass)).andReturn(pluginLoader); - EasyMock.expect(plugins.newConnector(connectorClass)) - .andReturn(sinkConnector); - EasyMock.expect(sinkConnector.version()).andReturn("1.0"); + Map taskProps = Collections.singletonMap("foo", "bar"); + when(sinkConnector.taskConfigs(2)).thenReturn(Arrays.asList(taskProps, taskProps)); - connectorProps.put(SinkConnectorConfig.TOPICS_CONFIG, "foo,bar"); - connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass); - - EasyMock.expect(sinkConnector.version()).andReturn("1.0"); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)) - .andReturn(delegatingLoader) - .times(4); - sinkConnector.initialize(anyObject(ConnectorContext.class)); - EasyMock.expectLastCall(); - sinkConnector.start(connectorProps); - EasyMock.expectLastCall(); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) - .andReturn(pluginLoader) - .times(4); - - connectorStatusListener.onStartup(CONNECTOR_ID); - EasyMock.expectLastCall(); - - // Reconfigure - EasyMock.>expect(sinkConnector.taskClass()).andReturn(TestSourceTask.class); - Map taskProps = new HashMap<>(); - taskProps.put("foo", "bar"); - EasyMock.expect(sinkConnector.taskConfigs(2)).andReturn(Arrays.asList(taskProps, taskProps)); - - // Remove - sinkConnector.stop(); - EasyMock.expectLastCall(); + // Use doReturn().when() syntax due to when().thenReturn() not being able to return wildcard generic types + doReturn(TestSourceTask.class).when(sinkConnector).taskClass(); - connectorStatusListener.onShutdown(CONNECTOR_ID); - EasyMock.expectLastCall(); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(pluginLoader)).thenReturn(delegatingLoader); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader); - ctx.close(); - expectLastCall(); - expectStopStorage(); - expectClusterId(); - - PowerMock.replayAll(); + connectorProps.put(SinkConnectorConfig.TOPICS_CONFIG, "foo,bar"); + connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass); worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); worker.herder = herder; @@ -593,7 +548,7 @@ public void testReconfigureConnectorTasks() throws Throwable { // Wait for the connector to actually start assertEquals(TargetState.STARTED, onFirstStart.get(1000, TimeUnit.MILLISECONDS)); assertStatistics(worker, 1, 0); - assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); + assertEquals(Collections.singleton(CONNECTOR_ID), worker.connectorNames()); FutureCallback onSecondStart = new FutureCallback<>(); worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onSecondStart); @@ -625,77 +580,52 @@ public void testReconfigureConnectorTasks() throws Throwable { worker.stop(); assertStatistics(worker, 0, 0); - PowerMock.verifyAll(); + verify(plugins, times(3)).currentThreadLoader(); + verify(plugins).delegatingLoader(); + verify(delegatingLoader).connectorLoader(connectorClass); + verify(plugins).newConnector(connectorClass); + verify(sinkConnector, times(2)).version(); + verify(sinkConnector).initialize(any(ConnectorContext.class)); + verify(sinkConnector).start(connectorProps); + verify(connectorStatusListener).onStartup(CONNECTOR_ID); + verify(sinkConnector).taskClass(); + verify(sinkConnector).taskConfigs(2); + verify(sinkConnector).stop(); + verify(connectorStatusListener).onShutdown(CONNECTOR_ID); + verify(ctx).close(); + + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(pluginLoader), times(3)); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader), times(3)); } @Test - public void testAddRemoveTask() throws Exception { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); - - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - expectNewWorkerTask(); - Map origProps = new HashMap<>(); - origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); - - TaskConfig taskConfig = new TaskConfig(origProps); - // We should expect this call, but the pluginLoader being swapped in is only mocked. - // EasyMock.expect(pluginLoader.loadClass(TestSourceTask.class.getName())) - // .andReturn((Class) TestSourceTask.class); - EasyMock.expect(plugins.newTask(TestSourceTask.class)).andReturn(task); - EasyMock.expect(task.version()).andReturn("1.0"); - - workerTask.initialize(taskConfig); - EasyMock.expectLastCall(); - - // Expect that the worker will create converters and will find them using the current classloader ... - assertNotNull(taskKeyConverter); - assertNotNull(taskValueConverter); - assertNotNull(taskHeaderConverter); - expectTaskKeyConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, taskKeyConverter); - expectTaskValueConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, taskValueConverter); - expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, taskHeaderConverter); - - EasyMock.expect(executorService.submit(workerTask)).andReturn(null); - - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) - .andReturn(pluginLoader); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader) - .times(2); - - EasyMock.expect(workerTask.loader()).andReturn(pluginLoader); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader) - .times(2); - plugins.connectorClass(WorkerTestConnector.class.getName()); - EasyMock.expectLastCall().andReturn(WorkerTestConnector.class); - // Remove - workerTask.stop(); - EasyMock.expectLastCall(); - EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andStubReturn(true); - EasyMock.expectLastCall(); - - workerTask.removeMetrics(); - EasyMock.expectLastCall(); - - expectStopStorage(); - expectClusterId(); - - PowerMock.replayAll(); + public void testAddRemoveTask() { + when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); + when(plugins.delegatingLoader()).thenReturn(delegatingLoader); + when(delegatingLoader.connectorLoader(SampleSourceConnector.class.getName())).thenReturn(pluginLoader); + + when(plugins.newTask(TestSourceTask.class)).thenReturn(task); + when(task.version()).thenReturn("1.0"); + mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, taskKeyConverter); + mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, taskValueConverter); + mockTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, taskHeaderConverter); + when(executorService.submit(any(WorkerSourceTask.class))).thenReturn(null); + doReturn(SampleSourceConnector.class).when(plugins).connectorClass(SampleSourceConnector.class.getName()); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(pluginLoader)).thenReturn(delegatingLoader); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader); + + Map origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService, - noneConnectorClientConfigOverridePolicy); + noneConnectorClientConfigOverridePolicy); worker.herder = herder; worker.start(); + assertStatistics(worker, 0, 0); assertEquals(Collections.emptySet(), worker.taskIds()); worker.startTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED); assertStatistics(worker, 0, 1); - assertEquals(new HashSet<>(Arrays.asList(TASK_ID)), worker.taskIds()); + assertEquals(Collections.singleton(TASK_ID), worker.taskIds()); worker.stopAndAwaitTask(TASK_ID); assertStatistics(worker, 0, 0); assertEquals(Collections.emptySet(), worker.taskIds()); @@ -703,88 +633,67 @@ public void testAddRemoveTask() throws Exception { worker.stop(); assertStatistics(worker, 0, 0); - PowerMock.verifyAll(); + verify(plugins, times(2)).currentThreadLoader(); + verify(plugins).newTask(TestSourceTask.class); + verify(task).version(); + verifyTaskConverter(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG); + verifyTaskConverter(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG); + verifyTaskHeaderConverter(); + + verify(executorService).submit(any(WorkerSourceTask.class)); + verify(plugins).delegatingLoader(); + verify(delegatingLoader).connectorLoader(SampleSourceConnector.class.getName()); + verify(plugins).connectorClass(SampleSourceConnector.class.getName()); + + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(pluginLoader), times(2)); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader), times(2)); + connectUtilsMockedStatic.verify(() -> ConnectUtils.lookupKafkaClusterId(any(WorkerConfig.class))); + } @Test - public void testTaskStatusMetricsStatuses() throws Exception { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); + public void testTaskStatusMetricsStatuses() { + mockInternalConverters(); + mockStorage(); + mockFileConfigProvider(); + - EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - expectNewWorkerTask(); - Map origProps = new HashMap<>(); - origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); + when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); + + Map origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); TaskConfig taskConfig = new TaskConfig(origProps); - // We should expect this call, but the pluginLoader being swapped in is only mocked. - // EasyMock.expect(pluginLoader.loadClass(TestSourceTask.class.getName())) - // .andReturn((Class) TestSourceTask.class); - EasyMock.expect(plugins.newTask(TestSourceTask.class)).andReturn(task); - EasyMock.expect(task.version()).andReturn("1.0"); - workerTask.initialize(taskConfig); - EasyMock.expectLastCall(); + when(plugins.newTask(TestSourceTask.class)).thenReturn(task); + when(task.version()).thenReturn("1.0"); // Expect that the worker will create converters and will find them using the current classloader ... assertNotNull(taskKeyConverter); assertNotNull(taskValueConverter); assertNotNull(taskHeaderConverter); - expectTaskKeyConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, taskKeyConverter); - expectTaskValueConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, taskValueConverter); - expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, taskHeaderConverter); - - EasyMock.expect(executorService.submit(workerTask)).andReturn(null); + mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, taskKeyConverter); + mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, taskValueConverter); + mockTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, taskHeaderConverter); - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) - .andReturn(pluginLoader); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader) - .times(2); + when(executorService.submit(any(WorkerSourceTask.class))).thenReturn(null); + when(plugins.delegatingLoader()).thenReturn(delegatingLoader); + when(delegatingLoader.connectorLoader(SampleSourceConnector.class.getName())).thenReturn(pluginLoader); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(pluginLoader)).thenReturn(delegatingLoader); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader); - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader) - .times(2); - plugins.connectorClass(WorkerTestConnector.class.getName()); - EasyMock.expectLastCall().andReturn(WorkerTestConnector.class); + doReturn(SampleSourceConnector.class).when(plugins).connectorClass(SampleSourceConnector.class.getName()); - EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andStubReturn(true); - EasyMock.expectLastCall(); - workerTask.removeMetrics(); - EasyMock.expectLastCall(); // Each time we check the task metrics, the worker will call the herder - herder.taskStatus(TASK_ID); - EasyMock.expectLastCall() - .andReturn(new ConnectorStateInfo.TaskState(0, "RUNNING", "worker", "msg")); - - herder.taskStatus(TASK_ID); - EasyMock.expectLastCall() - .andReturn(new ConnectorStateInfo.TaskState(0, "PAUSED", "worker", "msg")); - - herder.taskStatus(TASK_ID); - EasyMock.expectLastCall() - .andReturn(new ConnectorStateInfo.TaskState(0, "FAILED", "worker", "msg")); - - herder.taskStatus(TASK_ID); - EasyMock.expectLastCall() - .andReturn(new ConnectorStateInfo.TaskState(0, "DESTROYED", "worker", "msg")); - - herder.taskStatus(TASK_ID); - EasyMock.expectLastCall() - .andReturn(new ConnectorStateInfo.TaskState(0, "UNASSIGNED", "worker", "msg")); - - // Called when we stop the worker - EasyMock.expect(workerTask.loader()).andReturn(pluginLoader); - workerTask.stop(); - EasyMock.expectLastCall(); - - expectClusterId(); - - PowerMock.replayAll(); + when(herder.taskStatus(TASK_ID)).thenReturn( + new ConnectorStateInfo.TaskState(0, "RUNNING", "worker", "msg"), + new ConnectorStateInfo.TaskState(0, "PAUSED", "worker", "msg"), + new ConnectorStateInfo.TaskState(0, "FAILED", "worker", "msg"), + new ConnectorStateInfo.TaskState(0, "DESTROYED", "worker", "msg"), + new ConnectorStateInfo.TaskState(0, "UNASSIGNED", "worker", "msg") + ); worker = new Worker(WORKER_ID, new MockTime(), @@ -821,31 +730,40 @@ public void testTaskStatusMetricsStatuses() throws Exception { assertStatusMetrics(0L, "connector-destroyed-task-count"); assertStatusMetrics(0L, "connector-unassigned-task-count"); - PowerMock.verifyAll(); + WorkerSourceTask instantiatedTask = sourceTaskMockedConstruction.constructed().get(0); + verify(instantiatedTask).initialize(taskConfig); + verify(herder, times(5)).taskStatus(TASK_ID); + verify(plugins).delegatingLoader(); + verify(delegatingLoader).connectorLoader(SampleSourceConnector.class.getName()); + verify(executorService).submit(instantiatedTask); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(pluginLoader), times(2)); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader), times(2)); + verify(plugins).connectorClass(SampleSourceConnector.class.getName()); + verify(instantiatedTask, atLeastOnce()).id(); + verify(instantiatedTask).awaitStop(anyLong()); + verify(instantiatedTask).removeMetrics(); + + // Called when we stop the worker + verify(instantiatedTask).loader(); + verify(instantiatedTask).stop(); + verifyTaskConverter(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG); + verifyTaskConverter(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG); + verifyTaskHeaderConverter(); + verify(plugins, times(2)).currentThreadLoader(); + } @Test public void testConnectorStatusMetricsGroup_taskStatusCounter() { ConcurrentMap tasks = new ConcurrentHashMap<>(); - tasks.put(new ConnectorTaskId("c1", 0), workerTask); - tasks.put(new ConnectorTaskId("c1", 1), workerTask); - tasks.put(new ConnectorTaskId("c2", 0), workerTask); - - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader); - - taskStatusListener.onFailure(EasyMock.eq(TASK_ID), EasyMock.anyObject()); - EasyMock.expectLastCall(); + tasks.put(new ConnectorTaskId("c1", 0), mock(WorkerSourceTask.class)); + tasks.put(new ConnectorTaskId("c1", 1), mock(WorkerSourceTask.class)); + tasks.put(new ConnectorTaskId("c2", 0), mock(WorkerSourceTask.class)); - expectClusterId(); + mockInternalConverters(); + mockFileConfigProvider(); - PowerMock.replayAll(); + connectUtilsMockedStatic.when(() -> ConnectUtils.lookupKafkaClusterId(any())).thenReturn(CLUSTER_ID); worker = new Worker(WORKER_ID, new MockTime(), @@ -861,40 +779,23 @@ public void testConnectorStatusMetricsGroup_taskStatusCounter() { assertEquals(2L, (long) metricGroup.taskCounter("c1").metricValue(0L)); assertEquals(1L, (long) metricGroup.taskCounter("c2").metricValue(0L)); assertEquals(0L, (long) metricGroup.taskCounter("fakeConnector").metricValue(0L)); + + connectUtilsMockedStatic.verify(() -> ConnectUtils.lookupKafkaClusterId(any())); } @Test public void testStartTaskFailure() { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - Map origProps = new HashMap<>(); - origProps.put(TaskConfig.TASK_CLASS_CONFIG, "missing.From.This.Workers.Classpath"); - - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader); - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) - .andReturn(pluginLoader); - - // We would normally expect this since the plugin loader would have been swapped in. However, since we mock out - // all classloader changes, the call actually goes to the normal default classloader. However, this works out - // fine since we just wanted a ClassNotFoundException anyway. - // EasyMock.expect(pluginLoader.loadClass(origProps.get(TaskConfig.TASK_CLASS_CONFIG))) - // .andThrow(new ClassNotFoundException()); + mockInternalConverters(); + mockFileConfigProvider(); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)) - .andReturn(delegatingLoader); + Map origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, "missing.From.This.Workers.Classpath"); - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) - .andReturn(pluginLoader); + when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); + when(plugins.delegatingLoader()).thenReturn(delegatingLoader); + when(delegatingLoader.connectorLoader(SampleSourceConnector.class.getName())).thenReturn(pluginLoader); - taskStatusListener.onFailure(EasyMock.eq(TASK_ID), EasyMock.anyObject()); - EasyMock.expectLastCall(); - - expectClusterId(); - - PowerMock.replayAll(); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(pluginLoader)).thenReturn(delegatingLoader); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader); worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); worker.herder = herder; @@ -909,73 +810,44 @@ public void testStartTaskFailure() { assertStartupStatistics(worker, 0, 0, 1, 1); assertEquals(Collections.emptySet(), worker.taskIds()); - PowerMock.verifyAll(); + verify(taskStatusListener).onFailure(eq(TASK_ID), any(ConfigException.class)); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(pluginLoader)); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader)); } @Test - public void testCleanupTasksOnStop() throws Exception { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); - - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - expectNewWorkerTask(); - Map origProps = new HashMap<>(); - origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); + public void testCleanupTasksOnStop() { + mockInternalConverters(); + mockStorage(); + mockFileConfigProvider(); - TaskConfig taskConfig = new TaskConfig(origProps); - // We should expect this call, but the pluginLoader being swapped in is only mocked. - // EasyMock.expect(pluginLoader.loadClass(TestSourceTask.class.getName())) - // .andReturn((Class) TestSourceTask.class); - EasyMock.expect(plugins.newTask(TestSourceTask.class)).andReturn(task); - EasyMock.expect(task.version()).andReturn("1.0"); - - workerTask.initialize(taskConfig); - EasyMock.expectLastCall(); + when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); + when(plugins.newTask(TestSourceTask.class)).thenReturn(task); + when(task.version()).thenReturn("1.0"); // Expect that the worker will create converters and will not initially find them using the current classloader ... assertNotNull(taskKeyConverter); assertNotNull(taskValueConverter); assertNotNull(taskHeaderConverter); - expectTaskKeyConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, null); - expectTaskKeyConverters(ClassLoaderUsage.PLUGINS, taskKeyConverter); - expectTaskValueConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, null); - expectTaskValueConverters(ClassLoaderUsage.PLUGINS, taskValueConverter); - expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, null); - expectTaskHeaderConverter(ClassLoaderUsage.PLUGINS, taskHeaderConverter); - - EasyMock.expect(executorService.submit(workerTask)).andReturn(null); - - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) - .andReturn(pluginLoader); + mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, null); + mockTaskConverter(ClassLoaderUsage.PLUGINS, WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, taskKeyConverter); + mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, null); + mockTaskConverter(ClassLoaderUsage.PLUGINS, WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, taskValueConverter); + mockTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, null); + mockTaskHeaderConverter(ClassLoaderUsage.PLUGINS, taskHeaderConverter); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader) - .times(2); + when(executorService.submit(any(WorkerSourceTask.class))).thenReturn(null); - EasyMock.expect(workerTask.loader()).andReturn(pluginLoader); + when(plugins.delegatingLoader()).thenReturn(delegatingLoader); + when(delegatingLoader.connectorLoader(SampleSourceConnector.class.getName())).thenReturn(pluginLoader); + doReturn(SampleSourceConnector.class).when(plugins).connectorClass(SampleSourceConnector.class.getName()); - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader) - .times(2); - plugins.connectorClass(WorkerTestConnector.class.getName()); - EasyMock.expectLastCall().andReturn(WorkerTestConnector.class); - // Remove on Worker.stop() - workerTask.stop(); - EasyMock.expectLastCall(); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(pluginLoader)).thenReturn(delegatingLoader); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader); - EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andReturn(true); - // Note that in this case we *do not* commit offsets since it's an unclean shutdown - EasyMock.expectLastCall(); + Map origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); - workerTask.removeMetrics(); - EasyMock.expectLastCall(); - - expectStopStorage(); - expectClusterId(); - - PowerMock.replayAll(); + TaskConfig taskConfig = new TaskConfig(origProps); worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService, noneConnectorClientConfigOverridePolicy); @@ -987,72 +859,60 @@ public void testCleanupTasksOnStop() throws Exception { worker.stop(); assertStatistics(worker, 0, 0); - PowerMock.verifyAll(); + verifyStorage(); + + WorkerSourceTask constructedMockTask = sourceTaskMockedConstruction.constructed().get(0); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(pluginLoader), times(2)); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader), times(2)); + verify(plugins).newTask(TestSourceTask.class); + verify(plugins, times(2)).currentThreadLoader(); + + verify(plugins).delegatingLoader(); + verify(delegatingLoader).connectorLoader(SampleSourceConnector.class.getName()); + verify(plugins).connectorClass(SampleSourceConnector.class.getName()); + verify(constructedMockTask).initialize(taskConfig); + verify(constructedMockTask).loader(); + verify(constructedMockTask).stop(); + verify(constructedMockTask).awaitStop(anyLong()); + verify(constructedMockTask).removeMetrics(); + verifyConverters(); + + verify(executorService).submit(any(WorkerSourceTask.class)); } @Test - public void testConverterOverrides() throws Exception { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); - - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - expectNewWorkerTask(); - Map origProps = new HashMap<>(); - origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); + public void testConverterOverrides() { + mockInternalConverters(); + mockStorage(); + mockFileConfigProvider(); + when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); + Map origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); TaskConfig taskConfig = new TaskConfig(origProps); - // We should expect this call, but the pluginLoader being swapped in is only mocked. - // EasyMock.expect(pluginLoader.loadClass(TestSourceTask.class.getName())) - // .andReturn((Class) TestSourceTask.class); - EasyMock.expect(plugins.newTask(TestSourceTask.class)).andReturn(task); - EasyMock.expect(task.version()).andReturn("1.0"); - workerTask.initialize(taskConfig); - EasyMock.expectLastCall(); + when(plugins.newTask(TestSourceTask.class)).thenReturn(task); + when(task.version()).thenReturn("1.0"); // Expect that the worker will create converters and will not initially find them using the current classloader ... assertNotNull(taskKeyConverter); assertNotNull(taskValueConverter); assertNotNull(taskHeaderConverter); - expectTaskKeyConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, null); - expectTaskKeyConverters(ClassLoaderUsage.PLUGINS, taskKeyConverter); - expectTaskValueConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, null); - expectTaskValueConverters(ClassLoaderUsage.PLUGINS, taskValueConverter); - expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, null); - expectTaskHeaderConverter(ClassLoaderUsage.PLUGINS, taskHeaderConverter); - - EasyMock.expect(executorService.submit(workerTask)).andReturn(null); - - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) - .andReturn(pluginLoader); - - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader) - .times(2); + mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, null); + mockTaskConverter(ClassLoaderUsage.PLUGINS, WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, taskKeyConverter); + mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, null); + mockTaskConverter(ClassLoaderUsage.PLUGINS, WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, taskValueConverter); + mockTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, null); + mockTaskHeaderConverter(ClassLoaderUsage.PLUGINS, taskHeaderConverter); - EasyMock.expect(workerTask.loader()).andReturn(pluginLoader); + when(executorService.submit(any(WorkerSourceTask.class))).thenReturn(null); - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader) - .times(2); - plugins.connectorClass(WorkerTestConnector.class.getName()); - EasyMock.expectLastCall().andReturn(WorkerTestConnector.class); + when(plugins.delegatingLoader()).thenReturn(delegatingLoader); + when(delegatingLoader.connectorLoader(SampleSourceConnector.class.getName())).thenReturn(pluginLoader); + doReturn(SampleSourceConnector.class).when(plugins).connectorClass(SampleSourceConnector.class.getName()); - // Remove - workerTask.stop(); - EasyMock.expectLastCall(); - EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andStubReturn(true); - EasyMock.expectLastCall(); - - workerTask.removeMetrics(); - EasyMock.expectLastCall(); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(pluginLoader)).thenReturn(delegatingLoader); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader); - expectStopStorage(); - expectClusterId(); - - PowerMock.replayAll(); worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService, noneConnectorClientConfigOverridePolicy); @@ -1061,13 +921,11 @@ public void testConverterOverrides() throws Exception { assertStatistics(worker, 0, 0); assertEquals(Collections.emptySet(), worker.taskIds()); Map connProps = anyConnectorConfigMap(); - connProps.put(ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); - connProps.put("key.converter.extra.config", "foo"); - connProps.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, TestConfigurableConverter.class.getName()); - connProps.put("value.converter.extra.config", "bar"); + connProps.put(ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, SampleConverterWithHeaders.class.getName()); + connProps.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, SampleConverterWithHeaders.class.getName()); worker.startTask(TASK_ID, ClusterConfigState.EMPTY, connProps, origProps, taskStatusListener, TargetState.STARTED); assertStatistics(worker, 0, 1); - assertEquals(new HashSet<>(Arrays.asList(TASK_ID)), worker.taskIds()); + assertEquals(Collections.singleton(TASK_ID), worker.taskIds()); worker.stopAndAwaitTask(TASK_ID); assertStatistics(worker, 0, 0); assertEquals(Collections.emptySet(), worker.taskIds()); @@ -1076,20 +934,35 @@ public void testConverterOverrides() throws Exception { assertStatistics(worker, 0, 0); // We've mocked the Plugin.newConverter method, so we don't currently configure the converters + verify(plugins).newTask(TestSourceTask.class); + WorkerSourceTask instantiatedTask = sourceTaskMockedConstruction.constructed().get(0); + verify(instantiatedTask).initialize(taskConfig); + verify(executorService).submit(any(WorkerSourceTask.class)); + verify(plugins).delegatingLoader(); + verify(delegatingLoader).connectorLoader(SampleSourceConnector.class.getName()); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(pluginLoader), times(2)); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader), times(2)); + verify(plugins).connectorClass(SampleSourceConnector.class.getName()); - PowerMock.verifyAll(); + // Remove + verify(instantiatedTask).stop(); + verify(instantiatedTask).awaitStop(anyLong()); + verify(instantiatedTask).removeMetrics(); + + verify(plugins, times(2)).currentThreadLoader(); + verifyStorage(); } @Test public void testProducerConfigsWithoutOverrides() { - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).andReturn( - new HashMap<>()); - PowerMock.replayAll(); + when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).thenReturn(new HashMap<>()); Map expectedConfigs = new HashMap<>(defaultProducerConfigs); expectedConfigs.put("client.id", "connector-producer-job-0"); expectedConfigs.put("metrics.context.connect.kafka.cluster.id", CLUSTER_ID); assertEquals(expectedConfigs, Worker.producerConfigs(TASK_ID, "connector-producer-" + TASK_ID, config, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); + + verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX); } @Test @@ -1106,11 +979,10 @@ public void testProducerConfigsWithOverrides() { expectedConfigs.put("client.id", "producer-test-id"); expectedConfigs.put("metrics.context.connect.kafka.cluster.id", CLUSTER_ID); - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).andReturn( - new HashMap<>()); - PowerMock.replayAll(); + when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).thenReturn(new HashMap<>()); assertEquals(expectedConfigs, Worker.producerConfigs(TASK_ID, "connector-producer-" + TASK_ID, configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); + verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX); } @Test @@ -1131,11 +1003,13 @@ public void testProducerConfigsWithClientOverrides() { Map connConfig = new HashMap<>(); connConfig.put("linger.ms", "5000"); connConfig.put("batch.size", "1000"); - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)) - .andReturn(connConfig); - PowerMock.replayAll(); + + when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).thenReturn(connConfig); + assertEquals(expectedConfigs, Worker.producerConfigs(TASK_ID, "connector-producer-" + TASK_ID, configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); + + verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX); } @Test @@ -1145,10 +1019,11 @@ public void testConsumerConfigsWithoutOverrides() { expectedConfigs.put("client.id", "connector-consumer-test-1"); expectedConfigs.put("metrics.context.connect.kafka.cluster.id", CLUSTER_ID); - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)).andReturn(new HashMap<>()); - PowerMock.replayAll(); + when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)).thenReturn(new HashMap<>()); assertEquals(expectedConfigs, Worker.consumerConfigs(new ConnectorTaskId("test", 1), config, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); + + verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX); } @Test @@ -1166,11 +1041,12 @@ public void testConsumerConfigsWithOverrides() { expectedConfigs.put("client.id", "consumer-test-id"); expectedConfigs.put("metrics.context.connect.kafka.cluster.id", CLUSTER_ID); - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)).andReturn(new HashMap<>()); - PowerMock.replayAll(); + when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)).thenReturn(new HashMap<>()); + assertEquals(expectedConfigs, Worker.consumerConfigs(new ConnectorTaskId("test", 1), configWithOverrides, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); + verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX); } @Test @@ -1191,11 +1067,13 @@ public void testConsumerConfigsWithClientOverrides() { Map connConfig = new HashMap<>(); connConfig.put("max.poll.records", "5000"); connConfig.put("max.poll.interval.ms", "1000"); - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)) - .andReturn(connConfig); - PowerMock.replayAll(); + + when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)).thenReturn(connConfig); + assertEquals(expectedConfigs, Worker.consumerConfigs(new ConnectorTaskId("test", 1), configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); + + verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX); } @Test @@ -1208,11 +1086,12 @@ public void testConsumerConfigsClientOverridesWithNonePolicy() { Map connConfig = new HashMap<>(); connConfig.put("max.poll.records", "5000"); connConfig.put("max.poll.interval.ms", "1000"); - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)) - .andReturn(connConfig); - PowerMock.replayAll(); + when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)).thenReturn(connConfig); + assertThrows(ConnectException.class, () -> Worker.consumerConfigs(new ConnectorTaskId("test", 1), configWithOverrides, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); + + verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX); } @Test @@ -1224,22 +1103,20 @@ public void testAdminConfigsClientOverridesWithAllPolicy() { props.put("consumer.bootstrap.servers", "localhost:4761"); WorkerConfig configWithOverrides = new StandaloneConfig(props); - Map connConfig = new HashMap<>(); - connConfig.put("metadata.max.age.ms", "10000"); - + Map connConfig = Collections.singletonMap("metadata.max.age.ms", "10000"); Map expectedConfigs = new HashMap<>(workerProps); - expectedConfigs.put("bootstrap.servers", "localhost:9092"); expectedConfigs.put("client.id", "testid"); expectedConfigs.put("metadata.max.age.ms", "10000"); + //we added a config on the fly expectedConfigs.put("metrics.context.connect.kafka.cluster.id", CLUSTER_ID); - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)) - .andReturn(connConfig); - PowerMock.replayAll(); + when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)).thenReturn(connConfig); assertEquals(expectedConfigs, Worker.adminConfigs(new ConnectorTaskId("test", 1), "", configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); + + verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX); } @Test @@ -1249,63 +1126,21 @@ public void testAdminConfigsClientOverridesWithNonePolicy() { props.put("admin.metadata.max.age.ms", "5000"); WorkerConfig configWithOverrides = new StandaloneConfig(props); - Map connConfig = new HashMap<>(); - connConfig.put("metadata.max.age.ms", "10000"); + Map connConfig = Collections.singletonMap("metadata.max.age.ms", "10000"); - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)) - .andReturn(connConfig); - PowerMock.replayAll(); + when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)).thenReturn(connConfig); assertThrows(ConnectException.class, () -> Worker.adminConfigs(new ConnectorTaskId("test", 1), "", configWithOverrides, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); + verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX); } @Test public void testWorkerMetrics() throws Exception { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - // Create - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - EasyMock.expect(plugins.newConnector(WorkerTestConnector.class.getName())) - .andReturn(sourceConnector); - EasyMock.expect(sourceConnector.version()).andReturn("1.0"); - - Map props = new HashMap<>(); - props.put(SinkConnectorConfig.TOPICS_CONFIG, "foo,bar"); - props.put(ConnectorConfig.TASKS_MAX_CONFIG, "1"); - props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); - props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName()); - - EasyMock.expect(sourceConnector.version()).andReturn("1.0"); - - EasyMock.expect(plugins.compareAndSwapLoaders(sourceConnector)) - .andReturn(delegatingLoader) - .times(2); - sourceConnector.initialize(anyObject(ConnectorContext.class)); - EasyMock.expectLastCall(); - sourceConnector.start(props); - EasyMock.expectLastCall(); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) - .andReturn(pluginLoader).times(2); - - connectorStatusListener.onStartup(CONNECTOR_ID); - EasyMock.expectLastCall(); - - // Remove - sourceConnector.stop(); - EasyMock.expectLastCall(); - - connectorStatusListener.onShutdown(CONNECTOR_ID); - EasyMock.expectLastCall(); - - expectStopStorage(); - expectClusterId(); - - PowerMock.replayAll(); + mockInternalConverters(); + mockFileConfigProvider(); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader); Worker worker = new Worker("worker-1", Time.SYSTEM, plugins, @@ -1316,7 +1151,6 @@ public void testWorkerMetrics() throws Exception { MetricName name = worker.metrics().metrics().metricName("test.avg", "grp1"); worker.metrics().metrics().addMetric(name, new Avg()); MBeanServer server = ManagementFactory.getPlatformMBeanServer(); - Set ret = server.queryMBeans(null, null); List list = worker.metrics().metrics().reporters(); for (MetricsReporter reporter : list) { @@ -1375,165 +1209,63 @@ private void assertStartupStatistics(Worker worker, int connectorStartupAttempts assertEquals(taskStartupFailurePct, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-startup-failure-percentage"), 0.0001d); } - private void expectStartStorage() { - offsetBackingStore.configure(anyObject(WorkerConfig.class)); - EasyMock.expectLastCall(); - offsetBackingStore.start(); - EasyMock.expectLastCall(); - EasyMock.expect(herder.statusBackingStore()) - .andReturn(statusBackingStore).anyTimes(); + private void mockStorage() { + when(herder.statusBackingStore()).thenReturn(statusBackingStore); } - private void expectStopStorage() { - offsetBackingStore.stop(); - EasyMock.expectLastCall(); - } - private void expectConverters() { - expectConverters(JsonConverter.class, false); + private void verifyStorage() { + verify(offsetBackingStore).configure(any(WorkerConfig.class)); + verify(offsetBackingStore).start(); + verify(herder).statusBackingStore(); + verify(offsetBackingStore).stop(); } - private void expectConverters(Boolean expectDefaultConverters) { - expectConverters(JsonConverter.class, expectDefaultConverters); - } + private void mockInternalConverters() { + Converter internalKeyConverter = mock(JsonConverter.class); + Converter internalValueConverter = mock(JsonConverter.class); - private void expectConverters(Class converterClass, Boolean expectDefaultConverters) { - // As default converters are instantiated when a task starts, they are expected only if the `startTask` method is called - if (expectDefaultConverters) { + when(plugins.newInternalConverter(eq(true), anyString(), anyMap())) + .thenReturn(internalKeyConverter); - // Instantiate and configure default - EasyMock.expect(plugins.newConverter(config, WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, ClassLoaderUsage.PLUGINS)) - .andReturn(keyConverter); - EasyMock.expect(plugins.newConverter(config, WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, ClassLoaderUsage.PLUGINS)) - .andReturn(valueConverter); - EasyMock.expectLastCall(); - } + when(plugins.newInternalConverter(eq(false), anyString(), anyMap())) + .thenReturn(internalValueConverter); + } - //internal - Converter internalKeyConverter = PowerMock.createMock(converterClass); - Converter internalValueConverter = PowerMock.createMock(converterClass); - - // Instantiate and configure internal - EasyMock.expect( - plugins.newInternalConverter( - EasyMock.eq(true), - EasyMock.anyString(), - EasyMock.anyObject() - ) - ).andReturn(internalKeyConverter); - EasyMock.expect( - plugins.newInternalConverter( - EasyMock.eq(false), - EasyMock.anyString(), - EasyMock.anyObject() - ) - ).andReturn(internalValueConverter); - EasyMock.expectLastCall(); + private void verifyConverters() { + verify(plugins, times(1)).newInternalConverter(eq(true), anyString(), anyMap()); + verify(plugins).newInternalConverter(eq(false), anyString(), anyMap()); } - private void expectTaskKeyConverters(ClassLoaderUsage classLoaderUsage, Converter returning) { - EasyMock.expect( - plugins.newConverter( - anyObject(AbstractConfig.class), - eq(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG), - eq(classLoaderUsage))) - .andReturn(returning); + private void mockTaskConverter(ClassLoaderUsage classLoaderUsage, String converterClassConfig, Converter returning) { + when(plugins.newConverter(any(AbstractConfig.class), eq(converterClassConfig), eq(classLoaderUsage))) + .thenReturn(returning); } - private void expectTaskValueConverters(ClassLoaderUsage classLoaderUsage, Converter returning) { - EasyMock.expect( - plugins.newConverter( - anyObject(AbstractConfig.class), - eq(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG), - eq(classLoaderUsage))) - .andReturn(returning); + private void verifyTaskConverter(String converterClassConfig) { + verify(plugins).newConverter(any(AbstractConfig.class), eq(converterClassConfig), eq(ClassLoaderUsage.CURRENT_CLASSLOADER)); } - private void expectTaskHeaderConverter(ClassLoaderUsage classLoaderUsage, HeaderConverter returning) { - EasyMock.expect( - plugins.newHeaderConverter( - anyObject(AbstractConfig.class), - eq(WorkerConfig.HEADER_CONVERTER_CLASS_CONFIG), - eq(classLoaderUsage))) - .andReturn(returning); + private void mockTaskHeaderConverter(ClassLoaderUsage classLoaderUsage, HeaderConverter returning) { + when(plugins.newHeaderConverter(any(AbstractConfig.class), eq(WorkerConfig.HEADER_CONVERTER_CLASS_CONFIG), eq(classLoaderUsage))) + .thenReturn(returning); } + private void verifyTaskHeaderConverter() { + verify(plugins).newHeaderConverter(any(AbstractConfig.class), eq(WorkerConfig.HEADER_CONVERTER_CLASS_CONFIG), eq(ClassLoaderUsage.CURRENT_CLASSLOADER)); + } + + private Map anyConnectorConfigMap() { Map props = new HashMap<>(); props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); - props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName()); + props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, SampleSourceConnector.class.getName()); props.put(ConnectorConfig.TASKS_MAX_CONFIG, "1"); props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); return props; } - private void expectClusterId() { - PowerMock.mockStaticPartial(ConnectUtils.class, "lookupKafkaClusterId"); - EasyMock.expect(ConnectUtils.lookupKafkaClusterId(EasyMock.anyObject())).andReturn("test-cluster").anyTimes(); - } - - private void expectNewWorkerTask() throws Exception { - PowerMock.expectNew( - WorkerSourceTask.class, EasyMock.eq(TASK_ID), - EasyMock.eq(task), - anyObject(TaskStatus.Listener.class), - EasyMock.eq(TargetState.STARTED), - anyObject(JsonConverter.class), - anyObject(JsonConverter.class), - anyObject(JsonConverter.class), - EasyMock.eq(new TransformationChain<>(Collections.emptyList(), NOOP_OPERATOR)), - anyObject(KafkaProducer.class), - anyObject(TopicAdmin.class), - EasyMock.>anyObject(), - anyObject(OffsetStorageReader.class), - anyObject(OffsetStorageWriter.class), - EasyMock.eq(config), - anyObject(ClusterConfigState.class), - anyObject(ConnectMetrics.class), - EasyMock.eq(pluginLoader), - anyObject(Time.class), - anyObject(RetryWithToleranceOperator.class), - anyObject(StatusBackingStore.class), - anyObject(Executor.class)) - .andReturn(workerTask); - } - /* Name here needs to be unique as we are testing the aliasing mechanism */ - public static class WorkerTestConnector extends SourceConnector { - - private static final ConfigDef CONFIG_DEF = new ConfigDef() - .define("configName", ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, "Test configName."); - - @Override - public String version() { - return "1.0"; - } - - @Override - public void start(Map props) { - - } - - @Override - public Class taskClass() { - return null; - } - - @Override - public List> taskConfigs(int maxTasks) { - return null; - } - - @Override - public void stop() { - - } - - @Override - public ConfigDef config() { - return CONFIG_DEF; - } - } private static class TestSourceTask extends SourceTask { public TestSourceTask() { @@ -1558,51 +1290,5 @@ public void stop() { } } - public static class TestConverter implements Converter { - public Map configs; - - @Override - public void configure(Map configs, boolean isKey) { - this.configs = configs; - } - - @Override - public byte[] fromConnectData(String topic, Schema schema, Object value) { - return new byte[0]; - } - - @Override - public SchemaAndValue toConnectData(String topic, byte[] value) { - return null; - } - } - - public static class TestConfigurableConverter implements Converter, Configurable { - public Map configs; - public ConfigDef config() { - return JsonConverterConfig.configDef(); - } - - @Override - public void configure(Map configs) { - this.configs = configs; - new JsonConverterConfig(configs); // requires the `converter.type` config be set - } - - @Override - public void configure(Map configs, boolean isKey) { - this.configs = configs; - } - - @Override - public byte[] fromConnectData(String topic, Schema schema, Object value) { - return new byte[0]; - } - - @Override - public SchemaAndValue toConnectData(String topic, byte[] value) { - return null; - } - } } From eb6c5baf3ee764cd821f0a1d48b0fede445146f6 Mon Sep 17 00:00:00 2001 From: GauthamM-official Date: Mon, 14 Mar 2022 21:50:28 +0530 Subject: [PATCH 002/447] MINOR: Adding kafka-storage.bat file (similar to kafka-storage.sh) for windows. (#11816) Reviewers: Jun Rao --- bin/windows/kafka-storage.bat | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) create mode 100644 bin/windows/kafka-storage.bat diff --git a/bin/windows/kafka-storage.bat b/bin/windows/kafka-storage.bat new file mode 100644 index 000000000000..4a0e458a623b --- /dev/null +++ b/bin/windows/kafka-storage.bat @@ -0,0 +1,17 @@ +@echo off +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +"%~dp0kafka-run-class.bat" kafka.tools.StorageTool %* From b1f36360ed739ff1e29941952f219ee6d15258ec Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Mon, 14 Mar 2022 09:22:01 -0700 Subject: [PATCH 003/447] KAKFA-13699: new ProcessorContext is missing methods (#11877) We added `currentSystemTimeMs()` and `currentStreamTimeMs()` to the `ProcessorContext` via KIP-622, but forgot to add both to the new `api.ProcessorContext`. Reviewers: Ricardo Brasil , Guozhang Wang --- .../processor/api/ProcessorContext.java | 28 +++++++++++++++++++ .../processor/api/MockProcessorContext.java | 10 +++++++ 2 files changed, 38 insertions(+) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessorContext.java index d110a76ef7bb..494dac4771b9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessorContext.java @@ -17,7 +17,9 @@ package org.apache.kafka.streams.processor.api; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.processor.Cancellable; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; @@ -247,4 +249,30 @@ Cancellable schedule(final Duration interval, * @return the key/values matching the given prefix from the StreamsConfig properties. */ Map appConfigsWithPrefix(final String prefix); + + /** + * Return the current system timestamp (also called wall-clock time) in milliseconds. + * + *

Note: this method returns the internally cached system timestamp from the Kafka Stream runtime. + * Thus, it may return a different value compared to {@code System.currentTimeMillis()}. + * + * @return the current system timestamp in milliseconds + */ + long currentSystemTimeMs(); + + /** + * Return the current stream-time in milliseconds. + * + *

Stream-time is the maximum observed {@link TimestampExtractor record timestamp} so far + * (including the currently processed record), i.e., it can be considered a high-watermark. + * Stream-time is tracked on a per-task basis and is preserved across restarts and during task migration. + * + *

Note: this method is not supported for global processors (cf. {@link Topology#addGlobalStore} (...) + * and {@link StreamsBuilder#addGlobalStore} (...), + * because there is no concept of stream-time for this case. + * Calling this method in a global processor will result in an {@link UnsupportedOperationException}. + * + * @return the current stream-time in milliseconds + */ + long currentStreamTimeMs(); } diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java index acd946a9d48e..26c09b8fd7fa 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java @@ -284,6 +284,16 @@ public Map appConfigsWithPrefix(final String prefix) { return config.originalsWithPrefix(prefix); } + @Override + public long currentSystemTimeMs() { + throw new UnsupportedOperationException(); + } + + @Override + public long currentStreamTimeMs() { + throw new UnsupportedOperationException(); + } + @Override public Serde keySerde() { return config.defaultKeySerde(); From b916cb40bd7bfe7077ce9fd1c2b4270b1b97b5ed Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 14 Mar 2022 15:42:10 -0700 Subject: [PATCH 004/447] KAFKA-13690: Fix flaky test in EosIntegrationTest (#11887) I found a couple of flakiness with the integration test. IQv1 on stores failed although getting the store itself is covered with timeouts, since the InvalidStoreException is upon the query (store.all()). I changed to the util function with IQv2 whose timeout/retry covers the whole procedure. Example of such failure is: https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-11802/11/tests/ With ALOS we should not check that the output, as well as the state store content is exactly as of processed once, since it is possible that during processing we got spurious task-migrate exceptions and re-processed with duplicates. I actually cannot reproduce this error locally, but from the jenkins errors it seems possible indeed. Example of such failure is: https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-11433/4/tests/ Some minor cleanups. Reviewers: A. Sophie Blee-Goldman --- .../integration/EosIntegrationTest.java | 158 ++++++------------ 1 file changed, 55 insertions(+), 103 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java index f3dcc646418e..5a97579e70da 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java @@ -40,21 +40,17 @@ import org.apache.kafka.streams.kstream.Transformer; import org.apache.kafka.streams.kstream.TransformerSupplier; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.StreamThread; +import org.apache.kafka.streams.query.QueryResult; +import org.apache.kafka.streams.query.RangeQuery; +import org.apache.kafka.streams.query.StateQueryRequest; +import org.apache.kafka.streams.query.StateQueryResult; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.streams.state.QueryableStoreTypes; -import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.internals.OffsetCheckpoint; -import org.apache.kafka.streams.state.internals.RocksDBStore; -import org.apache.kafka.streams.state.internals.RocksDbKeyValueBytesStoreSupplier; import org.apache.kafka.test.IntegrationTest; -import org.apache.kafka.test.MockInternalProcessorContext; -import org.apache.kafka.test.MockKeyValueStore; import org.apache.kafka.test.StreamsTestUtils; import org.apache.kafka.test.TestUtils; import org.junit.AfterClass; @@ -71,7 +67,6 @@ import java.io.File; import java.io.IOException; -import java.math.BigInteger; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -91,6 +86,7 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForEmptyConsumerGroup; +import static org.apache.kafka.streams.query.StateQueryRequest.inStore; import static org.apache.kafka.test.StreamsTestUtils.startKafkaStreamsAndWaitForRunningState; import static org.apache.kafka.test.TestUtils.consumerConfig; import static org.apache.kafka.test.TestUtils.waitForCondition; @@ -98,7 +94,6 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @RunWith(Parameterized.class) @@ -284,20 +279,7 @@ private void runSimpleCopyTest(final int numberOfRestarts, try (final KafkaStreams streams = new KafkaStreams(builder.build(), config)) { startKafkaStreamsAndWaitForRunningState(streams, MAX_WAIT_TIME_MS); - final List> committedRecords = IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived( - TestUtils.consumerConfig( - CLUSTER.bootstrapServers(), - CONSUMER_GROUP_ID, - LongDeserializer.class, - LongDeserializer.class, - Utils.mkProperties(Collections.singletonMap( - ConsumerConfig.ISOLATION_LEVEL_CONFIG, - IsolationLevel.READ_COMMITTED.name().toLowerCase(Locale.ROOT))) - ), - outputTopic, - inputData.size() - ); - + final List> committedRecords = readResult(outputTopic, inputData.size(), CONSUMER_GROUP_ID); checkResultPerKey(committedRecords, inputData, "The committed records do not match what expected"); } } @@ -365,21 +347,7 @@ public void shouldBeAbleToPerformMultipleTransactions() throws Exception { CLUSTER.time ); - final List> firstCommittedRecords = - IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived( - TestUtils.consumerConfig( - CLUSTER.bootstrapServers(), - CONSUMER_GROUP_ID, - LongDeserializer.class, - LongDeserializer.class, - Utils.mkProperties(Collections.singletonMap( - ConsumerConfig.ISOLATION_LEVEL_CONFIG, - IsolationLevel.READ_COMMITTED.name().toLowerCase(Locale.ROOT))) - ), - SINGLE_PARTITION_OUTPUT_TOPIC, - firstBurstOfData.size() - ); - + final List> firstCommittedRecords = readResult(SINGLE_PARTITION_OUTPUT_TOPIC, firstBurstOfData.size(), CONSUMER_GROUP_ID); assertThat(firstCommittedRecords, equalTo(firstBurstOfData)); IntegrationTestUtils.produceKeyValuesSynchronously( @@ -389,21 +357,7 @@ public void shouldBeAbleToPerformMultipleTransactions() throws Exception { CLUSTER.time ); - final List> secondCommittedRecords = - IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived( - TestUtils.consumerConfig( - CLUSTER.bootstrapServers(), - CONSUMER_GROUP_ID, - LongDeserializer.class, - LongDeserializer.class, - Utils.mkProperties(Collections.singletonMap( - ConsumerConfig.ISOLATION_LEVEL_CONFIG, - IsolationLevel.READ_COMMITTED.name().toLowerCase(Locale.ROOT))) - ), - SINGLE_PARTITION_OUTPUT_TOPIC, - secondBurstOfData.size() - ); - + final List> secondCommittedRecords = readResult(SINGLE_PARTITION_OUTPUT_TOPIC, secondBurstOfData.size(), CONSUMER_GROUP_ID); assertThat(secondCommittedRecords, equalTo(secondBurstOfData)); } } @@ -445,7 +399,7 @@ public void shouldNotViolateEosIfOneTaskFails() throws Exception { // p-0: ---> 10 rec + C // p-1: ---> 10 rec + C - final List> committedRecords = readResult(committedDataBeforeFailure.size(), CONSUMER_GROUP_ID); + final List> committedRecords = readResult(SINGLE_PARTITION_OUTPUT_TOPIC, committedDataBeforeFailure.size(), CONSUMER_GROUP_ID); checkResultPerKey( committedRecords, committedDataBeforeFailure, @@ -458,7 +412,7 @@ public void shouldNotViolateEosIfOneTaskFails() throws Exception { // p-0: ---> 10 rec + C + 5 rec (pending) // p-1: ---> 10 rec + C + 5 rec (pending) - final List> uncommittedRecords = readResult(dataBeforeFailure.size(), null); + final List> uncommittedRecords = readResult(SINGLE_PARTITION_OUTPUT_TOPIC, dataBeforeFailure.size(), null); checkResultPerKey( uncommittedRecords, dataBeforeFailure, @@ -477,10 +431,12 @@ public void shouldNotViolateEosIfOneTaskFails() throws Exception { // p-1: ---> 10 rec + C + 5 rec + C + 5 rec + C final List> allCommittedRecords = readResult( + SINGLE_PARTITION_OUTPUT_TOPIC, committedDataBeforeFailure.size() + uncommittedDataBeforeFailure.size() + dataAfterFailure.size(), CONSUMER_GROUP_ID + "_ALL"); final List> committedRecordsAfterFailure = readResult( + SINGLE_PARTITION_OUTPUT_TOPIC, uncommittedDataBeforeFailure.size() + dataAfterFailure.size(), CONSUMER_GROUP_ID); @@ -551,7 +507,7 @@ public void shouldNotViolateEosIfOneTaskFailsWithState() throws Exception { // p-0: ---> 10 rec + C // p-1: ---> 10 rec + C - final List> committedRecords = readResult(committedDataBeforeFailure.size(), CONSUMER_GROUP_ID); + final List> committedRecords = readResult(SINGLE_PARTITION_OUTPUT_TOPIC, committedDataBeforeFailure.size(), CONSUMER_GROUP_ID); checkResultPerKey( committedRecords, computeExpectedResult(committedDataBeforeFailure), @@ -564,7 +520,7 @@ public void shouldNotViolateEosIfOneTaskFailsWithState() throws Exception { // p-0: ---> 10 rec + C + 5 rec (pending) // p-1: ---> 10 rec + C + 5 rec (pending) - final List> uncommittedRecords = readResult(dataBeforeFailure.size(), null); + final List> uncommittedRecords = readResult(SINGLE_PARTITION_OUTPUT_TOPIC, dataBeforeFailure.size(), null); final List> expectedResultBeforeFailure = computeExpectedResult(dataBeforeFailure); @@ -590,10 +546,12 @@ public void shouldNotViolateEosIfOneTaskFailsWithState() throws Exception { // p-1: ---> 10 rec + C + 5 rec + C + 5 rec + C final List> allCommittedRecords = readResult( + SINGLE_PARTITION_OUTPUT_TOPIC, committedDataBeforeFailure.size() + uncommittedDataBeforeFailure.size() + dataAfterFailure.size(), CONSUMER_GROUP_ID + "_ALL"); final List> committedRecordsAfterFailure = readResult( + SINGLE_PARTITION_OUTPUT_TOPIC, uncommittedDataBeforeFailure.size() + dataAfterFailure.size(), CONSUMER_GROUP_ID); @@ -671,7 +629,7 @@ public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances() th // p-0: ---> 10 rec + C // p-1: ---> 10 rec + C - final List> committedRecords = readResult(committedDataBeforeStall.size(), CONSUMER_GROUP_ID); + final List> committedRecords = readResult(SINGLE_PARTITION_OUTPUT_TOPIC, committedDataBeforeStall.size(), CONSUMER_GROUP_ID); checkResultPerKey( committedRecords, committedDataBeforeStall, @@ -684,7 +642,7 @@ public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances() th // p-0: ---> 10 rec + C + 5 rec (pending) // p-1: ---> 10 rec + C + 5 rec (pending) - final List> uncommittedRecords = readResult(dataBeforeStall.size(), null); + final List> uncommittedRecords = readResult(SINGLE_PARTITION_OUTPUT_TOPIC, dataBeforeStall.size(), null); checkResultPerKey( uncommittedRecords, dataBeforeStall, @@ -730,6 +688,7 @@ public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances() th // p-1: ---> 10 rec + C + 5 rec + C + 5 rec + C final List> committedRecordsAfterRebalance = readResult( + SINGLE_PARTITION_OUTPUT_TOPIC, uncommittedDataBeforeStall.size() + dataToTriggerFirstRebalance.size(), CONSUMER_GROUP_ID); @@ -766,6 +725,7 @@ public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances() th // p-1: ---> 10 rec + C + 5 rec + C + 5 rec + C + 10 rec + C final List> allCommittedRecords = readResult( + SINGLE_PARTITION_OUTPUT_TOPIC, committedDataBeforeStall.size() + uncommittedDataBeforeStall.size() + dataToTriggerFirstRebalance.size() + dataAfterSecondRebalance.size(), CONSUMER_GROUP_ID + "_ALL"); @@ -791,49 +751,36 @@ public void shouldWriteLatestOffsetsToCheckpointOnShutdown() throws Exception { final List> expectedResult = computeExpectedResult(writtenData); try (final KafkaStreams streams = getKafkaStreams("streams", true, "appDir", 1, eosConfig, MAX_POLL_INTERVAL_MS)) { + writeInputData(writtenData); startKafkaStreamsAndWaitForRunningState(streams, MAX_WAIT_TIME_MS); - writeInputData(writtenData); - waitForCondition( () -> commitRequested.get() == 2, MAX_WAIT_TIME_MS, "SteamsTasks did not request commit."); - final List> committedRecords = readResult(writtenData.size(), CONSUMER_GROUP_ID); + final List> committedRecords = readResult(SINGLE_PARTITION_OUTPUT_TOPIC, writtenData.size(), CONSUMER_GROUP_ID); - checkResultPerKey( - committedRecords, - expectedResult, - "The committed records do not match what expected"); + if (!eosConfig.equals(StreamsConfig.AT_LEAST_ONCE)) { + checkResultPerKey( + committedRecords, + expectedResult, + "The committed records do not match what expected"); - verifyStateStore( - streams, - getMaxPerKey(expectedResult), - "The state store content do not match what expected"); + verifyStateStore( + streams, + getMaxPerKey(expectedResult), + "The state store content do not match what expected"); + } } - final Set> expectedState = getMaxPerKey(expectedResult); - verifyStateIsInStoreAndOffsetsAreInCheckpoint(0, expectedState); - verifyStateIsInStoreAndOffsetsAreInCheckpoint(1, expectedState); - - assertThat("Not all expected state values were found in the state stores", expectedState.isEmpty()); + verifyOffsetsAreInCheckpoint(0); + verifyOffsetsAreInCheckpoint(1); } - private void verifyStateIsInStoreAndOffsetsAreInCheckpoint(final int partition, final Set> expectedState) throws IOException { + private void verifyOffsetsAreInCheckpoint(final int partition) throws IOException { final String stateStoreDir = stateTmpDir + File.separator + "appDir" + File.separator + applicationId + File.separator + "0_" + partition + File.separator; - // Verify that the data in the state store on disk is fully up-to-date - final StateStoreContext context = new MockInternalProcessorContext(new Properties(), new TaskId(0, 0), new File(stateStoreDir)); - final MockKeyValueStore stateStore = new MockKeyValueStore("store", false); - final RocksDBStore store = (RocksDBStore) new RocksDbKeyValueBytesStoreSupplier(storeName, false).get(); - store.init(context, stateStore); - - store.all().forEachRemaining(kv -> { - final KeyValue kv2 = new KeyValue<>(new BigInteger(kv.key.get()).longValue(), new BigInteger(kv.value).longValue()); - expectedState.remove(kv2); - }); - // Verify that the checkpointed offsets match exactly with max offset of the records in the changelog final OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(stateStoreDir + ".checkpoint")); final Map checkpointedOffsets = checkpoint.read(); @@ -866,8 +813,8 @@ private void verifyChangelogMaxRecordOffsetMatchesCheckpointedOffset(final Topic private List> prepareData(final long fromInclusive, final long toExclusive, final Long... keys) { - final Long dataSize = keys.length * (toExclusive - fromInclusive); - final List> data = new ArrayList<>(dataSize.intValue()); + final long dataSize = keys.length * (toExclusive - fromInclusive); + final List> data = new ArrayList<>((int) dataSize); for (final Long k : keys) { for (long v = fromInclusive; v < toExclusive; ++v) { @@ -1022,7 +969,8 @@ private void writeInputData(final List> records) { ); } - private List> readResult(final int numberOfRecords, + private List> readResult(final String topic, + final int numberOfRecords, final String groupId) throws Exception { if (groupId != null) { return IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived( @@ -1034,7 +982,7 @@ private List> readResult(final int numberOfRecords, Utils.mkProperties(Collections.singletonMap( ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.name().toLowerCase(Locale.ROOT)))), - SINGLE_PARTITION_OUTPUT_TOPIC, + topic, numberOfRecords ); } @@ -1042,7 +990,7 @@ private List> readResult(final int numberOfRecords, // read uncommitted return IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived( TestUtils.consumerConfig(CLUSTER.bootstrapServers(), LongDeserializer.class, LongDeserializer.class), - SINGLE_PARTITION_OUTPUT_TOPIC, + topic, numberOfRecords ); } @@ -1088,17 +1036,21 @@ private Set> getMaxPerKey(final List> private void verifyStateStore(final KafkaStreams streams, final Set> expectedStoreContent, - final String reason) throws Exception { - final ReadOnlyKeyValueStore store = IntegrationTestUtils - .getStore(300_000L, storeName, streams, QueryableStoreTypes.keyValueStore()); - assertNotNull(store); - - try (final KeyValueIterator it = store.all()) { - while (it.hasNext()) { - assertTrue(reason, expectedStoreContent.remove(it.next())); + final String reason) { + final StateQueryRequest> request = + inStore(storeName).withQuery(RangeQuery.withNoBounds()); + + final StateQueryResult> result = + IntegrationTestUtils.iqv2WaitForResult(streams, request); + + for (final QueryResult> partitionResult: result.getPartitionResults().values()) { + try (final KeyValueIterator it = partitionResult.getResult()) { + while (it.hasNext()) { + assertTrue(reason, expectedStoreContent.remove(it.next())); + } } - - assertTrue(reason, expectedStoreContent.isEmpty()); } + + assertTrue(reason, expectedStoreContent.isEmpty()); } } From 76cf7a5793702b55e2cfd98a375f8f1708ff32c3 Mon Sep 17 00:00:00 2001 From: Liam Clarke-Hutchinson Date: Tue, 15 Mar 2022 16:20:18 +1300 Subject: [PATCH 005/447] KAFKA-7077: Use default producer settings in Connect Worker (#11475) Reviewers: Luke Chen --- .../src/main/java/org/apache/kafka/connect/runtime/Worker.java | 2 -- .../test/java/org/apache/kafka/connect/runtime/WorkerTest.java | 2 -- 2 files changed, 4 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index 582271a19633..45c7d9f6d13a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -648,8 +648,6 @@ static Map producerConfigs(ConnectorTaskId id, // These settings will execute infinite retries on retriable exceptions. They *may* be overridden via configs passed to the worker, // but this may compromise the delivery guarantees of Kafka Connect. producerProps.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.toString(Long.MAX_VALUE)); - producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); - producerProps.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1"); producerProps.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, Integer.toString(Integer.MAX_VALUE)); producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, defaultClientId); // User-specified overrides diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index 2b210794207a..e57e5e33fea8 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -213,8 +213,6 @@ public void setup() { defaultProducerConfigs.put( ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); defaultProducerConfigs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.toString(Long.MAX_VALUE)); - defaultProducerConfigs.put(ProducerConfig.ACKS_CONFIG, "all"); - defaultProducerConfigs.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1"); defaultProducerConfigs.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, Integer.toString(Integer.MAX_VALUE)); defaultConsumerConfigs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); From cad4985a0a15c6372e9b42260c2468d18d93e3d6 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 14 Mar 2022 21:32:28 -0700 Subject: [PATCH 006/447] MINOR: Disable those flaky tests (#11895) I collected a list of the most flaky tests observed lately, checked / created their corresponding tickets, and mark them as ignored for now. Many of these failures are: 0. Failing very frequently in the past (at least in my observations). 1. not investigated for some time. 2. have a PR for review (mostly thanks to @showuon !), but not reviewed for some time. Since 0), these tests failures are hindering our development; and from 1/2) above, people are either too busy to look after them, or honestly the tests are not considered as providing values since otherwise people should care enough to panic and try to resolve. So I think it's reasonable to disable all these tests for now. If we later learned our lesson a hard way, it would motivate us to tackle flaky tests more diligently as well. I'm only disabling those tests that have been failed for a while, and if for such time no one have been looking into them, I'm concerned that just gossiping around about those flakiness would not bring people's attention to them either. So my psychological motivation is that "if people do not care about those failed tests for weeks (which, is not a good thing! :P), let's teach ourselves the lesson a hard way when it indeed buries a bug that bites us, or not learn the lesson at all --- that indicates those tests are indeed not valuable". For tests that I only very recently saw I did not disable them. Reviewers: John Roesler , Matthias J. Sax , Luke Chen , Randall Hauch --- .../RebalanceSourceConnectorsIntegrationTest.java | 4 ++-- .../scala/integration/kafka/api/ConsumerBounceTest.scala | 2 ++ .../kafka/server/DynamicBrokerReconfigurationTest.scala | 4 ++++ .../scala/unit/kafka/admin/LeaderElectionCommandTest.scala | 5 ++++- .../src/test/scala/unit/kafka/network/SocketServerTest.scala | 2 ++ 5 files changed, 14 insertions(+), 3 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RebalanceSourceConnectorsIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RebalanceSourceConnectorsIntegrationTest.java index ae3e20d2a8f0..855882c9e697 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RebalanceSourceConnectorsIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RebalanceSourceConnectorsIntegrationTest.java @@ -188,6 +188,7 @@ public void testReconfigConnector() throws Exception { } @Test + @Ignore // TODO: To be re-enabled once we can make it less flaky (KAFKA-8391) public void testDeleteConnector() throws Exception { // create test topic connect.kafka().createTopic(TOPIC_NAME, NUM_TOPIC_PARTITIONS); @@ -269,8 +270,7 @@ public void testRemovingWorker() throws Exception { WORKER_SETUP_DURATION_MS, "Connect and tasks are imbalanced between the workers."); } - // should enable it after KAFKA-12495 fixed - @Ignore + @Ignore // TODO: To be re-enabled once we can make it less flaky (KAFKA-12495, KAFKA-12283) @Test public void testMultipleWorkersRejoining() throws Exception { // create test topic diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index f675ef5314a8..b3afc5075826 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -211,6 +211,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { } @Test + @Disabled // TODO: To be re-enabled once we can make it less flaky (KAFKA-7540) def testClose(): Unit = { val numRecords = 10 val producer = createProducer() @@ -299,6 +300,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { * Then, 1 consumer should be left out of the group. */ @Test + @Disabled // TODO: To be re-enabled once we can make it less flaky (KAFKA-13421) def testRollingBrokerRestartsWithSmallerMaxGroupSizeConfigDisruptsBigGroup(): Unit = { val group = "group-max-size-test" val topic = "group-max-size-test" diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 0cc58628103d..1d64106f096f 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -552,6 +552,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } @Test + @Disabled // TODO: To be re-enabled once we can make it less flaky: KAFKA-6527 def testDefaultTopicConfig(): Unit = { val (producerThread, consumerThread) = startProduceConsume(retries = 0) @@ -665,6 +666,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } @Test + @Disabled // TODO: To be re-enabled once we can make it less flaky: KAFKA-8280 def testUncleanLeaderElectionEnable(): Unit = { val controller = servers.find(_.config.brokerId == TestUtils.waitUntilControllerElected(zkClient)).get val controllerId = controller.config.brokerId @@ -726,6 +728,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } @Test + @Disabled // TODO: To be re-enabled once we can make it less flaky (KAFKA-13672) def testThreadPoolResize(): Unit = { val requestHandlerPrefix = "data-plane-kafka-request-handler-" val networkThreadPrefix = "data-plane-kafka-network-thread-" @@ -864,6 +867,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } @Test + @Disabled // TODO: To be re-enabled once we can make it less flaky (KAFKA-7957) def testMetricsReporterUpdate(): Unit = { // Add a new metrics reporter val newProps = new Properties diff --git a/core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala b/core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala index 4a2a401655fc..e9efdabe8ef6 100644 --- a/core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.UnknownTopicOrPartitionException import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.extension.ExtendWith -import org.junit.jupiter.api.{BeforeEach, Tag} +import org.junit.jupiter.api.{BeforeEach, Disabled, Tag} @ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(clusterType = Type.BOTH, brokers = 3) @@ -86,6 +86,7 @@ final class LeaderElectionCommandTest(cluster: ClusterInstance) { } @ClusterTest + @Disabled // TODO: re-enable until we fixed KAFKA-8541 def testTopicPartition(): Unit = { val client = cluster.createAdminClient() val topic = "unclean-topic" @@ -119,6 +120,7 @@ final class LeaderElectionCommandTest(cluster: ClusterInstance) { } @ClusterTest + @Disabled // TODO: re-enable until we fixed KAFKA-8785 def testPathToJsonFile(): Unit = { val client = cluster.createAdminClient() val topic = "unclean-topic" @@ -153,6 +155,7 @@ final class LeaderElectionCommandTest(cluster: ClusterInstance) { } @ClusterTest + @Disabled // TODO: re-enable after KAFKA-13737 is fixed def testPreferredReplicaElection(): Unit = { val client = cluster.createAdminClient() val topic = "preferred-topic" diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 849646cb644c..33d15ad10b5a 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -1389,6 +1389,7 @@ class SocketServerTest { * buffered receive. */ @Test + @Disabled // TODO: re-enabled until KAFKA-13735 is fixed def remoteCloseWithoutBufferedReceives(): Unit = { verifyRemoteCloseWithBufferedReceives(numComplete = 0, hasIncomplete = false) } @@ -1426,6 +1427,7 @@ class SocketServerTest { * The channel must be closed after pending receives are processed. */ @Test + @Disabled // TODO: re-enable after KAFKA-13736 is fixed def closingChannelWithBufferedReceives(): Unit = { verifyRemoteCloseWithBufferedReceives(numComplete = 3, hasIncomplete = false, makeClosing = true) } From 418b1221502aaab0c9d9f627172927b5ff2d1005 Mon Sep 17 00:00:00 2001 From: Paolo Patierno Date: Tue, 15 Mar 2022 12:08:10 +0100 Subject: [PATCH 007/447] MINOR: Improve producer Javadoc about send with acks = 0 (#11882) Reviewers: Mickael Maison --- .../org/apache/kafka/clients/producer/KafkaProducer.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index f739336b8569..ef686de7cebe 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -768,8 +768,9 @@ public Future send(ProducerRecord record) { * response after each one. *

* The result of the send is a {@link RecordMetadata} specifying the partition the record was sent to, the offset - * it was assigned and the timestamp of the record. If - * {@link org.apache.kafka.common.record.TimestampType#CREATE_TIME CreateTime} is used by the topic, the timestamp + * it was assigned and the timestamp of the record. If the producer is configured with acks = 0, the {@link RecordMetadata} + * will have offset = -1 because the producer does not wait for the acknowledgement from the broker. + * If {@link org.apache.kafka.common.record.TimestampType#CREATE_TIME CreateTime} is used by the topic, the timestamp * will be the user provided timestamp or the record send time if the user did not specify a timestamp for the * record. If {@link org.apache.kafka.common.record.TimestampType#LOG_APPEND_TIME LogAppendTime} is used for the * topic, the timestamp will be the Kafka broker local time when the message is appended. From e8a762eee4f2dbe2564cbb2319e51277ba6b4b3a Mon Sep 17 00:00:00 2001 From: wangyap Date: Tue, 15 Mar 2022 19:40:11 +0800 Subject: [PATCH 008/447] MINOR: set batch-size option into batch.size config in consoleProducer (#11855) Reviewers: Luke Chen --- .../scala/kafka/tools/ConsoleProducer.scala | 11 +++ .../kafka/tools/ConsoleProducerTest.scala | 73 +++++++++++++++++++ 2 files changed, 84 insertions(+) diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala index c99f6aed16ef..6afd9a923eac 100644 --- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -110,6 +110,11 @@ object ConsoleProducer { props, ProducerConfig.SEND_BUFFER_CONFIG, config.options, config.socketBufferSizeOpt) CommandLineUtils.maybeMergeOptions( props, ProducerConfig.BUFFER_MEMORY_CONFIG, config.options, config.maxMemoryBytesOpt) + // We currently have 2 options to set the batch.size value. We'll deprecate/remove one of them in KIP-717. + CommandLineUtils.maybeMergeOptions( + props, ProducerConfig.BATCH_SIZE_CONFIG, config.options, config.batchSizeOpt) + CommandLineUtils.maybeMergeOptions( + props, ProducerConfig.BATCH_SIZE_CONFIG, config.options, config.maxPartitionMemoryBytesOpt) CommandLineUtils.maybeMergeOptions( props, ProducerConfig.METADATA_MAX_AGE_CONFIG, config.options, config.metadataExpiryMsOpt) CommandLineUtils.maybeMergeOptions( @@ -138,6 +143,12 @@ object ConsoleProducer { .withOptionalArg() .describedAs("compression-codec") .ofType(classOf[String]) + val batchSizeOpt = parser.accepts("batch-size", "Number of messages to send in a single batch if they are not being sent synchronously. "+ + "please note that this option will be replaced if max-partition-memory-bytes is also set") + .withRequiredArg + .describedAs("size") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(16 * 1024) val messageSendMaxRetriesOpt = parser.accepts("message-send-max-retries", "Brokers can fail receiving the message for multiple reasons, " + "and being unavailable transiently is just one of them. This property specifies the number of retries before the producer give up and drop this message. " + "This is the option to control `retries` in producer configs.") diff --git a/core/src/test/scala/unit/kafka/tools/ConsoleProducerTest.scala b/core/src/test/scala/unit/kafka/tools/ConsoleProducerTest.scala index 84aafa18ea06..f136c62b5ff6 100644 --- a/core/src/test/scala/unit/kafka/tools/ConsoleProducerTest.scala +++ b/core/src/test/scala/unit/kafka/tools/ConsoleProducerTest.scala @@ -67,6 +67,46 @@ class ConsoleProducerTest { "--producer-property", "client.id=producer-1" ) + val batchSizeOverriddenByMaxPartitionMemoryBytesValue: Array[String] = Array( + "--broker-list", + "localhost:1001", + "--bootstrap-server", + "localhost:1002", + "--topic", + "t3", + "--batch-size", + "123", + "--max-partition-memory-bytes", + "456" + ) + val btchSizeSetAndMaxPartitionMemoryBytesNotSet: Array[String] = Array( + "--broker-list", + "localhost:1001", + "--bootstrap-server", + "localhost:1002", + "--topic", + "t3", + "--batch-size", + "123" + ) + val batchSizeNotSetAndMaxPartitionMemoryBytesSet: Array[String] = Array( + "--broker-list", + "localhost:1001", + "--bootstrap-server", + "localhost:1002", + "--topic", + "t3", + "--max-partition-memory-bytes", + "456" + ) + val batchSizeDefault: Array[String] = Array( + "--broker-list", + "localhost:1001", + "--bootstrap-server", + "localhost:1002", + "--topic", + "t3" + ) @Test def testValidConfigsBrokerList(): Unit = { @@ -123,4 +163,37 @@ class ConsoleProducerTest { assertEquals("console-producer", producerConfig.getString(ProducerConfig.CLIENT_ID_CONFIG)) } + + @Test + def testBatchSizeOverriddenByMaxPartitionMemoryBytesValue(): Unit = { + val config = new ConsoleProducer.ProducerConfig(batchSizeOverriddenByMaxPartitionMemoryBytesValue) + val producerConfig = new ProducerConfig(ConsoleProducer.producerProps(config)) + assertEquals(456, + producerConfig.getInt(ProducerConfig.BATCH_SIZE_CONFIG)) + } + + @Test + def testBatchSizeSetAndMaxPartitionMemoryBytesNotSet(): Unit = { + val config = new ConsoleProducer.ProducerConfig(btchSizeSetAndMaxPartitionMemoryBytesNotSet) + val producerConfig = new ProducerConfig(ConsoleProducer.producerProps(config)) + assertEquals(123, + producerConfig.getInt(ProducerConfig.BATCH_SIZE_CONFIG)) + } + + @Test + def testDefaultBatchSize(): Unit = { + val config = new ConsoleProducer.ProducerConfig(batchSizeDefault) + val producerConfig = new ProducerConfig(ConsoleProducer.producerProps(config)) + assertEquals(16*1024, + producerConfig.getInt(ProducerConfig.BATCH_SIZE_CONFIG)) + } + + @Test + def testBatchSizeNotSetAndMaxPartitionMemoryBytesSet (): Unit = { + val config = new ConsoleProducer.ProducerConfig(batchSizeNotSetAndMaxPartitionMemoryBytesSet) + val producerConfig = new ProducerConfig(ConsoleProducer.producerProps(config)) + assertEquals(456, + producerConfig.getInt(ProducerConfig.BATCH_SIZE_CONFIG)) + } + } From 03411ca28b21d554d754b3f67a7857afe64f5bef Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Tue, 15 Mar 2022 09:37:01 -0700 Subject: [PATCH 009/447] KAFKA-13721: asymetric join-winodws should not emit spurious left/outer join results (#11875) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Reviewers: Sergio Peña , Guozhang Wang --- .../kstream/internals/KStreamKStreamJoin.java | 5 +- .../internals/KStreamKStreamJoinTest.java | 7 +- .../internals/KStreamKStreamLeftJoinTest.java | 101 +++++++-- .../KStreamKStreamOuterJoinTest.java | 208 ++++++++++++++++-- 4 files changed, 285 insertions(+), 36 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index 305cb3843c73..4d9ac2e1672d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -51,6 +51,7 @@ class KStreamKStreamJoin implements ProcessorSupplier implements ProcessorSupplier= sharedTimeTracker.streamTime - joinAfterMs - joinGraceMs) { + if (sharedTimeTracker.minTime >= sharedTimeTracker.streamTime - joinSpuriousLookBackTimeMs - joinGraceMs) { return; } // throttle the emit frequency to a (configurable) interval; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java index 2ffa0480daed..d4f716df0843 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java @@ -103,7 +103,6 @@ public void shouldLogAndMeterOnSkippedRecordsWithNullValueWithBuiltInMetricsVers } } - @Test public void shouldReuseRepartitionTopicWithGeneratedName() { final StreamsBuilder builder = new StreamsBuilder(); @@ -137,7 +136,6 @@ public void shouldCreateRepartitionTopicsWithUserProvidedName() { @Test public void shouldDisableLoggingOnStreamJoined() { - final JoinWindows joinWindows = JoinWindows.ofTimeDifferenceAndGrace(ofMillis(100), Duration.ofMillis(50)); final StreamJoined streamJoined = StreamJoined .with(Serdes.String(), Serdes.Integer(), Serdes.Integer()) @@ -164,7 +162,6 @@ public void shouldDisableLoggingOnStreamJoined() { @Test public void shouldEnableLoggingWithCustomConfigOnStreamJoined() { - final JoinWindows joinWindows = JoinWindows.ofTimeDifferenceAndGrace(ofMillis(100), Duration.ofMillis(50)); final StreamJoined streamJoined = StreamJoined .with(Serdes.String(), Serdes.Integer(), Serdes.Integer()) @@ -1197,7 +1194,7 @@ public void testAsymmetricWindowingAfter() { joined = stream1.join( stream2, MockValueJoiner.TOSTRING_JOINER, - JoinWindows.ofTimeDifferenceAndGrace(ofMillis(0), ofMillis(0)).after(ofMillis(100)), + JoinWindows.ofTimeDifferenceWithNoGrace(ofMillis(0)).after(ofMillis(100)), StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()) @@ -1866,4 +1863,4 @@ private WindowBytesStoreSupplier buildWindowBytesStoreSupplier(final String name " <-- KSTREAM-MERGE-0000000011\n" + " Sink: KSTREAM-SINK-0000000021 (topic: out-to)\n" + " <-- KSTREAM-MERGE-0000000020\n\n"; -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java index 2a29915f75a5..156b553455d4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.KeyValueTimestamp; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig.InternalConfig; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.TopologyWrapper; import org.apache.kafka.streams.kstream.Consumed; @@ -34,6 +35,7 @@ import org.apache.kafka.test.MockApiProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.StreamsTestUtils; +import org.junit.BeforeClass; import org.junit.Test; import java.time.Duration; @@ -44,6 +46,7 @@ import java.util.Properties; import java.util.Set; +import static java.time.Duration.ZERO; import static java.time.Duration.ofMillis; import static org.junit.Assert.assertEquals; @@ -53,9 +56,14 @@ public class KStreamKStreamLeftJoinTest { private final String topic1 = "topic1"; private final String topic2 = "topic2"; private final Consumed consumed = Consumed.with(Serdes.Integer(), Serdes.String()); - private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); + private final static Properties PROPS = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); - @SuppressWarnings("deprecation") + @BeforeClass + public static void beforeClass() { + PROPS.put(InternalConfig.EMIT_INTERVAL_MS_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, 0L); + } + + @SuppressWarnings("deprecation") // old join semantics; can be removed when `JoinWindows.of()` is removed @Test public void testLeftJoinWithSpuriousResultFixDisabledOldApi() { final StreamsBuilder builder = new StreamsBuilder(); @@ -77,7 +85,7 @@ public void testLeftJoinWithSpuriousResultFixDisabledOldApi() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(props), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(PROPS), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -115,7 +123,7 @@ public void testLeftJoinWithSpuriousResultFixDisabledOldApi() { } } - @SuppressWarnings("deprecation") + @SuppressWarnings("deprecation") // old join semantics; can be removed when `JoinWindows.of()` is removed @Test public void testLeftJoinDuplicatesWithSpuriousResultFixDisabledOldApi() { final StreamsBuilder builder = new StreamsBuilder(); @@ -135,7 +143,7 @@ public void testLeftJoinDuplicatesWithSpuriousResultFixDisabledOldApi() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(props), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(PROPS), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -177,7 +185,7 @@ public void testLeftJoinDuplicates() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -233,7 +241,7 @@ public void testLeftExpiredNonJoinedRecordsAreEmittedByTheLeftProcessor() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -288,7 +296,7 @@ public void testLeftExpiredNonJoinedRecordsAreEmittedByTheRightProcessor() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -343,7 +351,7 @@ public void testRightNonJoinedRecordsAreNeverEmittedByTheLeftProcessor() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -395,7 +403,7 @@ public void testRightNonJoinedRecordsAreNeverEmittedByTheRightProcessor() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -480,7 +488,7 @@ public void runLeftJoin(final StreamJoined streamJoined assertEquals(1, copartitionGroups.size()); assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -585,7 +593,7 @@ public void testOrdering() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -642,7 +650,7 @@ public void testGracePeriod() { assertEquals(1, copartitionGroups.size()); assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -714,7 +722,7 @@ public void testWindowing() { assertEquals(1, copartitionGroups.size()); assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -749,6 +757,70 @@ public void testWindowing() { } } + @Test + public void shouldNotEmitLeftJoinResultForAsymmetricWindow() { + final StreamsBuilder builder = new StreamsBuilder(); + final int[] expectedKeys = new int[] {0, 1, 2, 3}; + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockApiProcessorSupplier supplier = new MockApiProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.leftJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.ofTimeDifferenceWithNoGrace(ofMillis(100)).before(ZERO), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()) + ); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockApiProcessor processor = supplier.theCapturedProcessor(); + long time = 0L; + + // push two items to the primary stream; the other window is empty; this should not produce any items + // w1 = {} + // w2 = {} + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // --> w2 = {} + for (int i = 0; i < 2; i++) { + inputTopic1.pipeInput(expectedKeys[i], "A" + expectedKeys[i], time + i); + } + processor.checkAndClearProcessResult(); + + // push one item to the other stream; this should produce one full-join item + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // w2 = {} + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // --> w2 = { 0:a0 (ts: 100) } + time += 100L; + inputTopic2.pipeInput(expectedKeys[0], "a" + expectedKeys[0], time); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "A0+a0", 100L) + ); + + // push one item to the other stream; this should produce one left-join item + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // w2 = { 0:a0 (ts: 100) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // --> w2 = { 0:a0 (ts: 100), 1:a1 (ts: 102) } + time += 2; + inputTopic2.pipeInput(expectedKeys[1], "a" + expectedKeys[1], time); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(1, "A1+null", 1L) + ); + } + } + private void testUpperWindowBound(final int[] expectedKeys, final TopologyTestDriver driver, final MockApiProcessor processor) { @@ -877,7 +949,6 @@ private void testUpperWindowBound(final int[] expectedKeys, // push a dummy record to produce all left-join non-joined items time += 301L; - driver.advanceWallClockTime(Duration.ofMillis(1000L)); inputTopic1.pipeInput(0, "dummy", time); processor.checkAndClearProcessResult( new KeyValueTimestamp<>(0, "C0+null", 1101L), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java index 0fcbfeb0049a..acb4be46bbe9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.KeyValueTimestamp; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig.InternalConfig; import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.TopologyWrapper; @@ -34,6 +35,7 @@ import org.apache.kafka.test.MockApiProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.StreamsTestUtils; +import org.junit.BeforeClass; import org.junit.Test; import java.time.Duration; @@ -44,6 +46,7 @@ import java.util.Properties; import java.util.Set; +import static java.time.Duration.ZERO; import static java.time.Duration.ofMillis; import static org.junit.Assert.assertEquals; @@ -51,9 +54,14 @@ public class KStreamKStreamOuterJoinTest { private final String topic1 = "topic1"; private final String topic2 = "topic2"; private final Consumed consumed = Consumed.with(Serdes.Integer(), Serdes.String()); - private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); + private final static Properties PROPS = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); - @SuppressWarnings("deprecation") + @BeforeClass + public static void beforeClass() { + PROPS.put(InternalConfig.EMIT_INTERVAL_MS_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, 0L); + } + + @SuppressWarnings("deprecation") // old join semantics; can be removed when `JoinWindows.of()` is removed @Test public void testOuterJoinDuplicatesWithFixDisabledOldApi() { final StreamsBuilder builder = new StreamsBuilder(); @@ -73,7 +81,7 @@ public void testOuterJoinDuplicatesWithFixDisabledOldApi() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(props), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(PROPS), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -117,7 +125,7 @@ public void testOuterJoinDuplicates() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -155,7 +163,6 @@ public void testOuterJoinDuplicates() { // this record should expired non-joined records; only null+a0 will be emitted because // it did not have a join - driver.advanceWallClockTime(Duration.ofMillis(1000L)); inputTopic2.pipeInput(3, "dummy", 1500L); processor.checkAndClearProcessResult( @@ -184,7 +191,7 @@ public void testLeftExpiredNonJoinedRecordsAreEmittedByTheLeftProcessor() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -239,7 +246,7 @@ public void testLeftExpiredNonJoinedRecordsAreEmittedByTheRightProcessor() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -294,7 +301,7 @@ public void testRightExpiredNonJoinedRecordsAreEmittedByTheLeftProcessor() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -349,7 +356,7 @@ public void testRightExpiredNonJoinedRecordsAreEmittedByTheRightProcessor() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -404,7 +411,7 @@ public void testOrdering() { ); joined.process(supplier); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -460,7 +467,7 @@ public void testGracePeriod() { assertEquals(1, copartitionGroups.size()); assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -559,7 +566,7 @@ public void runOuterJoin(final StreamJoined streamJoine assertEquals(1, copartitionGroups.size()); assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -672,7 +679,7 @@ public void testWindowing() { assertEquals(1, copartitionGroups.size()); assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = @@ -708,6 +715,178 @@ public void testWindowing() { } } + @Test + public void shouldNotEmitLeftJoinResultForAsymmetricBeforeWindow() { + final StreamsBuilder builder = new StreamsBuilder(); + final int[] expectedKeys = new int[] {0, 1, 2, 3}; + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockApiProcessorSupplier supplier = new MockApiProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.ofTimeDifferenceWithNoGrace(ofMillis(100)).before(ZERO), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()) + ); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockApiProcessor processor = supplier.theCapturedProcessor(); + long time = 0L; + + // push two items to the primary stream; the other window is empty; this should not produce any items + // w1 = {} + // w2 = {} + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // --> w2 = {} + for (int i = 0; i < 2; i++) { + inputTopic1.pipeInput(expectedKeys[i], "A" + expectedKeys[i], time + i); + } + processor.checkAndClearProcessResult(); + + // push one item to the other stream; this should produce one full-join item + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // w2 = {} + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // --> w2 = { 0:a0 (ts: 100) } + time += 100L; + inputTopic2.pipeInput(expectedKeys[0], "a" + expectedKeys[0], time); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "A0+a0", 100L) + ); + + // push one item to the other stream; this should produce one left-join item + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // w2 = { 0:a0 (ts: 100) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // --> w2 = { 0:a0 (ts: 100), 1:a1 (ts: 102) } + time += 2; + inputTopic2.pipeInput(expectedKeys[1], "a" + expectedKeys[1], time); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(1, "A1+null", 1L) + ); + + // push one item to the other stream; this should not produce any items + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // w2 = { 0:a0 (ts: 100), 1:a1 (ts: 102) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // --> w2 = { 0:a0 (ts: 100), 1:a1 (ts: 102), 2:a2 (ts: 103) } + time += 1; + inputTopic2.pipeInput(expectedKeys[2], "a" + expectedKeys[2], time); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(1, "null+a1", 102L) + ); + + // push one item to the first stream; this should not produce one full-join item + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // w2 = { 0:a0 (ts: 100), 1:a1 (ts: 102), 2:a2 (ts: 103) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1), 2:A2 (ts: 103) } + // --> w2 = { 0:a0 (ts: 100), 1:a1 (ts: 102), 2:a2 (ts: 103) } + inputTopic1.pipeInput(expectedKeys[2], "A" + expectedKeys[2], time); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(2, "A2+a2", 103L) + ); + } + } + + @Test + public void shouldNotEmitLeftJoinResultForAsymmetricAfterWindow() { + final StreamsBuilder builder = new StreamsBuilder(); + final int[] expectedKeys = new int[] {0, 1, 2, 3}; + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockApiProcessorSupplier supplier = new MockApiProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.ofTimeDifferenceWithNoGrace(ofMillis(100)).after(ZERO), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()) + ); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockApiProcessor processor = supplier.theCapturedProcessor(); + long time = 0L; + + // push two items to the primary stream; the other window is empty; this should not produce any item + // w1 = {} + // w2 = {} + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // --> w2 = {} + for (int i = 0; i < 2; i++) { + inputTopic1.pipeInput(expectedKeys[i], "A" + expectedKeys[i], time + i); + } + processor.checkAndClearProcessResult(); + + // push one item to the other stream; this should produce one full-join item + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // w2 = {} + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // --> w2 = { 1:a1 (ts: 1) } + time += 1; + inputTopic2.pipeInput(expectedKeys[1], "a" + expectedKeys[1], time); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(1, "A1+a1", 1L) + ); + + // push one item to the other stream; this should produce one left-join item + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // w2 = { 1:a1 (ts: 1) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // --> w2 = { 1:a1 (ts: 1), 2:a2 (ts: 101) } + time += 100; + inputTopic2.pipeInput(expectedKeys[2], "a" + expectedKeys[2], time); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "A0+null", 0L) + ); + + // push one item to the other stream; this should not produce any item + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // w2 = { 1:a1 (ts: 1), 2:a2 (ts: 101) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // --> w2 = { 1:a1 (ts: 1), 2:a2 (ts: 101), 3:a3 (ts: 101) } + inputTopic2.pipeInput(expectedKeys[3], "a" + expectedKeys[3], time); + + processor.checkAndClearProcessResult(); + + // push one item to the first stream; this should produce one full-join item + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) } + // w2 = { 1:a1 (ts: 1), 2:a2 (ts: 101), 3:a3 (ts: 101) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1), 2:A2 (ts: 201) } + // --> w2 = { 1:a1 (ts: 1), 2:a2 (ts: 101), 3:a3 (ts: 101 } + time += 100; + inputTopic1.pipeInput(expectedKeys[2], "A" + expectedKeys[2], time); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(2, "A2+a2", 201L) + ); + } + } + private void testUpperWindowBound(final int[] expectedKeys, final TopologyTestDriver driver, final MockApiProcessor processor) { @@ -839,7 +1018,6 @@ private void testUpperWindowBound(final int[] expectedKeys, // push a dummy record to produce all left-join non-joined items time += 301L; - driver.advanceWallClockTime(Duration.ofMillis(1000L)); inputTopic1.pipeInput(0, "dummy", time); processor.checkAndClearProcessResult( new KeyValueTimestamp<>(0, "C0+null", 1101L), @@ -1033,4 +1211,4 @@ private void testLowerWindowBound(final int[] expectedKeys, new KeyValueTimestamp<>(0, "dummy+null", 1103L) ); } -} \ No newline at end of file +} From 76d287c96771159d9b86dcf2fa193ff69198dd5b Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 15 Mar 2022 12:26:23 -0700 Subject: [PATCH 010/447] KAFKA-13727; Preserve txn markers after partial segment cleaning (#11891) It is possible to clean a segment partially if the offset map is filled before reaching the end of the segment. The highest offset that is reached becomes the new dirty offset after the cleaning completes. The data above this offset is nevertheless copied over to the new partially cleaned segment. Hence we need to ensure that the transaction index reflects aborted transactions from both the cleaned and uncleaned portion of the segment. Prior to this patch, this was not the case. We only collected the aborted transactions from the cleaned portion, which means that the reconstructed index could be incomplete. This can cause the aborted data to become effectively committed. It can also cause the deletion of the abort marker before the corresponding data has been removed (i.e. the aborted transaction becomes hanging). Reviewers: Jun Rao --- .../common/record/RecordBatchIterator.java | 4 + .../src/main/scala/kafka/log/LogCleaner.scala | 4 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 110 ++++++++++++++++++ 3 files changed, 117 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordBatchIterator.java b/clients/src/main/java/org/apache/kafka/common/record/RecordBatchIterator.java index 88af039847d5..967cff80cc7a 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/RecordBatchIterator.java +++ b/clients/src/main/java/org/apache/kafka/common/record/RecordBatchIterator.java @@ -17,8 +17,10 @@ package org.apache.kafka.common.record; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.utils.AbstractIterator; +import java.io.EOFException; import java.io.IOException; class RecordBatchIterator extends AbstractIterator { @@ -36,6 +38,8 @@ protected T makeNext() { if (batch == null) return allDone(); return batch; + } catch (EOFException e) { + throw new CorruptRecordException("Unexpected EOF while attempting to read the next batch", e); } catch (IOException e) { throw new KafkaException(e); } diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 0d4cab9a0fa0..55d795243df0 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -577,8 +577,10 @@ private[log] class Cleaner(val id: Int, val currentSegment = currentSegmentOpt.get val nextSegmentOpt = if (iter.hasNext) Some(iter.next()) else None + // Note that it is important to collect aborted transactions from the full log segment + // range since we need to rebuild the full transaction index for the new segment. val startOffset = currentSegment.baseOffset - val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(map.latestOffset + 1) + val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(currentSegment.readNextOffset) val abortedTransactions = log.collectAbortedTransactions(startOffset, upperBoundOffset) transactionMetadata.addAbortedTransactions(abortedTransactions) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 30f3bd4ba6d1..b3a1a76f0d0a 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -270,6 +270,111 @@ class LogCleanerTest { assertEquals(2L, logAppendInfo.lastOffset) } + private def assertAllAbortedTxns( + expectedAbortedTxns: List[AbortedTxn], + log: UnifiedLog + ): Unit= { + val abortedTxns = log.collectAbortedTransactions(startOffset = 0L, upperBoundOffset = log.logEndOffset) + assertEquals(expectedAbortedTxns, abortedTxns) + } + + private def assertAllTransactionsComplete(log: UnifiedLog): Unit = { + assertTrue(log.activeProducers.forall(_.currentTxnStartOffset() == -1)) + } + + @Test + def testMultiPassSegmentCleaningWithAbortedTransactions(): Unit = { + // Verify that the log cleaner preserves aborted transaction state (including the index) + // even if the cleaner cannot clean the whole segment in one pass. + + val deleteRetentionMs = 50000 + val offsetMapSlots = 4 + val cleaner = makeCleaner(Int.MaxValue) + val logProps = new Properties() + logProps.put(LogConfig.DeleteRetentionMsProp, deleteRetentionMs.toString) + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) + + val producerEpoch = 0.toShort + val producerId1 = 1 + val producerId2 = 2 + + val appendProducer1 = appendTransactionalAsLeader(log, producerId1, producerEpoch) + val appendProducer2 = appendTransactionalAsLeader(log, producerId2, producerEpoch) + + def abort(producerId: Long): Unit = { + log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Replication) + } + + def commit(producerId: Long): Unit = { + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Replication) + } + + // Append some transaction data (offset range in parenthesis) + appendProducer1(Seq(1, 2)) // [0, 1] + appendProducer2(Seq(2, 3)) // [2, 3] + appendProducer1(Seq(3, 4)) // [4, 5] + commit(producerId1) // [6, 6] + commit(producerId2) // [7, 7] + appendProducer1(Seq(2, 3)) // [8, 9] + abort(producerId1) // [10, 10] + appendProducer2(Seq(4, 5)) // [11, 12] + appendProducer1(Seq(5, 6)) // [13, 14] + commit(producerId1) // [15, 15] + abort(producerId2) // [16, 16] + appendProducer2(Seq(6, 7)) // [17, 18] + commit(producerId2) // [19, 19] + + log.roll() + assertEquals(20L, log.logEndOffset) + + val expectedAbortedTxns = List( + new AbortedTxn(producerId=producerId1, firstOffset=8, lastOffset=10, lastStableOffset=11), + new AbortedTxn(producerId=producerId2, firstOffset=11, lastOffset=16, lastStableOffset=17) + ) + + assertAllTransactionsComplete(log) + assertAllAbortedTxns(expectedAbortedTxns, log) + + var dirtyOffset = 0L + def cleanSegments(): Unit = { + val offsetMap = new FakeOffsetMap(slots = offsetMapSlots) + val segments = log.logSegments(0, log.activeSegment.baseOffset).toSeq + val stats = new CleanerStats(time) + cleaner.buildOffsetMap(log, dirtyOffset, log.activeSegment.baseOffset, offsetMap, stats) + cleaner.cleanSegments(log, segments, offsetMap, time.milliseconds(), stats, new CleanedTransactionMetadata, Long.MaxValue) + dirtyOffset = offsetMap.latestOffset + 1 + } + + // On the first pass, we should see the data from the aborted transactions deleted, + // but the markers should remain until the deletion retention time has passed. + cleanSegments() + assertEquals(4L, dirtyOffset) + assertEquals(List(0, 2, 4, 6, 7, 10, 13, 15, 16, 17, 19), batchBaseOffsetsInLog(log)) + assertEquals(List(0, 2, 3, 4, 5, 6, 7, 10, 13, 14, 15, 16, 17, 18, 19), offsetsInLog(log)) + assertAllTransactionsComplete(log) + assertAllAbortedTxns(expectedAbortedTxns, log) + + // On the second pass, no data from the aborted transactions remains. The markers + // still cannot be removed from the log due to the retention time, but we do not + // need to record them in the transaction index since they are empty. + cleanSegments() + assertEquals(14, dirtyOffset) + assertEquals(List(0, 2, 4, 6, 7, 10, 13, 15, 16, 17, 19), batchBaseOffsetsInLog(log)) + assertEquals(List(0, 2, 4, 5, 6, 7, 10, 13, 14, 15, 16, 17, 18, 19), offsetsInLog(log)) + assertAllTransactionsComplete(log) + assertAllAbortedTxns(List(), log) + + // On the last pass, wait for the retention time to expire. The abort markers + // (offsets 10 and 16) should be deleted. + time.sleep(deleteRetentionMs) + cleanSegments() + assertEquals(20L, dirtyOffset) + assertEquals(List(0, 2, 4, 6, 7, 13, 15, 17, 19), batchBaseOffsetsInLog(log)) + assertEquals(List(0, 2, 4, 5, 6, 7, 13, 15, 17, 18, 19), offsetsInLog(log)) + assertAllTransactionsComplete(log) + assertAllAbortedTxns(List(), log) + } + @Test def testBasicTransactionAwareCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) @@ -1080,6 +1185,11 @@ class LogCleanerTest { assertEquals(numInvalidMessages, stats.invalidMessagesRead, "Cleaner should have seen %d invalid messages.") } + private def batchBaseOffsetsInLog(log: UnifiedLog): Iterable[Long] = { + for (segment <- log.logSegments; batch <- segment.log.batches.asScala) + yield batch.baseOffset + } + def lastOffsetsPerBatchInLog(log: UnifiedLog): Iterable[Long] = { for (segment <- log.logSegments; batch <- segment.log.batches.asScala) yield batch.lastOffset From bda5c34b030207f542c7987a5e0f9bcb23406c18 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Tue, 15 Mar 2022 12:50:53 -0700 Subject: [PATCH 011/447] MINOR: refactor how ConfigurationControl checks for resource existence (#11835) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ConfigurationControl methods should take a boolean indicating whether the resource is newly created, rather than taking an existence checker object. The boolean is easier to understand. Also add a unit test of existing checking failing (and succeeding). Reviewers: Kirk True , José Armando García Sancio --- .../ConfigurationControlManager.java | 27 ++++--- .../kafka/controller/QuorumController.java | 10 ++- .../controller/ReplicationControlManager.java | 3 +- .../ConfigurationControlManagerTest.java | 81 ++++++++++++++----- .../ReplicationControlManagerTest.java | 8 +- 5 files changed, 88 insertions(+), 41 deletions(-) diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java index f9caf2bb04cb..a16361343b23 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java @@ -51,11 +51,10 @@ public class ConfigurationControlManager { - final static Consumer NO_OP_EXISTENCE_CHECKER = __ -> { }; - private final Logger log; private final SnapshotRegistry snapshotRegistry; private final KafkaConfigSchema configSchema; + private final Consumer existenceChecker; private final Optional alterConfigPolicy; private final ConfigurationValidator validator; private final TimelineHashMap> configData; @@ -63,11 +62,13 @@ public class ConfigurationControlManager { ConfigurationControlManager(LogContext logContext, SnapshotRegistry snapshotRegistry, KafkaConfigSchema configSchema, + Consumer existenceChecker, Optional alterConfigPolicy, ConfigurationValidator validator) { this.log = logContext.logger(ConfigurationControlManager.class); this.snapshotRegistry = snapshotRegistry; this.configSchema = configSchema; + this.existenceChecker = existenceChecker; this.alterConfigPolicy = alterConfigPolicy; this.validator = validator; this.configData = new TimelineHashMap<>(snapshotRegistry, 0); @@ -88,14 +89,14 @@ public class ConfigurationControlManager { */ ControllerResult> incrementalAlterConfigs( Map>> configChanges, - Consumer existenceChecker) { + boolean newlyCreatedResource) { List outputRecords = new ArrayList<>(); Map outputResults = new HashMap<>(); for (Entry>> resourceEntry : configChanges.entrySet()) { incrementalAlterConfigResource(resourceEntry.getKey(), resourceEntry.getValue(), - existenceChecker, + newlyCreatedResource, outputRecords, outputResults); } @@ -104,7 +105,7 @@ ControllerResult> incrementalAlterConfigs( private void incrementalAlterConfigResource(ConfigResource configResource, Map> keysToOps, - Consumer existenceChecker, + boolean newlyCreatedResource, List outputRecords, Map outputResults) { List newRecords = new ArrayList<>(); @@ -153,7 +154,7 @@ private void incrementalAlterConfigResource(ConfigResource configResource, setValue(newValue), CONFIG_RECORD.highestSupportedVersion())); } } - ApiError error = validateAlterConfig(configResource, newRecords, existenceChecker); + ApiError error = validateAlterConfig(configResource, newRecords, newlyCreatedResource); if (error.isFailure()) { outputResults.put(configResource, error); return; @@ -164,7 +165,7 @@ private void incrementalAlterConfigResource(ConfigResource configResource, private ApiError validateAlterConfig(ConfigResource configResource, List newRecords, - Consumer existenceChecker) { + boolean newlyCreatedResource) { Map newConfigs = new HashMap<>(); TimelineHashMap existingConfigs = configData.get(configResource); if (existingConfigs != null) newConfigs.putAll(existingConfigs); @@ -178,7 +179,9 @@ private ApiError validateAlterConfig(ConfigResource configResource, } try { validator.validate(configResource, newConfigs); - existenceChecker.accept(configResource); + if (!newlyCreatedResource) { + existenceChecker.accept(configResource); + } if (alterConfigPolicy.isPresent()) { alterConfigPolicy.get().validate(new RequestMetadata(configResource, newConfigs)); } @@ -201,7 +204,7 @@ private ApiError validateAlterConfig(ConfigResource configResource, */ ControllerResult> legacyAlterConfigs( Map> newConfigs, - Consumer existenceChecker + boolean newlyCreatedResource ) { List outputRecords = new ArrayList<>(); Map outputResults = new HashMap<>(); @@ -209,7 +212,7 @@ ControllerResult> legacyAlterConfigs( newConfigs.entrySet()) { legacyAlterConfigResource(resourceEntry.getKey(), resourceEntry.getValue(), - existenceChecker, + newlyCreatedResource, outputRecords, outputResults); } @@ -218,7 +221,7 @@ ControllerResult> legacyAlterConfigs( private void legacyAlterConfigResource(ConfigResource configResource, Map newConfigs, - Consumer existenceChecker, + boolean newlyCreatedResource, List outputRecords, Map outputResults) { List newRecords = new ArrayList<>(); @@ -247,7 +250,7 @@ private void legacyAlterConfigResource(ConfigResource configResource, setValue(null), CONFIG_RECORD.highestSupportedVersion())); } } - ApiError error = validateAlterConfig(configResource, newRecords, existenceChecker); + ApiError error = validateAlterConfig(configResource, newRecords, newlyCreatedResource); if (error.isFailure()) { outputResults.put(configResource, error); return; 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 11594ed77d2c..5b52c2d2fd47 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -1226,7 +1226,11 @@ private QuorumController(LogContext logContext, this.purgatory = new ControllerPurgatory(); this.resourceExists = new ConfigResourceExistenceChecker(); this.configurationControl = new ConfigurationControlManager(logContext, - snapshotRegistry, configSchema, alterConfigPolicy, configurationValidator); + snapshotRegistry, + configSchema, + resourceExists, + alterConfigPolicy, + configurationValidator); this.clientQuotaControlManager = new ClientQuotaControlManager(snapshotRegistry); this.clusterControl = new ClusterControlManager(logContext, clusterId, time, snapshotRegistry, sessionTimeoutNs, replicaPlacer, controllerMetrics); @@ -1340,7 +1344,7 @@ public CompletableFuture> incrementalAlterConfigs( } return appendWriteEvent("incrementalAlterConfigs", () -> { ControllerResult> result = - configurationControl.incrementalAlterConfigs(configChanges, resourceExists); + configurationControl.incrementalAlterConfigs(configChanges, false); if (validateOnly) { return result.withoutRecords(); } else { @@ -1380,7 +1384,7 @@ public CompletableFuture> legacyAlterConfigs( } return appendWriteEvent("legacyAlterConfigs", () -> { ControllerResult> result = - configurationControl.legacyAlterConfigs(newConfigs, resourceExists); + configurationControl.legacyAlterConfigs(newConfigs, false); if (validateOnly) { return result.withoutRecords(); } else { diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index ad9e3c7ecc9a..c492ed9af079 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -114,7 +114,6 @@ import static org.apache.kafka.common.protocol.Errors.NO_REASSIGNMENT_IN_PROGRESS; import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_ID; import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_OR_PARTITION; -import static org.apache.kafka.controller.ConfigurationControlManager.NO_OP_EXISTENCE_CHECKER; import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER; import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER_CHANGE; @@ -378,7 +377,7 @@ public void replay(RemoveTopicRecord record) { Map>> configChanges = computeConfigChanges(topicErrors, request.topics()); ControllerResult> configResult = - configurationControl.incrementalAlterConfigs(configChanges, NO_OP_EXISTENCE_CHECKER); + configurationControl.incrementalAlterConfigs(configChanges, true); for (Entry entry : configResult.response().entrySet()) { if (entry.getValue().isFailure()) { topicErrors.put(entry.getKey().name(), entry.getValue()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java index 60bcb8299ffc..6a71aba1e679 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.errors.PolicyViolationException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.metadata.ConfigRecord; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; @@ -40,6 +41,7 @@ import java.util.Map.Entry; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -51,7 +53,6 @@ import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SUBTRACT; import static org.apache.kafka.common.config.ConfigResource.Type.BROKER; import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC; -import static org.apache.kafka.controller.ConfigurationControlManager.NO_OP_EXISTENCE_CHECKER; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -76,6 +77,17 @@ public class ConfigurationControlManagerTest { static final ConfigResource BROKER0 = new ConfigResource(BROKER, "0"); static final ConfigResource MYTOPIC = new ConfigResource(TOPIC, "mytopic"); + static class TestExistenceChecker implements Consumer { + static final TestExistenceChecker INSTANCE = new TestExistenceChecker(); + + @Override + public void accept(ConfigResource resource) { + if (!resource.name().startsWith("Existing")) { + throw new UnknownTopicOrPartitionException("Unknown resource."); + } + } + } + @SuppressWarnings("unchecked") private static Map toMap(Entry... entries) { Map map = new LinkedHashMap<>(); @@ -89,12 +101,26 @@ static Entry entry(A a, B b) { return new SimpleImmutableEntry<>(a, b); } + static ConfigurationControlManager newConfigurationControlManager() { + return newConfigurationControlManager(Optional.empty()); + } + + static ConfigurationControlManager newConfigurationControlManager( + Optional alterConfigPolicy + ) { + LogContext logContext = new LogContext(); + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); + return new ConfigurationControlManager(logContext, + snapshotRegistry, + SCHEMA, + TestExistenceChecker.INSTANCE, + alterConfigPolicy, + ConfigurationValidator.NO_OP); + } + @Test public void testReplay() throws Exception { - SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); - ConfigurationControlManager manager = - new ConfigurationControlManager(new LogContext(), snapshotRegistry, SCHEMA, - Optional.empty(), ConfigurationValidator.NO_OP); + ConfigurationControlManager manager = newConfigurationControlManager(); assertEquals(Collections.emptyMap(), manager.getConfigs(BROKER0)); manager.replay(new ConfigRecord(). setResourceType(BROKER.id()).setResourceName("0"). @@ -125,17 +151,14 @@ public void testReplay() throws Exception { @Test public void testIncrementalAlterConfigs() { - SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); - ConfigurationControlManager manager = - new ConfigurationControlManager(new LogContext(), snapshotRegistry, SCHEMA, - Optional.empty(), ConfigurationValidator.NO_OP); + ConfigurationControlManager manager = newConfigurationControlManager(); ControllerResult> result = manager. incrementalAlterConfigs(toMap(entry(BROKER0, toMap( entry("baz", entry(SUBTRACT, "abc")), entry("quux", entry(SET, "abc")))), entry(MYTOPIC, toMap(entry("abc", entry(APPEND, "123"))))), - NO_OP_EXISTENCE_CHECKER); + true); assertEquals(ControllerResult.atomicOf(Collections.singletonList(new ApiMessageAndVersion( new ConfigRecord().setResourceType(TOPIC.id()).setResourceName("mytopic"). @@ -152,7 +175,26 @@ public void testIncrementalAlterConfigs() { toMap(entry(MYTOPIC, ApiError.NONE))), manager.incrementalAlterConfigs(toMap(entry(MYTOPIC, toMap( entry("abc", entry(DELETE, "xyz"))))), - NO_OP_EXISTENCE_CHECKER)); + true)); + } + + @Test + public void testIncrementalAlterConfigsWithoutExistence() { + ConfigurationControlManager manager = newConfigurationControlManager(); + ConfigResource existingTopic = new ConfigResource(TOPIC, "ExistingTopic"); + + ControllerResult> result = manager. + incrementalAlterConfigs(toMap(entry(BROKER0, toMap( + entry("quux", entry(SET, "1")))), + entry(existingTopic, toMap(entry("def", entry(SET, "newVal"))))), + false); + + assertEquals(ControllerResult.atomicOf(Collections.singletonList(new ApiMessageAndVersion( + new ConfigRecord().setResourceType(TOPIC.id()).setResourceName("ExistingTopic"). + setName("def").setValue("newVal"), (short) 0)), + toMap(entry(BROKER0, new ApiError(Errors.UNKNOWN_TOPIC_OR_PARTITION, + "Unknown resource.")), + entry(existingTopic, ApiError.NONE))), result); } private static class MockAlterConfigsPolicy implements AlterConfigPolicy { @@ -190,14 +232,12 @@ public void configure(Map configs) { @Test public void testIncrementalAlterConfigsWithPolicy() { - SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); MockAlterConfigsPolicy policy = new MockAlterConfigsPolicy(asList( new RequestMetadata(MYTOPIC, Collections.emptyMap()), new RequestMetadata(BROKER0, toMap(entry("foo.bar", "123"), entry("quux", "456"))))); - ConfigurationControlManager manager = new ConfigurationControlManager( - new LogContext(), snapshotRegistry, SCHEMA, Optional.of(policy), - ConfigurationValidator.NO_OP); + + ConfigurationControlManager manager = newConfigurationControlManager(Optional.of(policy)); assertEquals(ControllerResult.atomicOf(asList(new ApiMessageAndVersion( new ConfigRecord().setResourceType(BROKER.id()).setResourceName("0"). @@ -215,15 +255,12 @@ public void testIncrementalAlterConfigsWithPolicy() { entry(BROKER0, toMap( entry("foo.bar", entry(SET, "123")), entry("quux", entry(SET, "456"))))), - NO_OP_EXISTENCE_CHECKER)); + true)); } @Test public void testLegacyAlterConfigs() { - SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); - ConfigurationControlManager manager = - new ConfigurationControlManager(new LogContext(), snapshotRegistry, SCHEMA, - Optional.empty(), ConfigurationValidator.NO_OP); + ConfigurationControlManager manager = newConfigurationControlManager(); List expectedRecords1 = asList( new ApiMessageAndVersion(new ConfigRecord(). setResourceType(TOPIC.id()).setResourceName("mytopic"). @@ -235,7 +272,7 @@ public void testLegacyAlterConfigs() { expectedRecords1, toMap(entry(MYTOPIC, ApiError.NONE))), manager.legacyAlterConfigs( toMap(entry(MYTOPIC, toMap(entry("abc", "456"), entry("def", "901")))), - NO_OP_EXISTENCE_CHECKER)); + true)); for (ApiMessageAndVersion message : expectedRecords1) { manager.replay((ConfigRecord) message.message()); } @@ -249,6 +286,6 @@ expectedRecords1, toMap(entry(MYTOPIC, ApiError.NONE))), (short) 0)), toMap(entry(MYTOPIC, ApiError.NONE))), manager.legacyAlterConfigs(toMap(entry(MYTOPIC, toMap(entry("def", "901")))), - NO_OP_EXISTENCE_CHECKER)); + true)); } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index 4a08b237b0c2..a6c8c1e66171 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -143,8 +143,12 @@ private static class ReplicationControlTestContext { new StripedReplicaPlacer(random), metrics); final ConfigurationControlManager configurationControl = new ConfigurationControlManager( - new LogContext(), snapshotRegistry, KafkaConfigSchema.EMPTY, Optional.empty(), - (__, ___) -> { }); + new LogContext(), + snapshotRegistry, + KafkaConfigSchema.EMPTY, + __ -> { }, + Optional.empty(), + (__, ___) -> { }); final ReplicationControlManager replicationControl; void replay(List records) throws Exception { From f708dc58ed7f7e9d7aed2d43cd36aadaf32c75ee Mon Sep 17 00:00:00 2001 From: Walker Carlson <18128741+wcarlson5@users.noreply.github.com> Date: Tue, 15 Mar 2022 15:54:48 -0500 Subject: [PATCH 012/447] MINOR: fix shouldWaitForMissingInputTopicsToBeCreated test (#11902) This test was falling occasionally. It does appear to be a matter of the tests assuming perfecting deduplication/caching when asserting the test output records, ie a bug in the test not in the real code. Since we are not assuming that it is going to be perfect I changed the test to make sure the records we expect arrive, instead of only those arrive. Reviewers: Guozhang Wang --- .../streams/integration/NamedTopologyIntegrationTest.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java index 23aa898a8cce..0e367d909e9f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java @@ -779,7 +779,11 @@ public void shouldWaitForMissingInputTopicsToBeCreated() throws Exception { CLUSTER.createTopic(NEW_STREAM, 2, 1); produceToInputTopics(NEW_STREAM, STANDARD_INPUT_DATA); - assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), equalTo(COUNT_OUTPUT_DATA)); + final List> output = + waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3); + output.retainAll(COUNT_OUTPUT_DATA); + + assertThat(output, equalTo(COUNT_OUTPUT_DATA)); // Make sure the threads were not actually killed and replaced assertThat(streams.metadataForLocalThreads().size(), equalTo(2)); From 9e8ace080990a6471ed058edffaafa74984649e4 Mon Sep 17 00:00:00 2001 From: Nick Telford Date: Tue, 15 Mar 2022 22:55:20 +0000 Subject: [PATCH 013/447] KAFKA-13549: Add repartition.purge.interval.ms (#11610) Implements KIP-811. Add a new config `repartition.purge.interval.ms` that limits how often data is purged from repartition topics. --- .../apache/kafka/streams/StreamsConfig.java | 13 +++ .../processor/internals/StreamThread.java | 6 +- .../processor/internals/StreamThreadTest.java | 81 +++++++++++++++++++ 3 files changed, 99 insertions(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 62e5e82e24da..672df55c99e4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -375,6 +375,13 @@ public class StreamsConfig extends AbstractConfig { " (Note, if processing.guarantee is set to " + EXACTLY_ONCE_V2 + ", " + EXACTLY_ONCE + ",the default value is " + EOS_DEFAULT_COMMIT_INTERVAL_MS + "," + " otherwise the default value is " + DEFAULT_COMMIT_INTERVAL_MS + "."; + /** {@code repartition.purge.interval.ms} */ + @SuppressWarnings("WeakerAccess") + public static final String REPARTITION_PURGE_INTERVAL_MS_CONFIG = "repartition.purge.interval.ms"; + private static final String REPARTITION_PURGE_INTERVAL_MS_DOC = "The frequency in milliseconds with which to delete fully consumed records from repartition topics." + + " Purging will occur after at least this value since the last purge, but may be delayed until later." + + " (Note, unlike commit.interval.ms, the default for this value remains unchanged when processing.guarantee is set to " + EXACTLY_ONCE_V2 + ")."; + /** {@code connections.max.idle.ms} */ @SuppressWarnings("WeakerAccess") public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; @@ -768,6 +775,12 @@ public class StreamsConfig extends AbstractConfig { atLeast(0), Importance.LOW, COMMIT_INTERVAL_MS_DOC) + .define(REPARTITION_PURGE_INTERVAL_MS_CONFIG, + Type.LONG, + DEFAULT_COMMIT_INTERVAL_MS, + atLeast(0), + Importance.LOW, + REPARTITION_PURGE_INTERVAL_MS_DOC) .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, ConfigDef.Type.LONG, 9 * 60 * 1000L, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 8345a4ade921..7893a06fb79f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -261,6 +261,7 @@ public boolean isRunning() { public final Object stateLock; private final Duration pollTime; private final long commitTimeMs; + private final long purgeTimeMs; private final int maxPollTimeMs; private final String originalReset; private final TaskManager taskManager; @@ -288,6 +289,7 @@ public boolean isRunning() { private long now; private long lastPollMs; private long lastCommitMs; + private long lastPurgeMs; private long lastPartitionAssignedMs = -1L; private int numIterations; private volatile State state = State.CREATED; @@ -517,6 +519,7 @@ public StreamThread(final Time time, this.maxPollTimeMs = new InternalConsumerConfig(config.getMainConsumerConfigs("dummyGroupId", "dummyClientId", dummyThreadIdx)) .getInt(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG); this.commitTimeMs = config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG); + this.purgeTimeMs = config.getLong(StreamsConfig.REPARTITION_PURGE_INTERVAL_MS_CONFIG); this.numIterations = 1; this.eosEnabled = eosEnabled(config); @@ -1067,9 +1070,10 @@ int maybeCommit() { .collect(Collectors.toSet()) ); - if (committed > 0) { + if (committed > 0 && (now - lastPurgeMs) > purgeTimeMs) { // try to purge the committed records for repartition topics if possible taskManager.maybePurgeCommittedRecords(); + lastPurgeMs = now; } if (committed == -1) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 8fa632bb5ba8..c0cf9a20bd8a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -464,6 +464,36 @@ public void shouldNotCommitBeforeTheCommitInterval() { verify(taskManager); } + @Test + public void shouldNotPurgeBeforeThePurgeInterval() { + final long commitInterval = 1000L; + final long purgeInterval = 2000L; + final Properties props = configProps(false); + props.setProperty(StreamsConfig.STATE_DIR_CONFIG, stateDir); + props.setProperty(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(commitInterval)); + props.setProperty(StreamsConfig.REPARTITION_PURGE_INTERVAL_MS_CONFIG, Long.toString(purgeInterval)); + + final StreamsConfig config = new StreamsConfig(props); + final Consumer consumer = EasyMock.createNiceMock(Consumer.class); + final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class); + expect(consumer.groupMetadata()).andStubReturn(consumerGroupMetadata); + expect(consumerGroupMetadata.groupInstanceId()).andReturn(Optional.empty()); + final TaskManager taskManager = mockTaskManagerPurge(1); + taskManager.maybePurgeCommittedRecords(); + EasyMock.replay(consumer, consumerGroupMetadata); + + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); + final StreamThread thread = buildStreamThread(consumer, taskManager, config, topologyMetadata); + thread.setNow(mockTime.milliseconds()); + thread.maybeCommit(); + mockTime.sleep(purgeInterval - 10L); + thread.setNow(mockTime.milliseconds()); + thread.maybeCommit(); + + verify(taskManager); + } + @Test public void shouldEnforceRebalanceAfterNextScheduledProbingRebalanceTime() throws InterruptedException { final StreamsConfig config = new StreamsConfig(configProps(false)); @@ -758,6 +788,41 @@ int commit(final Collection tasksToCommit) { assertTrue(committed.get()); } + @Test + public void shouldPurgeAfterPurgeInterval() { + final long commitInterval = 100L; + final long purgeInterval = 200L; + + final Properties props = configProps(false); + props.setProperty(StreamsConfig.STATE_DIR_CONFIG, stateDir); + props.setProperty(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(commitInterval)); + props.setProperty(StreamsConfig.REPARTITION_PURGE_INTERVAL_MS_CONFIG, Long.toString(purgeInterval)); + + final StreamsConfig config = new StreamsConfig(props); + final Consumer consumer = EasyMock.createNiceMock(Consumer.class); + final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class); + expect(consumer.groupMetadata()).andStubReturn(consumerGroupMetadata); + expect(consumerGroupMetadata.groupInstanceId()).andReturn(Optional.empty()); + + final TaskManager taskManager = mockTaskManagerPurge(2); + + EasyMock.replay(consumer, consumerGroupMetadata); + + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); + final StreamThread thread = buildStreamThread(consumer, taskManager, config, topologyMetadata); + + thread.setNow(mockTime.milliseconds()); + thread.maybeCommit(); + + mockTime.sleep(purgeInterval + 1); + + thread.setNow(mockTime.milliseconds()); + thread.maybeCommit(); + + verify(taskManager); + } + @Test public void shouldRecordCommitLatency() { final Consumer consumer = EasyMock.createNiceMock(Consumer.class); @@ -2749,6 +2814,22 @@ void runOnce() { assertThat(failedThreads.metricValue(), is(shouldFail ? 1.0 : 0.0)); } + private TaskManager mockTaskManagerPurge(final int numberOfPurges) { + final TaskManager taskManager = EasyMock.createNiceMock(TaskManager.class); + final Task runningTask = mock(Task.class); + final TaskId taskId = new TaskId(0, 0); + + expect(runningTask.state()).andReturn(Task.State.RUNNING).anyTimes(); + expect(runningTask.id()).andReturn(taskId).anyTimes(); + expect(taskManager.tasks()) + .andReturn(Collections.singletonMap(taskId, runningTask)).anyTimes(); + expect(taskManager.commit(Collections.singleton(runningTask))).andReturn(1).anyTimes(); + taskManager.maybePurgeCommittedRecords(); + EasyMock.expectLastCall().times(numberOfPurges); + EasyMock.replay(taskManager, runningTask); + return taskManager; + } + private TaskManager mockTaskManagerCommit(final Consumer consumer, final int numberOfCommits, final int commits) { From 620f1d88d80fdf8150bd0b75be307bc4a2d3a0ea Mon Sep 17 00:00:00 2001 From: liym <43025881+ijliym@users.noreply.github.com> Date: Wed, 16 Mar 2022 07:58:47 +0800 Subject: [PATCH 014/447] Polish Javadoc for EpochState (#11897) Polish Javadoc for EpochState Reviewers: Bill Bejeck --- raft/src/main/java/org/apache/kafka/raft/EpochState.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/EpochState.java b/raft/src/main/java/org/apache/kafka/raft/EpochState.java index 89e8f0ac235b..9cf231c42131 100644 --- a/raft/src/main/java/org/apache/kafka/raft/EpochState.java +++ b/raft/src/main/java/org/apache/kafka/raft/EpochState.java @@ -27,7 +27,7 @@ default Optional highWatermark() { /** * Decide whether to grant a vote to a candidate, it is the responsibility of the caller to invoke - * {@link QuorumState##transitionToVoted(int, int)} if vote is granted. + * {@link QuorumState#transitionToVoted(int, int)} if vote is granted. * * @param candidateId The ID of the voter who attempt to become leader * @param isLogUpToDate Whether the candidate’s log is at least as up-to-date as receiver’s log, it From 1783fb14df6d83caf8af27bf5029e27674d53363 Mon Sep 17 00:00:00 2001 From: Mickael Maison Date: Wed, 16 Mar 2022 11:43:37 +0100 Subject: [PATCH 015/447] MINOR: Bump latest 3.0 version to 3.0.1 (#11885) Reviewers: Matthias J. Sax --- gradle/dependencies.gradle | 2 +- tests/docker/Dockerfile | 4 ++-- tests/kafkatest/version.py | 3 ++- vagrant/base.sh | 4 ++-- 4 files changed, 7 insertions(+), 6 deletions(-) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 3581e1c04256..3c317b53ba29 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -98,7 +98,7 @@ versions += [ kafka_26: "2.6.2", kafka_27: "2.7.1", kafka_28: "2.8.1", - kafka_30: "3.0.0", + kafka_30: "3.0.1", kafka_31: "3.1.0", lz4: "1.8.0", mavenArtifact: "3.8.4", diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index ac4c39b210ed..3f312d254cf8 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -63,7 +63,7 @@ RUN mkdir -p "/opt/kafka-2.5.1" && chmod a+rw /opt/kafka-2.5.1 && curl -s "$KAFK RUN mkdir -p "/opt/kafka-2.6.2" && chmod a+rw /opt/kafka-2.6.2 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.6.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.6.2" RUN mkdir -p "/opt/kafka-2.7.1" && chmod a+rw /opt/kafka-2.7.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.7.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.7.1" RUN mkdir -p "/opt/kafka-2.8.1" && chmod a+rw /opt/kafka-2.8.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.8.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.8.1" -RUN mkdir -p "/opt/kafka-3.0.0" && chmod a+rw /opt/kafka-3.0.0 && curl -s "$KAFKA_MIRROR/kafka_2.12-3.0.0.tgz" | tar xz --strip-components=1 -C "/opt/kafka-3.0.0" +RUN mkdir -p "/opt/kafka-3.0.1" && chmod a+rw /opt/kafka-3.0.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-3.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-3.0.1" RUN mkdir -p "/opt/kafka-3.1.0" && chmod a+rw /opt/kafka-3.1.0 && curl -s "$KAFKA_MIRROR/kafka_2.12-3.1.0.tgz" | tar xz --strip-components=1 -C "/opt/kafka-3.1.0" # Streams test dependencies @@ -82,7 +82,7 @@ RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.5.1-test.jar" -o /opt/kafka-2.5.1/lib RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.6.2-test.jar" -o /opt/kafka-2.6.2/libs/kafka-streams-2.6.2-test.jar RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.7.1-test.jar" -o /opt/kafka-2.7.1/libs/kafka-streams-2.7.1-test.jar RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.8.1-test.jar" -o /opt/kafka-2.8.1/libs/kafka-streams-2.8.1-test.jar -RUN curl -s "$KAFKA_MIRROR/kafka-streams-3.0.0-test.jar" -o /opt/kafka-3.0.0/libs/kafka-streams-3.0.0-test.jar +RUN curl -s "$KAFKA_MIRROR/kafka-streams-3.0.1-test.jar" -o /opt/kafka-3.0.1/libs/kafka-streams-3.0.1-test.jar RUN curl -s "$KAFKA_MIRROR/kafka-streams-3.1.0-test.jar" -o /opt/kafka-3.1.0/libs/kafka-streams-3.1.0-test.jar # The version of Kibosh to use for testing. diff --git a/tests/kafkatest/version.py b/tests/kafkatest/version.py index 95100b0532b9..2d7fe0377e01 100644 --- a/tests/kafkatest/version.py +++ b/tests/kafkatest/version.py @@ -214,7 +214,8 @@ def get_version(node=None): # 3.0.x versions V_3_0_0 = KafkaVersion("3.0.0") -LATEST_3_0 = V_3_0_0 +V_3_0_1 = KafkaVersion("3.0.1") +LATEST_3_0 = V_3_0_1 # 3.1.x versions V_3_1_0 = KafkaVersion("3.1.0") diff --git a/vagrant/base.sh b/vagrant/base.sh index 24eecdb9f6bf..13cc9ff0b767 100755 --- a/vagrant/base.sh +++ b/vagrant/base.sh @@ -148,8 +148,8 @@ get_kafka 2.7.1 2.12 chmod a+rw /opt/kafka-2.7.1 get_kafka 2.8.1 2.12 chmod a+rw /opt/kafka-2.8.1 -get_kafka 3.0.0 2.12 -chmod a+rw /opt/kafka-3.0.0 +get_kafka 3.0.1 2.12 +chmod a+rw /opt/kafka-3.0.1 get_kafka 3.1.0 2.12 chmod a+rw /opt/kafka-3.1.0 From 5c1dd493d6f608b566fdad5ab3a896cb13622bce Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 16 Mar 2022 11:54:02 -0400 Subject: [PATCH 016/447] Don't generate Uuid with a leading "-" (#11901) --- .../main/java/org/apache/kafka/common/Uuid.java | 17 +++++++++++------ .../java/org/apache/kafka/common/UuidTest.java | 5 ++++- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/Uuid.java b/clients/src/main/java/org/apache/kafka/common/Uuid.java index a639f3edd7fc..83b8f0f0b164 100644 --- a/clients/src/main/java/org/apache/kafka/common/Uuid.java +++ b/clients/src/main/java/org/apache/kafka/common/Uuid.java @@ -31,13 +31,11 @@ public class Uuid implements Comparable { * A UUID for the metadata topic in KRaft mode. Will never be returned by the randomUuid method. */ public static final Uuid METADATA_TOPIC_ID = new Uuid(0L, 1L); - private static final java.util.UUID METADATA_TOPIC_ID_INTERNAL = new java.util.UUID(0L, 1L); /** * A UUID that represents a null or empty UUID. Will never be returned by the randomUuid method. */ public static final Uuid ZERO_UUID = new Uuid(0L, 0L); - private static final java.util.UUID ZERO_ID_INTERNAL = new java.util.UUID(0L, 0L); private final long mostSignificantBits; private final long leastSignificantBits; @@ -51,15 +49,22 @@ public Uuid(long mostSigBits, long leastSigBits) { this.leastSignificantBits = leastSigBits; } + private static Uuid unsafeRandomUuid() { + java.util.UUID jUuid = java.util.UUID.randomUUID(); + return new Uuid(jUuid.getMostSignificantBits(), jUuid.getLeastSignificantBits()); + } + /** * Static factory to retrieve a type 4 (pseudo randomly generated) UUID. + * + * This will not generate a UUID equal to 0, 1, or one whose string representation starts with a dash ("-") */ public static Uuid randomUuid() { - java.util.UUID uuid = java.util.UUID.randomUUID(); - while (uuid.equals(METADATA_TOPIC_ID_INTERNAL) || uuid.equals(ZERO_ID_INTERNAL)) { - uuid = java.util.UUID.randomUUID(); + Uuid uuid = unsafeRandomUuid(); + while (uuid.equals(METADATA_TOPIC_ID) || uuid.equals(ZERO_UUID) || uuid.toString().startsWith("-")) { + uuid = unsafeRandomUuid(); } - return new Uuid(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits()); + return uuid; } /** diff --git a/clients/src/test/java/org/apache/kafka/common/UuidTest.java b/clients/src/test/java/org/apache/kafka/common/UuidTest.java index 232b9927c896..67d841925cef 100644 --- a/clients/src/test/java/org/apache/kafka/common/UuidTest.java +++ b/clients/src/test/java/org/apache/kafka/common/UuidTest.java @@ -16,11 +16,13 @@ */ package org.apache.kafka.common; +import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.Test; import java.util.Base64; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -72,12 +74,13 @@ public void testStringConversion() { assertEquals(Uuid.fromString(zeroIdString), Uuid.ZERO_UUID); } - @Test + @RepeatedTest(100) public void testRandomUuid() { Uuid randomID = Uuid.randomUuid(); assertNotEquals(randomID, Uuid.ZERO_UUID); assertNotEquals(randomID, Uuid.METADATA_TOPIC_ID); + assertFalse(randomID.toString().startsWith("-")); } @Test From b68463c250ec99d86ffacdbd45e58059d0ca51e1 Mon Sep 17 00:00:00 2001 From: Levani Kokhreidze Date: Wed, 16 Mar 2022 19:02:24 +0200 Subject: [PATCH 017/447] KAFKA-6718 / Add rack awareness configurations to StreamsConfig (#11837) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR is part of KIP-708 and adds rack aware standby task assignment logic. Rack aware standby task assignment won't be functional until all parts of this KIP gets merged. Splitting PRs into three smaller PRs to make the review process easier to follow. Overall plan is the following: ⏭️ Rack aware standby task assignment logic #10851 ⏭️ Protocol change, add clientTags to SubscriptionInfoData #10802 👉 Add required configurations to StreamsConfig (public API change, at this point we should have full functionality) This PR implements last point of the above mentioned plan. Reviewers: Luke Chen , Bruno Cadonna --- .../apache/kafka/common/config/ConfigDef.java | 21 + .../kafka/common/config/ConfigDefTest.java | 42 +- .../apache/kafka/streams/StreamsConfig.java | 92 ++++ .../processor/internals/StreamThread.java | 1 + .../internals/StreamsPartitionAssignor.java | 18 +- .../assignment/AssignorConfiguration.java | 6 +- .../internals/assignment/ClientState.java | 8 +- .../ClientTagAwareStandbyTaskAssignor.java | 56 ++- .../DefaultStandbyTaskAssignor.java | 18 +- .../HighAvailabilityTaskAssignor.java | 11 +- .../assignment/ReferenceContainer.java | 2 + .../StandbyTaskAssignmentUtils.java | 19 +- .../StandbyTaskAssignorFactory.java | 29 ++ .../kafka/streams/StreamsConfigTest.java | 105 +++++ .../RackAwarenessIntegrationTest.java | 436 ++++++++++++++++++ .../StreamsPartitionAssignorTest.java | 21 +- .../assignment/AssignmentTestUtils.java | 9 + ...ClientTagAwareStandbyTaskAssignorTest.java | 42 +- .../HighAvailabilityTaskAssignorTest.java | 25 - .../StandbyTaskAssignmentUtilsTest.java | 54 ++- .../StandbyTaskAssignorFactoryTest.java | 52 +++ 21 files changed, 946 insertions(+), 121 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorFactory.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/RackAwarenessIntegrationTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorFactoryTest.java diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index 85b0103e59ae..8c91a2523e5d 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -1121,6 +1121,27 @@ public String toString() { } } + public static class ListSize implements Validator { + final int maxSize; + + private ListSize(final int maxSize) { + this.maxSize = maxSize; + } + + public static ListSize atMostOfSize(final int maxSize) { + return new ListSize(maxSize); + } + + @Override + public void ensureValid(final String name, final Object value) { + @SuppressWarnings("unchecked") + List values = (List) value; + if (values.size() > maxSize) { + throw new ConfigException(name, value, "exceeds maximum list size of [" + maxSize + "]."); + } + } + } + public static class ConfigKey { public final String name; public final Type type; diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index 893f68b89e6b..0e5af1f5cb38 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.config.ConfigDef.CaseInsensitiveValidString; import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.ListSize; import org.apache.kafka.common.config.ConfigDef.Range; import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.ValidString; @@ -38,6 +39,8 @@ import java.util.Set; import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; +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.assertNull; @@ -426,7 +429,7 @@ public void testNames() { public void testMissingDependentConfigs() { // Should not be possible to parse a config if a dependent config has not been defined final ConfigDef configDef = new ConfigDef() - .define("parent", Type.STRING, Importance.HIGH, "parent docs", "group", 1, Width.LONG, "Parent", Collections.singletonList("child")); + .define("parent", Type.STRING, Importance.HIGH, "parent docs", "group", 1, Width.LONG, "Parent", singletonList("child")); assertThrows(ConfigException.class, () -> configDef.parse(Collections.emptyMap())); } @@ -438,7 +441,7 @@ public void testBaseConfigDefDependents() { assertEquals(new HashSet<>(Arrays.asList("a")), baseConfigDef.getConfigsWithNoParent()); final ConfigDef configDef = new ConfigDef(baseConfigDef) - .define("parent", Type.STRING, Importance.HIGH, "parent docs", "group", 1, Width.LONG, "Parent", Collections.singletonList("child")) + .define("parent", Type.STRING, Importance.HIGH, "parent docs", "group", 1, Width.LONG, "Parent", singletonList("child")) .define("child", Type.STRING, Importance.HIGH, "docs"); assertEquals(new HashSet<>(Arrays.asList("a", "parent")), configDef.getConfigsWithNoParent()); @@ -541,7 +544,7 @@ public void toEnrichedRst() { .define("opt2.of.group2", Type.BOOLEAN, false, Importance.HIGH, "Doc doc doc doc.", "Group Two", 1, Width.NONE, "..", Collections.emptyList()) .define("opt1.of.group2", Type.BOOLEAN, false, Importance.HIGH, "Doc doc doc doc doc.", - "Group Two", 0, Width.NONE, "..", Collections.singletonList("some.option")) + "Group Two", 0, Width.NONE, "..", singletonList("some.option")) .define("poor.opt", Type.STRING, "foo", Importance.HIGH, "Doc doc doc doc."); final String expectedRst = "" + @@ -722,4 +725,37 @@ public void testNiceTimeUnits() { assertEquals(" (365 days)", ConfigDef.niceTimeUnits(Duration.ofDays(365).toMillis())); } + @Test + public void testThrowsExceptionWhenListSizeExceedsLimit() { + final ConfigException exception = assertThrows(ConfigException.class, () -> new ConfigDef().define("lst", + Type.LIST, + asList("a", "b"), + ListSize.atMostOfSize(1), + Importance.HIGH, + "lst doc")); + assertEquals("Invalid value [a, b] for configuration lst: exceeds maximum list size of [1].", + exception.getMessage()); + } + + @Test + public void testNoExceptionIsThrownWhenListSizeEqualsTheLimit() { + final List lst = asList("a", "b", "c"); + assertDoesNotThrow(() -> new ConfigDef().define("lst", + Type.LIST, + lst, + ListSize.atMostOfSize(lst.size()), + Importance.HIGH, + "lst doc")); + } + + @Test + public void testNoExceptionIsThrownWhenListSizeIsBelowTheLimit() { + assertDoesNotThrow(() -> new ConfigDef().define("lst", + Type.LIST, + asList("a", "b"), + ListSize.atMostOfSize(3), + Importance.HIGH, + "lst doc")); + } + } diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 672df55c99e4..66cee6aed7c8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -48,12 +48,16 @@ import java.time.Duration; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Objects; import java.util.Properties; import java.util.Set; +import java.util.stream.Collectors; import static org.apache.kafka.common.IsolationLevel.READ_COMMITTED; +import static org.apache.kafka.common.config.ConfigDef.ListSize.atMostOfSize; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; import static org.apache.kafka.common.config.ConfigDef.Range.between; import static org.apache.kafka.common.config.ConfigDef.ValidString.in; @@ -148,6 +152,12 @@ public class StreamsConfig extends AbstractConfig { public static final int DUMMY_THREAD_INDEX = 1; public static final long MAX_TASK_IDLE_MS_DISABLED = -1; + // We impose these limitations because client tags are encoded into the subscription info, + // which is part of the group metadata message that is persisted into the internal topic. + public static final int MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE = 5; + public static final int MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH = 20; + public static final int MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH = 30; + /** * Prefix used to provide default topic configs to be applied when creating internal topics. * These should be valid properties from {@link org.apache.kafka.common.config.TopicConfig TopicConfig}. @@ -212,6 +222,15 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String ADMIN_CLIENT_PREFIX = "admin."; + /** + * Prefix used to add arbitrary tags to a Kafka Stream's instance as key-value pairs. + * Example: + * client.tag.zone=zone1 + * client.tag.cluster=cluster1 + */ + @SuppressWarnings("WeakerAccess") + public static final String CLIENT_TAG_PREFIX = "client.tag."; + /** * Config value for parameter {@link #TOPOLOGY_OPTIMIZATION_CONFIG "topology.optimization"} for disabling topology optimization */ @@ -511,6 +530,13 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG; + /** {@code rack.aware.assignment.tags} */ + @SuppressWarnings("WeakerAccess") + public static final String RACK_AWARE_ASSIGNMENT_TAGS_CONFIG = "rack.aware.assignment.tags"; + private static final String RACK_AWARE_ASSIGNMENT_TAGS_DOC = "List of client tag keys used to distribute standby replicas across Kafka Streams instances." + + " When configured, Kafka Streams will make a best-effort to distribute" + + " the standby tasks over each client tag dimension."; + /** {@code reconnect.backoff.ms} */ @SuppressWarnings("WeakerAccess") public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; @@ -726,6 +752,12 @@ public class StreamsConfig extends AbstractConfig { in(AT_LEAST_ONCE, EXACTLY_ONCE, EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2), Importance.MEDIUM, PROCESSING_GUARANTEE_DOC) + .define(RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, + Type.LIST, + Collections.emptyList(), + atMostOfSize(MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE), + Importance.MEDIUM, + RACK_AWARE_ASSIGNMENT_TAGS_DOC) .define(REPLICATION_FACTOR_CONFIG, Type.INT, -1, @@ -1040,6 +1072,16 @@ public static String restoreConsumerPrefix(final String consumerProp) { return RESTORE_CONSUMER_PREFIX + consumerProp; } + /** + * Prefix a client tag key with {@link #CLIENT_TAG_PREFIX}. + * + * @param clientTagKey client tag key + * @return {@link #CLIENT_TAG_PREFIX} + {@code clientTagKey} + */ + public static String clientTagPrefix(final String clientTagKey) { + return CLIENT_TAG_PREFIX + clientTagKey; + } + /** * Prefix a property with {@link #GLOBAL_CONSUMER_PREFIX}. This is used to isolate {@link ConsumerConfig global consumer configs} * from other client configs. @@ -1159,9 +1201,43 @@ protected Map postProcessParsedConfig(final Map configUpdates.put(COMMIT_INTERVAL_MS_CONFIG, EOS_DEFAULT_COMMIT_INTERVAL_MS); } + validateRackAwarenessConfiguration(); + return configUpdates; } + private void validateRackAwarenessConfiguration() { + final List rackAwareAssignmentTags = getList(RACK_AWARE_ASSIGNMENT_TAGS_CONFIG); + final Map clientTags = getClientTags(); + + if (clientTags.size() > MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE) { + throw new ConfigException("At most " + MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE + " client tags " + + "can be specified using " + CLIENT_TAG_PREFIX + " prefix."); + } + + for (final String rackAwareAssignmentTag : rackAwareAssignmentTags) { + if (!clientTags.containsKey(rackAwareAssignmentTag)) { + throw new ConfigException(RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, + rackAwareAssignmentTags, + "Contains invalid value [" + rackAwareAssignmentTag + "] " + + "which doesn't have corresponding tag set via [" + CLIENT_TAG_PREFIX + "] prefix."); + } + } + + clientTags.forEach((tagKey, tagValue) -> { + if (tagKey.length() > MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH) { + throw new ConfigException(CLIENT_TAG_PREFIX, + tagKey, + "Tag key exceeds maximum length of " + MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH + "."); + } + if (tagValue.length() > MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH) { + throw new ConfigException(CLIENT_TAG_PREFIX, + tagValue, + "Tag value exceeds maximum length of " + MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH + "."); + } + }); + } + private Map getCommonConsumerConfigs() { final Map clientProvidedProps = getClientPropsWithPrefix(CONSUMER_PREFIX, ConsumerConfig.configNames()); @@ -1295,6 +1371,7 @@ public Map getMainConsumerConfigs(final String groupId, final St consumerProps.put(PROBING_REBALANCE_INTERVAL_MS_CONFIG, getLong(PROBING_REBALANCE_INTERVAL_MS_CONFIG)); consumerProps.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, StreamsPartitionAssignor.class.getName()); consumerProps.put(WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, getLong(WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG)); + consumerProps.put(RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, getList(RACK_AWARE_ASSIGNMENT_TAGS_CONFIG)); // disable auto topic creation consumerProps.put(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, "false"); @@ -1441,6 +1518,21 @@ public Map getAdminConfigs(final String clientId) { return props; } + /** + * Get the configured client tags set with {@link #CLIENT_TAG_PREFIX} prefix. + * + * @return Map of the client tags. + */ + @SuppressWarnings("WeakerAccess") + public Map getClientTags() { + return originalsWithPrefix(CLIENT_TAG_PREFIX).entrySet().stream().collect( + Collectors.toMap( + Map.Entry::getKey, + tagEntry -> Objects.toString(tagEntry.getValue()) + ) + ); + } + private Map getClientPropsWithPrefix(final String prefix, final Set configNames) { final Map props = clientProps(configNames, originals()); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 7893a06fb79f..7401e539c4f7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -345,6 +345,7 @@ public static StreamThread create(final TopologyMetadata topologyMetadata, referenceContainer.adminClient = adminClient; referenceContainer.streamsMetadataState = streamsMetadataState; referenceContainer.time = time; + referenceContainer.clientTags = config.getClientTags(); log.info("Creating restore consumer client"); final Map restoreConsumerConfigs = config.getRestoreConsumerConfigs(getRestoreConsumerClientId(threadId)); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index d2fa90524f50..2af2fba71870 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -128,7 +128,7 @@ private static class ClientMetadata { private final ClientState state; private final SortedSet consumers; - ClientMetadata(final String endPoint) { + ClientMetadata(final String endPoint, final Map clientTags) { // get the host info, or null if no endpoint is configured (ie endPoint == null) hostInfo = HostInfo.buildFromEndpoint(endPoint); @@ -136,8 +136,8 @@ private static class ClientMetadata { // initialize the consumer memberIds consumers = new TreeSet<>(); - // initialize the client state - state = new ClientState(); + // initialize the client state with client tags + state = new ClientState(clientTags); } void addConsumer(final String consumerMemberId, final List ownedPartitions) { @@ -189,6 +189,7 @@ public String toString() { private Supplier taskAssignorSupplier; private byte uniqueField; + private Map clientTags; /** * We need to have the PartitionAssignor and its StreamThread to be mutually accessible since the former needs @@ -223,6 +224,7 @@ public void configure(final Map configs) { taskAssignorSupplier = assignorConfiguration::taskAssignor; assignmentListener = assignorConfiguration.assignmentListener(); uniqueField = 0; + clientTags = referenceContainer.clientTags; } @Override @@ -265,7 +267,7 @@ public ByteBuffer subscriptionUserData(final Set topics) { taskOffsetSums, uniqueField, assignmentErrorCode.get(), - Collections.emptyMap() + clientTags ).encode(); } @@ -338,7 +340,7 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr futureMetadataVersion = usedVersion; processId = FUTURE_ID; if (!clientMetadataMap.containsKey(FUTURE_ID)) { - clientMetadataMap.put(FUTURE_ID, new ClientMetadata(null)); + clientMetadataMap.put(FUTURE_ID, new ClientMetadata(null, Collections.emptyMap())); } } else { processId = info.processId(); @@ -348,7 +350,7 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr // create the new client metadata if necessary if (clientMetadata == null) { - clientMetadata = new ClientMetadata(info.userEndPoint()); + clientMetadata = new ClientMetadata(info.userEndPoint(), info.clientTags()); clientMetadataMap.put(info.processId(), clientMetadata); } @@ -1462,6 +1464,10 @@ protected byte uniqueField() { return uniqueField; } + protected Map clientTags() { + return clientTags; + } + protected void handleRebalanceStart(final Set topics) { taskManager.handleRebalanceStart(topics); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java index 65cc7ae1930b..5e317efdba9f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java @@ -29,7 +29,6 @@ import org.apache.kafka.streams.processor.internals.InternalTopicManager; import org.slf4j.Logger; -import java.util.Collections; import java.util.List; import java.util.Map; @@ -250,7 +249,7 @@ private AssignmentConfigs(final StreamsConfig configs) { maxWarmupReplicas = configs.getInt(StreamsConfig.MAX_WARMUP_REPLICAS_CONFIG); numStandbyReplicas = configs.getInt(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG); probingRebalanceIntervalMs = configs.getLong(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG); - rackAwareAssignmentTags = Collections.emptyList(); + rackAwareAssignmentTags = configs.getList(StreamsConfig.RACK_AWARE_ASSIGNMENT_TAGS_CONFIG); } AssignmentConfigs(final Long acceptableRecoveryLag, @@ -262,7 +261,7 @@ private AssignmentConfigs(final StreamsConfig configs) { this.maxWarmupReplicas = validated(StreamsConfig.MAX_WARMUP_REPLICAS_CONFIG, maxWarmupReplicas); this.numStandbyReplicas = validated(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, numStandbyReplicas); this.probingRebalanceIntervalMs = validated(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, probingRebalanceIntervalMs); - this.rackAwareAssignmentTags = rackAwareAssignmentTags; + this.rackAwareAssignmentTags = validated(StreamsConfig.RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, rackAwareAssignmentTags); } private static T validated(final String configKey, final T value) { @@ -280,6 +279,7 @@ public String toString() { "\n maxWarmupReplicas=" + maxWarmupReplicas + "\n numStandbyReplicas=" + numStandbyReplicas + "\n probingRebalanceIntervalMs=" + probingRebalanceIntervalMs + + "\n rackAwareAssignmentTags=" + rackAwareAssignmentTags + "\n}"; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java index 5ee0e93e6aa4..b8ba4ce27e10 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java @@ -29,17 +29,16 @@ import java.util.Map; import java.util.Set; import java.util.SortedSet; -import java.util.stream.Collectors; import java.util.TreeMap; import java.util.TreeSet; import java.util.UUID; +import java.util.stream.Collectors; import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; import static java.util.Collections.unmodifiableSet; import static java.util.Comparator.comparing; import static java.util.Comparator.comparingLong; - import static org.apache.kafka.common.utils.Utils.union; import static org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo.UNKNOWN_OFFSET_SUM; @@ -65,6 +64,10 @@ public ClientState() { this(0); } + public ClientState(final Map clientTags) { + this(0, clientTags); + } + ClientState(final int capacity) { this(capacity, Collections.emptyMap()); } @@ -422,6 +425,7 @@ public String toString() { ") prevStandbyTasks: (" + previousStandbyTasks.taskIds() + ") changelogOffsetTotalsByTask: (" + taskOffsetSums.entrySet() + ") taskLagTotals: (" + taskLagTotals.entrySet() + + ") clientTags: (" + clientTags.entrySet() + ") capacity: " + capacity + " assigned: " + assignedTaskCount() + "]"; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java index c7399d7ed8a5..cabfa545b101 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java @@ -60,7 +60,7 @@ public boolean assign(final Map clients, final Map tasksToRemainingStandbys = computeTasksToRemainingStandbys( numStandbyReplicas, - allTaskIds + statefulTaskIds ); final Map> tagKeyToValues = new HashMap<>(); @@ -79,6 +79,7 @@ public boolean assign(final Map clients, if (clientState.activeTasks().contains(statefulTaskId)) { assignStandbyTasksToClientsWithDifferentTags( + numStandbyReplicas, standbyTaskClientsByTaskLoad, statefulTaskId, clientId, @@ -94,17 +95,10 @@ public boolean assign(final Map clients, } if (!tasksToRemainingStandbys.isEmpty()) { - log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " + - "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " + - "Will distribute the remaining standby tasks to least loaded clients.", - tasksToRemainingStandbys, pendingStandbyTasksToClientId); - assignPendingStandbyTasksToLeastLoadedClients(clients, numStandbyReplicas, - rackAwareAssignmentTags, standbyTaskClientsByTaskLoad, - tasksToRemainingStandbys, - pendingStandbyTasksToClientId); + tasksToRemainingStandbys); } // returning false, because standby task assignment will never require a follow-up probing rebalance. @@ -113,34 +107,22 @@ public boolean assign(final Map clients, private static void assignPendingStandbyTasksToLeastLoadedClients(final Map clients, final int numStandbyReplicas, - final Set rackAwareAssignmentTags, final ConstrainedPrioritySet standbyTaskClientsByTaskLoad, - final Map pendingStandbyTaskToNumberRemainingStandbys, - final Map pendingStandbyTaskToClientId) { + final Map pendingStandbyTaskToNumberRemainingStandbys) { // We need to re offer all the clients to find the least loaded ones standbyTaskClientsByTaskLoad.offerAll(clients.keySet()); for (final Entry pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) { final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey(); - final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId); - final int numberOfRemainingStandbys = pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks( + pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks( + numStandbyReplicas, clients, pendingStandbyTaskToNumberRemainingStandbys, standbyTaskClientsByTaskLoad, - activeTaskId + activeTaskId, + log ); - - if (numberOfRemainingStandbys > 0) { - log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " + - "There is not enough available capacity. You should " + - "increase the number of application instances " + - "on different client tag dimensions " + - "to maintain the requested number of standby replicas. " + - "Rack awareness is configured with [{}] tags.", - numberOfRemainingStandbys, numStandbyReplicas, activeTaskId, - clients.get(clientId).clientTags(), rackAwareAssignmentTags); - } } } @@ -174,7 +156,8 @@ static void fillClientsTagStatistics(final Map clientStates, } // Visible for testing - static void assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad, + static void assignStandbyTasksToClientsWithDifferentTags(final int numberOfStandbyClients, + final ConstrainedPrioritySet standbyTaskClientsByTaskLoad, final TaskId activeTaskId, final UUID activeTaskClient, final Set rackAwareAssignmentTags, @@ -211,17 +194,32 @@ static void assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPriori break; } - clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId); - + final ClientState clientStateOnUsedTagDimensions = clientStates.get(clientOnUnusedTagDimensions); countOfUsedClients++; numRemainingStandbys--; + log.debug("Assigning {} out of {} standby tasks for an active task [{}] with client tags {}. " + + "Standby task client tags are {}.", + numberOfStandbyClients - numRemainingStandbys, numberOfStandbyClients, activeTaskId, + clientStates.get(activeTaskClient).clientTags(), clientStateOnUsedTagDimensions.clientTags()); + + clientStateOnUsedTagDimensions.assignStandby(activeTaskId); lastUsedClient = clientOnUnusedTagDimensions; } while (numRemainingStandbys > 0); if (numRemainingStandbys > 0) { pendingStandbyTasksToClientId.put(activeTaskId, activeTaskClient); tasksToRemainingStandbys.put(activeTaskId, numRemainingStandbys); + log.warn("Rack aware standby task assignment was not able to assign {} of {} standby tasks for the " + + "active task [{}] with the rack aware assignment tags {}. " + + "This may happen when there aren't enough application instances on different tag " + + "dimensions compared to an active and corresponding standby task. " + + "Consider launching application instances on different tag dimensions than [{}]. " + + "Standby task assignment will fall back to assigning standby tasks to the least loaded clients.", + numRemainingStandbys, numberOfStandbyClients, + activeTaskId, rackAwareAssignmentTags, + clientStates.get(activeTaskClient).clientTags()); + } else { tasksToRemainingStandbys.remove(activeTaskId); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultStandbyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultStandbyTaskAssignor.java index db6cb4e26cef..680a056a826f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultStandbyTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultStandbyTaskAssignor.java @@ -50,18 +50,12 @@ public boolean assign(final Map clients, standbyTaskClientsByTaskLoad.offerAll(clients.keySet()); for (final TaskId task : statefulTaskIds) { - final int numRemainingStandbys = pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks(clients, - tasksToRemainingStandbys, - standbyTaskClientsByTaskLoad, - task); - - if (numRemainingStandbys > 0) { - log.warn("Unable to assign {} of {} standby tasks for task [{}]. " + - "There is not enough available capacity. You should " + - "increase the number of application instances " + - "to maintain the requested number of standby replicas.", - numRemainingStandbys, numStandbyReplicas, task); - } + pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks(numStandbyReplicas, + clients, + tasksToRemainingStandbys, + standbyTaskClientsByTaskLoad, + task, + log); } // returning false, because standby task assignment will never require a follow-up probing rebalance. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java index d0bb50b66ff6..7111ae28e5fd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java @@ -129,7 +129,7 @@ private void assignStandbyReplicaTasks(final TreeMap clientSt return; } - final StandbyTaskAssignor standbyTaskAssignor = createStandbyTaskAssignor(configs); + final StandbyTaskAssignor standbyTaskAssignor = StandbyTaskAssignorFactory.create(configs); standbyTaskAssignor.assign(clientStates, allTaskIds, statefulTasks, configs); @@ -142,15 +142,6 @@ private void assignStandbyReplicaTasks(final TreeMap clientSt ); } - // Visible for testing - static StandbyTaskAssignor createStandbyTaskAssignor(final AssignmentConfigs configs) { - if (!configs.rackAwareAssignmentTags.isEmpty()) { - return new ClientTagAwareStandbyTaskAssignor(); - } else { - return new DefaultStandbyTaskAssignor(); - } - } - private static void balanceTasksOverThreads(final SortedMap clientStates, final Function> currentAssignmentAccessor, final BiConsumer taskUnassignor, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ReferenceContainer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ReferenceContainer.java index 9b46eeb71080..19011d865a1c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ReferenceContainer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ReferenceContainer.java @@ -24,6 +24,7 @@ import org.apache.kafka.streams.processor.internals.TaskManager; import java.util.LinkedList; +import java.util.Map; import java.util.Queue; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -37,4 +38,5 @@ public class ReferenceContainer { public final AtomicLong nextScheduledRebalanceMs = new AtomicLong(Long.MAX_VALUE); public final Queue nonFatalExceptionsToHandle = new LinkedList<>(); public Time time; + public Map clientTags; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignmentUtils.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignmentUtils.java index 7ed6f5dec1b1..3f34e5ef8c17 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignmentUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignmentUtils.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.processor.internals.assignment; import org.apache.kafka.streams.processor.TaskId; +import org.slf4j.Logger; import java.util.Map; import java.util.Set; @@ -33,10 +34,12 @@ static ConstrainedPrioritySet createLeastLoadedPrioritySetConstrainedByAssignedT client -> clients.get(client).assignedTaskLoad()); } - static int pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks(final Map clients, - final Map tasksToRemainingStandbys, - final ConstrainedPrioritySet standbyTaskClientsByTaskLoad, - final TaskId activeTaskId) { + static void pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks(final int numStandbyReplicas, + final Map clients, + final Map tasksToRemainingStandbys, + final ConstrainedPrioritySet standbyTaskClientsByTaskLoad, + final TaskId activeTaskId, + final Logger log) { int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId); while (numRemainingStandbys > 0) { final UUID client = standbyTaskClientsByTaskLoad.poll(activeTaskId); @@ -49,7 +52,13 @@ static int pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks(final Map 0) { + log.warn("Unable to assign {} of {} standby tasks for task [{}]. " + + "There is not enough available capacity. You should " + + "increase the number of application instances " + + "to maintain the requested number of standby replicas.", + numRemainingStandbys, numStandbyReplicas, activeTaskId); + } } static Map computeTasksToRemainingStandbys(final int numStandbyReplicas, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorFactory.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorFactory.java new file mode 100644 index 000000000000..30c78f33f38d --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorFactory.java @@ -0,0 +1,29 @@ +/* + * 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.streams.processor.internals.assignment; + +class StandbyTaskAssignorFactory { + private StandbyTaskAssignorFactory() {} + + static StandbyTaskAssignor create(final AssignorConfiguration.AssignmentConfigs configs) { + if (!configs.rackAwareAssignmentTags.isEmpty()) { + return new ClientTagAwareStandbyTaskAssignor(); + } else { + return new DefaultStandbyTaskAssignor(); + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index 2e1b0d842e14..05b13a52e50b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -38,19 +38,25 @@ import java.io.File; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Properties; +import static java.util.Collections.nCopies; import static org.apache.kafka.common.IsolationLevel.READ_COMMITTED; import static org.apache.kafka.common.IsolationLevel.READ_UNCOMMITTED; +import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.streams.StreamsConfig.AT_LEAST_ONCE; import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE; import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE_BETA; import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE_V2; +import static org.apache.kafka.streams.StreamsConfig.MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH; +import static org.apache.kafka.streams.StreamsConfig.MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH; import static org.apache.kafka.streams.StreamsConfig.STATE_DIR_CONFIG; import static org.apache.kafka.streams.StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG; import static org.apache.kafka.streams.StreamsConfig.adminClientPrefix; @@ -1124,6 +1130,105 @@ public void shouldThrowConfigExceptionIfProbingRebalanceIntervalIsOutsideBounds( assertThrows(ConfigException.class, () -> new StreamsConfig(props)); } + @Test + public void shouldDefaultToEmptyListIfRackAwareAssignmentTagsIsNotSet() { + final StreamsConfig config = new StreamsConfig(props); + assertTrue(config.getList(StreamsConfig.RACK_AWARE_ASSIGNMENT_TAGS_CONFIG).isEmpty()); + } + + @Test + public void shouldThrowExceptionWhenClientTagsExceedTheLimit() { + final int limit = StreamsConfig.MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE + 1; + for (int i = 0; i < limit; i++) { + props.put(StreamsConfig.clientTagPrefix("k" + i), "v" + i); + } + final ConfigException exception = assertThrows(ConfigException.class, () -> new StreamsConfig(props)); + assertEquals( + String.format("At most %s client tags can be specified using %s prefix.", + StreamsConfig.MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE, + StreamsConfig.CLIENT_TAG_PREFIX + ), exception.getMessage() + ); + } + + @Test + public void shouldThrowExceptionWhenRackAwareAssignmentTagsExceedsMaxListSize() { + final int limit = StreamsConfig.MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE + 1; + final List rackAwareAssignmentTags = new ArrayList<>(); + for (int i = 0; i < limit; i++) { + final String clientTagKey = "k" + i; + rackAwareAssignmentTags.add(clientTagKey); + props.put(StreamsConfig.clientTagPrefix(clientTagKey), "v" + i); + } + + props.put(StreamsConfig.RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, String.join(",", rackAwareAssignmentTags)); + final ConfigException exception = assertThrows(ConfigException.class, () -> new StreamsConfig(props)); + assertEquals( + String.format("Invalid value %s for configuration %s: exceeds maximum list size of [%s].", + rackAwareAssignmentTags, + StreamsConfig.RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, + StreamsConfig.MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE), + exception.getMessage() + ); + } + + @Test + public void shouldSetRackAwareAssignmentTags() { + props.put(StreamsConfig.clientTagPrefix("cluster"), "cluster-1"); + props.put(StreamsConfig.clientTagPrefix("zone"), "eu-central-1a"); + props.put(StreamsConfig.RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, "cluster,zone"); + final StreamsConfig config = new StreamsConfig(props); + assertEquals(new HashSet<>(config.getList(StreamsConfig.RACK_AWARE_ASSIGNMENT_TAGS_CONFIG)), + mkSet("cluster", "zone")); + } + + @Test + public void shouldGetEmptyMapIfClientTagsAreNotSet() { + final StreamsConfig config = new StreamsConfig(props); + assertTrue(config.getClientTags().isEmpty()); + } + + @Test + public void shouldGetClientTagsMapWhenSet() { + props.put(StreamsConfig.clientTagPrefix("zone"), "eu-central-1a"); + props.put(StreamsConfig.clientTagPrefix("cluster"), "cluster-1"); + final StreamsConfig config = new StreamsConfig(props); + final Map clientTags = config.getClientTags(); + assertEquals(clientTags.size(), 2); + assertEquals(clientTags.get("zone"), "eu-central-1a"); + assertEquals(clientTags.get("cluster"), "cluster-1"); + } + + @Test + public void shouldThrowExceptionWhenClientTagRackAwarenessIsConfiguredWithUnknownTags() { + props.put(StreamsConfig.RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, "cluster"); + assertThrows(ConfigException.class, () -> new StreamsConfig(props)); + } + + @Test + public void shouldThrowExceptionWhenClientTagKeyExceedMaxLimit() { + final String key = String.join("", nCopies(MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH + 1, "k")); + props.put(StreamsConfig.clientTagPrefix(key), "eu-central-1a"); + final ConfigException exception = assertThrows(ConfigException.class, () -> new StreamsConfig(props)); + assertEquals( + String.format("Invalid value %s for configuration %s: Tag key exceeds maximum length of %s.", + key, StreamsConfig.CLIENT_TAG_PREFIX, StreamsConfig.MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH), + exception.getMessage() + ); + } + + @Test + public void shouldThrowExceptionWhenClientTagValueExceedMaxLimit() { + final String value = String.join("", nCopies(MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH + 1, "v")); + props.put(StreamsConfig.clientTagPrefix("x"), value); + final ConfigException exception = assertThrows(ConfigException.class, () -> new StreamsConfig(props)); + assertEquals( + String.format("Invalid value %s for configuration %s: Tag value exceeds maximum length of %s.", + value, StreamsConfig.CLIENT_TAG_PREFIX, StreamsConfig.MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH), + exception.getMessage() + ); + } + static class MisconfiguredSerde implements Serde { @Override public void configure(final Map configs, final boolean isKey) { diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RackAwarenessIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RackAwarenessIntegrationTest.java new file mode 100644 index 000000000000..f4afbe8012c5 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RackAwarenessIntegrationTest.java @@ -0,0 +1,436 @@ +/* + * 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.streams.integration; + +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.ThreadMetadata; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.Repartitioned; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.StoreBuilder; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +@Category({IntegrationTest.class}) +public class RackAwarenessIntegrationTest { + private static final int NUM_BROKERS = 1; + + private static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); + + private static final String TAG_VALUE_K8_CLUSTER_1 = "k8s-cluster-1"; + private static final String TAG_VALUE_K8_CLUSTER_2 = "k8s-cluster-2"; + private static final String TAG_VALUE_K8_CLUSTER_3 = "k8s-cluster-3"; + private static final String TAG_VALUE_EU_CENTRAL_1A = "eu-central-1a"; + private static final String TAG_VALUE_EU_CENTRAL_1B = "eu-central-1b"; + private static final String TAG_VALUE_EU_CENTRAL_1C = "eu-central-1c"; + + private static final int DEFAULT_NUMBER_OF_STATEFUL_SUB_TOPOLOGIES = 1; + private static final int DEFAULT_NUMBER_OF_PARTITIONS_OF_SUB_TOPOLOGIES = 2; + + @Rule + public TestName testName = new TestName(); + + private static final String INPUT_TOPIC = "input-topic"; + + private static final String TAG_ZONE = "zone"; + private static final String TAG_CLUSTER = "cluster"; + + private List kafkaStreamsInstances; + private Properties baseConfiguration; + private Topology topology; + + @BeforeClass + public static void createTopics() throws Exception { + CLUSTER.start(); + CLUSTER.createTopic(INPUT_TOPIC, 6, 1); + } + + @Before + public void setup() { + kafkaStreamsInstances = new ArrayList<>(); + baseConfiguration = new Properties(); + final String safeTestName = safeUniqueTestName(getClass(), testName); + final String applicationId = "app-" + safeTestName; + baseConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); + baseConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + baseConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); + baseConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); + baseConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); + } + + @After + public void cleanup() throws IOException { + for (final KafkaStreamsWithConfiguration kafkaStreamsWithConfiguration : kafkaStreamsInstances) { + kafkaStreamsWithConfiguration.kafkaStreams.close(Duration.ofMillis(IntegrationTestUtils.DEFAULT_TIMEOUT)); + IntegrationTestUtils.purgeLocalStreamsState(kafkaStreamsWithConfiguration.configuration); + } + kafkaStreamsInstances.clear(); + } + + @Test + public void shouldDoRebalancingWithMaximumNumberOfClientTags() throws Exception { + initTopology(3, 3); + final int numberOfStandbyReplicas = 1; + + final List clientTagKeys = new ArrayList<>(); + final Map clientTags1 = new HashMap<>(); + final Map clientTags2 = new HashMap<>(); + + for (int i = 0; i < StreamsConfig.MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE; i++) { + clientTagKeys.add("key-" + i); + } + + for (int i = 0; i < clientTagKeys.size(); i++) { + final String key = clientTagKeys.get(i); + clientTags1.put(key, "value-1-" + i); + clientTags2.put(key, "value-2-" + i); + } + + assertEquals(StreamsConfig.MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE, clientTagKeys.size()); + Stream.of(clientTags1, clientTags2) + .forEach(clientTags -> assertEquals(String.format("clientsTags with content '%s' " + + "did not match expected size", clientTags), + StreamsConfig.MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE, + clientTags.size())); + + createAndStart(clientTags1, clientTagKeys, numberOfStandbyReplicas); + createAndStart(clientTags1, clientTagKeys, numberOfStandbyReplicas); + createAndStart(clientTags2, clientTagKeys, numberOfStandbyReplicas); + + waitUntilAllKafkaStreamsClientsAreRunning(); + assertTrue(isIdealTaskDistributionReachedForTags(clientTagKeys)); + + stopKafkaStreamsInstanceWithIndex(0); + + waitUntilAllKafkaStreamsClientsAreRunning(); + + assertTrue(isIdealTaskDistributionReachedForTags(clientTagKeys)); + } + + @Test + public void shouldDistributeStandbyReplicasWhenAllClientsAreLocatedOnASameClusterTag() throws Exception { + initTopology(); + final int numberOfStandbyReplicas = 1; + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1A, TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1A, TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1A, TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1B, TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1B, TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1B, TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + + waitUntilAllKafkaStreamsClientsAreRunning(); + assertTrue(isIdealTaskDistributionReachedForTags(singletonList(TAG_ZONE))); + } + + @Test + public void shouldDistributeStandbyReplicasOverMultipleClientTags() throws Exception { + initTopology(); + final int numberOfStandbyReplicas = 2; + + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1A, TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1B, TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1C, TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1A, TAG_VALUE_K8_CLUSTER_2), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1B, TAG_VALUE_K8_CLUSTER_2), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1C, TAG_VALUE_K8_CLUSTER_2), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1A, TAG_VALUE_K8_CLUSTER_3), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1B, TAG_VALUE_K8_CLUSTER_3), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1C, TAG_VALUE_K8_CLUSTER_3), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + + waitUntilAllKafkaStreamsClientsAreRunning(); + assertTrue(isIdealTaskDistributionReachedForTags(asList(TAG_ZONE, TAG_CLUSTER))); + } + + @Test + public void shouldDistributeStandbyReplicasWhenIdealDistributionCanNotBeAchieved() throws Exception { + initTopology(); + final int numberOfStandbyReplicas = 2; + + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1A, TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1B, TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1C, TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1A, TAG_VALUE_K8_CLUSTER_2), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1B, TAG_VALUE_K8_CLUSTER_2), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + createAndStart(buildClientTags(TAG_VALUE_EU_CENTRAL_1C, TAG_VALUE_K8_CLUSTER_2), asList(TAG_ZONE, TAG_CLUSTER), numberOfStandbyReplicas); + + waitUntilAllKafkaStreamsClientsAreRunning(); + + assertTrue(isIdealTaskDistributionReachedForTags(singletonList(TAG_ZONE))); + assertTrue(isPartialTaskDistributionReachedForTags(singletonList(TAG_CLUSTER))); + } + + private void stopKafkaStreamsInstanceWithIndex(final int index) { + kafkaStreamsInstances.get(index).kafkaStreams.close(Duration.ofMillis(IntegrationTestUtils.DEFAULT_TIMEOUT)); + kafkaStreamsInstances.remove(index); + } + + private void waitUntilAllKafkaStreamsClientsAreRunning() throws Exception { + waitUntilAllKafkaStreamsClientsAreRunning(Duration.ofMillis(IntegrationTestUtils.DEFAULT_TIMEOUT)); + } + + private void waitUntilAllKafkaStreamsClientsAreRunning(final Duration timeout) throws Exception { + IntegrationTestUtils.waitForApplicationState(kafkaStreamsInstances.stream().map(it -> it.kafkaStreams).collect(Collectors.toList()), + KafkaStreams.State.RUNNING, + timeout); + } + + private boolean isPartialTaskDistributionReachedForTags(final Collection tagsToCheck) { + final Predicate partialTaskClientTagDistributionTest = taskClientTagDistribution -> { + final Map activeTaskClientTags = taskClientTagDistribution.activeTaskClientTags.clientTags; + return tagsAmongstActiveAndAtLeastOneStandbyTaskIsDifferent(taskClientTagDistribution.standbyTasksClientTags, activeTaskClientTags, tagsToCheck); + }; + + return isTaskDistributionTestSuccessful(partialTaskClientTagDistributionTest); + } + + private boolean isIdealTaskDistributionReachedForTags(final Collection tagsToCheck) { + final Predicate idealTaskClientTagDistributionTest = taskClientTagDistribution -> { + final Map activeTaskClientTags = taskClientTagDistribution.activeTaskClientTags.clientTags; + return tagsAmongstStandbyTasksAreDifferent(taskClientTagDistribution.standbyTasksClientTags, tagsToCheck) + && tagsAmongstActiveAndAllStandbyTasksAreDifferent(taskClientTagDistribution.standbyTasksClientTags, + activeTaskClientTags, + tagsToCheck); + }; + + return isTaskDistributionTestSuccessful(idealTaskClientTagDistributionTest); + } + + private boolean isTaskDistributionTestSuccessful(final Predicate taskClientTagDistributionPredicate) { + final List tasksClientTagDistributions = getTasksClientTagDistributions(); + + if (tasksClientTagDistributions.isEmpty()) { + return false; + } + + return tasksClientTagDistributions.stream().allMatch(taskClientTagDistributionPredicate); + } + + private static boolean tagsAmongstActiveAndAllStandbyTasksAreDifferent(final Collection standbyTasks, + final Map activeTaskClientTags, + final Collection tagsToCheck) { + return standbyTasks.stream().allMatch(standbyTask -> tagsToCheck.stream().noneMatch(tag -> activeTaskClientTags.get(tag).equals(standbyTask.clientTags.get(tag)))); + } + + private static boolean tagsAmongstActiveAndAtLeastOneStandbyTaskIsDifferent(final Collection standbyTasks, + final Map activeTaskClientTags, + final Collection tagsToCheck) { + return standbyTasks.stream().anyMatch(standbyTask -> tagsToCheck.stream().noneMatch(tag -> activeTaskClientTags.get(tag).equals(standbyTask.clientTags.get(tag)))); + } + + private static boolean tagsAmongstStandbyTasksAreDifferent(final Collection standbyTasks, final Collection tagsToCheck) { + final Map statistics = new HashMap<>(); + + for (final TaskClientTags standbyTask : standbyTasks) { + for (final String tag : tagsToCheck) { + final String tagValue = standbyTask.clientTags.get(tag); + final Integer tagValueOccurrence = statistics.getOrDefault(tagValue, 0); + statistics.put(tagValue, tagValueOccurrence + 1); + } + } + + return statistics.values().stream().noneMatch(occurrence -> occurrence > 1); + } + + private void initTopology() { + initTopology(DEFAULT_NUMBER_OF_PARTITIONS_OF_SUB_TOPOLOGIES, DEFAULT_NUMBER_OF_STATEFUL_SUB_TOPOLOGIES); + } + + private void initTopology(final int numberOfPartitionsOfSubTopologies, final int numberOfStatefulSubTopologies) { + final StreamsBuilder builder = new StreamsBuilder(); + final String stateStoreName = "myTransformState"; + + final StoreBuilder> keyValueStoreBuilder = Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore(stateStoreName), + Serdes.Integer(), + Serdes.Integer() + ); + + builder.addStateStore(keyValueStoreBuilder); + + final KStream stream = builder.stream(INPUT_TOPIC, Consumed.with(Serdes.Integer(), Serdes.Integer())); + + // Stateless sub-topology + stream.repartition(Repartitioned.numberOfPartitions(numberOfPartitionsOfSubTopologies)).filter((k, v) -> true); + + // Stateful sub-topologies + for (int i = 0; i < numberOfStatefulSubTopologies; i++) { + stream.repartition(Repartitioned.numberOfPartitions(numberOfPartitionsOfSubTopologies)) + .groupByKey() + .reduce(Integer::sum); + } + + topology = builder.build(); + } + + private List getTasksClientTagDistributions() { + final List taskClientTags = new ArrayList<>(); + + for (final KafkaStreamsWithConfiguration kafkaStreamsInstance : kafkaStreamsInstances) { + final StreamsConfig config = new StreamsConfig(kafkaStreamsInstance.configuration); + for (final ThreadMetadata localThreadsMetadata : kafkaStreamsInstance.kafkaStreams.metadataForLocalThreads()) { + localThreadsMetadata.activeTasks().forEach(activeTask -> { + final TaskId activeTaskId = activeTask.taskId(); + final Map clientTags = config.getClientTags(); + + final List standbyTasks = findStandbysForActiveTask(activeTaskId); + + if (!standbyTasks.isEmpty()) { + final TaskClientTags activeTaskView = new TaskClientTags(activeTaskId, clientTags); + taskClientTags.add(new TaskClientTagDistribution(activeTaskView, standbyTasks)); + } + }); + + } + } + + return taskClientTags; + } + + private List findStandbysForActiveTask(final TaskId taskId) { + final List standbyTasks = new ArrayList<>(); + + for (final KafkaStreamsWithConfiguration kafkaStreamsInstance : kafkaStreamsInstances) { + for (final ThreadMetadata localThreadsMetadata : kafkaStreamsInstance.kafkaStreams.metadataForLocalThreads()) { + localThreadsMetadata.standbyTasks().forEach(standbyTask -> { + final TaskId standbyTaskId = standbyTask.taskId(); + if (taskId.equals(standbyTaskId)) { + final StreamsConfig config = new StreamsConfig(kafkaStreamsInstance.configuration); + standbyTasks.add(new TaskClientTags(standbyTaskId, config.getClientTags())); + } + }); + } + } + + return standbyTasks; + } + + private static Map buildClientTags(final String zone, final String cluster) { + final Map clientTags = new HashMap<>(); + + clientTags.put(TAG_ZONE, zone); + clientTags.put(TAG_CLUSTER, cluster); + + return clientTags; + } + + private void createAndStart(final Map clientTags, + final Collection rackAwareAssignmentTags, + final int numberOfStandbyReplicas) { + final Properties streamsConfiguration = createStreamsConfiguration(clientTags, rackAwareAssignmentTags, numberOfStandbyReplicas); + final KafkaStreams kafkaStreams = new KafkaStreams(topology, streamsConfiguration); + + kafkaStreamsInstances.add(new KafkaStreamsWithConfiguration(streamsConfiguration, kafkaStreams)); + + kafkaStreams.start(); + } + + private Properties createStreamsConfiguration(final Map clientTags, + final Collection rackAwareAssignmentTags, + final int numStandbyReplicas) { + final Properties streamsConfiguration = new Properties(); + streamsConfiguration.putAll(baseConfiguration); + streamsConfiguration.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, numStandbyReplicas); + streamsConfiguration.put(StreamsConfig.RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, String.join(",", rackAwareAssignmentTags)); + clientTags.forEach((key, value) -> streamsConfiguration.put(StreamsConfig.clientTagPrefix(key), value)); + streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(String.join("-", clientTags.values())).getPath()); + return streamsConfiguration; + } + + private static final class KafkaStreamsWithConfiguration { + private final Properties configuration; + private final KafkaStreams kafkaStreams; + + KafkaStreamsWithConfiguration(final Properties configuration, final KafkaStreams kafkaStreams) { + this.configuration = configuration; + this.kafkaStreams = kafkaStreams; + } + } + + private static final class TaskClientTagDistribution { + private final TaskClientTags activeTaskClientTags; + private final List standbyTasksClientTags; + + TaskClientTagDistribution(final TaskClientTags activeTaskClientTags, final List standbyTasksClientTags) { + this.activeTaskClientTags = activeTaskClientTags; + this.standbyTasksClientTags = standbyTasksClientTags; + } + + @Override + public String toString() { + return "TaskDistribution{" + + "activeTaskClientTagsView=" + activeTaskClientTags + + ", standbyTasks=" + standbyTasksClientTags + + '}'; + } + } + + private static final class TaskClientTags { + private final TaskId taskId; + private final Map clientTags; + + TaskClientTags(final TaskId taskId, final Map clientTags) { + this.taskId = taskId; + this.clientTags = clientTags; + } + + @Override + public String toString() { + return "TaskClientTags{" + + "taskId=" + taskId + + ", clientTags=" + clientTags + + '}'; + } + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java index d11f3e056cf5..e2c08ed4b62d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.streams.processor.internals; -import java.time.Duration; -import java.util.Properties; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.ListOffsetsResult; @@ -79,6 +77,7 @@ import org.junit.runners.Parameterized; import java.nio.ByteBuffer; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -86,6 +85,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; import java.util.SortedSet; import java.util.UUID; @@ -196,6 +196,7 @@ public class StreamsPartitionAssignorTest { private StreamsMetadataState streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class); private final Map subscriptions = new HashMap<>(); private final Class taskAssignor; + private Map clientTags; private final ReferenceContainer referenceContainer = new ReferenceContainer(); private final MockTime time = new MockTime(); @@ -210,6 +211,7 @@ private Map configProps() { referenceContainer.taskManager = taskManager; referenceContainer.streamsMetadataState = streamsMetadataState; referenceContainer.time = time; + referenceContainer.clientTags = clientTags != null ? clientTags : EMPTY_CLIENT_TAGS; configurationMap.put(InternalConfig.REFERENCE_CONTAINER_PARTITION_ASSIGNOR, referenceContainer); configurationMap.put(InternalConfig.INTERNAL_TASK_ASSIGNOR_CLASS, taskAssignor.getName()); return configurationMap; @@ -2190,6 +2192,21 @@ public void shouldThrowTaskAssignmentExceptionWhenUnableToResolvePartitionCount( equalTo(AssignorError.ASSIGNMENT_ERROR.code())); } + @Test + public void testClientTags() { + clientTags = mkMap(mkEntry("cluster", "cluster1"), mkEntry("zone", "az1")); + createDefaultMockTaskManager(); + configureDefaultPartitionAssignor(); + final Set topics = mkSet("input"); + final Subscription subscription = new Subscription(new ArrayList<>(topics), + partitionAssignor.subscriptionUserData(topics)); + final SubscriptionInfo info = getInfo(UUID_1, EMPTY_TASKS, EMPTY_TASKS, uniqueField, clientTags); + + assertEquals(singletonList("input"), subscription.topics()); + assertEquals(info, SubscriptionInfo.decode(subscription.userData())); + assertEquals(clientTags, partitionAssignor.clientTags()); + } + private static class CorruptedInternalTopologyBuilder extends InternalTopologyBuilder { private Map corruptedTopicGroups; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java index cf8a6b297ab7..42a32c04b95b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java @@ -166,6 +166,15 @@ public static SubscriptionInfo getInfo(final UUID processId, LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, processId, null, getTaskOffsetSums(prevTasks, standbyTasks), uniqueField, 0, EMPTY_CLIENT_TAGS); } + public static SubscriptionInfo getInfo(final UUID processId, + final Set prevTasks, + final Set standbyTasks, + final byte uniqueField, + final Map clientTags) { + return new SubscriptionInfo( + LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, processId, null, getTaskOffsetSums(prevTasks, standbyTasks), uniqueField, 0, clientTags); + } + // Stub offset sums for when we only care about the prev/standby task sets, not the actual offsets private static Map getTaskOffsetSums(final Collection activeTasks, final Collection standbyTasks) { final Map taskOffsetSums = activeTasks.stream().collect(Collectors.toMap(t -> t, t -> Task.LATEST_OFFSET)); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignorTest.java index 8a983dee9be2..631430c6a82f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignorTest.java @@ -81,8 +81,43 @@ public void setup() { standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor(); } + @Test + public void shouldNotAssignStatelessTasksToAnyClients() { + final Set statefulTasks = mkSet( + TASK_1_0, + TASK_1_1, + TASK_1_2 + ); + + final Map clientStates = mkMap( + mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0, TASK_1_0)), + mkEntry(UUID_2, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)))), + mkEntry(UUID_3, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)))), + + mkEntry(UUID_4, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_2)), TASK_0_1, TASK_1_1)), + mkEntry(UUID_5, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_2)))), + mkEntry(UUID_6, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_2)))), + + mkEntry(UUID_7, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_3)), TASK_0_2, TASK_1_2)), + mkEntry(UUID_8, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_3)))), + mkEntry(UUID_9, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_3)))) + ); + + final Set allActiveTasks = findAllActiveTasks(clientStates); + + final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, ZONE_TAG, CLUSTER_TAG); + + standbyTaskAssignor.assign(clientStates, allActiveTasks, statefulTasks, assignmentConfigs); + + final Set statelessTasks = allActiveTasks.stream().filter(taskId -> !statefulTasks.contains(taskId)).collect(Collectors.toSet()); + assertTrue( + clientStates.values().stream().allMatch(clientState -> statelessTasks.stream().noneMatch(clientState::hasStandbyTask)) + ); + } + @Test public void shouldRemoveClientToRemainingStandbysAndNotPopulatePendingStandbyTasksToClientIdWhenAllStandbyTasksWereAssigned() { + final int numStandbyReplicas = 2; final Set rackAwareAssignmentTags = mkSet(ZONE_TAG, CLUSTER_TAG); final Map clientStates = mkMap( mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0)), @@ -102,10 +137,11 @@ public void shouldRemoveClientToRemainingStandbysAndNotPopulatePendingStandbyTas fillClientsTagStatistics(clientStates, tagEntryToClients, tagKeyToValues); final Map pendingStandbyTasksToClientId = new HashMap<>(); - final Map tasksToRemainingStandbys = computeTasksToRemainingStandbys(2, allActiveTasks); + final Map tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas, allActiveTasks); for (final TaskId activeTaskId : allActiveTasks) { assignStandbyTasksToClientsWithDifferentTags( + numStandbyReplicas, constrainedPrioritySet, activeTaskId, taskToClientId.get(activeTaskId), @@ -132,6 +168,7 @@ public void shouldUpdateClientToRemainingStandbysAndPendingStandbyTasksToClientI ); final ConstrainedPrioritySet constrainedPrioritySet = createLeastLoadedPrioritySetConstrainedByAssignedTask(clientStates); + final int numStandbyReplicas = 3; final Set allActiveTasks = findAllActiveTasks(clientStates); final Map taskToClientId = mkMap(mkEntry(TASK_0_0, UUID_1), mkEntry(TASK_0_1, UUID_2), @@ -143,10 +180,11 @@ public void shouldUpdateClientToRemainingStandbysAndPendingStandbyTasksToClientI fillClientsTagStatistics(clientStates, tagEntryToClients, tagKeyToValues); final Map pendingStandbyTasksToClientId = new HashMap<>(); - final Map tasksToRemainingStandbys = computeTasksToRemainingStandbys(3, allActiveTasks); + final Map tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas, allActiveTasks); for (final TaskId activeTaskId : allActiveTasks) { assignStandbyTasksToClientsWithDifferentTags( + numStandbyReplicas, constrainedPrioritySet, activeTaskId, taskToClientId.get(activeTaskId), diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignorTest.java index bf78db6457f2..36ae42fded37 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignorTest.java @@ -20,10 +20,8 @@ import org.apache.kafka.streams.processor.internals.assignment.AssignorConfiguration.AssignmentConfigs; import org.junit.Test; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; @@ -31,7 +29,6 @@ import static java.util.Collections.emptySet; import static java.util.Collections.singleton; -import static java.util.Collections.singletonList; import static java.util.Collections.singletonMap; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -68,7 +65,6 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; public class HighAvailabilityTaskAssignorTest { @@ -814,27 +810,6 @@ public void shouldDistributeStatelessTasksEvenlyWithPreviousAssignmentAndNoState assertThat(probingRebalanceNeeded, is(false)); } - @Test - public void shouldReturnClientTagAwareStandbyTaskAssignorWhenRackAwareAssignmentTagsIsSet() { - final StandbyTaskAssignor standbyTaskAssignor = HighAvailabilityTaskAssignor.createStandbyTaskAssignor(newAssignmentConfigs(1, singletonList("az"))); - assertTrue(standbyTaskAssignor instanceof ClientTagAwareStandbyTaskAssignor); - } - - @Test - public void shouldReturnDefaultStandbyTaskAssignorWhenRackAwareAssignmentTagsIsEmpty() { - final StandbyTaskAssignor standbyTaskAssignor = HighAvailabilityTaskAssignor.createStandbyTaskAssignor(newAssignmentConfigs(1, Collections.emptyList())); - assertTrue(standbyTaskAssignor instanceof DefaultStandbyTaskAssignor); - } - - private static AssignorConfiguration.AssignmentConfigs newAssignmentConfigs(final int numStandbyReplicas, - final List rackAwareAssignmentTags) { - return new AssignorConfiguration.AssignmentConfigs(0L, - 1, - numStandbyReplicas, - 60000L, - rackAwareAssignmentTags); - } - private static void assertHasNoActiveTasks(final ClientState... clients) { for (final ClientState client : clients) { assertThat(client.activeTasks(), is(empty())); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignmentUtilsTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignmentUtilsTest.java index 1abf1b926353..b13f04b2bd8e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignmentUtilsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignmentUtilsTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.junit.Before; import org.junit.Test; +import org.slf4j.Logger; import java.util.Map; import java.util.Set; @@ -36,8 +37,16 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; public class StandbyTaskAssignmentUtilsTest { + private static final Set ACTIVE_TASKS = mkSet(TASK_0_0, TASK_0_1, TASK_0_2); private Map clients; @@ -45,6 +54,7 @@ public class StandbyTaskAssignmentUtilsTest { @Before public void setup() { + clients = getClientStatesMap(ACTIVE_TASKS.stream().map(StandbyTaskAssignmentUtilsTest::mkState).toArray(ClientState[]::new)); clientsByTaskLoad = new ConstrainedPrioritySet( (client, task) -> !clients.get(client).hasAssignedTask(task), @@ -55,38 +65,38 @@ public void setup() { @Test public void shouldReturnNumberOfStandbyTasksThatWereNotAssigned() { - final Map tasksToRemainingStandbys = computeTasksToRemainingStandbys(3, ACTIVE_TASKS); - - assertTrue(tasksToRemainingStandbys.keySet() - .stream() - .map(taskId -> pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks( - clients, - tasksToRemainingStandbys, - clientsByTaskLoad, - taskId - )) - .allMatch(numRemainingStandbys -> numRemainingStandbys == 1)); + final Logger logMock = mock(Logger.class); + final int numStandbyReplicas = 3; + final Map tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas, ACTIVE_TASKS); + + tasksToRemainingStandbys.keySet().forEach(taskId -> pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks(numStandbyReplicas, + clients, + tasksToRemainingStandbys, + clientsByTaskLoad, + taskId, + logMock)); assertTrue(ACTIVE_TASKS.stream().allMatch(activeTask -> tasksToRemainingStandbys.get(activeTask) == 1)); assertTrue(areStandbyTasksPresentForAllActiveTasks(2)); + verify(logMock, times(ACTIVE_TASKS.size())).warn(anyString(), anyInt(), anyInt(), any()); } @Test public void shouldReturnZeroWhenAllStandbyTasksWereSuccessfullyAssigned() { - final Map tasksToRemainingStandbys = computeTasksToRemainingStandbys(1, ACTIVE_TASKS); - - assertTrue(tasksToRemainingStandbys.keySet() - .stream() - .map(taskId -> pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks( - clients, - tasksToRemainingStandbys, - clientsByTaskLoad, - taskId - )) - .allMatch(numRemainingStandbys -> numRemainingStandbys == 0)); + final Logger logMock = mock(Logger.class); + final int numStandbyReplicas = 1; + final Map tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas, ACTIVE_TASKS); + + tasksToRemainingStandbys.keySet().forEach(taskId -> pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks(numStandbyReplicas, + clients, + tasksToRemainingStandbys, + clientsByTaskLoad, + taskId, + logMock)); assertTrue(ACTIVE_TASKS.stream().allMatch(activeTask -> tasksToRemainingStandbys.get(activeTask) == 0)); assertTrue(areStandbyTasksPresentForAllActiveTasks(1)); + verifyNoInteractions(logMock); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorFactoryTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorFactoryTest.java new file mode 100644 index 000000000000..fdd7fa1d4737 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorFactoryTest.java @@ -0,0 +1,52 @@ +/* + * 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.streams.processor.internals.assignment; + +import org.junit.Test; + +import java.util.Collections; +import java.util.List; + +import static java.util.Collections.singletonList; +import static org.junit.Assert.assertTrue; + +public class StandbyTaskAssignorFactoryTest { + private static final long ACCEPTABLE_RECOVERY_LAG = 0L; + private static final int MAX_WARMUP_REPLICAS = 1; + private static final int NUMBER_OF_STANDBY_REPLICAS = 1; + private static final long PROBING_REBALANCE_INTERVAL_MS = 60000L; + + @Test + public void shouldReturnClientTagAwareStandbyTaskAssignorWhenRackAwareAssignmentTagsIsSet() { + final StandbyTaskAssignor standbyTaskAssignor = StandbyTaskAssignorFactory.create(newAssignmentConfigs(singletonList("az"))); + assertTrue(standbyTaskAssignor instanceof ClientTagAwareStandbyTaskAssignor); + } + + @Test + public void shouldReturnDefaultStandbyTaskAssignorWhenRackAwareAssignmentTagsIsEmpty() { + final StandbyTaskAssignor standbyTaskAssignor = StandbyTaskAssignorFactory.create(newAssignmentConfigs(Collections.emptyList())); + assertTrue(standbyTaskAssignor instanceof DefaultStandbyTaskAssignor); + } + + private static AssignorConfiguration.AssignmentConfigs newAssignmentConfigs(final List rackAwareAssignmentTags) { + return new AssignorConfiguration.AssignmentConfigs(ACCEPTABLE_RECOVERY_LAG, + MAX_WARMUP_REPLICAS, + NUMBER_OF_STANDBY_REPLICAS, + PROBING_REBALANCE_INTERVAL_MS, + rackAwareAssignmentTags); + } +} \ No newline at end of file From fbe7fb941173c0907792a8b48e8e9122aabecbd8 Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Thu, 17 Mar 2022 10:19:42 +0800 Subject: [PATCH 018/447] KAFKA-9847: add config to set default store type (KIP-591) (#11705) Reviewers: Hao Li , A. Sophie Blee-Goldman , Guozhang Wang , Matthias J. Sax --- .../apache/kafka/streams/StreamsBuilder.java | 10 +- .../apache/kafka/streams/StreamsConfig.java | 14 + .../org/apache/kafka/streams/Topology.java | 4 + .../namedtopology => }/TopologyConfig.java | 89 ++- .../kafka/streams/kstream/Materialized.java | 46 +- .../internals/MaterializedInternal.java | 17 + .../SessionWindowedCogroupedKStreamImpl.java | 22 +- .../internals/SessionWindowedKStreamImpl.java | 22 +- .../SlidingWindowedCogroupedKStreamImpl.java | 29 +- .../internals/SlidingWindowedKStreamImpl.java | 29 +- .../TimeWindowedCogroupedKStreamImpl.java | 26 +- .../internals/TimeWindowedKStreamImpl.java | 28 +- .../TimestampedKeyValueStoreMaterializer.java | 13 +- .../internals/InternalTopologyBuilder.java | 2 +- .../processor/internals/StandbyTask.java | 2 +- .../processor/internals/StreamTask.java | 2 +- .../processor/internals/TopologyMetadata.java | 2 +- .../namedtopology/NamedTopology.java | 1 + .../namedtopology/NamedTopologyBuilder.java | 1 + .../kafka/streams/StreamsConfigTest.java | 25 +- .../apache/kafka/streams/TopologyTest.java | 677 +++++++++++++++++- .../streams/kstream/MaterializedTest.java | 17 + .../internals/KGroupedStreamImplTest.java | 2 +- .../internals/MaterializedInternalTest.java | 22 + .../internals/ActiveTaskCreatorTest.java | 2 +- .../InternalTopologyBuilderTest.java | 5 +- .../processor/internals/StandbyTaskTest.java | 2 +- .../processor/internals/StreamTaskTest.java | 2 +- .../StreamThreadStateStoreProviderTest.java | 2 +- .../kafka/streams/TopologyTestDriver.java | 2 +- 30 files changed, 1009 insertions(+), 108 deletions(-) rename streams/src/main/java/org/apache/kafka/streams/{processor/internals/namedtopology => }/TopologyConfig.java (78%) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java index 5f5d0b7853aa..e913728984e1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java @@ -38,7 +38,6 @@ import org.apache.kafka.streams.processor.internals.ProcessorAdapter; import org.apache.kafka.streams.processor.internals.ProcessorNode; import org.apache.kafka.streams.processor.internals.SourceNode; -import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; @@ -81,14 +80,19 @@ public StreamsBuilder() { internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder); } - protected StreamsBuilder(final TopologyConfig topologyConfigs) { + /** + * Create a {@code StreamsBuilder} instance. + * + * @param topologyConfigs the streams configs that apply at the topology level. Please refer to {@link TopologyConfig} for more detail + */ + public StreamsBuilder(final TopologyConfig topologyConfigs) { topology = getNewTopology(topologyConfigs); internalTopologyBuilder = topology.internalTopologyBuilder; internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder); } protected Topology getNewTopology(final TopologyConfig topologyConfigs) { - return new Topology(); + return new Topology(topologyConfigs); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 66cee6aed7c8..58ee252abf0a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -415,6 +415,14 @@ public class StreamsConfig extends AbstractConfig { public static final String DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG = "default.production.exception.handler"; private static final String DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the org.apache.kafka.streams.errors.ProductionExceptionHandler interface."; + /** {@code default.dsl.store} */ + @SuppressWarnings("WeakerAccess") + public static final String DEFAULT_DSL_STORE_CONFIG = "default.dsl.store"; + public static final String DEFAULT_DSL_STORE_DOC = "The default state store type used by DSL operators."; + + public static final String ROCKS_DB = "rocksDB"; + public static final String IN_MEMORY = "in_memory"; + /** {@code default.windowed.key.serde.inner} */ @SuppressWarnings("WeakerAccess") @Deprecated @@ -818,6 +826,12 @@ public class StreamsConfig extends AbstractConfig { 9 * 60 * 1000L, ConfigDef.Importance.LOW, CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC) + .define(DEFAULT_DSL_STORE_CONFIG, + Type.STRING, + ROCKS_DB, + in(ROCKS_DB, IN_MEMORY), + Importance.LOW, + DEFAULT_DSL_STORE_DOC) .define(METADATA_MAX_AGE_CONFIG, ConfigDef.Type.LONG, 5 * 60 * 1000L, diff --git a/streams/src/main/java/org/apache/kafka/streams/Topology.java b/streams/src/main/java/org/apache/kafka/streams/Topology.java index 0eb5e2b3861b..314d2190f666 100644 --- a/streams/src/main/java/org/apache/kafka/streams/Topology.java +++ b/streams/src/main/java/org/apache/kafka/streams/Topology.java @@ -60,6 +60,10 @@ public Topology() { this(new InternalTopologyBuilder()); } + public Topology(final TopologyConfig topologyConfigs) { + this(new InternalTopologyBuilder(topologyConfigs)); + } + protected Topology(final InternalTopologyBuilder internalTopologyBuilder) { this.internalTopologyBuilder = internalTopologyBuilder; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java similarity index 78% rename from streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java rename to streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java index 2587a0ab5bc2..f046e71c8e9c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java @@ -14,22 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.streams.processor.internals.namedtopology; +package org.apache.kafka.streams; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; -import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; +import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.internals.StreamsConfigUtils; import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Properties; import java.util.function.Supplier; +import static org.apache.kafka.common.config.ConfigDef.ValidString.in; import static org.apache.kafka.streams.StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG; import static org.apache.kafka.streams.StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_DOC; import static org.apache.kafka.streams.StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG; @@ -42,6 +44,10 @@ import static org.apache.kafka.streams.StreamsConfig.MAX_TASK_IDLE_MS_DOC; import static org.apache.kafka.streams.StreamsConfig.TASK_TIMEOUT_MS_CONFIG; import static org.apache.kafka.streams.StreamsConfig.TASK_TIMEOUT_MS_DOC; +import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_CONFIG; +import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_DOC; +import static org.apache.kafka.streams.StreamsConfig.ROCKS_DB; +import static org.apache.kafka.streams.StreamsConfig.IN_MEMORY; /** * Streams configs that apply at the topology level. The values in the {@link StreamsConfig} parameter of the @@ -53,36 +59,42 @@ public class TopologyConfig extends AbstractConfig { private static final ConfigDef CONFIG; static { CONFIG = new ConfigDef() - .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, - Type.INT, - null, - Importance.LOW, - BUFFERED_RECORDS_PER_PARTITION_DOC) + .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, + Type.INT, + null, + Importance.LOW, + BUFFERED_RECORDS_PER_PARTITION_DOC) .define(CACHE_MAX_BYTES_BUFFERING_CONFIG, - Type.LONG, - null, - Importance.MEDIUM, - CACHE_MAX_BYTES_BUFFERING_DOC) - .define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, - Type.CLASS, - null, - Importance.MEDIUM, - DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC) - .define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, - Type.CLASS, - null, - Importance.MEDIUM, - DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC) - .define(MAX_TASK_IDLE_MS_CONFIG, - Type.LONG, - null, - Importance.MEDIUM, - MAX_TASK_IDLE_MS_DOC) - .define(TASK_TIMEOUT_MS_CONFIG, - Type.LONG, - null, - Importance.MEDIUM, - TASK_TIMEOUT_MS_DOC); + Type.LONG, + null, + Importance.MEDIUM, + CACHE_MAX_BYTES_BUFFERING_DOC) + .define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, + Type.CLASS, + null, + Importance.MEDIUM, + DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC) + .define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, + Type.CLASS, + null, + Importance.MEDIUM, + DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC) + .define(MAX_TASK_IDLE_MS_CONFIG, + Type.LONG, + null, + Importance.MEDIUM, + MAX_TASK_IDLE_MS_DOC) + .define(TASK_TIMEOUT_MS_CONFIG, + Type.LONG, + null, + Importance.MEDIUM, + TASK_TIMEOUT_MS_DOC) + .define(DEFAULT_DSL_STORE_CONFIG, + Type.STRING, + ROCKS_DB, + in(ROCKS_DB, IN_MEMORY), + Importance.LOW, + DEFAULT_DSL_STORE_DOC); } private final Logger log = LoggerFactory.getLogger(TopologyConfig.class); @@ -96,6 +108,7 @@ public class TopologyConfig extends AbstractConfig { public final long cacheSize; public final long maxTaskIdleMs; public final long taskTimeoutMs; + public final String storeType; public final Supplier timestampExtractorSupplier; public final Supplier deserializationExceptionHandlerSupplier; @@ -153,6 +166,20 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo } else { deserializationExceptionHandlerSupplier = () -> globalAppConfigs.getConfiguredInstance(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, DeserializationExceptionHandler.class); } + + if (isTopologyOverride(DEFAULT_DSL_STORE_CONFIG, topologyOverrides)) { + storeType = getString(DEFAULT_DSL_STORE_CONFIG); + log.info("Topology {} is overriding {} to {}", topologyName, DEFAULT_DSL_STORE_CONFIG, storeType); + } else { + storeType = globalAppConfigs.getString(DEFAULT_DSL_STORE_CONFIG); + } + } + + public Materialized.StoreType parseStoreType() { + if (storeType.equals(IN_MEMORY)) { + return Materialized.StoreType.IN_MEMORY; + } + return Materialized.StoreType.ROCKS_DB; } public boolean isNamedTopology() { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Materialized.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Materialized.java index 82b38000e09b..f63b6b9773df 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Materialized.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Materialized.java @@ -64,6 +64,13 @@ public class Materialized { protected boolean cachingEnabled = true; protected Map topicConfig = new HashMap<>(); protected Duration retention; + public StoreType storeType; + + // the built-in state store types + public enum StoreType { + ROCKS_DB, + IN_MEMORY + } private Materialized(final StoreSupplier storeSupplier) { this.storeSupplier = storeSupplier; @@ -73,6 +80,10 @@ private Materialized(final String storeName) { this.storeName = storeName; } + private Materialized(final StoreType storeType) { + this.storeType = storeType; + } + /** * Copy constructor. * @param materialized the {@link Materialized} instance to copy. @@ -86,6 +97,21 @@ protected Materialized(final Materialized materialized) { this.cachingEnabled = materialized.cachingEnabled; this.topicConfig = materialized.topicConfig; this.retention = materialized.retention; + this.storeType = materialized.storeType; + } + + /** + * Materialize a {@link StateStore} with the given {@link StoreType}. + * + * @param storeType the type of the state store + * @param key type of the store + * @param value type of the store + * @param type of the {@link StateStore} + * @return a new {@link Materialized} instance with the given storeName + */ + public static Materialized as(final StoreType storeType) { + Objects.requireNonNull(storeType, "store type can't be null"); + return new Materialized<>(storeType); } /** @@ -251,12 +277,28 @@ public Materialized withCachingDisabled() { */ public Materialized withRetention(final Duration retention) throws IllegalArgumentException { final String msgPrefix = prepareMillisCheckFailMsgPrefix(retention, "retention"); - final long retenationMs = validateMillisecondDuration(retention, msgPrefix); + final long retentionMs = validateMillisecondDuration(retention, msgPrefix); - if (retenationMs < 0) { + if (retentionMs < 0) { throw new IllegalArgumentException("Retention must not be negative."); } this.retention = retention; return this; } + + /** + * Set the type of the materialized {@link StateStore}. + * + * @param storeType the store type {@link StoreType} to use. + * @return itself + * @throws IllegalArgumentException if store supplier is also pre-configured + */ + public Materialized withStoreType(final StoreType storeType) throws IllegalArgumentException { + Objects.requireNonNull(storeType, "store type can't be null"); + if (storeSupplier != null) { + throw new IllegalArgumentException("Cannot set store type when store supplier is pre-configured."); + } + this.storeType = storeType; + return this; + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/MaterializedInternal.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/MaterializedInternal.java index 4a3cbb25cd24..e81934716b29 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/MaterializedInternal.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/MaterializedInternal.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.TopologyConfig; import org.apache.kafka.streams.state.StoreSupplier; import java.time.Duration; @@ -43,6 +44,18 @@ public MaterializedInternal(final Materialized materialized, if (!queryable && nameProvider != null) { storeName = nameProvider.newStoreName(generatedStorePrefix); } + + // if store type is not configured during creating Materialized, then try to get the topologyConfigs from nameProvider + // otherwise, set to default rocksDB + if (storeType == null) { + storeType = StoreType.ROCKS_DB; + if (nameProvider instanceof InternalStreamsBuilder) { + final TopologyConfig topologyConfig = ((InternalStreamsBuilder) nameProvider).internalTopologyBuilder.topologyConfigs(); + if (topologyConfig != null) { + storeType = topologyConfig.parseStoreType(); + } + } + } } public String queryableStoreName() { @@ -56,6 +69,10 @@ public String storeName() { return storeName; } + public StoreType storeType() { + return storeType; + } + public StoreSupplier storeSupplier() { return storeSupplier; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindowedCogroupedKStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindowedCogroupedKStreamImpl.java index a78bcd3dae44..4279224fa812 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindowedCogroupedKStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindowedCogroupedKStreamImpl.java @@ -121,11 +121,25 @@ private StoreBuilder> materialize(final MaterializedInternal + " grace=[" + sessionWindows.gracePeriodMs() + "]," + " retention=[" + retentionPeriod + "]"); } - supplier = Stores.persistentSessionStore( - materialized.storeName(), - Duration.ofMillis(retentionPeriod) - ); + + switch (materialized.storeType()) { + case IN_MEMORY: + supplier = Stores.inMemorySessionStore( + materialized.storeName(), + Duration.ofMillis(retentionPeriod) + ); + break; + case ROCKS_DB: + supplier = Stores.persistentSessionStore( + materialized.storeName(), + Duration.ofMillis(retentionPeriod) + ); + break; + default: + throw new IllegalStateException("Unknown store type: " + materialized.storeType()); + } } + final StoreBuilder> builder = Stores.sessionStoreBuilder( supplier, materialized.keySerde(), diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImpl.java index fe9a3a1f8643..f18c6ef56802 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImpl.java @@ -237,11 +237,25 @@ private StoreBuilder> materialize(final MaterializedInt + " grace=[" + windows.gracePeriodMs() + "]," + " retention=[" + retentionPeriod + "]"); } - supplier = Stores.persistentSessionStore( - materialized.storeName(), - Duration.ofMillis(retentionPeriod) - ); + + switch (materialized.storeType()) { + case IN_MEMORY: + supplier = Stores.inMemorySessionStore( + materialized.storeName(), + Duration.ofMillis(retentionPeriod) + ); + break; + case ROCKS_DB: + supplier = Stores.persistentSessionStore( + materialized.storeName(), + Duration.ofMillis(retentionPeriod) + ); + break; + default: + throw new IllegalStateException("Unknown store type: " + materialized.storeType()); + } } + final StoreBuilder> builder = Stores.sessionStoreBuilder( supplier, materialized.keySerde(), diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedCogroupedKStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedCogroupedKStreamImpl.java index a432b1f1462b..383fed70a690 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedCogroupedKStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedCogroupedKStreamImpl.java @@ -32,6 +32,7 @@ import org.apache.kafka.streams.state.TimestampedWindowStore; import org.apache.kafka.streams.state.WindowBytesStoreSupplier; import org.apache.kafka.streams.state.WindowStore; + import java.time.Duration; import java.util.Map; import java.util.Objects; @@ -113,14 +114,28 @@ private StoreBuilder> materialize(final Materialize + "]"); } - supplier = Stores.persistentTimestampedWindowStore( - materialized.storeName(), - Duration.ofMillis(retentionPeriod), - Duration.ofMillis(windows.timeDifferenceMs()), - false - ); - + switch (materialized.storeType()) { + case IN_MEMORY: + supplier = Stores.inMemoryWindowStore( + materialized.storeName(), + Duration.ofMillis(retentionPeriod), + Duration.ofMillis(windows.timeDifferenceMs()), + false + ); + break; + case ROCKS_DB: + supplier = Stores.persistentTimestampedWindowStore( + materialized.storeName(), + Duration.ofMillis(retentionPeriod), + Duration.ofMillis(windows.timeDifferenceMs()), + false + ); + break; + default: + throw new IllegalStateException("Unknown store type: " + materialized.storeType()); + } } + final StoreBuilder> builder = Stores .timestampedWindowStoreBuilder( supplier, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImpl.java index ddfe9abc8bc5..e29483f2b27a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImpl.java @@ -34,6 +34,7 @@ import org.apache.kafka.streams.state.TimestampedWindowStore; import org.apache.kafka.streams.state.WindowBytesStoreSupplier; import org.apache.kafka.streams.state.WindowStore; + import java.time.Duration; import java.util.Objects; import java.util.Set; @@ -204,13 +205,29 @@ private StoreBuilder> materialize(final Mater + " grace=[" + windows.gracePeriodMs() + "]," + " retention=[" + retentionPeriod + "]"); } - supplier = Stores.persistentTimestampedWindowStore( - materialized.storeName(), - Duration.ofMillis(retentionPeriod), - Duration.ofMillis(windows.timeDifferenceMs()), - false - ); + + switch (materialized.storeType()) { + case IN_MEMORY: + supplier = Stores.inMemoryWindowStore( + materialized.storeName(), + Duration.ofMillis(retentionPeriod), + Duration.ofMillis(windows.timeDifferenceMs()), + false + ); + break; + case ROCKS_DB: + supplier = Stores.persistentTimestampedWindowStore( + materialized.storeName(), + Duration.ofMillis(retentionPeriod), + Duration.ofMillis(windows.timeDifferenceMs()), + false + ); + break; + default: + throw new IllegalStateException("Unknown store type: " + materialized.storeType()); + } } + final StoreBuilder> builder = Stores.timestampedWindowStoreBuilder( supplier, materialized.keySerde(), diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedCogroupedKStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedCogroupedKStreamImpl.java index 8cef89f6197e..07b75bd1454f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedCogroupedKStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedCogroupedKStreamImpl.java @@ -120,12 +120,26 @@ private StoreBuilder> materialize( + "]"); } - supplier = Stores.persistentTimestampedWindowStore( - materialized.storeName(), - Duration.ofMillis(retentionPeriod), - Duration.ofMillis(windows.size()), - false - ); + switch (materialized.storeType()) { + case IN_MEMORY: + supplier = Stores.inMemoryWindowStore( + materialized.storeName(), + Duration.ofMillis(retentionPeriod), + Duration.ofMillis(windows.size()), + false + ); + break; + case ROCKS_DB: + supplier = Stores.persistentTimestampedWindowStore( + materialized.storeName(), + Duration.ofMillis(retentionPeriod), + Duration.ofMillis(windows.size()), + false + ); + break; + default: + throw new IllegalStateException("Unknown store type: " + materialized.storeType()); + } } final StoreBuilder> builder = Stores diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java index 2282672a1365..7a82d0834731 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java @@ -206,8 +206,6 @@ public KTable, V> reduce(final Reducer reducer, materializedInternal.queryableStoreName(), materializedInternal.keySerde() != null ? new FullTimeWindowedSerde<>(materializedInternal.keySerde(), windows.size()) : null, materializedInternal.valueSerde()); - - } private StoreBuilder> materialize(final MaterializedInternal> materialized) { @@ -224,12 +222,26 @@ private StoreBuilder> materialize(final Mater + " retention=[" + retentionPeriod + "]"); } - supplier = Stores.persistentTimestampedWindowStore( - materialized.storeName(), - Duration.ofMillis(retentionPeriod), - Duration.ofMillis(windows.size()), - false - ); + switch (materialized.storeType()) { + case IN_MEMORY: + supplier = Stores.inMemoryWindowStore( + materialized.storeName(), + Duration.ofMillis(retentionPeriod), + Duration.ofMillis(windows.size()), + false + ); + break; + case ROCKS_DB: + supplier = Stores.persistentTimestampedWindowStore( + materialized.storeName(), + Duration.ofMillis(retentionPeriod), + Duration.ofMillis(windows.size()), + false + ); + break; + default: + throw new IllegalStateException("Unknown store type: " + materialized.storeType()); + } } final StoreBuilder> builder = Stores.timestampedWindowStoreBuilder( diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimestampedKeyValueStoreMaterializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimestampedKeyValueStoreMaterializer.java index fb40b464ce74..edbde7e11c8b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimestampedKeyValueStoreMaterializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimestampedKeyValueStoreMaterializer.java @@ -36,9 +36,18 @@ public TimestampedKeyValueStoreMaterializer(final MaterializedInternal> materialize() { KeyValueBytesStoreSupplier supplier = (KeyValueBytesStoreSupplier) materialized.storeSupplier(); if (supplier == null) { - final String name = materialized.storeName(); - supplier = Stores.persistentTimestampedKeyValueStore(name); + switch (materialized.storeType()) { + case IN_MEMORY: + supplier = Stores.inMemoryKeyValueStore(materialized.storeName()); + break; + case ROCKS_DB: + supplier = Stores.persistentTimestampedKeyValueStore(materialized.storeName()); + break; + default: + throw new IllegalStateException("Unknown store type: " + materialized.storeType()); + } } + final StoreBuilder> builder = Stores.timestampedKeyValueStoreBuilder( supplier, materialized.keySerde(), diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 8fa9b15a312e..4c9c97ae1db4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -31,7 +31,7 @@ import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopology; -import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; +import org.apache.kafka.streams.TopologyConfig; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.internals.SessionStoreBuilder; import org.apache.kafka.streams.state.internals.TimestampedWindowStoreBuilder; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java index 6b3d6794e18d..ea946b2341c1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java @@ -27,7 +27,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; -import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig.TaskConfig; +import org.apache.kafka.streams.TopologyConfig.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import java.util.Collections; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index f86e89f73ff6..4185fcff47c4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -43,7 +43,7 @@ import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; -import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig.TaskConfig; +import org.apache.kafka.streams.TopologyConfig.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import java.io.IOException; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java index 91f1768d86fc..3c0710a847c3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java @@ -29,8 +29,8 @@ import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo; +import org.apache.kafka.streams.TopologyConfig.TaskConfig; import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopology; -import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig.TaskConfig; import java.util.ArrayList; import java.util.Collection; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java index a1debbe4b562..208aa9f6e03a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.processor.internals.namedtopology; import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.TopologyConfig; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import java.util.List; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java index ddd9192d53e5..42af69e25f50 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java @@ -18,6 +18,7 @@ import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.TopologyConfig; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import java.util.Properties; diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index 05b13a52e50b..e569727ca315 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -52,6 +52,7 @@ import static org.apache.kafka.common.IsolationLevel.READ_UNCOMMITTED; import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.streams.StreamsConfig.AT_LEAST_ONCE; +import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_CONFIG; import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE; import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE_BETA; import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE_V2; @@ -1013,7 +1014,7 @@ public void shouldSpecifyNoOptimizationWhenNotExplicitlyAddedToConfigs() { } @Test - public void shouldSpecifyOptimizationWhenNotExplicitlyAddedToConfigs() { + public void shouldSpecifyOptimizationWhenExplicitlyAddedToConfigs() { final String expectedOptimizeConfig = "all"; props.put(TOPOLOGY_OPTIMIZATION_CONFIG, "all"); final StreamsConfig config = new StreamsConfig(props); @@ -1027,6 +1028,28 @@ public void shouldThrowConfigExceptionWhenOptimizationConfigNotValueInRange() { assertThrows(ConfigException.class, () -> new StreamsConfig(props)); } + @Test + public void shouldSpecifyRocksdbWhenNotExplicitlyAddedToConfigs() { + final String expectedDefaultStoreType = StreamsConfig.ROCKS_DB; + final String actualDefaultStoreType = streamsConfig.getString(DEFAULT_DSL_STORE_CONFIG); + assertEquals("default.dsl.store should be \"rocksDB\"", expectedDefaultStoreType, actualDefaultStoreType); + } + + @Test + public void shouldSpecifyInMemoryWhenExplicitlyAddedToConfigs() { + final String expectedDefaultStoreType = StreamsConfig.IN_MEMORY; + props.put(DEFAULT_DSL_STORE_CONFIG, expectedDefaultStoreType); + final StreamsConfig config = new StreamsConfig(props); + final String actualDefaultStoreType = config.getString(DEFAULT_DSL_STORE_CONFIG); + assertEquals("default.dsl.store should be \"in_memory\"", expectedDefaultStoreType, actualDefaultStoreType); + } + + @Test + public void shouldThrowConfigExceptionWhenStoreTypeConfigNotValueInRange() { + props.put(DEFAULT_DSL_STORE_CONFIG, "bad_config"); + assertThrows(ConfigException.class, () -> new StreamsConfig(props)); + } + @SuppressWarnings("deprecation") @Test public void shouldLogWarningWhenEosAlphaIsUsed() { diff --git a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java index b332f6c3530b..726d8b29a581 100644 --- a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.SessionWindows; +import org.apache.kafka.streams.kstream.SlidingWindows; import org.apache.kafka.streams.kstream.StreamJoined; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.processor.RecordContext; @@ -35,15 +36,19 @@ import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.SubtopologyDescription; +import org.apache.kafka.streams.processor.internals.ProcessorTopology; import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.SessionStore; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.WindowBytesStoreSupplier; +import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; import org.apache.kafka.test.MockApiProcessorSupplier; import org.apache.kafka.test.MockKeyValueStore; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; +import org.apache.kafka.test.StreamsTestUtils; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; @@ -58,6 +63,7 @@ import static java.time.Duration.ofMillis; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; @@ -82,7 +88,7 @@ public void shouldNotAllowNullNameWhenAddingSourceWithPattern() { } @Test - public void shouldNotAllowNullTopicsWhenAddingSoureWithTopic() { + public void shouldNotAllowNullTopicsWhenAddingSourceWithTopic() { assertThrows(NullPointerException.class, () -> topology.addSource("source", (String[]) null)); } @@ -1153,7 +1159,9 @@ public void kGroupedStreamZeroArgCountShouldPreserveTopologyStructure() { builder.stream("input-topic") .groupByKey() .count(); - final TopologyDescription describe = builder.build().describe(); + final Topology topology = builder.build(); + + final TopologyDescription describe = topology.describe(); assertEquals( "Topologies:\n" + " Sub-topology: 0\n" + @@ -1164,6 +1172,8 @@ public void kGroupedStreamZeroArgCountShouldPreserveTopologyStructure() { " <-- KSTREAM-SOURCE-0000000000\n\n", describe.toString() ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(true)); } @Test @@ -1171,8 +1181,10 @@ public void kGroupedStreamNamedMaterializedCountShouldPreserveTopologyStructure( final StreamsBuilder builder = new StreamsBuilder(); builder.stream("input-topic") .groupByKey() - .count(Materialized.as("count-store")); - final TopologyDescription describe = builder.build().describe(); + .count(Materialized.>as("count-store") + .withStoreType(Materialized.StoreType.IN_MEMORY)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); assertEquals( "Topologies:\n" + " Sub-topology: 0\n" + @@ -1183,6 +1195,8 @@ public void kGroupedStreamNamedMaterializedCountShouldPreserveTopologyStructure( " <-- KSTREAM-SOURCE-0000000000\n\n", describe.toString() ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); } @Test @@ -1190,8 +1204,33 @@ public void kGroupedStreamAnonymousMaterializedCountShouldPreserveTopologyStruct final StreamsBuilder builder = new StreamsBuilder(); builder.stream("input-topic") .groupByKey() - .count(Materialized.with(null, Serdes.Long())); - final TopologyDescription describe = builder.build().describe(); + .count(Materialized.>with(null, Serdes.Long()) + // set store type explicitly with default rocksDB + .withStoreType(Materialized.StoreType.ROCKS_DB)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> KSTREAM-AGGREGATE-0000000003\n" + + " Processor: KSTREAM-AGGREGATE-0000000003 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000002])\n" + + " --> none\n" + + " <-- KSTREAM-SOURCE-0000000000\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(true)); + } + + @Test + public void kGroupedStreamAnonymousStoreTypedMaterializedCountShouldPreserveTopologyStructure() { + final StreamsBuilder builder = new StreamsBuilder(); + builder.stream("input-topic") + .groupByKey() + .count(Materialized.as(Materialized.StoreType.IN_MEMORY)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); assertEquals( "Topologies:\n" + " Sub-topology: 0\n" + @@ -1202,6 +1241,32 @@ public void kGroupedStreamAnonymousMaterializedCountShouldPreserveTopologyStruct " <-- KSTREAM-SOURCE-0000000000\n\n", describe.toString() ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); + } + + @Test + public void kGroupedStreamZeroArgCountWithTopologyConfigShouldPreserveTopologyStructure() { + // override the default store into in-memory + final StreamsBuilder builder = new StreamsBuilder(overrideDefaultStore(StreamsConfig.IN_MEMORY)); + builder.stream("input-topic") + .groupByKey() + .count(); + final Topology topology = builder.build(); + + final TopologyDescription describe = topology.describe(); + assertEquals( + "Topology: my-topology:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> KSTREAM-AGGREGATE-0000000002\n" + + " Processor: KSTREAM-AGGREGATE-0000000002 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000001])\n" + + " --> none\n" + + " <-- KSTREAM-SOURCE-0000000000\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); } @Test @@ -1211,7 +1276,8 @@ public void timeWindowZeroArgCountShouldPreserveTopologyStructure() { .groupByKey() .windowedBy(TimeWindows.of(ofMillis(1))) .count(); - final TopologyDescription describe = builder.build().describe(); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); assertEquals( "Topologies:\n" + " Sub-topology: 0\n" + @@ -1222,6 +1288,8 @@ public void timeWindowZeroArgCountShouldPreserveTopologyStructure() { " <-- KSTREAM-SOURCE-0000000000\n\n", describe.toString() ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(true)); } @Test @@ -1230,8 +1298,9 @@ public void timeWindowNamedMaterializedCountShouldPreserveTopologyStructure() { builder.stream("input-topic") .groupByKey() .windowedBy(TimeWindows.of(ofMillis(1))) - .count(Materialized.as("count-store")); - final TopologyDescription describe = builder.build().describe(); + .count(Materialized.>as("count-store").withStoreType(Materialized.StoreType.IN_MEMORY)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); assertEquals( "Topologies:\n" + " Sub-topology: 0\n" + @@ -1242,6 +1311,8 @@ public void timeWindowNamedMaterializedCountShouldPreserveTopologyStructure() { " <-- KSTREAM-SOURCE-0000000000\n\n", describe.toString() ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); } @Test @@ -1250,8 +1321,10 @@ public void timeWindowAnonymousMaterializedCountShouldPreserveTopologyStructure( builder.stream("input-topic") .groupByKey() .windowedBy(TimeWindows.of(ofMillis(1))) - .count(Materialized.with(null, Serdes.Long())); - final TopologyDescription describe = builder.build().describe(); + .count(Materialized.>with(null, Serdes.Long()) + .withStoreType(Materialized.StoreType.ROCKS_DB)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); assertEquals( "Topologies:\n" + " Sub-topology: 0\n" + @@ -1262,6 +1335,381 @@ public void timeWindowAnonymousMaterializedCountShouldPreserveTopologyStructure( " <-- KSTREAM-SOURCE-0000000000\n\n", describe.toString() ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(true)); + } + + @Test + public void timeWindowAnonymousStoreTypeMaterializedCountShouldPreserveTopologyStructure() { + final StreamsBuilder builder = new StreamsBuilder(); + builder.stream("input-topic") + .groupByKey() + .windowedBy(TimeWindows.of(ofMillis(1))) + .count(Materialized.as(Materialized.StoreType.IN_MEMORY)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> KSTREAM-AGGREGATE-0000000003\n" + + " Processor: KSTREAM-AGGREGATE-0000000003 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000002])\n" + + " --> none\n" + + " <-- KSTREAM-SOURCE-0000000000\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); + } + + @Test + public void timeWindowZeroArgCountWithTopologyConfigShouldPreserveTopologyStructure() { + // override the default store into in-memory + final StreamsBuilder builder = new StreamsBuilder(overrideDefaultStore(StreamsConfig.IN_MEMORY)); + builder.stream("input-topic") + .groupByKey() + .windowedBy(TimeWindows.of(ofMillis(1))) + .count(); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + assertEquals( + "Topology: my-topology:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> KSTREAM-AGGREGATE-0000000002\n" + + " Processor: KSTREAM-AGGREGATE-0000000002 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000001])\n" + + " --> none\n" + + " <-- KSTREAM-SOURCE-0000000000\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); + } + + @Test + public void slidingWindowZeroArgCountShouldPreserveTopologyStructure() { + final StreamsBuilder builder = new StreamsBuilder(); + builder.stream("input-topic") + .groupByKey() + .windowedBy(TimeWindows.of(ofMillis(1))) + .count(); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> KSTREAM-AGGREGATE-0000000002\n" + + " Processor: KSTREAM-AGGREGATE-0000000002 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000001])\n" + + " --> none\n" + + " <-- KSTREAM-SOURCE-0000000000\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(true)); + } + + @Test + public void slidingWindowNamedMaterializedCountShouldPreserveTopologyStructure() { + final StreamsBuilder builder = new StreamsBuilder(); + builder.stream("input-topic") + .groupByKey() + .windowedBy(TimeWindows.of(ofMillis(1))) + .count(Materialized.>as("count-store").withStoreType(Materialized.StoreType.IN_MEMORY)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> KSTREAM-AGGREGATE-0000000001\n" + + " Processor: KSTREAM-AGGREGATE-0000000001 (stores: [count-store])\n" + + " --> none\n" + + " <-- KSTREAM-SOURCE-0000000000\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); + } + + @Test + public void slidingWindowZeroArgCountWithTopologyConfigShouldPreserveTopologyStructure() { + // override the default store into in-memory + final StreamsBuilder builder = new StreamsBuilder(overrideDefaultStore(StreamsConfig.IN_MEMORY)); + builder.stream("input-topic") + .groupByKey() + .windowedBy(SlidingWindows.ofTimeDifferenceWithNoGrace(ofMillis(1))) + .count(); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + assertEquals( + "Topology: my-topology:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> KSTREAM-AGGREGATE-0000000002\n" + + " Processor: KSTREAM-AGGREGATE-0000000002 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000001])\n" + + " --> none\n" + + " <-- KSTREAM-SOURCE-0000000000\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); + } + + @Test + public void timeWindowedCogroupedZeroArgCountShouldPreserveTopologyStructure() { + final StreamsBuilder builder = new StreamsBuilder(); + builder.stream("input-topic") + .groupByKey() + .cogroup((key, value, aggregate) -> value) + .windowedBy(TimeWindows.ofSizeWithNoGrace(ofMillis(1))) + .aggregate(() -> ""); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> COGROUPKSTREAM-AGGREGATE-0000000002\n" + + " Processor: COGROUPKSTREAM-AGGREGATE-0000000002 (stores: [COGROUPKSTREAM-AGGREGATE-STATE-STORE-0000000001])\n" + + " --> COGROUPKSTREAM-MERGE-0000000003\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: COGROUPKSTREAM-MERGE-0000000003 (stores: [])\n" + + " --> none\n" + + " <-- COGROUPKSTREAM-AGGREGATE-0000000002\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(true)); + } + + @Test + public void timeWindowedCogroupedNamedMaterializedCountShouldPreserveTopologyStructure() { + final StreamsBuilder builder = new StreamsBuilder(); + builder.stream("input-topic") + .groupByKey() + .cogroup((key, value, aggregate) -> value) + .windowedBy(TimeWindows.ofSizeWithNoGrace(ofMillis(1))) + .aggregate(() -> "", Materialized.>as("aggregate-store") + .withStoreType(Materialized.StoreType.IN_MEMORY)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> COGROUPKSTREAM-AGGREGATE-0000000001\n" + + " Processor: COGROUPKSTREAM-AGGREGATE-0000000001 (stores: [aggregate-store])\n" + + " --> COGROUPKSTREAM-MERGE-0000000002\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: COGROUPKSTREAM-MERGE-0000000002 (stores: [])\n" + + " --> none\n" + + " <-- COGROUPKSTREAM-AGGREGATE-0000000001\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); + } + + @Test + public void timeWindowedCogroupedZeroArgCountWithTopologyConfigShouldPreserveTopologyStructure() { + // override the default store into in-memory + final StreamsBuilder builder = new StreamsBuilder(overrideDefaultStore(StreamsConfig.IN_MEMORY)); + builder.stream("input-topic") + .groupByKey() + .cogroup((key, value, aggregate) -> value) + .windowedBy(TimeWindows.ofSizeWithNoGrace(ofMillis(1))) + .aggregate(() -> ""); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + + assertEquals( + "Topology: my-topology:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> COGROUPKSTREAM-AGGREGATE-0000000002\n" + + " Processor: COGROUPKSTREAM-AGGREGATE-0000000002 (stores: [COGROUPKSTREAM-AGGREGATE-STATE-STORE-0000000001])\n" + + " --> COGROUPKSTREAM-MERGE-0000000003\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: COGROUPKSTREAM-MERGE-0000000003 (stores: [])\n" + + " --> none\n" + + " <-- COGROUPKSTREAM-AGGREGATE-0000000002\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); + } + + @Test + public void slidingWindowedCogroupedZeroArgCountShouldPreserveTopologyStructure() { + final StreamsBuilder builder = new StreamsBuilder(); + builder.stream("input-topic") + .groupByKey() + .cogroup((key, value, aggregate) -> value) + .windowedBy(SlidingWindows.ofTimeDifferenceWithNoGrace(ofMillis(1))) + .aggregate(() -> ""); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> COGROUPKSTREAM-AGGREGATE-0000000002\n" + + " Processor: COGROUPKSTREAM-AGGREGATE-0000000002 (stores: [COGROUPKSTREAM-AGGREGATE-STATE-STORE-0000000001])\n" + + " --> COGROUPKSTREAM-MERGE-0000000003\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: COGROUPKSTREAM-MERGE-0000000003 (stores: [])\n" + + " --> none\n" + + " <-- COGROUPKSTREAM-AGGREGATE-0000000002\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(true)); + } + + @Test + public void slidingWindowedCogroupedNamedMaterializedCountShouldPreserveTopologyStructure() { + final StreamsBuilder builder = new StreamsBuilder(); + builder.stream("input-topic") + .groupByKey() + .cogroup((key, value, aggregate) -> value) + .windowedBy(SlidingWindows.ofTimeDifferenceWithNoGrace(ofMillis(1))) + .aggregate(() -> "", Materialized.>as("aggregate-store") + .withStoreType(Materialized.StoreType.IN_MEMORY)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> COGROUPKSTREAM-AGGREGATE-0000000001\n" + + " Processor: COGROUPKSTREAM-AGGREGATE-0000000001 (stores: [aggregate-store])\n" + + " --> COGROUPKSTREAM-MERGE-0000000002\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: COGROUPKSTREAM-MERGE-0000000002 (stores: [])\n" + + " --> none\n" + + " <-- COGROUPKSTREAM-AGGREGATE-0000000001\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); + } + + @Test + public void slidingWindowedCogroupedZeroArgCountWithTopologyConfigShouldPreserveTopologyStructure() { + // override the default store into in-memory + final StreamsBuilder builder = new StreamsBuilder(overrideDefaultStore(StreamsConfig.IN_MEMORY)); + builder.stream("input-topic") + .groupByKey() + .cogroup((key, value, aggregate) -> value) + .windowedBy(SlidingWindows.ofTimeDifferenceWithNoGrace(ofMillis(1))) + .aggregate(() -> ""); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + assertEquals( + "Topology: my-topology:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> COGROUPKSTREAM-AGGREGATE-0000000002\n" + + " Processor: COGROUPKSTREAM-AGGREGATE-0000000002 (stores: [COGROUPKSTREAM-AGGREGATE-STATE-STORE-0000000001])\n" + + " --> COGROUPKSTREAM-MERGE-0000000003\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: COGROUPKSTREAM-MERGE-0000000003 (stores: [])\n" + + " --> none\n" + + " <-- COGROUPKSTREAM-AGGREGATE-0000000002\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); + } + + @Test + public void sessionWindowedCogroupedZeroArgCountShouldPreserveTopologyStructure() { + final StreamsBuilder builder = new StreamsBuilder(); + builder.stream("input-topic") + .groupByKey() + .cogroup((key, value, aggregate) -> value) + .windowedBy(SessionWindows.ofInactivityGapWithNoGrace(ofMillis(1))) + .aggregate(() -> "", (aggKey, aggOne, aggTwo) -> ""); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> COGROUPKSTREAM-AGGREGATE-0000000002\n" + + " Processor: COGROUPKSTREAM-AGGREGATE-0000000002 (stores: [COGROUPKSTREAM-AGGREGATE-STATE-STORE-0000000001])\n" + + " --> COGROUPKSTREAM-MERGE-0000000003\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: COGROUPKSTREAM-MERGE-0000000003 (stores: [])\n" + + " --> none\n" + + " <-- COGROUPKSTREAM-AGGREGATE-0000000002\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(true)); + } + + @Test + public void sessionWindowedCogroupedNamedMaterializedCountShouldPreserveTopologyStructure() { + final StreamsBuilder builder = new StreamsBuilder(); + builder.stream("input-topic") + .groupByKey() + .cogroup((key, value, aggregate) -> value) + .windowedBy(SessionWindows.ofInactivityGapWithNoGrace(ofMillis(1))) + .aggregate(() -> "", (aggKey, aggOne, aggTwo) -> "", Materialized.>as("aggregate-store") + .withStoreType(Materialized.StoreType.IN_MEMORY)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> COGROUPKSTREAM-AGGREGATE-0000000001\n" + + " Processor: COGROUPKSTREAM-AGGREGATE-0000000001 (stores: [aggregate-store])\n" + + " --> COGROUPKSTREAM-MERGE-0000000002\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: COGROUPKSTREAM-MERGE-0000000002 (stores: [])\n" + + " --> none\n" + + " <-- COGROUPKSTREAM-AGGREGATE-0000000001\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); + } + + @Test + public void sessionWindowedCogroupedZeroArgCountWithTopologyConfigShouldPreserveTopologyStructure() { + // override the default store into in-memory + final StreamsBuilder builder = new StreamsBuilder(overrideDefaultStore(StreamsConfig.IN_MEMORY)); + builder.stream("input-topic") + .groupByKey() + .cogroup((key, value, aggregate) -> value) + .windowedBy(SessionWindows.ofInactivityGapWithNoGrace(ofMillis(1))) + .aggregate(() -> "", (aggKey, aggOne, aggTwo) -> ""); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + assertEquals( + "Topology: my-topology:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> COGROUPKSTREAM-AGGREGATE-0000000002\n" + + " Processor: COGROUPKSTREAM-AGGREGATE-0000000002 (stores: [COGROUPKSTREAM-AGGREGATE-STATE-STORE-0000000001])\n" + + " --> COGROUPKSTREAM-MERGE-0000000003\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: COGROUPKSTREAM-MERGE-0000000003 (stores: [])\n" + + " --> none\n" + + " <-- COGROUPKSTREAM-AGGREGATE-0000000002\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); } @Test @@ -1271,7 +1719,8 @@ public void sessionWindowZeroArgCountShouldPreserveTopologyStructure() { .groupByKey() .windowedBy(SessionWindows.with(ofMillis(1))) .count(); - final TopologyDescription describe = builder.build().describe(); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); assertEquals( "Topologies:\n" + " Sub-topology: 0\n" + @@ -1282,6 +1731,8 @@ public void sessionWindowZeroArgCountShouldPreserveTopologyStructure() { " <-- KSTREAM-SOURCE-0000000000\n\n", describe.toString() ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(true)); } @Test @@ -1290,8 +1741,10 @@ public void sessionWindowNamedMaterializedCountShouldPreserveTopologyStructure() builder.stream("input-topic") .groupByKey() .windowedBy(SessionWindows.with(ofMillis(1))) - .count(Materialized.as("count-store")); - final TopologyDescription describe = builder.build().describe(); + .count(Materialized.>as("count-store") + .withStoreType(Materialized.StoreType.IN_MEMORY)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); assertEquals( "Topologies:\n" + " Sub-topology: 0\n" + @@ -1302,6 +1755,8 @@ public void sessionWindowNamedMaterializedCountShouldPreserveTopologyStructure() " <-- KSTREAM-SOURCE-0000000000\n\n", describe.toString() ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); } @Test @@ -1310,8 +1765,33 @@ public void sessionWindowAnonymousMaterializedCountShouldPreserveTopologyStructu builder.stream("input-topic") .groupByKey() .windowedBy(SessionWindows.with(ofMillis(1))) - .count(Materialized.with(null, Serdes.Long())); - final TopologyDescription describe = builder.build().describe(); + .count(Materialized.>with(null, Serdes.Long()) + .withStoreType(Materialized.StoreType.ROCKS_DB)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> KSTREAM-AGGREGATE-0000000003\n" + + " Processor: KSTREAM-AGGREGATE-0000000003 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000002])\n" + + " --> none\n" + + " <-- KSTREAM-SOURCE-0000000000\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(true)); + } + + @Test + public void sessionWindowAnonymousStoreTypedMaterializedCountShouldPreserveTopologyStructure() { + final StreamsBuilder builder = new StreamsBuilder(); + builder.stream("input-topic") + .groupByKey() + .windowedBy(SessionWindows.with(ofMillis(1))) + .count(Materialized.as(Materialized.StoreType.IN_MEMORY)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); assertEquals( "Topologies:\n" + " Sub-topology: 0\n" + @@ -1322,6 +1802,32 @@ public void sessionWindowAnonymousMaterializedCountShouldPreserveTopologyStructu " <-- KSTREAM-SOURCE-0000000000\n\n", describe.toString() ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); + } + + @Test + public void sessionWindowZeroArgCountWithTopologyConfigShouldPreserveTopologyStructure() { + // override the default store into in-memory + final StreamsBuilder builder = new StreamsBuilder(overrideDefaultStore(StreamsConfig.IN_MEMORY)); + builder.stream("input-topic") + .groupByKey() + .windowedBy(SessionWindows.with(ofMillis(1))) + .count(); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + assertEquals( + "Topology: my-topology:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic])\n" + + " --> KSTREAM-AGGREGATE-0000000002\n" + + " Processor: KSTREAM-AGGREGATE-0000000002 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000001])\n" + + " --> none\n" + + " <-- KSTREAM-SOURCE-0000000000\n\n", + describe.toString() + ); + + assertThat(topology.internalTopologyBuilder.setApplicationId("test").buildTopology().hasPersistentLocalStore(), is(false)); } @Test @@ -1330,8 +1836,8 @@ public void tableZeroArgCountShouldPreserveTopologyStructure() { builder.table("input-topic") .groupBy((key, value) -> null) .count(); - final TopologyDescription describe = builder.build().describe(); - + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); assertEquals( "Topologies:\n" + " Sub-topology: 0\n" + @@ -1355,6 +1861,14 @@ public void tableZeroArgCountShouldPreserveTopologyStructure() { "\n", describe.toString() ); + + final ProcessorTopology processorTopology = topology.internalTopologyBuilder.setApplicationId("test").buildTopology(); + // one for ktable, and one for count operation + assertThat(processorTopology.stateStores().size(), is(2)); + // ktable store is rocksDB (default) + assertThat(processorTopology.stateStores().get(0).persistent(), is(true)); + // count store is rocksDB (default) + assertThat(processorTopology.stateStores().get(1).persistent(), is(true)); } @Test @@ -1362,8 +1876,10 @@ public void tableNamedMaterializedCountShouldPreserveTopologyStructure() { final StreamsBuilder builder = new StreamsBuilder(); builder.table("input-topic") .groupBy((key, value) -> null) - .count(Materialized.as("count-store")); - final TopologyDescription describe = builder.build().describe(); + .count(Materialized.>as("count-store") + .withStoreType(Materialized.StoreType.IN_MEMORY)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); assertEquals( "Topologies:\n" + " Sub-topology: 0\n" + @@ -1387,6 +1903,57 @@ public void tableNamedMaterializedCountShouldPreserveTopologyStructure() { "\n", describe.toString() ); + + final ProcessorTopology processorTopology = topology.internalTopologyBuilder.setApplicationId("test").buildTopology(); + // one for ktable, and one for count operation + assertThat(processorTopology.stateStores().size(), is(2)); + // ktable store is rocksDB (default) + assertThat(processorTopology.stateStores().get(0).persistent(), is(true)); + // count store is in-memory + assertThat(processorTopology.stateStores().get(1).persistent(), is(false)); + } + + @Test + public void tableNamedMaterializedCountWithTopologyConfigShouldPreserveTopologyStructure() { + // override the default store into in-memory + final StreamsBuilder builder = new StreamsBuilder(overrideDefaultStore(StreamsConfig.IN_MEMORY)); + builder.table("input-topic") + .groupBy((key, value) -> null) + // can still override the default store dynamically + .count(Materialized.as(Materialized.StoreType.ROCKS_DB)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + assertEquals( + "Topology: my-topology:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000001 (topics: [input-topic])\n" + + " --> KTABLE-SOURCE-0000000002\n" + + " Processor: KTABLE-SOURCE-0000000002 (stores: [input-topic-STATE-STORE-0000000000])\n" + + " --> KTABLE-SELECT-0000000003\n" + + " <-- KSTREAM-SOURCE-0000000001\n" + + " Processor: KTABLE-SELECT-0000000003 (stores: [])\n" + + " --> KSTREAM-SINK-0000000005\n" + + " <-- KTABLE-SOURCE-0000000002\n" + + " Sink: KSTREAM-SINK-0000000005 (topic: KTABLE-AGGREGATE-STATE-STORE-0000000004-repartition)\n" + + " <-- KTABLE-SELECT-0000000003\n" + + "\n" + + " Sub-topology: 1\n" + + " Source: KSTREAM-SOURCE-0000000006 (topics: [KTABLE-AGGREGATE-STATE-STORE-0000000004-repartition])\n" + + " --> KTABLE-AGGREGATE-0000000007\n" + + " Processor: KTABLE-AGGREGATE-0000000007 (stores: [KTABLE-AGGREGATE-STATE-STORE-0000000004])\n" + + " --> none\n" + + " <-- KSTREAM-SOURCE-0000000006\n" + + "\n", + describe.toString() + ); + + final ProcessorTopology processorTopology = topology.internalTopologyBuilder.setApplicationId("test").buildTopology(); + // one for ktable, and one for count operation + assertThat(processorTopology.stateStores().size(), is(2)); + // ktable store is in-memory (default is in-memory) + assertThat(processorTopology.stateStores().get(0).persistent(), is(false)); + // count store is rocksDB + assertThat(processorTopology.stateStores().get(1).persistent(), is(true)); } @Test @@ -1394,8 +1961,10 @@ public void tableAnonymousMaterializedCountShouldPreserveTopologyStructure() { final StreamsBuilder builder = new StreamsBuilder(); builder.table("input-topic") .groupBy((key, value) -> null) - .count(Materialized.with(null, Serdes.Long())); - final TopologyDescription describe = builder.build().describe(); + .count(Materialized.>with(null, Serdes.Long()) + .withStoreType(Materialized.StoreType.ROCKS_DB)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); assertEquals( "Topologies:\n" + " Sub-topology: 0\n" + @@ -1419,6 +1988,55 @@ public void tableAnonymousMaterializedCountShouldPreserveTopologyStructure() { "\n", describe.toString() ); + + final ProcessorTopology processorTopology = topology.internalTopologyBuilder.setApplicationId("test").buildTopology(); + // one for ktable, and one for count operation + assertThat(processorTopology.stateStores().size(), is(2)); + // ktable store is rocksDB (default) + assertThat(processorTopology.stateStores().get(0).persistent(), is(true)); + // count store is rocksDB + assertThat(processorTopology.stateStores().get(1).persistent(), is(true)); + } + + @Test + public void tableAnonymousStoreTypedMaterializedCountShouldPreserveTopologyStructure() { + final StreamsBuilder builder = new StreamsBuilder(); + builder.table("input-topic") + .groupBy((key, value) -> null) + .count(Materialized.as(Materialized.StoreType.IN_MEMORY)); + final Topology topology = builder.build(); + final TopologyDescription describe = topology.describe(); + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000001 (topics: [input-topic])\n" + + " --> KTABLE-SOURCE-0000000002\n" + + " Processor: KTABLE-SOURCE-0000000002 (stores: [input-topic-STATE-STORE-0000000000])\n" + + " --> KTABLE-SELECT-0000000003\n" + + " <-- KSTREAM-SOURCE-0000000001\n" + + " Processor: KTABLE-SELECT-0000000003 (stores: [])\n" + + " --> KSTREAM-SINK-0000000005\n" + + " <-- KTABLE-SOURCE-0000000002\n" + + " Sink: KSTREAM-SINK-0000000005 (topic: KTABLE-AGGREGATE-STATE-STORE-0000000004-repartition)\n" + + " <-- KTABLE-SELECT-0000000003\n" + + "\n" + + " Sub-topology: 1\n" + + " Source: KSTREAM-SOURCE-0000000006 (topics: [KTABLE-AGGREGATE-STATE-STORE-0000000004-repartition])\n" + + " --> KTABLE-AGGREGATE-0000000007\n" + + " Processor: KTABLE-AGGREGATE-0000000007 (stores: [KTABLE-AGGREGATE-STATE-STORE-0000000004])\n" + + " --> none\n" + + " <-- KSTREAM-SOURCE-0000000006\n" + + "\n", + describe.toString() + ); + + final ProcessorTopology processorTopology = topology.internalTopologyBuilder.setApplicationId("test").buildTopology(); + // one for ktable, and one for count operation + assertThat(processorTopology.stateStores().size(), is(2)); + // ktable store is rocksDB (default) + assertThat(processorTopology.stateStores().get(0).persistent(), is(true)); + // count store is in-memory + assertThat(processorTopology.stateStores().get(1).persistent(), is(false)); } @Test @@ -1447,7 +2065,8 @@ public void kTableAnonymousMaterializedMapValuesShouldPreserveTopologyStructure( final KTable table = builder.table("input-topic"); table.mapValues( (readOnlyKey, value) -> null, - Materialized.with(null, null)); + Materialized.>with(null, null) + .withStoreType(Materialized.StoreType.IN_MEMORY)); final TopologyDescription describe = builder.build().describe(); Assert.assertEquals( "Topologies:\n" + @@ -1682,4 +2301,16 @@ private void addGlobalStoreToTopologyAndExpectedDescription(final String globalS expectedDescription.addGlobalStore(expectedGlobalStore); } + + private TopologyConfig overrideDefaultStore(final String defaultStore) { + final Properties topologyOverrides = new Properties(); + // change default store as in-memory + topologyOverrides.put(StreamsConfig.DEFAULT_DSL_STORE_CONFIG, defaultStore); + final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); + + return new TopologyConfig( + "my-topology", + config, + topologyOverrides); + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/MaterializedTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/MaterializedTest.java index 2630f35b75a6..0a00cfbf9407 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/MaterializedTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/MaterializedTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; import org.apache.kafka.streams.state.SessionBytesStoreSupplier; +import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.WindowBytesStoreSupplier; import org.junit.Test; @@ -64,6 +65,14 @@ public void shouldThrowNullPointerIfKeyValueBytesStoreSupplierIsNull() { assertEquals(e.getMessage(), "supplier can't be null"); } + @Test + public void shouldThrowNullPointerIfStoreTypeIsNull() { + final NullPointerException e = assertThrows(NullPointerException.class, + () -> Materialized.as((Materialized.StoreType) null)); + + assertEquals(e.getMessage(), "store type can't be null"); + } + @Test public void shouldThrowNullPointerIfSessionBytesStoreSupplierIsNull() { final NullPointerException e = assertThrows(NullPointerException.class, @@ -80,6 +89,14 @@ public void shouldThrowIllegalArgumentExceptionIfRetentionIsNegative() { assertEquals(e.getMessage(), "Retention must not be negative."); } + @Test + public void shouldThrowIllegalArgumentExceptionIfStoreSupplierAndStoreTypeBothSet() { + final IllegalArgumentException e = assertThrows(IllegalArgumentException.class, + () -> Materialized.as(Stores.persistentKeyValueStore("test")).withStoreType(Materialized.StoreType.ROCKS_DB)); + + assertEquals(e.getMessage(), "Cannot set store type when store supplier is pre-configured."); + } + @Test public void shouldThrowTopologyExceptionIfStoreNameExceedsMaxAllowedLength() { final StringBuffer invalidStoreNameBuffer = new StringBuffer(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java index 5025e7f4c576..354fbcac3189 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java @@ -486,7 +486,7 @@ public void shouldNotAcceptInvalidStoreNameWhenReducingSessionWindows() { public void shouldNotAcceptNullStateStoreSupplierWhenReducingSessionWindows() { assertThrows(NullPointerException.class, () -> groupedStream .windowedBy(SessionWindows.ofInactivityGapWithNoGrace(ofMillis(30))) - .reduce(null, Materialized.>as(null)) + .reduce(null, Materialized.>as((String) null)) ); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/MaterializedInternalTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/MaterializedInternalTest.java index 5d5e8884d397..302845a97399 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/MaterializedInternalTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/MaterializedInternalTest.java @@ -18,10 +18,14 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; +import org.apache.kafka.streams.TopologyConfig; import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.test.StreamsTestUtils; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; import org.easymock.Mock; @@ -29,6 +33,8 @@ import org.junit.Test; import org.junit.runner.RunWith; +import java.util.Properties; + import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; @@ -73,4 +79,20 @@ public void shouldUseStoreNameOfSupplierWhenProvided() { new MaterializedInternal<>(Materialized.as(supplier), nameProvider, prefix); assertThat(materialized.storeName(), equalTo(storeName)); } + + @Test + public void shouldUseStoreTypeWhenProvidedViaTopologyConfig() { + final Properties topologyOverrides = new Properties(); + topologyOverrides.put(StreamsConfig.DEFAULT_DSL_STORE_CONFIG, StreamsConfig.IN_MEMORY); + final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); + + final InternalTopologyBuilder topologyBuilder = new InternalTopologyBuilder( + new TopologyConfig("my-topology", config, topologyOverrides)); + + final InternalStreamsBuilder internalStreamsBuilder = new InternalStreamsBuilder(topologyBuilder); + + final MaterializedInternal> materialized = + new MaterializedInternal<>(Materialized.as(supplier), internalStreamsBuilder, prefix); + assertThat(materialized.storeType(), equalTo(Materialized.StoreType.IN_MEMORY)); + } } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java index 4b4ff571673f..e2ef0d16d46e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java @@ -30,7 +30,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; -import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; +import org.apache.kafka.streams.TopologyConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.test.MockClientSupplier; import org.easymock.EasyMockRunner; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index 419a7ae15c4d..f185f1aff800 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -25,12 +25,13 @@ import org.apache.kafka.streams.TopologyDescription; import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler; import org.apache.kafka.streams.errors.TopologyException; +import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TopicNameExtractor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.SubtopologyDescription; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; -import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; +import org.apache.kafka.streams.TopologyConfig; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; @@ -948,6 +949,7 @@ public void shouldOverrideGlobalStreamsConfigWhenGivenNamedTopologyProps() { topologyOverrides.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15); topologyOverrides.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class); topologyOverrides.put(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueExceptionHandler.class); + topologyOverrides.put(StreamsConfig.DEFAULT_DSL_STORE_CONFIG, StreamsConfig.IN_MEMORY); final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); final InternalTopologyBuilder topologyBuilder = new InternalTopologyBuilder( @@ -963,6 +965,7 @@ public void shouldOverrideGlobalStreamsConfigWhenGivenNamedTopologyProps() { assertThat(topologyBuilder.topologyConfigs().getTaskConfig().maxBufferedSize, equalTo(15)); assertThat(topologyBuilder.topologyConfigs().getTaskConfig().timestampExtractor.getClass(), equalTo(MockTimestampExtractor.class)); assertThat(topologyBuilder.topologyConfigs().getTaskConfig().deserializationExceptionHandler.getClass(), equalTo(LogAndContinueExceptionHandler.class)); + assertThat(topologyBuilder.topologyConfigs().parseStoreType(), equalTo(Materialized.StoreType.IN_MEMORY)); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 00daaa6e9d82..9a66f27df3a5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -36,7 +36,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; -import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; +import org.apache.kafka.streams.TopologyConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.test.MockKeyValueStore; import org.apache.kafka.test.MockKeyValueStoreBuilder; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 72dc1bc8e445..71537ae2802a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -59,7 +59,7 @@ import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; -import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; +import org.apache.kafka.streams.TopologyConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.test.MockKeyValueStore; import org.apache.kafka.test.MockProcessorNode; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java index 722e71eed899..5b0479d32502 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java @@ -46,7 +46,7 @@ import org.apache.kafka.streams.processor.internals.StreamsProducer; import org.apache.kafka.streams.processor.internals.Task; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; -import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; +import org.apache.kafka.streams.TopologyConfig; import org.apache.kafka.streams.state.QueryableStoreTypes; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; import org.apache.kafka.streams.state.ReadOnlySessionStore; diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index c9745d8d982d..dba1cbca1132 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -70,7 +70,7 @@ import org.apache.kafka.streams.processor.internals.Task; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; -import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig.TaskConfig; +import org.apache.kafka.streams.TopologyConfig.TaskConfig; import org.apache.kafka.streams.query.Position; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; From 3dacdc5694da5db283524889d2270695defebbaa Mon Sep 17 00:00:00 2001 From: dengziming Date: Fri, 18 Mar 2022 00:30:37 +0800 Subject: [PATCH 019/447] MINOR: Replace EasyMock with Mockito in connect:file (#11471) Reviewers: Mickael Maison , Ismael Juma --- build.gradle | 3 +- .../file/FileStreamSourceConnectorTest.java | 21 ++------- .../file/FileStreamSourceTaskTest.java | 47 +++++++++---------- 3 files changed, 26 insertions(+), 45 deletions(-) diff --git a/build.gradle b/build.gradle index 8e1ae634e253..ff4bab922c5e 100644 --- a/build.gradle +++ b/build.gradle @@ -2517,9 +2517,8 @@ project(':connect:file') { implementation project(':connect:api') implementation libs.slf4jApi - testImplementation libs.easymock testImplementation libs.junitJupiter - testImplementation libs.mockitoInline // supports mocking static methods, final classes, etc. + testImplementation libs.mockitoCore testRuntimeOnly libs.slf4jlog4j testImplementation project(':clients').sourceSets.test.output diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceConnectorTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceConnectorTest.java index 3550d5c8ab12..8e4661d13d15 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceConnectorTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceConnectorTest.java @@ -19,20 +19,19 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.connect.connector.ConnectorContext; -import org.easymock.EasyMock; -import org.easymock.EasyMockSupport; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.mock; import java.util.HashMap; import java.util.List; import java.util.Map; -public class FileStreamSourceConnectorTest extends EasyMockSupport { +public class FileStreamSourceConnectorTest { private static final String SINGLE_TOPIC = "test"; private static final String MULTIPLE_TOPICS = "test1,test2"; @@ -45,7 +44,7 @@ public class FileStreamSourceConnectorTest extends EasyMockSupport { @BeforeEach public void setup() { connector = new FileStreamSourceConnector(); - ctx = createMock(ConnectorContext.class); + ctx = mock(ConnectorContext.class); connector.initialize(ctx); sourceProperties = new HashMap<>(); @@ -55,18 +54,14 @@ public void setup() { @Test public void testConnectorConfigValidation() { - replayAll(); List configValues = connector.config().validate(sourceProperties); for (ConfigValue val : configValues) { assertEquals(0, val.errorMessages().size(), "Config property errors: " + val.errorMessages()); } - verifyAll(); } @Test public void testSourceTasks() { - replayAll(); - connector.start(sourceProperties); List> taskConfigs = connector.taskConfigs(1); assertEquals(1, taskConfigs.size()); @@ -82,21 +77,15 @@ public void testSourceTasks() { taskConfigs.get(0).get(FileStreamSourceConnector.FILE_CONFIG)); assertEquals(SINGLE_TOPIC, taskConfigs.get(0).get(FileStreamSourceConnector.TOPIC_CONFIG)); - - verifyAll(); } @Test public void testSourceTasksStdin() { - EasyMock.replay(ctx); - sourceProperties.remove(FileStreamSourceConnector.FILE_CONFIG); connector.start(sourceProperties); List> taskConfigs = connector.taskConfigs(1); assertEquals(1, taskConfigs.size()); assertNull(taskConfigs.get(0).get(FileStreamSourceConnector.FILE_CONFIG)); - - EasyMock.verify(ctx); } @Test @@ -107,12 +96,8 @@ public void testMultipleSourcesInvalid() { @Test public void testTaskClass() { - EasyMock.replay(ctx); - connector.start(sourceProperties); assertEquals(FileStreamSourceTask.class, connector.taskClass()); - - EasyMock.verify(ctx); } @Test diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java index d02463d75a04..c77f8b0acf3a 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java @@ -19,8 +19,6 @@ import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceTaskContext; import org.apache.kafka.connect.storage.OffsetStorageReader; -import org.easymock.EasyMock; -import org.easymock.EasyMockSupport; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -38,8 +36,12 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; +import static org.mockito.ArgumentMatchers.anyMap; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; -public class FileStreamSourceTaskTest extends EasyMockSupport { +public class FileStreamSourceTaskTest { private static final String TOPIC = "test"; @@ -49,8 +51,6 @@ public class FileStreamSourceTaskTest extends EasyMockSupport { private SourceTaskContext context; private FileStreamSourceTask task; - private boolean verifyMocks = false; - @BeforeEach public void setup() throws IOException { tempFile = File.createTempFile("file-stream-source-task-test", null); @@ -59,28 +59,19 @@ public void setup() throws IOException { config.put(FileStreamSourceConnector.TOPIC_CONFIG, TOPIC); config.put(FileStreamSourceConnector.TASK_BATCH_SIZE_CONFIG, String.valueOf(FileStreamSourceConnector.DEFAULT_TASK_BATCH_SIZE)); task = new FileStreamSourceTask(2); - offsetStorageReader = createMock(OffsetStorageReader.class); - context = createMock(SourceTaskContext.class); + offsetStorageReader = mock(OffsetStorageReader.class); + context = mock(SourceTaskContext.class); task.initialize(context); } @AfterEach public void teardown() { tempFile.delete(); - - if (verifyMocks) - verifyAll(); - } - - private void replay() { - replayAll(); - verifyMocks = true; } @Test public void testNormalLifecycle() throws InterruptedException, IOException { expectOffsetLookupReturnNone(); - replay(); task.start(config); @@ -128,12 +119,13 @@ public void testNormalLifecycle() throws InterruptedException, IOException { os.close(); task.stop(); + + verifyAll(); } @Test public void testBatchSize() throws IOException, InterruptedException { expectOffsetLookupReturnNone(); - replay(); config.put(FileStreamSourceConnector.TASK_BATCH_SIZE_CONFIG, "5000"); task.start(config); @@ -154,13 +146,13 @@ public void testBatchSize() throws IOException, InterruptedException { os.close(); task.stop(); + verifyAll(); } @Test public void testBufferResize() throws IOException, InterruptedException { int batchSize = 1000; expectOffsetLookupReturnNone(); - replay(); config.put(FileStreamSourceConnector.TASK_BATCH_SIZE_CONFIG, Integer.toString(batchSize)); task.start(config); @@ -181,6 +173,8 @@ public void testBufferResize() throws IOException, InterruptedException { writeAndAssertBufferSize(batchSize, os, "9 \n".getBytes(), 2048); os.close(); task.stop(); + + verifyAll(); } private void writeAndAssertBufferSize(int batchSize, OutputStream os, byte[] bytes, int expectBufferSize) @@ -203,9 +197,7 @@ private void writeTimesAndFlush(OutputStream os, int times, byte[] line) throws } @Test - public void testMissingFile() throws InterruptedException { - replay(); - + public void testUsingSystemInputSourceOnMissingFile() throws InterruptedException { String data = "line\n"; System.setIn(new ByteArrayInputStream(data.getBytes())); @@ -220,17 +212,22 @@ public void testMissingFile() throws InterruptedException { task.stop(); } + @Test public void testInvalidFile() throws InterruptedException { config.put(FileStreamSourceConnector.FILE_CONFIG, "bogusfilename"); task.start(config); // Currently the task retries indefinitely if the file isn't found, but shouldn't return any data. - for (int i = 0; i < 100; i++) + for (int i = 0; i < 3; i++) assertNull(task.poll()); } - private void expectOffsetLookupReturnNone() { - EasyMock.expect(context.offsetStorageReader()).andReturn(offsetStorageReader); - EasyMock.expect(offsetStorageReader.offset(EasyMock.>anyObject())).andReturn(null); + when(context.offsetStorageReader()).thenReturn(offsetStorageReader); + when(offsetStorageReader.offset(anyMap())).thenReturn(null); + } + + private void verifyAll() { + verify(context).offsetStorageReader(); + verify(offsetStorageReader).offset(anyMap()); } } From 5cebe12a664d95c6dc5b5aeacd3cee0913ce2c4f Mon Sep 17 00:00:00 2001 From: dengziming Date: Fri, 18 Mar 2022 00:53:37 +0800 Subject: [PATCH 020/447] KAFKA-13509; Support max timestamp in GetOffsetShell (KIP-815) (#11173) This patch implements KIP-815 as described here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-815%3A++Support+max-timestamp+in+GetOffsetShell. Reviewers: Luke Chen , Justine Olshan , David Jacot --- .../scala/kafka/tools/GetOffsetShell.scala | 203 ++++++++--- .../tools/GetOffsetShellParsingTest.scala | 343 ++++++++++-------- .../kafka/tools/GetOffsetShellTest.scala | 65 +++- 3 files changed, 415 insertions(+), 196 deletions(-) diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index dfd5a227689e..d4e81bcaf5c4 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -18,18 +18,18 @@ */ package kafka.tools -import java.util.Properties import joptsimple._ import kafka.utils.{CommandLineUtils, Exit, IncludeList, ToolsUtils} -import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} +import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, ListTopicsOptions, OffsetSpec} +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.requests.ListOffsetsRequest -import org.apache.kafka.common.{PartitionInfo, TopicPartition} -import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.utils.Utils +import java.util.Properties +import java.util.concurrent.ExecutionException import java.util.regex.Pattern -import scala.jdk.CollectionConverters._ import scala.collection.Seq +import scala.jdk.CollectionConverters._ import scala.math.Ordering.Implicits.infixOrderingOps object GetOffsetShell { @@ -45,7 +45,7 @@ object GetOffsetShell { } } - private def fetchOffsets(args: Array[String]): Unit = { + private[tools] def fetchOffsets(args: Array[String]): Unit = { val parser = new OptionParser(false) val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The server(s) to connect to in the form HOST1:PORT1,HOST2:PORT2.") .withRequiredArg @@ -72,10 +72,10 @@ object GetOffsetShell { .ofType(classOf[String]) val timeOpt = parser.accepts("time", "timestamp of the offsets before that. [Note: No offset is returned, if the timestamp greater than recently committed record timestamp is given.]") .withRequiredArg - .describedAs("timestamp/-1(latest)/-2(earliest)") - .ofType(classOf[java.lang.Long]) - .defaultsTo(-1L) - val commandConfigOpt = parser.accepts("command-config", s"Property file containing configs to be passed to Consumer Client.") + .describedAs(" / -1 or latest / -2 or earliest / -3 or max-timestamp") + .ofType(classOf[String]) + .defaultsTo("latest") + val commandConfigOpt = parser.accepts("command-config", s"Property file containing configs to be passed to Admin Client.") .withRequiredArg .describedAs("config file") .ofType(classOf[String]) @@ -103,17 +103,14 @@ object GetOffsetShell { throw new IllegalArgumentException("--topic-partitions cannot be used with --topic or --partitions") } - val listOffsetsTimestamp = options.valueOf(timeOpt).longValue + val offsetSpec = parseOffsetSpec(options.valueOf(timeOpt)) val topicPartitionFilter = if (options.has(topicPartitionsOpt)) { - createTopicPartitionFilterWithPatternList(options.valueOf(topicPartitionsOpt), excludeInternalTopics) + createTopicPartitionFilterWithPatternList(options.valueOf(topicPartitionsOpt)) } else { - val partitionIdsRequested = createPartitionSet(options.valueOf(partitionsOpt)) - createTopicPartitionFilterWithTopicAndPartitionPattern( if (options.has(topicOpt)) Some(options.valueOf(topicOpt)) else None, - excludeInternalTopics, - partitionIdsRequested + options.valueOf(partitionsOpt) ) } @@ -121,41 +118,61 @@ object GetOffsetShell { Utils.loadProps(options.valueOf(commandConfigOpt)) else new Properties - config.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) - config.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, clientId) - val consumer = new KafkaConsumer(config, new ByteArrayDeserializer, new ByteArrayDeserializer) + config.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + config.setProperty(AdminClientConfig.CLIENT_ID_CONFIG, clientId) + val adminClient = Admin.create(config) try { - val partitionInfos = listPartitionInfos(consumer, topicPartitionFilter) + val partitionInfos = listPartitionInfos(adminClient, topicPartitionFilter, excludeInternalTopics) if (partitionInfos.isEmpty) { throw new IllegalArgumentException("Could not match any topic-partitions with the specified filters") } - val topicPartitions = partitionInfos.flatMap { p => - if (p.leader == null) { - System.err.println(s"Error: topic-partition ${p.topic}:${p.partition} does not have a leader. Skip getting offsets") - None - } else - Some(new TopicPartition(p.topic, p.partition)) - } + val timestampsToSearch = partitionInfos.map(tp => tp -> offsetSpec).toMap.asJava - /* Note that the value of the map can be null */ - val partitionOffsets: collection.Map[TopicPartition, java.lang.Long] = listOffsetsTimestamp match { - case ListOffsetsRequest.EARLIEST_TIMESTAMP => consumer.beginningOffsets(topicPartitions.asJava).asScala - case ListOffsetsRequest.LATEST_TIMESTAMP => consumer.endOffsets(topicPartitions.asJava).asScala - case _ => - val timestampsToSearch = topicPartitions.map(tp => tp -> (listOffsetsTimestamp: java.lang.Long)).toMap.asJava - consumer.offsetsForTimes(timestampsToSearch).asScala.map { case (k, x) => - if (x == null) (k, null) else (k, x.offset: java.lang.Long) - } + val listOffsetsResult = adminClient.listOffsets(timestampsToSearch) + val partitionOffsets = partitionInfos.flatMap { tp => + try { + val partitionInfo = listOffsetsResult.partitionResult(tp).get + Some((tp, partitionInfo.offset)) + } catch { + case e: ExecutionException => + e.getCause match { + case cause: KafkaException => + System.err.println(s"Skip getting offsets for topic-partition ${tp.topic}:${tp.partition} due to error: ${cause.getMessage}") + case _ => + throw e + } + None + } } - partitionOffsets.toSeq.sortWith((tp1, tp2) => compareTopicPartitions(tp1._1, tp2._1)).foreach { + partitionOffsets.sortWith((tp1, tp2) => compareTopicPartitions(tp1._1, tp2._1)).foreach { case (tp, offset) => println(s"${tp.topic}:${tp.partition}:${Option(offset).getOrElse("")}") } } finally { - consumer.close() + adminClient.close() + } + } + + private def parseOffsetSpec(listOffsetsTimestamp: String): OffsetSpec = { + listOffsetsTimestamp match { + case "earliest" => OffsetSpec.earliest() + case "latest" => OffsetSpec.latest() + case "max-timestamp" => OffsetSpec.maxTimestamp() + case _ => + try { + listOffsetsTimestamp.toLong match { + case ListOffsetsRequest.EARLIEST_TIMESTAMP => OffsetSpec.earliest() + case ListOffsetsRequest.LATEST_TIMESTAMP => OffsetSpec.latest() + case ListOffsetsRequest.MAX_TIMESTAMP => OffsetSpec.maxTimestamp() + case value => OffsetSpec.forTimestamp(value) + } + } catch { + case e: NumberFormatException => + throw new IllegalArgumentException(s"Malformed time argument $listOffsetsTimestamp, please use -1 or latest / -2 or earliest / -3 or max-timestamp, or a specified long format timestamp", e) + } } } @@ -171,13 +188,15 @@ object GetOffsetShell { * TopicPattern: REGEX * PartitionPattern: NUMBER | NUMBER-(NUMBER)? | -NUMBER */ - def createTopicPartitionFilterWithPatternList(topicPartitions: String, excludeInternalTopics: Boolean): PartitionInfo => Boolean = { + def createTopicPartitionFilterWithPatternList( + topicPartitions: String + ): TopicPartitionFilter = { val ruleSpecs = topicPartitions.split(",") - val rules = ruleSpecs.map(ruleSpec => parseRuleSpec(ruleSpec, excludeInternalTopics)) - tp => rules.exists { rule => rule.apply(tp) } + val rules = ruleSpecs.map(ruleSpec => parseRuleSpec(ruleSpec)) + CompositeTopicPartitionFilter(rules) } - def parseRuleSpec(ruleSpec: String, excludeInternalTopics: Boolean): PartitionInfo => Boolean = { + def parseRuleSpec(ruleSpec: String): TopicPartitionFilter = { val matcher = TopicPartitionPattern.matcher(ruleSpec) if (!matcher.matches()) throw new IllegalArgumentException(s"Invalid rule specification: $ruleSpec") @@ -189,22 +208,29 @@ object GetOffsetShell { val topicFilter = IncludeList(group(1).getOrElse(".*")) val partitionFilter = group(2).map(_.toInt) match { case Some(partition) => - (p: Int) => p == partition + UniquePartitionFilter(partition) case None => val lowerRange = group(3).map(_.toInt).getOrElse(0) val upperRange = group(4).map(_.toInt).getOrElse(Int.MaxValue) - (p: Int) => p >= lowerRange && p < upperRange + PartitionRangeFilter(lowerRange, upperRange) } - - tp => topicFilter.isTopicAllowed(tp.topic, excludeInternalTopics) && partitionFilter(tp.partition) + TopicFilterAndPartitionFilter( + topicFilter, + partitionFilter + ) } /** * Creates a topic-partition filter based on a topic pattern and a set of partition ids. */ - def createTopicPartitionFilterWithTopicAndPartitionPattern(topicOpt: Option[String], excludeInternalTopics: Boolean, partitionIds: Set[Int]): PartitionInfo => Boolean = { - val topicsFilter = IncludeList(topicOpt.getOrElse(".*")) - t => topicsFilter.isTopicAllowed(t.topic, excludeInternalTopics) && (partitionIds.isEmpty || partitionIds.contains(t.partition)) + def createTopicPartitionFilterWithTopicAndPartitionPattern( + topicOpt: Option[String], + partitionIds: String + ): TopicFilterAndPartitionFilter = { + TopicFilterAndPartitionFilter( + IncludeList(topicOpt.getOrElse(".*")), + PartitionsSetFilter(createPartitionSet(partitionIds)) + ) } def createPartitionSet(partitionsString: String): Set[Int] = { @@ -224,9 +250,82 @@ object GetOffsetShell { /** * Return the partition infos. Filter them with topicPartitionFilter. */ - private def listPartitionInfos(consumer: KafkaConsumer[_, _], topicPartitionFilter: PartitionInfo => Boolean): Seq[PartitionInfo] = { - consumer.listTopics.asScala.values.flatMap { partitions => - partitions.asScala.filter(topicPartitionFilter) + private def listPartitionInfos( + client: Admin, + topicPartitionFilter: TopicPartitionFilter, + excludeInternalTopics: Boolean + ): Seq[TopicPartition] = { + val listTopicsOptions = new ListTopicsOptions().listInternal(!excludeInternalTopics) + val topics = client.listTopics(listTopicsOptions).names.get + val filteredTopics = topics.asScala.filter(topicPartitionFilter.isTopicAllowed) + + client.describeTopics(filteredTopics.asJava).allTopicNames.get.asScala.flatMap { case (topic, description) => + description + .partitions + .asScala + .map(tp => new TopicPartition(topic, tp.partition)) + .filter(topicPartitionFilter.isTopicPartitionAllowed) }.toBuffer } } + +trait PartitionFilter { + + /** + * Used to filter partitions based on a certain criteria, for example, a set of partition ids. + */ + def isPartitionAllowed(partition: Int): Boolean +} + +case class PartitionsSetFilter(partitionIds: Set[Int]) extends PartitionFilter { + override def isPartitionAllowed(partition: Int): Boolean = partitionIds.isEmpty || partitionIds.contains(partition) +} + +case class UniquePartitionFilter(partition: Int) extends PartitionFilter { + override def isPartitionAllowed(partition: Int): Boolean = partition == this.partition +} + +case class PartitionRangeFilter(lowerRange: Int, upperRange: Int) extends PartitionFilter { + override def isPartitionAllowed(partition: Int): Boolean = partition >= lowerRange && partition < upperRange +} + +trait TopicPartitionFilter { + + /** + * Used to filter topics based on a certain criteria, for example, a set of topic names or a regular expression. + */ + def isTopicAllowed(topic: String): Boolean + + /** + * Used to filter topic-partitions based on a certain criteria, for example, a topic pattern and a set of partition ids. + */ + def isTopicPartitionAllowed(partition: TopicPartition): Boolean +} + +/** + * Creates a topic-partition filter based on a topic filter and a partition filter + */ +case class TopicFilterAndPartitionFilter( + topicFilter: IncludeList, + partitionFilter: PartitionFilter +) extends TopicPartitionFilter { + + override def isTopicPartitionAllowed(partition: TopicPartition): Boolean = { + isTopicAllowed(partition.topic) && partitionFilter.isPartitionAllowed(partition.partition) + } + + override def isTopicAllowed(topic: String): Boolean = { + topicFilter.isTopicAllowed(topic, false) + } +} + +case class CompositeTopicPartitionFilter(filters: Array[TopicPartitionFilter]) extends TopicPartitionFilter { + + override def isTopicAllowed(topic: String): Boolean = { + filters.exists(_.isTopicAllowed(topic)) + } + + override def isTopicPartitionAllowed(tp: TopicPartition): Boolean = { + filters.exists(_.isTopicPartitionAllowed(tp)) + } +} diff --git a/core/src/test/scala/kafka/tools/GetOffsetShellParsingTest.scala b/core/src/test/scala/kafka/tools/GetOffsetShellParsingTest.scala index edfadea401ec..889631be1933 100644 --- a/core/src/test/scala/kafka/tools/GetOffsetShellParsingTest.scala +++ b/core/src/test/scala/kafka/tools/GetOffsetShellParsingTest.scala @@ -17,191 +17,248 @@ package kafka.tools -import org.apache.kafka.common.PartitionInfo -import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue} +import org.apache.kafka.common.TopicPartition +import org.junit.jupiter.api.Assertions.{assertFalse, assertThrows, assertTrue} import org.junit.jupiter.api.Test -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource class GetOffsetShellParsingTest { - @ParameterizedTest - @ValueSource(booleans = Array(true, false)) - def testTopicPartitionFilterForTopicName(excludeInternal: Boolean): Unit = { - val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList("test", excludeInternal) - assertTrue(filter.apply(partitionInfo("test", 0))) - assertTrue(filter.apply(partitionInfo("test", 1))) - assertFalse(filter.apply(partitionInfo("test1", 0))) - assertFalse(filter.apply(partitionInfo("__consumer_offsets", 0))) - } - - @ParameterizedTest - @ValueSource(booleans = Array(true, false)) - def testTopicPartitionFilterForInternalTopicName(excludeInternal: Boolean): Unit = { - val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList("__consumer_offsets", excludeInternal) - assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 0))) - assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 1))) - assertFalse(filter.apply(partitionInfo("test1", 0))) - assertFalse(filter.apply(partitionInfo("test2", 0))) - } - - @ParameterizedTest - @ValueSource(booleans = Array(true, false)) - def testTopicPartitionFilterForTopicNameList(excludeInternal: Boolean): Unit = { - val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList("test,test1,__consumer_offsets", excludeInternal) - assertTrue(filter.apply(partitionInfo("test", 0))) - assertTrue(filter.apply(partitionInfo("test1", 1))) - assertFalse(filter.apply(partitionInfo("test2", 0))) - - assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 0))) - } - - @ParameterizedTest - @ValueSource(booleans = Array(true, false)) - def testTopicPartitionFilterForRegex(excludeInternal: Boolean): Unit = { - val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList("test.*", excludeInternal) - assertTrue(filter.apply(partitionInfo("test", 0))) - assertTrue(filter.apply(partitionInfo("test1", 1))) - assertTrue(filter.apply(partitionInfo("test2", 0))) - assertFalse(filter.apply(partitionInfo("__consumer_offsets", 0))) - } - - @ParameterizedTest - @ValueSource(booleans = Array(true, false)) - def testTopicPartitionFilterForPartitionIndexSpec(excludeInternal: Boolean): Unit = { - val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":0", excludeInternal) - assertTrue(filter.apply(partitionInfo("test", 0))) - assertTrue(filter.apply(partitionInfo("test1", 0))) - assertFalse(filter.apply(partitionInfo("test2", 1))) - - assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 0))) - assertFalse(filter.apply(partitionInfo("__consumer_offsets", 1))) - } - - @ParameterizedTest - @ValueSource(booleans = Array(true, false)) - def testTopicPartitionFilterForPartitionRangeSpec(excludeInternal: Boolean): Unit = { - val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":1-3", excludeInternal) - assertTrue(filter.apply(partitionInfo("test", 1))) - assertTrue(filter.apply(partitionInfo("test1", 2))) - assertFalse(filter.apply(partitionInfo("test2", 0))) - assertFalse(filter.apply(partitionInfo("test2", 3))) - - assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 2))) - assertFalse(filter.apply(partitionInfo("__consumer_offsets", 3))) - } - - @ParameterizedTest - @ValueSource(booleans = Array(true, false)) - def testTopicPartitionFilterForPartitionLowerBoundSpec(excludeInternal: Boolean): Unit = { - val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":1-", excludeInternal) - assertTrue(filter.apply(partitionInfo("test", 1))) - assertTrue(filter.apply(partitionInfo("test1", 2))) - assertFalse(filter.apply(partitionInfo("test2", 0))) - - assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 2))) - assertFalse(filter.apply(partitionInfo("__consumer_offsets", 0))) - } - - @ParameterizedTest - @ValueSource(booleans = Array(true, false)) - def testTopicPartitionFilterForPartitionUpperBoundSpec(excludeInternal: Boolean): Unit = { - val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":-3", excludeInternal) - assertTrue(filter.apply(partitionInfo("test", 0))) - assertTrue(filter.apply(partitionInfo("test1", 1))) - assertTrue(filter.apply(partitionInfo("test2", 2))) - assertFalse(filter.apply(partitionInfo("test3", 3))) - - assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 2))) - assertFalse(filter.apply(partitionInfo("__consumer_offsets", 3))) - } - - @ParameterizedTest - @ValueSource(booleans = Array(true, false)) - def testTopicPartitionFilterComplex(excludeInternal: Boolean): Unit = { - val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList("test.*:0,__consumer_offsets:1-2,.*:3", excludeInternal) - assertTrue(filter.apply(partitionInfo("test", 0))) - assertTrue(filter.apply(partitionInfo("test", 3))) - assertFalse(filter.apply(partitionInfo("test", 1))) - - assertTrue(filter.apply(partitionInfo("test1", 0))) - assertTrue(filter.apply(partitionInfo("test1", 3))) - assertFalse(filter.apply(partitionInfo("test1", 1))) - assertTrue(filter.apply(partitionInfo("custom", 3))) - assertFalse(filter.apply(partitionInfo("custom", 0))) + @Test + def testTopicPartitionFilterForTopicName(): Unit = { + val topicPartitionFilter = GetOffsetShell.createTopicPartitionFilterWithPatternList("test") + + assertTrue(topicPartitionFilter.isTopicAllowed("test")) + assertFalse(topicPartitionFilter.isTopicAllowed("test1")) + assertFalse(topicPartitionFilter.isTopicAllowed("__consumer_offsets")) + + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 0))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 1))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test1", 0))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 0))) + } + + @Test + def testTopicPartitionFilterForInternalTopicName(): Unit = { + val topicPartitionFilter = GetOffsetShell.createTopicPartitionFilterWithPatternList("__consumer_offsets") + + assertTrue(topicPartitionFilter.isTopicAllowed("__consumer_offsets")) + assertFalse(topicPartitionFilter.isTopicAllowed("test1")) + assertFalse(topicPartitionFilter.isTopicAllowed("test2")) + + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 0))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 1))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test1", 0))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test2", 0))) + } + + @Test + def testTopicPartitionFilterForTopicNameList(): Unit = { + val topicPartitionFilter = GetOffsetShell.createTopicPartitionFilterWithPatternList("test,test1,__consumer_offsets") + + assertTrue(topicPartitionFilter.isTopicAllowed("test")) + assertTrue(topicPartitionFilter.isTopicAllowed("test1")) + assertTrue(topicPartitionFilter.isTopicAllowed("__consumer_offsets")) + assertFalse(topicPartitionFilter.isTopicAllowed("test2")) + + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 0))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test1", 1))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 0))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test2", 0))) + } + + @Test + def testTopicPartitionFilterForRegex(): Unit = { + val topicPartitionFilter = GetOffsetShell.createTopicPartitionFilterWithPatternList("test.*") + + assertTrue(topicPartitionFilter.isTopicAllowed("test")) + assertTrue(topicPartitionFilter.isTopicAllowed("test1")) + assertTrue(topicPartitionFilter.isTopicAllowed("test2")) + assertFalse(topicPartitionFilter.isTopicAllowed("__consumer_offsets")) + + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 0))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test1", 1))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test2", 0))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 0))) + } + + @Test + def testTopicPartitionFilterForPartitionIndexSpec(): Unit = { + val topicPartitionFilter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":0") + + assertTrue(topicPartitionFilter.isTopicAllowed("test")) + assertTrue(topicPartitionFilter.isTopicAllowed("test1")) + assertTrue(topicPartitionFilter.isTopicAllowed("test2")) + assertTrue(topicPartitionFilter.isTopicAllowed("__consumer_offsets")) + + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 0))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test1", 0))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test2", 1))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 0))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 1))) + } + + @Test + def testTopicPartitionFilterForPartitionRangeSpec(): Unit = { + val topicPartitionFilter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":1-3") + + assertTrue(topicPartitionFilter.isTopicAllowed("test")) + assertTrue(topicPartitionFilter.isTopicAllowed("test1")) + assertTrue(topicPartitionFilter.isTopicAllowed("__consumer_offsets")) + assertTrue(topicPartitionFilter.isTopicAllowed("test2")) - assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 1))) - assertEquals(!excludeInternal, filter.apply(partitionInfo("__consumer_offsets", 3))) - assertFalse(filter.apply(partitionInfo("__consumer_offsets", 0))) - assertFalse(filter.apply(partitionInfo("__consumer_offsets", 2))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 1))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test1", 2))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 2))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test2", 0))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test2", 3))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 3))) + } + + @Test + def testTopicPartitionFilterForPartitionLowerBoundSpec(): Unit = { + val topicPartitionFilter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":1-") + + assertTrue(topicPartitionFilter.isTopicAllowed("test")) + assertTrue(topicPartitionFilter.isTopicAllowed("test1")) + assertTrue(topicPartitionFilter.isTopicAllowed("test2")) + assertTrue(topicPartitionFilter.isTopicAllowed("__consumer_offsets")) + + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 1))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test1", 2))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 2))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test2", 0))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 0))) + } + + @Test + def testTopicPartitionFilterForPartitionUpperBoundSpec(): Unit = { + val topicPartitionFilter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":-3") + assertTrue(topicPartitionFilter.isTopicAllowed("test")) + assertTrue(topicPartitionFilter.isTopicAllowed("test1")) + assertTrue(topicPartitionFilter.isTopicAllowed("test2")) + assertTrue(topicPartitionFilter.isTopicAllowed("test3")) + assertTrue(topicPartitionFilter.isTopicAllowed("__consumer_offsets")) + + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 0))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test1", 1))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test2", 2))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 2))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test3", 3))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 3))) + } + + @Test + def testTopicPartitionFilterComplex(): Unit = { + val topicPartitionFilter = GetOffsetShell.createTopicPartitionFilterWithPatternList("test.*:0,__consumer_offsets:1-2,.*:3") + + assertTrue(topicPartitionFilter.isTopicAllowed("test")) + assertTrue(topicPartitionFilter.isTopicAllowed("test1")) + assertTrue(topicPartitionFilter.isTopicAllowed("custom")) + assertTrue(topicPartitionFilter.isTopicAllowed("__consumer_offsets")) + + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 0))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test1", 0))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 1))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test1", 1))) + + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("custom", 3))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("custom", 0))) + + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 1))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 3))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 0))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("__consumer_offsets", 2))) } @Test def testPartitionFilterForSingleIndex(): Unit = { - val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":1", excludeInternalTopics = false) - assertTrue(filter.apply(partitionInfo("test", 1))) - assertFalse(filter.apply(partitionInfo("test", 0))) - assertFalse(filter.apply(partitionInfo("test", 2))) + val topicPartitionFilter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":1") + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 1))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 0))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 2))) } @Test def testPartitionFilterForRange(): Unit = { - val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":1-3", excludeInternalTopics = false) - assertFalse(filter.apply(partitionInfo("test", 0))) - assertTrue(filter.apply(partitionInfo("test", 1))) - assertTrue(filter.apply(partitionInfo("test", 2))) - assertFalse(filter.apply(partitionInfo("test", 3))) - assertFalse(filter.apply(partitionInfo("test", 4))) - assertFalse(filter.apply(partitionInfo("test", 5))) + val topicPartitionFilter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":1-3") + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 0))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 1))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 2))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 3))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 4))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 5))) } @Test def testPartitionFilterForLowerBound(): Unit = { - val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":3-", excludeInternalTopics = false) - assertFalse(filter.apply(partitionInfo("test", 0))) - assertFalse(filter.apply(partitionInfo("test", 1))) - assertFalse(filter.apply(partitionInfo("test", 2))) - assertTrue(filter.apply(partitionInfo("test", 3))) - assertTrue(filter.apply(partitionInfo("test", 4))) - assertTrue(filter.apply(partitionInfo("test", 5))) + val topicPartitionFilter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":3-") + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 0))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 1))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 2))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 3))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 4))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 5))) } @Test def testPartitionFilterForUpperBound(): Unit = { - val filter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":-3", excludeInternalTopics = false) - assertTrue(filter.apply(partitionInfo("test", 0))) - assertTrue(filter.apply(partitionInfo("test", 1))) - assertTrue(filter.apply(partitionInfo("test", 2))) - assertFalse(filter.apply(partitionInfo("test", 3))) - assertFalse(filter.apply(partitionInfo("test", 4))) - assertFalse(filter.apply(partitionInfo("test", 5))) + val topicPartitionFilter = GetOffsetShell.createTopicPartitionFilterWithPatternList(":-3") + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 0))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 1))) + assertTrue(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 2))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 3))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 4))) + assertFalse(topicPartitionFilter.isTopicPartitionAllowed(topicPartition("test", 5))) + } + + @Test + def testPartitionsSetFilter(): Unit = { + val partitionsSetFilter = GetOffsetShell.createTopicPartitionFilterWithTopicAndPartitionPattern(Some("topic"), "1,3,5") + + assertFalse(partitionsSetFilter.isTopicPartitionAllowed(topicPartition("topic", 0))) + assertFalse(partitionsSetFilter.isTopicPartitionAllowed(topicPartition("topic", 2))) + assertFalse(partitionsSetFilter.isTopicPartitionAllowed(topicPartition("topic", 4))) + + assertFalse(partitionsSetFilter.isTopicPartitionAllowed(topicPartition("topic1", 1))) + assertFalse(partitionsSetFilter.isTopicAllowed("topic1")) + + assertTrue(partitionsSetFilter.isTopicPartitionAllowed(topicPartition("topic", 1))) + assertTrue(partitionsSetFilter.isTopicPartitionAllowed(topicPartition("topic", 3))) + assertTrue(partitionsSetFilter.isTopicPartitionAllowed(topicPartition("topic", 5))) + assertTrue(partitionsSetFilter.isTopicAllowed("topic")) } @Test def testPartitionFilterForInvalidSingleIndex(): Unit = { assertThrows(classOf[IllegalArgumentException], - () => GetOffsetShell.createTopicPartitionFilterWithPatternList(":a", excludeInternalTopics = false)) + () => GetOffsetShell.createTopicPartitionFilterWithPatternList(":a")) } @Test def testPartitionFilterForInvalidRange(): Unit = { assertThrows(classOf[IllegalArgumentException], - () => GetOffsetShell.createTopicPartitionFilterWithPatternList(":a-b", excludeInternalTopics = false)) + () => GetOffsetShell.createTopicPartitionFilterWithPatternList(":a-b")) } @Test def testPartitionFilterForInvalidLowerBound(): Unit = { assertThrows(classOf[IllegalArgumentException], - () => GetOffsetShell.createTopicPartitionFilterWithPatternList(":a-", excludeInternalTopics = false)) + () => GetOffsetShell.createTopicPartitionFilterWithPatternList(":a-")) } @Test def testPartitionFilterForInvalidUpperBound(): Unit = { assertThrows(classOf[IllegalArgumentException], - () => GetOffsetShell.createTopicPartitionFilterWithPatternList(":-b", excludeInternalTopics = false)) + () => GetOffsetShell.createTopicPartitionFilterWithPatternList(":-b")) + } + + @Test + def testInvalidTimeValue(): Unit = { + assertThrows(classOf[IllegalArgumentException], + () => GetOffsetShell.fetchOffsets(Array("--bootstrap-server", "localhost:9092", "--time", "invalid"))) } - private def partitionInfo(topic: String, partition: Int): PartitionInfo = { - new PartitionInfo(topic, partition, null, null, null) + private def topicPartition(topic: String, partition: Int): TopicPartition = { + new TopicPartition(topic, partition) } } diff --git a/core/src/test/scala/kafka/tools/GetOffsetShellTest.scala b/core/src/test/scala/kafka/tools/GetOffsetShellTest.scala index 8e646d633e5e..021642663793 100644 --- a/core/src/test/scala/kafka/tools/GetOffsetShellTest.scala +++ b/core/src/test/scala/kafka/tools/GetOffsetShellTest.scala @@ -24,8 +24,10 @@ import kafka.utils.{Exit, Logging, TestUtils} import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.serialization.StringSerializer -import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource class GetOffsetShellTest extends KafkaServerTestHarness with Logging { private val topicCount = 4 @@ -109,6 +111,61 @@ class GetOffsetShellTest extends KafkaServerTestHarness with Logging { ) } + @ParameterizedTest + @ValueSource(strings = Array("-1", "latest")) + def testGetLatestOffsets(time: String): Unit = { + val offsets = executeAndParse(Array("--topic-partitions", "topic.*:0", "--time", time)) + assertEquals( + List( + ("topic1", 0, Some(1)), + ("topic2", 0, Some(2)), + ("topic3", 0, Some(3)), + ("topic4", 0, Some(4)) + ), + offsets + ) + } + + @ParameterizedTest + @ValueSource(strings = Array("-2", "earliest")) + def testGetEarliestOffsets(time: String): Unit = { + val offsets = executeAndParse(Array("--topic-partitions", "topic.*:0", "--time", time)) + assertEquals( + List( + ("topic1", 0, Some(0)), + ("topic2", 0, Some(0)), + ("topic3", 0, Some(0)), + ("topic4", 0, Some(0)) + ), + offsets + ) + } + + @ParameterizedTest + @ValueSource(strings = Array("-3", "max-timestamp")) + def testGetOffsetsByMaxTimestamp(time: String): Unit = { + val offsets = executeAndParse(Array("--topic-partitions", "topic.*", "--time", time)) + offsets.foreach { case (topic, _, timestampOpt) => + // We can't know the exact offsets with max timestamp + assertTrue(timestampOpt.get >= 0 && timestampOpt.get <= topic.replace("topic", "").toInt) + } + } + + @Test + def testGetOffsetsByTimestamp(): Unit = { + val time = (System.currentTimeMillis() / 2).toString + val offsets = executeAndParse(Array("--topic-partitions", "topic.*:0", "--time", time)) + assertEquals( + List( + ("topic1", 0, Some(0)), + ("topic2", 0, Some(0)), + ("topic3", 0, Some(0)), + ("topic4", 0, Some(0)) + ), + offsets + ) + } + @Test def testTopicPartitionsArgWithInternalExcluded(): Unit = { val offsets = executeAndParse(Array("--topic-partitions", @@ -124,6 +181,12 @@ class GetOffsetShellTest extends KafkaServerTestHarness with Logging { ) } + @Test + def testTopicPartitionsArgWithInternalIncluded(): Unit = { + val offsets = executeAndParse(Array("--topic-partitions", "__.*:0")) + assertEquals(List(("__consumer_offsets", 0, Some(0))), offsets) + } + @Test def testTopicPartitionsNotFoundForNonExistentTopic(): Unit = { assertExitCodeIsOne(Array("--topic", "some_nonexistent_topic")) From 7afdb069bf5539ec404d9305239849ac35ad2d82 Mon Sep 17 00:00:00 2001 From: Justine Olshan Date: Thu, 17 Mar 2022 10:00:27 -0700 Subject: [PATCH 021/447] KAFKA-13750; Client Compatability KafkaTest uses invalid idempotency configs (#11909) Reviewers: Luke Chen , David Jacot --- .../client/client_compatibility_features_test.py | 2 ++ .../apache/kafka/tools/ClientCompatibilityTest.java | 12 ++++++++++++ 2 files changed, 14 insertions(+) diff --git a/tests/kafkatest/tests/client/client_compatibility_features_test.py b/tests/kafkatest/tests/client/client_compatibility_features_test.py index 434c8280cb80..6adb9751e323 100644 --- a/tests/kafkatest/tests/client/client_compatibility_features_test.py +++ b/tests/kafkatest/tests/client/client_compatibility_features_test.py @@ -43,9 +43,11 @@ def get_broker_features(broker_version): if broker_version < V_0_11_0_0: features["describe-acls-supported"] = False features["describe-configs-supported"] = False + features["idempotent-producer-supported"] = False else: features["describe-acls-supported"] = True features["describe-configs-supported"] = True + features["idempotent-producer-supported"] = True return features def run_command(node, cmd, ssh_log_file): diff --git a/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java b/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java index e40ca7ae82ec..9a04f068a66f 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java +++ b/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java @@ -88,6 +88,7 @@ static class TestConfig { final boolean createTopicsSupported; final boolean describeAclsSupported; final boolean describeConfigsSupported; + final boolean idempotentProducerSupported; TestConfig(Namespace res) { this.bootstrapServer = res.getString("bootstrapServer"); @@ -99,6 +100,7 @@ static class TestConfig { this.createTopicsSupported = res.getBoolean("createTopicsSupported"); this.describeAclsSupported = res.getBoolean("describeAclsSupported"); this.describeConfigsSupported = res.getBoolean("describeConfigsSupported"); + this.idempotentProducerSupported = res.get("idempotentProducerSupported"); } } @@ -172,6 +174,13 @@ public static void main(String[] args) throws Exception { .dest("describeConfigsSupported") .metavar("DESCRIBE_CONFIGS_SUPPORTED") .help("Whether describeConfigs is supported in the AdminClient."); + parser.addArgument("--idempotent-producer-supported") + .action(store()) + .required(true) + .type(Boolean.class) + .dest("idempotentProducerSupported") + .metavar("IDEMPOTENT_PRODUCER_SUPPORTED") + .help("Whether the producer supports idempotency."); Namespace res = null; try { @@ -243,6 +252,9 @@ void run() throws Throwable { public void testProduce() throws Exception { Properties producerProps = new Properties(); producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, testConfig.bootstrapServer); + if (!testConfig.idempotentProducerSupported) { + producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false"); + } ByteArraySerializer serializer = new ByteArraySerializer(); KafkaProducer producer = new KafkaProducer<>(producerProps, serializer, serializer); ProducerRecord record1 = new ProducerRecord<>(testConfig.topic, message1); From df963ee0a98c3107093b22c670a6bc8d8f459e05 Mon Sep 17 00:00:00 2001 From: Ludovic DEHON Date: Fri, 18 Mar 2022 03:00:03 +0100 Subject: [PATCH 022/447] MINOR: Fix incorrect log for out-of-order KTable (#11905) Reviewers: Luke Chen --- .../apache/kafka/streams/kstream/internals/KTableSource.java | 2 +- .../kafka/streams/kstream/internals/KTableSourceTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java index 6236c4c5c1fc..6de8ede316b8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java @@ -131,7 +131,7 @@ public void process(final Record record) { + "topic=[{}] partition=[{}] offset=[{}].", store.name(), oldValueAndTimestamp.timestamp(), record.timestamp(), - recordMetadata.topic(), recordMetadata.offset(), recordMetadata.partition() + recordMetadata.topic(), recordMetadata.partition(), recordMetadata.offset() ); } else { LOG.warn( diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java index 68c005a8c604..70e1bccdbe65 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java @@ -184,7 +184,7 @@ public void kTableShouldLogOnOutOfOrder() { .filter(e -> e.getLevel().equals("WARN")) .map(Event::getMessage) .collect(Collectors.toList()), - hasItem("Detected out-of-order KTable update for store, old timestamp=[10] new timestamp=[5]. topic=[topic] partition=[1] offset=[0].") + hasItem("Detected out-of-order KTable update for store, old timestamp=[10] new timestamp=[5]. topic=[topic] partition=[0] offset=[1].") ); } } From 03641e6a28ce56898f28610f0d58c59182befc2b Mon Sep 17 00:00:00 2001 From: Jules Ivanic Date: Fri, 18 Mar 2022 16:20:08 +0800 Subject: [PATCH 023/447] MINOR: Fix `ConsumerConfig.ISOLATION_LEVEL_DOC` (#11915) Reviewers: David Jacot --- .../java/org/apache/kafka/clients/consumer/ConsumerConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index ca24c281dc0d..48f1ccbf1951 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -312,7 +312,7 @@ public class ConsumerConfig extends AbstractConfig { " read_committed mode, consumer.poll() will only return messages up to the last stable offset (LSO), which is the one less than the offset of the first open transaction." + " In particular any messages appearing after messages belonging to ongoing transactions will be withheld until the relevant transaction has been completed. As a result, read_committed" + " consumers will not be able to read up to the high watermark when there are in flight transactions.

Further, when in read_committed the seekToEnd method will" + - " return the LSO"; + " return the LSO

"; public static final String DEFAULT_ISOLATION_LEVEL = IsolationLevel.READ_UNCOMMITTED.toString().toLowerCase(Locale.ROOT); From 43bf46423245383238998031b8c5b4294335cff6 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 18 Mar 2022 05:27:49 -0400 Subject: [PATCH 024/447] KAFKA-13497: Add trace logging to RegexRouter (#11903) This patch adds runtime logging to the RegexRouter to show exactly which topics get routed where. Reviewers: David Jacot --- .../org/apache/kafka/connect/transforms/RegexRouter.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java index 74a19cdb82df..a79f5c17dad6 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java @@ -20,6 +20,8 @@ import org.apache.kafka.connect.connector.ConnectRecord; import org.apache.kafka.connect.transforms.util.RegexValidator; import org.apache.kafka.connect.transforms.util.SimpleConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Map; import java.util.regex.Matcher; @@ -27,6 +29,8 @@ public class RegexRouter> implements Transformation { + private static final Logger log = LoggerFactory.getLogger(RegexRouter.class); + public static final String OVERVIEW_DOC = "Update the record topic using the configured regular expression and replacement string." + "

Under the hood, the regex is compiled to a java.util.regex.Pattern. " + "If the pattern matches the input topic, java.util.regex.Matcher#replaceFirst() is used with the replacement string to obtain the new topic."; @@ -57,7 +61,10 @@ public R apply(R record) { final Matcher matcher = regex.matcher(record.topic()); if (matcher.matches()) { final String topic = matcher.replaceFirst(replacement); + log.trace("Rerouting from topic '{}' to new topic '{}'", record.topic(), topic); return record.newRecord(topic, record.kafkaPartition(), record.keySchema(), record.key(), record.valueSchema(), record.value(), record.timestamp()); + } else { + log.trace("Not rerouting topic '{}' as it does not match the configured regex", record.topic()); } return record; } From 52621613fd386203773ba93903abd50b46fa093a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 18 Mar 2022 09:24:11 -0700 Subject: [PATCH 025/447] KAFKA-13587; Implement leader recovery for KIP-704 (#11733) Implementation of the protocol for starting and stopping leader recovery after an unclean leader election. This includes the management of state in the controllers (legacy and KRaft) and propagating this information to the brokers. This change doesn't implement log recovery after an unclean leader election. Protocol Changes ================ For the topic partition state znode, the new field "leader_recovery_state" was added. If the field is missing the value is assumed to be RECOVERED. ALTER_PARTITION was renamed from ALTER_ISR. The CurrentIsrVersion field was renamed to PartitionEpoch. The new field LeaderRecoveryState was added. The new field LeaderRecoverState was added to the LEADER_AND_ISR request. The inter broker protocol version is used to determine which version to send to the brokers. A new tagged field for LeaderRecoveryState was added to both the PartitionRecord and PartitionChangeRecord. Controller ========== For both the KRaft and legacy controller the LeaderRecoveryState is set to RECOVERING, if the leader was elected out of the ISR, also known as unclean leader election. The controller sets the state back to RECOVERED after receiving an ALTER_PARTITION request with version 0, or with version 1 and with the LeaderRecoveryState set to RECOVERED. Both controllers preserve the leader recovery state even if the unclean leader goes offline and comes back online before an RECOVERED ALTER_PARTITION is sent. The controllers reply with INVALID_REQUEST if the ALTER_PARTITION either: 1. Attempts to increase the ISR while the partition is still RECOVERING 2. Attempts to change the leader recovery state to RECOVERING from a RECOVERED state. Topic Partition Leader ====================== The topic partition leader doesn't implement any log recovery in this change. The topic partition leader immediately marks the partition as RECOVERED and sends that state in the next ALTER_PARTITION request. Reviewers: Jason Gustafson --- .../apache/kafka/common/protocol/ApiKeys.java | 2 +- .../common/requests/AbstractRequest.java | 4 +- .../common/requests/AbstractResponse.java | 4 +- ...equest.java => AlterPartitionRequest.java} | 32 +-- ...ponse.java => AlterPartitionResponse.java} | 16 +- ...equest.json => AlterPartitionRequest.json} | 12 +- ...ponse.json => AlterPartitionResponse.json} | 12 +- .../common/message/LeaderAndIsrRequest.json | 8 +- .../common/message/LeaderAndIsrResponse.json | 4 +- .../kafka/common/protocol/ApiKeysTest.java | 4 +- .../common/requests/RequestResponseTest.java | 66 +++-- .../src/main/scala/kafka/api/ApiVersion.scala | 13 +- .../main/scala/kafka/api/LeaderAndIsr.scala | 26 +- .../main/scala/kafka/cluster/Partition.scala | 186 ++++++++------ .../controller/ControllerChannelManager.scala | 13 +- .../scala/kafka/controller/Election.scala | 20 +- .../kafka/controller/KafkaController.scala | 115 ++++++--- .../controller/PartitionStateMachine.scala | 10 +- .../kafka/network/RequestConvertToJson.scala | 4 +- .../scala/kafka/server/AlterIsrManager.scala | 110 ++++---- .../scala/kafka/server/BrokerServer.scala | 3 +- .../scala/kafka/server/ControllerApis.scala | 12 +- .../scala/kafka/server/ControllerServer.scala | 2 + .../main/scala/kafka/server/KafkaApis.scala | 12 +- .../main/scala/kafka/server/KafkaServer.scala | 6 +- .../scala/kafka/server/ReplicaManager.scala | 1 - core/src/main/scala/kafka/zk/ZkData.scala | 31 ++- .../test/java/kafka/test/MockController.java | 6 +- .../scala/kafka/api/LeaderAndIsrTest.scala | 34 +++ .../zk/TopicPartitionStateZNodeTest.scala | 74 ++++++ .../scala/unit/kafka/api/ApiVersionTest.scala | 4 + .../unit/kafka/cluster/PartitionTest.scala | 157 ++++++++---- .../ControllerChannelManagerTest.scala | 36 ++- .../ControllerIntegrationTest.scala | 39 ++- .../MockPartitionStateMachine.scala | 14 +- .../PartitionStateMachineTest.scala | 39 ++- .../controller/TopicDeletionManagerTest.scala | 24 +- .../network/RequestConvertToJsonTest.scala | 6 +- .../kafka/server/AlterIsrManagerTest.scala | 179 ++++++++----- .../kafka/server/ControllerApisTest.scala | 6 +- .../server/HighwatermarkPersistenceTest.scala | 4 +- .../unit/kafka/server/IsrExpirationTest.scala | 4 +- .../unit/kafka/server/KafkaApisTest.scala | 4 +- .../ReplicaManagerConcurrencyTest.scala | 10 +- .../server/ReplicaManagerQuotasTest.scala | 8 +- .../kafka/server/ReplicaManagerTest.scala | 2 +- .../unit/kafka/server/RequestQuotaTest.scala | 4 +- .../BrokerMetadataPublisherTest.scala | 6 +- .../kafka/utils/ReplicationUtilsTest.scala | 9 +- .../unit/kafka/zk/KafkaZkClientTest.scala | 44 ++-- .../apache/kafka/controller/Controller.java | 10 +- .../controller/PartitionChangeBuilder.java | 23 +- .../kafka/controller/QuorumController.java | 25 +- .../controller/ReplicationControlManager.java | 236 +++++++++++------- .../kafka/metadata/LeaderRecoveryState.java | 74 ++++++ .../kafka/metadata/PartitionRegistration.java | 23 +- .../metadata/PartitionChangeRecord.json | 4 +- .../common/metadata/PartitionRecord.json | 2 + .../PartitionChangeBuilderTest.java | 129 +++++++++- .../PartitionReassignmentRevertTest.java | 11 +- .../controller/QuorumControllerTest.java | 20 +- .../ReplicationControlManagerTest.java | 208 ++++++++------- .../apache/kafka/image/TopicsImageTest.java | 15 +- .../metadata/LeaderRecoveryStateTest.java | 87 +++++++ .../metadata/PartitionRegistrationTest.java | 18 +- 65 files changed, 1619 insertions(+), 707 deletions(-) rename clients/src/main/java/org/apache/kafka/common/requests/{AlterIsrRequest.java => AlterPartitionRequest.java} (62%) rename clients/src/main/java/org/apache/kafka/common/requests/{AlterIsrResponse.java => AlterPartitionResponse.java} (75%) rename clients/src/main/resources/common/message/{AlterIsrRequest.json => AlterPartitionRequest.json} (77%) rename clients/src/main/resources/common/message/{AlterIsrResponse.json => AlterPartitionResponse.json} (81%) create mode 100644 core/src/test/scala/kafka/api/LeaderAndIsrTest.scala create mode 100644 core/src/test/scala/kafka/zk/TopicPartitionStateZNodeTest.scala create mode 100644 metadata/src/main/java/org/apache/kafka/metadata/LeaderRecoveryState.java create mode 100644 metadata/src/test/java/org/apache/kafka/metadata/LeaderRecoveryStateTest.java diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 5aa6186f4342..ad96cdfa428b 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -97,7 +97,7 @@ public enum ApiKeys { BEGIN_QUORUM_EPOCH(ApiMessageType.BEGIN_QUORUM_EPOCH, true, RecordBatch.MAGIC_VALUE_V0, false), END_QUORUM_EPOCH(ApiMessageType.END_QUORUM_EPOCH, true, RecordBatch.MAGIC_VALUE_V0, false), DESCRIBE_QUORUM(ApiMessageType.DESCRIBE_QUORUM, true, RecordBatch.MAGIC_VALUE_V0, true), - ALTER_ISR(ApiMessageType.ALTER_ISR, true), + ALTER_PARTITION(ApiMessageType.ALTER_PARTITION, true), UPDATE_FEATURES(ApiMessageType.UPDATE_FEATURES), ENVELOPE(ApiMessageType.ENVELOPE, true, RecordBatch.MAGIC_VALUE_V0, false), FETCH_SNAPSHOT(ApiMessageType.FETCH_SNAPSHOT, false, RecordBatch.MAGIC_VALUE_V0, false), diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index 0c38e998fe24..0d96d842d6df 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -279,8 +279,8 @@ private static AbstractRequest doParseRequest(ApiKeys apiKey, short apiVersion, return EndQuorumEpochRequest.parse(buffer, apiVersion); case DESCRIBE_QUORUM: return DescribeQuorumRequest.parse(buffer, apiVersion); - case ALTER_ISR: - return AlterIsrRequest.parse(buffer, apiVersion); + case ALTER_PARTITION: + return AlterPartitionRequest.parse(buffer, apiVersion); case UPDATE_FEATURES: return UpdateFeaturesRequest.parse(buffer, apiVersion); case ENVELOPE: diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index 47f2b3c7f309..cd99f472ebb0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -223,8 +223,8 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, ByteBuffer response return EndQuorumEpochResponse.parse(responseBuffer, version); case DESCRIBE_QUORUM: return DescribeQuorumResponse.parse(responseBuffer, version); - case ALTER_ISR: - return AlterIsrResponse.parse(responseBuffer, version); + case ALTER_PARTITION: + return AlterPartitionResponse.parse(responseBuffer, version); case UPDATE_FEATURES: return UpdateFeaturesResponse.parse(responseBuffer, version); case ENVELOPE: diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionRequest.java similarity index 62% rename from clients/src/main/java/org/apache/kafka/common/requests/AlterIsrRequest.java rename to clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionRequest.java index 516c2ce76aa8..091bed617355 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionRequest.java @@ -17,25 +17,25 @@ package org.apache.kafka.common.requests; -import org.apache.kafka.common.message.AlterIsrRequestData; -import org.apache.kafka.common.message.AlterIsrResponseData; +import org.apache.kafka.common.message.AlterPartitionRequestData; +import org.apache.kafka.common.message.AlterPartitionResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import java.nio.ByteBuffer; -public class AlterIsrRequest extends AbstractRequest { +public class AlterPartitionRequest extends AbstractRequest { - private final AlterIsrRequestData data; + private final AlterPartitionRequestData data; - public AlterIsrRequest(AlterIsrRequestData data, short apiVersion) { - super(ApiKeys.ALTER_ISR, apiVersion); + public AlterPartitionRequest(AlterPartitionRequestData data, short apiVersion) { + super(ApiKeys.ALTER_PARTITION, apiVersion); this.data = data; } @Override - public AlterIsrRequestData data() { + public AlterPartitionRequestData data() { return data; } @@ -44,27 +44,27 @@ public AlterIsrRequestData data() { */ @Override public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { - return new AlterIsrResponse(new AlterIsrResponseData() + return new AlterPartitionResponse(new AlterPartitionResponseData() .setThrottleTimeMs(throttleTimeMs) .setErrorCode(Errors.forException(e).code())); } - public static AlterIsrRequest parse(ByteBuffer buffer, short version) { - return new AlterIsrRequest(new AlterIsrRequestData(new ByteBufferAccessor(buffer), version), version); + public static AlterPartitionRequest parse(ByteBuffer buffer, short version) { + return new AlterPartitionRequest(new AlterPartitionRequestData(new ByteBufferAccessor(buffer), version), version); } - public static class Builder extends AbstractRequest.Builder { + public static class Builder extends AbstractRequest.Builder { - private final AlterIsrRequestData data; + private final AlterPartitionRequestData data; - public Builder(AlterIsrRequestData data) { - super(ApiKeys.ALTER_ISR); + public Builder(AlterPartitionRequestData data) { + super(ApiKeys.ALTER_PARTITION); this.data = data; } @Override - public AlterIsrRequest build(short version) { - return new AlterIsrRequest(data, version); + public AlterPartitionRequest build(short version) { + return new AlterPartitionRequest(data, version); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionResponse.java similarity index 75% rename from clients/src/main/java/org/apache/kafka/common/requests/AlterIsrResponse.java rename to clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionResponse.java index c3106ed94cbd..d2ace4112f4c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionResponse.java @@ -17,7 +17,7 @@ package org.apache.kafka.common.requests; -import org.apache.kafka.common.message.AlterIsrResponseData; +import org.apache.kafka.common.message.AlterPartitionResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; @@ -26,17 +26,17 @@ import java.util.HashMap; import java.util.Map; -public class AlterIsrResponse extends AbstractResponse { +public class AlterPartitionResponse extends AbstractResponse { - private final AlterIsrResponseData data; + private final AlterPartitionResponseData data; - public AlterIsrResponse(AlterIsrResponseData data) { - super(ApiKeys.ALTER_ISR); + public AlterPartitionResponse(AlterPartitionResponseData data) { + super(ApiKeys.ALTER_PARTITION); this.data = data; } @Override - public AlterIsrResponseData data() { + public AlterPartitionResponseData data() { return data; } @@ -55,7 +55,7 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } - public static AlterIsrResponse parse(ByteBuffer buffer, short version) { - return new AlterIsrResponse(new AlterIsrResponseData(new ByteBufferAccessor(buffer), version)); + public static AlterPartitionResponse parse(ByteBuffer buffer, short version) { + return new AlterPartitionResponse(new AlterPartitionResponseData(new ByteBufferAccessor(buffer), version)); } } diff --git a/clients/src/main/resources/common/message/AlterIsrRequest.json b/clients/src/main/resources/common/message/AlterPartitionRequest.json similarity index 77% rename from clients/src/main/resources/common/message/AlterIsrRequest.json rename to clients/src/main/resources/common/message/AlterPartitionRequest.json index 70736dbe8a5a..97f02457de97 100644 --- a/clients/src/main/resources/common/message/AlterIsrRequest.json +++ b/clients/src/main/resources/common/message/AlterPartitionRequest.json @@ -17,8 +17,8 @@ "apiKey": 56, "type": "request", "listeners": ["zkBroker", "controller"], - "name": "AlterIsrRequest", - "validVersions": "0", + "name": "AlterPartitionRequest", + "validVersions": "0-1", "flexibleVersions": "0+", "fields": [ { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId", @@ -34,9 +34,11 @@ { "name": "LeaderEpoch", "type": "int32", "versions": "0+", "about": "The leader epoch of this partition" }, { "name": "NewIsr", "type": "[]int32", "versions": "0+", "entityType": "brokerId", - "about": "The ISR for this partition"}, - { "name": "CurrentIsrVersion", "type": "int32", "versions": "0+", - "about": "The expected version of ISR which is being updated"} + "about": "The ISR for this partition" }, + { "name": "LeaderRecoveryState", "type": "int8", "versions": "1+", "default": "0", + "about": "1 if the partition is recovering from an unclean leader election; 0 otherwise." }, + { "name": "PartitionEpoch", "type": "int32", "versions": "0+", + "about": "The expected epoch of the partition which is being updated. For legacy cluster this is the ZkVersion in the LeaderAndIsr request." } ]} ]} ] diff --git a/clients/src/main/resources/common/message/AlterIsrResponse.json b/clients/src/main/resources/common/message/AlterPartitionResponse.json similarity index 81% rename from clients/src/main/resources/common/message/AlterIsrResponse.json rename to clients/src/main/resources/common/message/AlterPartitionResponse.json index 33837996d3fb..aaeb5cfad760 100644 --- a/clients/src/main/resources/common/message/AlterIsrResponse.json +++ b/clients/src/main/resources/common/message/AlterPartitionResponse.json @@ -16,8 +16,8 @@ { "apiKey": 56, "type": "response", - "name": "AlterIsrResponse", - "validVersions": "0", + "name": "AlterPartitionResponse", + "validVersions": "0-1", "flexibleVersions": "0+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", @@ -38,9 +38,11 @@ "about": "The leader epoch." }, { "name": "Isr", "type": "[]int32", "versions": "0+", "entityType": "brokerId", "about": "The in-sync replica IDs." }, - { "name": "CurrentIsrVersion", "type": "int32", "versions": "0+", - "about": "The current ISR version." } + { "name": "LeaderRecoveryState", "type": "int8", "versions": "1+", "default": "0", "ignorable": true, + "about": "1 if the partition is recovering from an unclean leader election; 0 otherwise." }, + { "name": "PartitionEpoch", "type": "int32", "versions": "0+", + "about": "The current epoch for the partition for KRaft controllers. The current ZK version for the legacy controllers." } ]} ]} ] -} \ No newline at end of file +} diff --git a/clients/src/main/resources/common/message/LeaderAndIsrRequest.json b/clients/src/main/resources/common/message/LeaderAndIsrRequest.json index c38f21e59fa8..663811055170 100644 --- a/clients/src/main/resources/common/message/LeaderAndIsrRequest.json +++ b/clients/src/main/resources/common/message/LeaderAndIsrRequest.json @@ -27,7 +27,9 @@ // Version 4 is the first flexible version. // // Version 5 adds Topic ID and Type to the TopicStates, as described in KIP-516. - "validVersions": "0-5", + // + // Version 6 adds ElectionState as described in KIP-704. + "validVersions": "0-6", "flexibleVersions": "4+", "fields": [ { "name": "ControllerId", "type": "int32", "versions": "0+", "entityType": "brokerId", @@ -85,7 +87,9 @@ { "name": "RemovingReplicas", "type": "[]int32", "versions": "3+", "ignorable": true, "entityType": "brokerId", "about": "The replica IDs that we are removing this partition from, or null if no replicas are being removed." }, { "name": "IsNew", "type": "bool", "versions": "1+", "default": "false", "ignorable": true, - "about": "Whether the replica should have existed on the broker or not." } + "about": "Whether the replica should have existed on the broker or not." }, + { "name": "LeaderRecoveryState", "type": "int8", "versions": "6+", "default": "0", + "about": "1 if the partition is recovering from an unclean leader election; 0 otherwise." } ]} ] } diff --git a/clients/src/main/resources/common/message/LeaderAndIsrResponse.json b/clients/src/main/resources/common/message/LeaderAndIsrResponse.json index 958448be2744..79ad819f5e6d 100644 --- a/clients/src/main/resources/common/message/LeaderAndIsrResponse.json +++ b/clients/src/main/resources/common/message/LeaderAndIsrResponse.json @@ -25,9 +25,9 @@ // // Version 4 is the first flexible version. // - // Version 5 removes TopicName and replaces it with TopicId and reorganizes + // Version 5 removes TopicName and replaces it with TopicId and reorganizes // the partitions by topic, as described by KIP-516. - "validVersions": "0-5", + "validVersions": "0-6", "flexibleVersions": "4+", "fields": [ { "name": "ErrorCode", "type": "int16", "versions": "0+", diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java index 3c66b211bec4..ec6259ae725d 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java @@ -45,7 +45,7 @@ public void testForIdWithInvalidIdHigh() { @Test public void testAlterIsrIsClusterAction() { - assertTrue(ApiKeys.ALTER_ISR.clusterAction); + assertTrue(ApiKeys.ALTER_PARTITION.clusterAction); } /** @@ -62,7 +62,7 @@ public void testAlterIsrIsClusterAction() { public void testResponseThrottleTime() { Set authenticationKeys = EnumSet.of(ApiKeys.SASL_HANDSHAKE, ApiKeys.SASL_AUTHENTICATE); // Newer protocol apis include throttle time ms even for cluster actions - Set clusterActionsWithThrottleTimeMs = EnumSet.of(ApiKeys.ALTER_ISR, ApiKeys.ALLOCATE_PRODUCER_IDS); + Set clusterActionsWithThrottleTimeMs = EnumSet.of(ApiKeys.ALTER_PARTITION, ApiKeys.ALLOCATE_PRODUCER_IDS); for (ApiKeys apiKey: ApiKeys.zkBrokerApis()) { Schema responseSchema = apiKey.messageType.responseSchemas()[apiKey.latestVersion()]; BoundField throttleTimeField = responseSchema.get("throttle_time_ms"); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 8b5ada9ef860..7b54dfcd3d67 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -41,8 +41,8 @@ import org.apache.kafka.common.message.AllocateProducerIdsResponseData; import org.apache.kafka.common.message.AlterClientQuotasResponseData; import org.apache.kafka.common.message.AlterConfigsResponseData; -import org.apache.kafka.common.message.AlterIsrRequestData; -import org.apache.kafka.common.message.AlterIsrResponseData; +import org.apache.kafka.common.message.AlterPartitionRequestData; +import org.apache.kafka.common.message.AlterPartitionResponseData; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; import org.apache.kafka.common.message.AlterReplicaLogDirsRequestData; @@ -1029,7 +1029,7 @@ private AbstractRequest getRequest(ApiKeys apikey, short version) { case BEGIN_QUORUM_EPOCH: return createBeginQuorumEpochRequest(version); case END_QUORUM_EPOCH: return createEndQuorumEpochRequest(version); case DESCRIBE_QUORUM: return createDescribeQuorumRequest(version); - case ALTER_ISR: return createAlterIsrRequest(version); + case ALTER_PARTITION: return createAlterPartitionRequest(version); case UPDATE_FEATURES: return createUpdateFeaturesRequest(version); case ENVELOPE: return createEnvelopeRequest(version); case FETCH_SNAPSHOT: return createFetchSnapshotRequest(version); @@ -1103,7 +1103,7 @@ private AbstractResponse getResponse(ApiKeys apikey, short version) { case BEGIN_QUORUM_EPOCH: return createBeginQuorumEpochResponse(); case END_QUORUM_EPOCH: return createEndQuorumEpochResponse(); case DESCRIBE_QUORUM: return createDescribeQuorumResponse(); - case ALTER_ISR: return createAlterIsrResponse(); + case ALTER_PARTITION: return createAlterPartitionResponse(version); case UPDATE_FEATURES: return createUpdateFeaturesResponse(); case ENVELOPE: return createEnvelopeResponse(); case FETCH_SNAPSHOT: return createFetchSnapshotResponse(); @@ -1305,33 +1305,47 @@ private DescribeUserScramCredentialsResponse createDescribeUserScramCredentialsR return new DescribeUserScramCredentialsResponse(data); } - private AlterIsrRequest createAlterIsrRequest(short version) { - AlterIsrRequestData data = new AlterIsrRequestData() - .setBrokerEpoch(123L) - .setBrokerId(1) - .setTopics(singletonList(new AlterIsrRequestData.TopicData() - .setName("topic1") - .setPartitions(singletonList(new AlterIsrRequestData.PartitionData() - .setPartitionIndex(1) - .setCurrentIsrVersion(2) - .setLeaderEpoch(3) - .setNewIsr(asList(1, 2)))))); - return new AlterIsrRequest.Builder(data).build(version); + private AlterPartitionRequest createAlterPartitionRequest(short version) { + AlterPartitionRequestData.PartitionData partitionData = new AlterPartitionRequestData.PartitionData() + .setPartitionIndex(1) + .setPartitionEpoch(2) + .setLeaderEpoch(3) + .setNewIsr(asList(1, 2)); + + if (version >= 1) { + // Use the none default value; 1 - RECOVERING + partitionData.setLeaderRecoveryState((byte) 1); + } + + AlterPartitionRequestData data = new AlterPartitionRequestData() + .setBrokerEpoch(123L) + .setBrokerId(1) + .setTopics(singletonList(new AlterPartitionRequestData.TopicData() + .setName("topic1") + .setPartitions(singletonList(partitionData)))); + return new AlterPartitionRequest.Builder(data).build(version); } - private AlterIsrResponse createAlterIsrResponse() { - AlterIsrResponseData data = new AlterIsrResponseData() + private AlterPartitionResponse createAlterPartitionResponse(int version) { + AlterPartitionResponseData.PartitionData partitionData = new AlterPartitionResponseData.PartitionData() + .setPartitionEpoch(1) + .setIsr(asList(0, 1, 2)) + .setErrorCode(Errors.NONE.code()) + .setLeaderEpoch(2) + .setLeaderId(3); + + if (version >= 1) { + // Use the none default value; 1 - RECOVERING + partitionData.setLeaderRecoveryState((byte) 1); + } + + AlterPartitionResponseData data = new AlterPartitionResponseData() .setErrorCode(Errors.NONE.code()) .setThrottleTimeMs(123) - .setTopics(singletonList(new AlterIsrResponseData.TopicData() + .setTopics(singletonList(new AlterPartitionResponseData.TopicData() .setName("topic1") - .setPartitions(singletonList(new AlterIsrResponseData.PartitionData() - .setCurrentIsrVersion(1) - .setIsr(asList(0, 1, 2)) - .setErrorCode(Errors.NONE.code()) - .setLeaderEpoch(2) - .setLeaderId(3))))); - return new AlterIsrResponse(data); + .setPartitions(singletonList(partitionData)))); + return new AlterPartitionResponse(data); } private UpdateFeaturesRequest createUpdateFeaturesRequest(short version) { diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala index 8165e6c6f1e5..1084fb4d11b0 100644 --- a/core/src/main/scala/kafka/api/ApiVersion.scala +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -107,7 +107,7 @@ object ApiVersion { KAFKA_2_7_IV0, // Bup Fetch protocol for Raft protocol (KIP-595) KAFKA_2_7_IV1, - // Introduced AlterIsr (KIP-497) + // Introduced AlterPartition (KIP-497) KAFKA_2_7_IV2, // Flexible versioning on ListOffsets, WriteTxnMarkers and OffsetsForLeaderEpoch. Also adds topic IDs (KIP-516) KAFKA_2_8_IV0, @@ -119,7 +119,9 @@ object ApiVersion { // Assume message format version is 3.0 (KIP-724) KAFKA_3_0_IV1, // Adds topic IDs to Fetch requests/responses (KIP-516) - KAFKA_3_1_IV0 + KAFKA_3_1_IV0, + // Support for leader recovery for unclean leader election (KIP-704) + KAFKA_3_2_IV0 ) // Map keys are the union of the short and full versions @@ -477,6 +479,13 @@ case object KAFKA_3_1_IV0 extends DefaultApiVersion { val id: Int = 35 } +case object KAFKA_3_2_IV0 extends DefaultApiVersion { + val shortVersion: String = "3.2" + val subVersion = "IV0" + val recordVersion = RecordVersion.V2 + val id: Int = 36 +} + object ApiVersionValidator extends Validator { override def ensureValid(name: String, value: Any): Unit = { diff --git a/core/src/main/scala/kafka/api/LeaderAndIsr.scala b/core/src/main/scala/kafka/api/LeaderAndIsr.scala index 05952aa58d3a..0cd8df092a51 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsr.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsr.scala @@ -17,6 +17,8 @@ package kafka.api +import org.apache.kafka.metadata.LeaderRecoveryState + object LeaderAndIsr { val initialLeaderEpoch: Int = 0 val initialZKVersion: Int = 0 @@ -25,7 +27,9 @@ object LeaderAndIsr { val LeaderDuringDelete: Int = -2 val EpochDuringDelete: Int = -2 - def apply(leader: Int, isr: List[Int]): LeaderAndIsr = LeaderAndIsr(leader, initialLeaderEpoch, isr, initialZKVersion) + def apply(leader: Int, isr: List[Int]): LeaderAndIsr = { + LeaderAndIsr(leader, initialLeaderEpoch, isr, LeaderRecoveryState.RECOVERED, initialZKVersion) + } def duringDelete(isr: List[Int]): LeaderAndIsr = LeaderAndIsr(LeaderDuringDelete, isr) } @@ -33,14 +37,21 @@ object LeaderAndIsr { case class LeaderAndIsr(leader: Int, leaderEpoch: Int, isr: List[Int], + leaderRecoveryState: LeaderRecoveryState, zkVersion: Int) { - def withZkVersion(zkVersion: Int) = copy(zkVersion = zkVersion) + def withZkVersion(zkVersion: Int): LeaderAndIsr = copy(zkVersion = zkVersion) - def newLeader(leader: Int) = newLeaderAndIsr(leader, isr) + def newLeader(leader: Int): LeaderAndIsr = newLeaderAndIsr(leader, isr) - def newLeaderAndIsr(leader: Int, isr: List[Int]) = LeaderAndIsr(leader, leaderEpoch + 1, isr, zkVersion) + def newLeaderAndIsr(leader: Int, isr: List[Int]): LeaderAndIsr = { + LeaderAndIsr(leader, leaderEpoch + 1, isr, leaderRecoveryState, zkVersion) + } + + def newRecoveringLeaderAndIsr(leader: Int, isr: List[Int]): LeaderAndIsr = { + LeaderAndIsr(leader, leaderEpoch + 1, isr, LeaderRecoveryState.RECOVERING, zkVersion) + } - def newEpochAndZkVersion = newLeaderAndIsr(leader, isr) + def newEpoch: LeaderAndIsr = newLeaderAndIsr(leader, isr) def leaderOpt: Option[Int] = { if (leader == LeaderAndIsr.NoLeader) None else Some(leader) @@ -52,11 +63,12 @@ case class LeaderAndIsr(leader: Int, } else if (other == null) { false } else { - leader == other.leader && leaderEpoch == other.leaderEpoch && isr.equals(other.isr) + leader == other.leader && leaderEpoch == other.leaderEpoch && isr.equals(other.isr) && + leaderRecoveryState == other.leaderRecoveryState } } override def toString: String = { - s"LeaderAndIsr(leader=$leader, leaderEpoch=$leaderEpoch, isr=$isr, zkVersion=$zkVersion)" + s"LeaderAndIsr(leader=$leader, leaderEpoch=$leaderEpoch, isr=$isr, leaderRecoveryState=$leaderRecoveryState, zkVersion=$zkVersion)" } } diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 150432d99c00..429e73b4d834 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -41,6 +41,7 @@ import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET} import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{IsolationLevel, TopicPartition, Uuid} +import org.apache.kafka.metadata.LeaderRecoveryState import scala.collection.{Map, Seq} import scala.jdk.CollectionConverters._ @@ -130,8 +131,7 @@ case class OngoingReassignmentState(addingReplicas: Seq[Int], case class SimpleAssignmentState(replicas: Seq[Int]) extends AssignmentState - -sealed trait IsrState { +sealed trait PartitionState { /** * Includes only the in-sync replicas which have been committed to ZK. */ @@ -147,28 +147,36 @@ sealed trait IsrState { def maximalIsr: Set[Int] /** - * Indicates if we have an AlterIsr request inflight. + * The leader recovery state. See the description for LeaderRecoveryState for details on the different values. + */ + def leaderRecoveryState: LeaderRecoveryState + + /** + * Indicates if we have an AlterPartition request inflight. */ def isInflight: Boolean } -sealed trait PendingIsrChange extends IsrState { +sealed trait PendingPartitionChange extends PartitionState { def sentLeaderAndIsr: LeaderAndIsr + + override val leaderRecoveryState: LeaderRecoveryState = LeaderRecoveryState.RECOVERED } case class PendingExpandIsr( isr: Set[Int], newInSyncReplicaId: Int, sentLeaderAndIsr: LeaderAndIsr -) extends PendingIsrChange { +) extends PendingPartitionChange { val maximalIsr = isr + newInSyncReplicaId val isInflight = true override def toString: String = { s"PendingExpandIsr(isr=$isr" + - s", newInSyncReplicaId=$newInSyncReplicaId" + - s", sentLeaderAndIsr=$sentLeaderAndIsr" + - ")" + s", newInSyncReplicaId=$newInSyncReplicaId" + + s", sentLeaderAndIsr=$sentLeaderAndIsr" + + s", leaderRecoveryState=$leaderRecoveryState" + + ")" } } @@ -176,27 +184,30 @@ case class PendingShrinkIsr( isr: Set[Int], outOfSyncReplicaIds: Set[Int], sentLeaderAndIsr: LeaderAndIsr -) extends PendingIsrChange { +) extends PendingPartitionChange { val maximalIsr = isr val isInflight = true override def toString: String = { s"PendingShrinkIsr(isr=$isr" + - s", outOfSyncReplicaIds=$outOfSyncReplicaIds" + - s", sentLeaderAndIsr=$sentLeaderAndIsr" + - ")" + s", outOfSyncReplicaIds=$outOfSyncReplicaIds" + + s", sentLeaderAndIsr=$sentLeaderAndIsr" + + s", leaderRecoveryState=$leaderRecoveryState" + + ")" } } -case class CommittedIsr( - isr: Set[Int] -) extends IsrState { +case class CommittedPartitionState( + isr: Set[Int], + override val leaderRecoveryState: LeaderRecoveryState +) extends PartitionState { val maximalIsr = isr val isInflight = false override def toString: String = { - s"CommittedIsr(isr=$isr" + - ")" + s"CommittedPartitionState(isr=$isr" + + s", leaderRecoveryState=$leaderRecoveryState" + + ")" } } @@ -246,7 +257,7 @@ class Partition(val topicPartition: TopicPartition, // defined when this broker is leader for partition @volatile private var leaderEpochStartOffsetOpt: Option[Long] = None @volatile var leaderReplicaIdOpt: Option[Int] = None - @volatile private[cluster] var isrState: IsrState = CommittedIsr(Set.empty) + @volatile private[cluster] var partitionState: PartitionState = CommittedPartitionState(Set.empty, LeaderRecoveryState.RECOVERED) @volatile var assignmentState: AssignmentState = SimpleAssignmentState(Seq.empty) // Logs belonging to this partition. Majority of time it will be only one log, but if log directory @@ -268,7 +279,7 @@ class Partition(val topicPartition: TopicPartition, private val tags = Map("topic" -> topic, "partition" -> partitionId.toString) newGauge("UnderReplicated", () => if (isUnderReplicated) 1 else 0, tags) - newGauge("InSyncReplicasCount", () => if (isLeader) isrState.isr.size else 0, tags) + newGauge("InSyncReplicasCount", () => if (isLeader) partitionState.isr.size else 0, tags) newGauge("UnderMinIsr", () => if (isUnderMinIsr) 1 else 0, tags) newGauge("AtMinIsr", () => if (isAtMinIsr) 1 else 0, tags) newGauge("ReplicasCount", () => if (isLeader) assignmentState.replicationFactor else 0, tags) @@ -276,11 +287,11 @@ class Partition(val topicPartition: TopicPartition, def hasLateTransaction(currentTimeMs: Long): Boolean = leaderLogIfLocal.exists(_.hasLateTransaction(currentTimeMs)) - def isUnderReplicated: Boolean = isLeader && (assignmentState.replicationFactor - isrState.isr.size) > 0 + def isUnderReplicated: Boolean = isLeader && (assignmentState.replicationFactor - partitionState.isr.size) > 0 - def isUnderMinIsr: Boolean = leaderLogIfLocal.exists { isrState.isr.size < _.config.minInSyncReplicas } + def isUnderMinIsr: Boolean = leaderLogIfLocal.exists { partitionState.isr.size < _.config.minInSyncReplicas } - def isAtMinIsr: Boolean = leaderLogIfLocal.exists { isrState.isr.size == _.config.minInSyncReplicas } + def isAtMinIsr: Boolean = leaderLogIfLocal.exists { partitionState.isr.size == _.config.minInSyncReplicas } def isReassigning: Boolean = assignmentState.isInstanceOf[OngoingReassignmentState] @@ -288,7 +299,7 @@ class Partition(val topicPartition: TopicPartition, def isAddingReplica(replicaId: Int): Boolean = assignmentState.isAddingReplica(replicaId) - def inSyncReplicaIds: Set[Int] = isrState.isr + def inSyncReplicaIds: Set[Int] = partitionState.isr /** * Create the future replica if 1) the current replica is not in the given log directory and 2) the future replica @@ -510,7 +521,7 @@ class Partition(val topicPartition: TopicPartition, assignmentState = SimpleAssignmentState(Seq.empty) log = None futureLog = None - isrState = CommittedIsr(Set.empty) + partitionState = CommittedPartitionState(Set.empty, LeaderRecoveryState.RECOVERED) leaderReplicaIdOpt = None leaderEpochStartOffsetOpt = None Partition.removeMetrics(topicPartition) @@ -538,11 +549,18 @@ class Partition(val topicPartition: TopicPartition, val addingReplicas = partitionState.addingReplicas.asScala.map(_.toInt) val removingReplicas = partitionState.removingReplicas.asScala.map(_.toInt) + if (partitionState.leaderRecoveryState == LeaderRecoveryState.RECOVERING) { + stateChangeLogger.info( + s"The topic partition $topicPartition was marked as RECOVERING. Leader log recovery is not implemented. " + + "Marking the topic partition as RECOVERED." + ) + } updateAssignmentAndIsr( assignment = partitionState.replicas.asScala.map(_.toInt), isr = isr, addingReplicas = addingReplicas, - removingReplicas = removingReplicas + removingReplicas = removingReplicas, + LeaderRecoveryState.RECOVERED ) try { createLogIfNotExists(partitionState.isNew, isFutureReplica = false, highWatermarkCheckpoints, topicId) @@ -577,7 +595,7 @@ class Partition(val topicPartition: TopicPartition, val curTimeMs = time.milliseconds // initialize lastCaughtUpTime of replicas as well as their lastFetchTimeMs and lastFetchLeaderLogEndOffset. remoteReplicas.foreach { replica => - val lastCaughtUpTimeMs = if (isrState.isr.contains(replica.brokerId)) curTimeMs else 0L + val lastCaughtUpTimeMs = if (partitionState.isr.contains(replica.brokerId)) curTimeMs else 0L replica.resetLastCaughtUpTime(leaderEpochStartOffset, curTimeMs, lastCaughtUpTimeMs) } @@ -622,7 +640,8 @@ class Partition(val topicPartition: TopicPartition, assignment = partitionState.replicas.asScala.iterator.map(_.toInt).toSeq, isr = Set.empty[Int], addingReplicas = partitionState.addingReplicas.asScala.map(_.toInt), - removingReplicas = partitionState.removingReplicas.asScala.map(_.toInt) + removingReplicas = partitionState.removingReplicas.asScala.map(_.toInt), + LeaderRecoveryState.of(partitionState.leaderRecoveryState) ) try { createLogIfNotExists(partitionState.isNew, isFutureReplica = false, highWatermarkCheckpoints, topicId) @@ -726,7 +745,8 @@ class Partition(val topicPartition: TopicPartition, def updateAssignmentAndIsr(assignment: Seq[Int], isr: Set[Int], addingReplicas: Seq[Int], - removingReplicas: Seq[Int]): Unit = { + removingReplicas: Seq[Int], + leaderRecoveryState: LeaderRecoveryState): Unit = { val newRemoteReplicas = assignment.filter(_ != localBrokerId) val removedReplicas = remoteReplicasMap.keys.filter(!newRemoteReplicas.contains(_)) @@ -739,7 +759,7 @@ class Partition(val topicPartition: TopicPartition, assignmentState = OngoingReassignmentState(addingReplicas, removingReplicas, assignment) else assignmentState = SimpleAssignmentState(assignment) - isrState = CommittedIsr(isr) + partitionState = CommittedPartitionState(isr, leaderRecoveryState) } /** @@ -757,21 +777,21 @@ class Partition(val topicPartition: TopicPartition, * This function can be triggered when a replica's LEO has incremented. */ private def maybeExpandIsr(followerReplica: Replica): Unit = { - val needsIsrUpdate = !isrState.isInflight && canAddReplicaToIsr(followerReplica.brokerId) && inReadLock(leaderIsrUpdateLock) { + val needsIsrUpdate = !partitionState.isInflight && canAddReplicaToIsr(followerReplica.brokerId) && inReadLock(leaderIsrUpdateLock) { needsExpandIsr(followerReplica) } if (needsIsrUpdate) { val alterIsrUpdateOpt = inWriteLock(leaderIsrUpdateLock) { // check if this replica needs to be added to the ISR - if (!isrState.isInflight && needsExpandIsr(followerReplica)) { + if (!partitionState.isInflight && needsExpandIsr(followerReplica)) { Some(prepareIsrExpand(followerReplica.brokerId)) } else { None } } - // Send the AlterIsr request outside of the LeaderAndIsr lock since the completion logic + // Send the AlterPartition request outside of the LeaderAndIsr lock since the completion logic // may increment the high watermark (and consequently complete delayed operations). - alterIsrUpdateOpt.foreach(submitAlterIsr) + alterIsrUpdateOpt.foreach(submitAlterPartition) } } @@ -780,7 +800,7 @@ class Partition(val topicPartition: TopicPartition, } private def canAddReplicaToIsr(followerReplicaId: Int): Boolean = { - val current = isrState + val current = partitionState !current.isInflight && !current.isr.contains(followerReplicaId) } @@ -803,7 +823,7 @@ class Partition(val topicPartition: TopicPartition, leaderLogIfLocal match { case Some(leaderLog) => // keep the current immutable replica list reference - val curMaximalIsr = isrState.maximalIsr + val curMaximalIsr = partitionState.maximalIsr if (isTraceEnabled) { def logEndOffsetString: ((Int, Long)) => String = { @@ -851,7 +871,7 @@ class Partition(val topicPartition: TopicPartition, * follower's log end offset may keep falling behind the HW (determined by the leader's log end offset) and therefore * will never be added to ISR. * - * With the addition of AlterIsr, we also consider newly added replicas as part of the ISR when advancing + * With the addition of AlterPartition, we also consider newly added replicas as part of the ISR when advancing * the HW. These replicas have not yet been committed to the ISR by the controller, so we could revert to the previously * committed ISR. However, adding additional replicas to the ISR makes it more restrictive and therefor safe. We call * this set the "maximal" ISR. See KIP-497 for more details @@ -867,7 +887,7 @@ class Partition(val topicPartition: TopicPartition, remoteReplicasMap.values.foreach { replica => // Note here we are using the "maximal", see explanation above if (replica.logEndOffsetMetadata.messageOffset < newHighWatermark.messageOffset && - (curTime - replica.lastCaughtUpTimeMs <= replicaLagTimeMaxMs || isrState.maximalIsr.contains(replica.brokerId))) { + (curTime - replica.lastCaughtUpTimeMs <= replicaLagTimeMaxMs || partitionState.maximalIsr.contains(replica.brokerId))) { newHighWatermark = replica.logEndOffsetMetadata } } @@ -925,7 +945,7 @@ class Partition(val topicPartition: TopicPartition, def maybeShrinkIsr(): Unit = { def needsIsrUpdate: Boolean = { - !isrState.isInflight && inReadLock(leaderIsrUpdateLock) { + !partitionState.isInflight && inReadLock(leaderIsrUpdateLock) { needsShrinkIsr() } } @@ -934,15 +954,15 @@ class Partition(val topicPartition: TopicPartition, val alterIsrUpdateOpt = inWriteLock(leaderIsrUpdateLock) { leaderLogIfLocal.flatMap { leaderLog => val outOfSyncReplicaIds = getOutOfSyncReplicas(replicaLagTimeMaxMs) - if (!isrState.isInflight && outOfSyncReplicaIds.nonEmpty) { + if (!partitionState.isInflight && outOfSyncReplicaIds.nonEmpty) { val outOfSyncReplicaLog = outOfSyncReplicaIds.map { replicaId => val logEndOffsetMessage = getReplica(replicaId) .map(_.logEndOffset.toString) .getOrElse("unknown") s"(brokerId: $replicaId, endOffset: $logEndOffsetMessage)" }.mkString(" ") - val newIsrLog = (isrState.isr -- outOfSyncReplicaIds).mkString(",") - info(s"Shrinking ISR from ${isrState.isr.mkString(",")} to $newIsrLog. " + + val newIsrLog = (partitionState.isr -- outOfSyncReplicaIds).mkString(",") + info(s"Shrinking ISR from ${partitionState.isr.mkString(",")} to $newIsrLog. " + s"Leader: (highWatermark: ${leaderLog.highWatermark}, " + s"endOffset: ${leaderLog.logEndOffset}). " + s"Out of sync replicas: $outOfSyncReplicaLog.") @@ -952,9 +972,9 @@ class Partition(val topicPartition: TopicPartition, } } } - // Send the AlterIsr request outside of the LeaderAndIsr lock since the completion logic + // Send the AlterPartition request outside of the LeaderAndIsr lock since the completion logic // may increment the high watermark (and consequently complete delayed operations). - alterIsrUpdateOpt.foreach(submitAlterIsr) + alterIsrUpdateOpt.foreach(submitAlterPartition) } } @@ -986,7 +1006,7 @@ class Partition(val topicPartition: TopicPartition, * If an ISR update is in-flight, we will return an empty set here **/ def getOutOfSyncReplicas(maxLagMs: Long): Set[Int] = { - val current = isrState + val current = partitionState if (!current.isInflight) { val candidateReplicaIds = current.isr - localBrokerId val currentTimeMs = time.milliseconds() @@ -1047,11 +1067,11 @@ class Partition(val topicPartition: TopicPartition, leaderLogIfLocal match { case Some(leaderLog) => val minIsr = leaderLog.config.minInSyncReplicas - val inSyncSize = isrState.isr.size + val inSyncSize = partitionState.isr.size // Avoid writing to leader if there are not enough insync replicas to make it safe if (inSyncSize < minIsr && requiredAcks == -1) { - throw new NotEnoughReplicasException(s"The size of the current ISR ${isrState.isr} " + + throw new NotEnoughReplicasException(s"The size of the current ISR ${partitionState.isr} " + s"is insufficient to satisfy the min.isr requirement of $minIsr for partition $topicPartition") } @@ -1330,25 +1350,25 @@ class Partition(val topicPartition: TopicPartition, // reflect the updated ISR even if there is a delay before we receive the confirmation. // Alternatively, if the update fails, no harm is done since the expanded ISR puts // a stricter requirement for advancement of the HW. - val isrToSend = isrState.isr + newInSyncReplicaId - val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, isrToSend.toList, zkVersion) - val updatedState = PendingExpandIsr(isrState.isr, newInSyncReplicaId, newLeaderAndIsr) - isrState = updatedState + val isrToSend = partitionState.isr + newInSyncReplicaId + val newLeaderAndIsr = LeaderAndIsr(localBrokerId, leaderEpoch, isrToSend.toList, partitionState.leaderRecoveryState, zkVersion) + val updatedState = PendingExpandIsr(partitionState.isr, newInSyncReplicaId, newLeaderAndIsr) + partitionState = updatedState updatedState } private[cluster] def prepareIsrShrink(outOfSyncReplicaIds: Set[Int]): PendingShrinkIsr = { // When shrinking the ISR, we cannot assume that the update will succeed as this could - // erroneously advance the HW if the `AlterIsr` were to fail. Hence the "maximal ISR" + // erroneously advance the HW if the `AlterPartition` were to fail. Hence the "maximal ISR" // for `PendingShrinkIsr` is the the current ISR. - val isrToSend = isrState.isr -- outOfSyncReplicaIds - val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, isrToSend.toList, zkVersion) - val updatedState = PendingShrinkIsr(isrState.isr, outOfSyncReplicaIds, newLeaderAndIsr) - isrState = updatedState + val isrToSend = partitionState.isr -- outOfSyncReplicaIds + val newLeaderAndIsr = LeaderAndIsr(localBrokerId, leaderEpoch, isrToSend.toList, partitionState.leaderRecoveryState, zkVersion) + val updatedState = PendingShrinkIsr(partitionState.isr, outOfSyncReplicaIds, newLeaderAndIsr) + partitionState = updatedState updatedState } - private def submitAlterIsr(proposedIsrState: PendingIsrChange): CompletableFuture[LeaderAndIsr] = { + private def submitAlterPartition(proposedIsrState: PendingPartitionChange): CompletableFuture[LeaderAndIsr] = { debug(s"Submitting ISR state change $proposedIsrState") val future = alterIsrManager.submit(topicPartition, proposedIsrState.sentLeaderAndIsr, controllerEpoch) future.whenComplete { (leaderAndIsr, e) => @@ -1356,16 +1376,16 @@ class Partition(val topicPartition: TopicPartition, var shouldRetry = false inWriteLock(leaderIsrUpdateLock) { - if (isrState != proposedIsrState) { - // This means isrState was updated through leader election or some other mechanism - // before we got the AlterIsr response. We don't know what happened on the controller + if (partitionState != proposedIsrState) { + // This means partitionState was updated through leader election or some other mechanism + // before we got the AlterPartition response. We don't know what happened on the controller // exactly, but we do know this response is out of date so we ignore it. debug(s"Ignoring failed ISR update to $proposedIsrState since we have already " + - s"updated state to $isrState") + s"updated state to $partitionState") } else if (leaderAndIsr != null) { - hwIncremented = handleAlterIsrUpdate(proposedIsrState, leaderAndIsr) + hwIncremented = handleAlterPartitionUpdate(proposedIsrState, leaderAndIsr) } else { - shouldRetry = handleAlterIsrError(proposedIsrState, Errors.forException(e)) + shouldRetry = handleAlterPartitionError(proposedIsrState, Errors.forException(e)) } } @@ -1373,45 +1393,48 @@ class Partition(val topicPartition: TopicPartition, tryCompleteDelayedRequests() } - // Send the AlterIsr request outside of the LeaderAndIsr lock since the completion logic + // Send the AlterPartition request outside of the LeaderAndIsr lock since the completion logic // may increment the high watermark (and consequently complete delayed operations). if (shouldRetry) { - submitAlterIsr(proposedIsrState) + submitAlterPartition(proposedIsrState) } } } /** - * Handle a failed `AlterIsr` request. For errors which are non-retriable, we simply give up. - * This leaves [[Partition.isrState]] in a pending state. Since the error was non-retriable, + * Handle a failed `AlterPartition` request. For errors which are non-retriable, we simply give up. + * This leaves [[Partition.partitionState]] in a pending state. Since the error was non-retriable, * we are okay staying in this state until we see new metadata from LeaderAndIsr (or an update * to the KRaft metadata log). * * @param proposedIsrState The ISR state change that was requested * @param error The error returned from [[AlterIsrManager]] - * @return true if the `AlterIsr` request should be retried, false otherwise + * @return true if the `AlterPartition` request should be retried, false otherwise */ - private def handleAlterIsrError( - proposedIsrState: PendingIsrChange, + private def handleAlterPartitionError( + proposedIsrState: PendingPartitionChange, error: Errors ): Boolean = { isrChangeListener.markFailed() error match { case Errors.OPERATION_NOT_ATTEMPTED => // Since the operation was not attempted, it is safe to reset back to the committed state. - isrState = CommittedIsr(proposedIsrState.isr) - debug(s"Failed to update ISR to $proposedIsrState since there is a pending ISR update still inflight. " + - s"ISR state has been reset to the latest committed state $isrState") + partitionState = CommittedPartitionState(proposedIsrState.isr, LeaderRecoveryState.RECOVERED) + debug(s"Failed to alter partition to $proposedIsrState since there is a pending AlterPartition still inflight. " + + s"partition state has been reset to the latest committed state $partitionState") false case Errors.UNKNOWN_TOPIC_OR_PARTITION => - debug(s"Failed to update ISR to $proposedIsrState since the controller doesn't know about " + + debug(s"Failed to alter partition to $proposedIsrState since the controller doesn't know about " + "this topic or partition. Giving up.") false case Errors.FENCED_LEADER_EPOCH => - debug(s"Failed to update ISR to $proposedIsrState since the leader epoch is old. Giving up.") + debug(s"Failed to alter partition to $proposedIsrState since the leader epoch is old. Giving up.") false case Errors.INVALID_UPDATE_VERSION => - debug(s"Failed to update ISR to $proposedIsrState because the version is invalid. Giving up.") + debug(s"Failed to alter partition to $proposedIsrState because the partition epoch is invalid. Giving up.") + false + case Errors.INVALID_REQUEST => + debug(s"Failed to alter partition to $proposedIsrState because the request is invalid. Giving up.") false case _ => warn(s"Failed to update ISR to $proposedIsrState due to unexpected $error. Retrying.") @@ -1420,14 +1443,14 @@ class Partition(val topicPartition: TopicPartition, } /** - * Handle a successful `AlterIsr` response. + * Handle a successful `AlterPartition` response. * * @param proposedIsrState The ISR state change that was requested * @param leaderAndIsr The updated LeaderAndIsr state * @return true if the high watermark was successfully incremented following, false otherwise */ - private def handleAlterIsrUpdate( - proposedIsrState: PendingIsrChange, + private def handleAlterPartitionUpdate( + proposedIsrState: PendingPartitionChange, leaderAndIsr: LeaderAndIsr ): Boolean = { // Success from controller, still need to check a few things @@ -1445,9 +1468,9 @@ class Partition(val topicPartition: TopicPartition, // 2) leaderAndIsr.zkVersion == zkVersion: No update was performed since proposed and actual state are the same. // In both cases, we want to move from Pending to Committed state to ensure new updates are processed. - isrState = CommittedIsr(leaderAndIsr.isr.toSet) + partitionState = CommittedPartitionState(leaderAndIsr.isr.toSet, leaderAndIsr.leaderRecoveryState) zkVersion = leaderAndIsr.zkVersion - info(s"ISR updated to ${isrState.isr.mkString(",")} and version updated to $zkVersion") + info(s"ISR updated to ${partitionState.isr.mkString(",")} and version updated to $zkVersion") proposedIsrState match { case PendingExpandIsr(_, _, _) => isrChangeListener.markExpand() @@ -1473,13 +1496,14 @@ class Partition(val topicPartition: TopicPartition, partitionString.append("; Partition: " + partitionId) partitionString.append("; Leader: " + leaderReplicaIdOpt) partitionString.append("; Replicas: " + assignmentState.replicas.mkString(",")) - partitionString.append("; ISR: " + isrState.isr.mkString(",")) + partitionString.append("; ISR: " + partitionState.isr.mkString(",")) assignmentState match { case OngoingReassignmentState(adding, removing, _) => partitionString.append("; AddingReplicas: " + adding.mkString(",")) partitionString.append("; RemovingReplicas: " + removing.mkString(",")) case _ => } + partitionString.append("; LeaderRecoveryState: " + partitionState.leaderRecoveryState) partitionString.toString } } diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 2f10710ddfef..a6ac94780c4d 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -377,7 +377,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, val result = leaderAndIsrRequestMap.getOrElseUpdate(brokerId, mutable.Map.empty) val alreadyNew = result.get(topicPartition).exists(_.isNew) val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr - result.put(topicPartition, new LeaderAndIsrPartitionState() + val partitionState = new LeaderAndIsrPartitionState() .setTopicName(topicPartition.topic) .setPartitionIndex(topicPartition.partition) .setControllerEpoch(leaderIsrAndControllerEpoch.controllerEpoch) @@ -388,7 +388,13 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, .setReplicas(replicaAssignment.replicas.map(Integer.valueOf).asJava) .setAddingReplicas(replicaAssignment.addingReplicas.map(Integer.valueOf).asJava) .setRemovingReplicas(replicaAssignment.removingReplicas.map(Integer.valueOf).asJava) - .setIsNew(isNew || alreadyNew)) + .setIsNew(isNew || alreadyNew) + + if (config.interBrokerProtocolVersion >= KAFKA_3_2_IV0) { + partitionState.setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value) + } + + result.put(topicPartition, partitionState) } addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(topicPartition)) @@ -454,7 +460,8 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, private def sendLeaderAndIsrRequest(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = { val leaderAndIsrRequestVersion: Short = - if (config.interBrokerProtocolVersion >= KAFKA_2_8_IV1) 5 + if (config.interBrokerProtocolVersion >= KAFKA_3_2_IV0) 6 + else if (config.interBrokerProtocolVersion >= KAFKA_2_8_IV1) 5 else if (config.interBrokerProtocolVersion >= KAFKA_2_4_IV1) 4 else if (config.interBrokerProtocolVersion >= KAFKA_2_4_IV0) 3 else if (config.interBrokerProtocolVersion >= KAFKA_2_2_IV0) 2 diff --git a/core/src/main/scala/kafka/controller/Election.scala b/core/src/main/scala/kafka/controller/Election.scala index dffa88841aac..1e1ee4e5b469 100644 --- a/core/src/main/scala/kafka/controller/Election.scala +++ b/core/src/main/scala/kafka/controller/Election.scala @@ -28,6 +28,7 @@ object Election { private def leaderForOffline(partition: TopicPartition, leaderAndIsrOpt: Option[LeaderAndIsr], uncleanLeaderElectionEnabled: Boolean, + isLeaderRecoverySupported: Boolean, controllerContext: ControllerContext): ElectionResult = { val assignment = controllerContext.partitionReplicaAssignment(partition) @@ -40,7 +41,14 @@ object Election { val newLeaderAndIsrOpt = leaderOpt.map { leader => val newIsr = if (isr.contains(leader)) isr.filter(replica => controllerContext.isReplicaOnline(replica, partition)) else List(leader) - leaderAndIsr.newLeaderAndIsr(leader, newIsr) + + if (!isr.contains(leader) && isLeaderRecoverySupported) { + // The new leader is not in the old ISR so mark the partition a RECOVERING + leaderAndIsr.newRecoveringLeaderAndIsr(leader, newIsr) + } else { + // Elect a new leader but keep the previous leader recovery state + leaderAndIsr.newLeaderAndIsr(leader, newIsr) + } } ElectionResult(partition, newLeaderAndIsrOpt, liveReplicas) @@ -53,7 +61,8 @@ object Election { * Elect leaders for new or offline partitions. * * @param controllerContext Context with the current state of the cluster - * @param partitionsWithUncleanLeaderElectionState A sequence of tuples representing the partitions + * @param isLeaderRecoverySupported true leader recovery is support and should be set if election is unclean + * @param partitionsWithUncleanLeaderRecoveryState A sequence of tuples representing the partitions * that need election, their leader/ISR state, and whether * or not unclean leader election is enabled * @@ -61,11 +70,12 @@ object Election { */ def leaderForOffline( controllerContext: ControllerContext, - partitionsWithUncleanLeaderElectionState: Seq[(TopicPartition, Option[LeaderAndIsr], Boolean)] + isLeaderRecoverySupported: Boolean, + partitionsWithUncleanLeaderRecoveryState: Seq[(TopicPartition, Option[LeaderAndIsr], Boolean)] ): Seq[ElectionResult] = { - partitionsWithUncleanLeaderElectionState.map { + partitionsWithUncleanLeaderRecoveryState.map { case (partition, leaderAndIsrOpt, uncleanLeaderElectionEnabled) => - leaderForOffline(partition, leaderAndIsrOpt, uncleanLeaderElectionEnabled, controllerContext) + leaderForOffline(partition, leaderAndIsrOpt, uncleanLeaderElectionEnabled, isLeaderRecoverySupported, controllerContext) } } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index c8d973aed180..7aeeded87fbd 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -21,7 +21,7 @@ import java.util.concurrent.TimeUnit import kafka.admin.AdminOperationException import kafka.api._ import kafka.common._ -import kafka.controller.KafkaController.AlterIsrCallback +import kafka.controller.KafkaController.AlterPartitionCallback import kafka.cluster.Broker import kafka.controller.KafkaController.{AlterReassignmentsCallback, ElectLeadersCallback, ListReassignmentsCallback, UpdateFeaturesCallback} import kafka.coordinator.transaction.ZkProducerIdManager @@ -37,12 +37,13 @@ import org.apache.kafka.common.ElectionType import org.apache.kafka.common.KafkaException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{BrokerNotAvailableException, ControllerMovedException, StaleBrokerEpochException} -import org.apache.kafka.common.message.{AllocateProducerIdsRequestData, AllocateProducerIdsResponseData, AlterIsrRequestData, AlterIsrResponseData, UpdateFeaturesRequestData} import org.apache.kafka.common.feature.{Features, FinalizedVersionRange} +import org.apache.kafka.common.message.{AllocateProducerIdsRequestData, AllocateProducerIdsResponseData, AlterPartitionRequestData, AlterPartitionResponseData, UpdateFeaturesRequestData} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AbstractControlRequest, ApiError, LeaderAndIsrResponse, UpdateFeaturesRequest, UpdateMetadataResponse} import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.server.common.ProducerIdsBlock import org.apache.zookeeper.KeeperException import org.apache.zookeeper.KeeperException.Code @@ -64,7 +65,7 @@ object KafkaController extends Logging { type ElectLeadersCallback = Map[TopicPartition, Either[ApiError, Int]] => Unit type ListReassignmentsCallback = Either[Map[TopicPartition, ReplicaAssignment], ApiError] => Unit type AlterReassignmentsCallback = Either[Map[TopicPartition, ApiError], ApiError] => Unit - type AlterIsrCallback = Either[Map[TopicPartition, Either[Errors, LeaderAndIsr]], Errors] => Unit + type AlterPartitionCallback = Either[Map[TopicPartition, Either[Errors, LeaderAndIsr]], Errors] => Unit type UpdateFeaturesCallback = Either[ApiError, Map[String, ApiError]] => Unit } @@ -1219,7 +1220,7 @@ class KafkaController(val config: KafkaConfig, s"controller was elected with epoch $controllerEpoch. Aborting state change by this controller") // increment the leader epoch even if there are no leader or isr changes to allow the leader to cache the expanded // assigned replica list - val newLeaderAndIsr = leaderAndIsr.newEpochAndZkVersion + val newLeaderAndIsr = leaderAndIsr.newEpoch // update the new leadership decision in zookeeper or retry val UpdateLeaderAndIsrResult(finishedUpdates, _) = zkClient.updateLeaderAndIsr(immutable.Map(partition -> newLeaderAndIsr), epoch, controllerContext.epochZkVersion) @@ -2132,7 +2133,7 @@ class KafkaController(val config: KafkaConfig, processUpdateNotifications(partitions) // During a partial upgrade, the controller may be on an IBP which assumes - // ISR changes through the `AlterIsr` API while some brokers are on an older + // ISR changes through the `AlterPartition` API while some brokers are on an older // IBP which assumes notification through Zookeeper. In this case, since the // controller will not have registered watches for reassigning partitions, we // can still rely on the batch ISR change notification path in order to @@ -2245,19 +2246,26 @@ class KafkaController(val config: KafkaConfig, } } - def alterIsrs(alterIsrRequest: AlterIsrRequestData, callback: AlterIsrResponseData => Unit): Unit = { - val isrsToAlter = mutable.Map[TopicPartition, LeaderAndIsr]() + def alterPartitions(alterPartitionRequest: AlterPartitionRequestData, callback: AlterPartitionResponseData => Unit): Unit = { + val partitionsToAlter = mutable.Map[TopicPartition, LeaderAndIsr]() - alterIsrRequest.topics.forEach { topicReq => + alterPartitionRequest.topics.forEach { topicReq => topicReq.partitions.forEach { partitionReq => - val tp = new TopicPartition(topicReq.name, partitionReq.partitionIndex) - val newIsr = partitionReq.newIsr().asScala.toList.map(_.toInt) - isrsToAlter.put(tp, new LeaderAndIsr(alterIsrRequest.brokerId, partitionReq.leaderEpoch, newIsr, partitionReq.currentIsrVersion)) + partitionsToAlter.put( + new TopicPartition(topicReq.name, partitionReq.partitionIndex), + LeaderAndIsr( + alterPartitionRequest.brokerId, + partitionReq.leaderEpoch, + partitionReq.newIsr().asScala.toList.map(_.toInt), + LeaderRecoveryState.of(partitionReq.leaderRecoveryState), + partitionReq.partitionEpoch + ) + ) } } def responseCallback(results: Either[Map[TopicPartition, Either[Errors, LeaderAndIsr]], Errors]): Unit = { - val resp = new AlterIsrResponseData() + val resp = new AlterPartitionResponseData() results match { case Right(error) => resp.setErrorCode(error.code) @@ -2267,7 +2275,7 @@ class KafkaController(val config: KafkaConfig, .groupBy { case (tp, _) => tp.topic } // Group by topic .foreach { case (topic, partitions) => // Add each topic part to the response - val topicResp = new AlterIsrResponseData.TopicData() + val topicResp = new AlterPartitionResponseData.TopicData() .setName(topic) .setPartitions(new util.ArrayList()) resp.topics.add(topicResp) @@ -2275,16 +2283,24 @@ class KafkaController(val config: KafkaConfig, // Add each partition part to the response (new ISR or error) errorOrIsr match { case Left(error) => topicResp.partitions.add( - new AlterIsrResponseData.PartitionData() + new AlterPartitionResponseData.PartitionData() .setPartitionIndex(tp.partition) .setErrorCode(error.code)) - case Right(leaderAndIsr) => topicResp.partitions.add( - new AlterIsrResponseData.PartitionData() - .setPartitionIndex(tp.partition) - .setLeaderId(leaderAndIsr.leader) - .setLeaderEpoch(leaderAndIsr.leaderEpoch) - .setIsr(leaderAndIsr.isr.map(Integer.valueOf).asJava) - .setCurrentIsrVersion(leaderAndIsr.zkVersion)) + case Right(leaderAndIsr) => + /* Setting the LeaderRecoveryState field is always safe because it will always be the same + * as the value set in the request. For version 0, that is always the default RECOVERED + * which is ignored when serializing to version 0. For any other version, the + * LeaderRecoveryState field is supported. + */ + topicResp.partitions.add( + new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(tp.partition) + .setLeaderId(leaderAndIsr.leader) + .setLeaderEpoch(leaderAndIsr.leaderEpoch) + .setIsr(leaderAndIsr.isr.map(Integer.valueOf).asJava) + .setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value) + .setPartitionEpoch(leaderAndIsr.zkVersion) + ) } } } @@ -2292,12 +2308,17 @@ class KafkaController(val config: KafkaConfig, callback.apply(resp) } - eventManager.put(AlterIsrReceived(alterIsrRequest.brokerId, alterIsrRequest.brokerEpoch, isrsToAlter, responseCallback)) + eventManager.put( + AlterPartitionReceived(alterPartitionRequest.brokerId, alterPartitionRequest.brokerEpoch, partitionsToAlter, responseCallback) + ) } - private def processAlterIsr(brokerId: Int, brokerEpoch: Long, - isrsToAlter: Map[TopicPartition, LeaderAndIsr], - callback: AlterIsrCallback): Unit = { + private def processAlterPartition( + brokerId: Int, + brokerEpoch: Long, + partitionsToAlter: Map[TopicPartition, LeaderAndIsr], + callback: AlterPartitionCallback + ): Unit = { // Handle a few short-circuits if (!isActive) { @@ -2307,13 +2328,13 @@ class KafkaController(val config: KafkaConfig, val brokerEpochOpt = controllerContext.liveBrokerIdAndEpochs.get(brokerId) if (brokerEpochOpt.isEmpty) { - info(s"Ignoring AlterIsr due to unknown broker $brokerId") + info(s"Ignoring AlterPartition due to unknown broker $brokerId") callback.apply(Right(Errors.STALE_BROKER_EPOCH)) return } if (!brokerEpochOpt.contains(brokerEpoch)) { - info(s"Ignoring AlterIsr due to stale broker epoch $brokerEpoch and local broker epoch $brokerEpochOpt for broker $brokerId") + info(s"Ignoring AlterPartition due to stale broker epoch $brokerEpoch and local broker epoch $brokerEpochOpt for broker $brokerId") callback.apply(Right(Errors.STALE_BROKER_EPOCH)) return } @@ -2322,12 +2343,28 @@ class KafkaController(val config: KafkaConfig, val partitionResponses = mutable.HashMap[TopicPartition, Either[Errors, LeaderAndIsr]]() // Determine which partitions we will accept the new ISR for - val adjustedIsrs: Map[TopicPartition, LeaderAndIsr] = isrsToAlter.flatMap { + val adjustedIsrs: Map[TopicPartition, LeaderAndIsr] = partitionsToAlter.flatMap { case (tp: TopicPartition, newLeaderAndIsr: LeaderAndIsr) => controllerContext.partitionLeadershipInfo(tp) match { case Some(leaderIsrAndControllerEpoch) => val currentLeaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr - if (newLeaderAndIsr.leaderEpoch < currentLeaderAndIsr.leaderEpoch) { + 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: " + + s"$newLeaderAndIsr" + ) + None + } else if (currentLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERED && + newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING) { + + partitionResponses(tp) = Left(Errors.INVALID_REQUEST) + info( + s"Rejecting AlterPartition from node $brokerId for $tp because the leader recovery state cannot change from " + + s"RECOVERED to RECOVERING: $newLeaderAndIsr" + ) + None + } else if (newLeaderAndIsr.leaderEpoch < currentLeaderAndIsr.leaderEpoch) { partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH) None } else if (newLeaderAndIsr.equalsIgnoreZk(currentLeaderAndIsr)) { @@ -2374,21 +2411,21 @@ class KafkaController(val config: KafkaConfig, // Update our cache and send out metadata updates updateLeaderAndIsrCache(successfulUpdates.keys.toSeq) - processUpdateNotifications(isrsToAlter.keys.toSeq) + processUpdateNotifications(partitionsToAlter.keys.toSeq) Left(partitionResponses) } catch { case e: Throwable => - error(s"Error when processing AlterIsr for partitions: ${isrsToAlter.keys.toSeq}", e) + error(s"Error when processing AlterPartition for partitions: ${partitionsToAlter.keys.toSeq}", e) Right(Errors.UNKNOWN_SERVER_ERROR) } callback.apply(response) - // After we have returned the result of the `AlterIsr` request, we should check whether + // After we have returned the result of the `AlterPartition` request, we should check whether // there are any reassignments which can be completed by a successful ISR expansion. - response.left.foreach { alterIsrResponses => - alterIsrResponses.forKeyValue { (topicPartition, partitionResponse) => + response.left.foreach { alterPartitionResponses => + alterPartitionResponses.forKeyValue { (topicPartition, partitionResponse) => if (controllerContext.partitionsBeingReassigned.contains(topicPartition)) { val isSuccessfulUpdate = partitionResponse.isRight if (isSuccessfulUpdate) { @@ -2523,8 +2560,8 @@ class KafkaController(val config: KafkaConfig, processPartitionReassignmentIsrChange(partition) case IsrChangeNotification => processIsrChangeNotification() - case AlterIsrReceived(brokerId, brokerEpoch, isrsToAlter, callback) => - processAlterIsr(brokerId, brokerEpoch, isrsToAlter, callback) + case AlterPartitionReceived(brokerId, brokerEpoch, partitionsToAlter, callback) => + processAlterPartition(brokerId, brokerEpoch, partitionsToAlter, callback) case AllocateProducerIds(brokerId, brokerEpoch, callback) => processAllocateProducerIds(brokerId, brokerEpoch, callback) case Startup => @@ -2643,6 +2680,7 @@ case class LeaderIsrAndControllerEpoch(leaderAndIsr: LeaderAndIsr, controllerEpo val leaderAndIsrInfo = new StringBuilder leaderAndIsrInfo.append("(Leader:" + leaderAndIsr.leader) leaderAndIsrInfo.append(",ISR:" + leaderAndIsr.isr.mkString(",")) + leaderAndIsrInfo.append(",LeaderRecoveryState:" + leaderAndIsr.leaderRecoveryState) leaderAndIsrInfo.append(",LeaderEpoch:" + leaderAndIsr.leaderEpoch) leaderAndIsrInfo.append(",ZkVersion:" + leaderAndIsr.zkVersion) leaderAndIsrInfo.append(",ControllerEpoch:" + controllerEpoch + ")") @@ -2785,8 +2823,9 @@ case object IsrChangeNotification extends ControllerEvent { override def preempt(): Unit = {} } -case class AlterIsrReceived(brokerId: Int, brokerEpoch: Long, isrsToAlter: Map[TopicPartition, LeaderAndIsr], - callback: AlterIsrCallback) extends ControllerEvent { +case class AlterPartitionReceived( + brokerId: Int, brokerEpoch: Long, partitionsToAlter: Map[TopicPartition, LeaderAndIsr], callback: AlterPartitionCallback +) extends ControllerEvent { override def state: ControllerState = ControllerState.IsrChange override def preempt(): Unit = {} } diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index 105e158f348d..3316e057979a 100755 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -16,6 +16,7 @@ */ package kafka.controller +import kafka.api.KAFKA_3_2_IV0 import kafka.api.LeaderAndIsr import kafka.common.StateChangeFailedException import kafka.controller.Election._ @@ -131,6 +132,8 @@ class ZkPartitionStateMachine(config: KafkaConfig, controllerBrokerRequestBatch: ControllerBrokerRequestBatch) extends PartitionStateMachine(controllerContext) { + private val isLeaderRecoverySupported = config.interBrokerProtocolVersion >= KAFKA_3_2_IV0 + private val controllerId = config.brokerId this.logIdent = s"[PartitionStateMachine controllerId=$controllerId] " @@ -410,7 +413,12 @@ class ZkPartitionStateMachine(config: KafkaConfig, validLeaderAndIsrs, allowUnclean ) - leaderForOffline(controllerContext, partitionsWithUncleanLeaderElectionState).partition(_.leaderAndIsr.isEmpty) + leaderForOffline( + controllerContext, + isLeaderRecoverySupported, + partitionsWithUncleanLeaderElectionState + ).partition(_.leaderAndIsr.isEmpty) + case ReassignPartitionLeaderElectionStrategy => leaderForReassign(controllerContext, validLeaderAndIsrs).partition(_.leaderAndIsr.isEmpty) case PreferredReplicaPartitionLeaderElectionStrategy => diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index bb8e327b1890..6a374e7afb28 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -32,7 +32,7 @@ object RequestConvertToJson { case req: AllocateProducerIdsRequest => AllocateProducerIdsRequestDataJsonConverter.write(req.data, request.version) case req: AlterClientQuotasRequest => AlterClientQuotasRequestDataJsonConverter.write(req.data, request.version) case req: AlterConfigsRequest => AlterConfigsRequestDataJsonConverter.write(req.data, request.version) - case req: AlterIsrRequest => AlterIsrRequestDataJsonConverter.write(req.data, request.version) + case req: AlterPartitionRequest => AlterPartitionRequestDataJsonConverter.write(req.data, request.version) case req: AlterPartitionReassignmentsRequest => AlterPartitionReassignmentsRequestDataJsonConverter.write(req.data, request.version) case req: AlterReplicaLogDirsRequest => AlterReplicaLogDirsRequestDataJsonConverter.write(req.data, request.version) case res: AlterUserScramCredentialsRequest => AlterUserScramCredentialsRequestDataJsonConverter.write(res.data, request.version) @@ -107,7 +107,7 @@ object RequestConvertToJson { case res: AllocateProducerIdsResponse => AllocateProducerIdsResponseDataJsonConverter.write(res.data, version) case res: AlterClientQuotasResponse => AlterClientQuotasResponseDataJsonConverter.write(res.data, version) case res: AlterConfigsResponse => AlterConfigsResponseDataJsonConverter.write(res.data, version) - case res: AlterIsrResponse => AlterIsrResponseDataJsonConverter.write(res.data, version) + case res: AlterPartitionResponse => AlterPartitionResponseDataJsonConverter.write(res.data, version) case res: AlterPartitionReassignmentsResponse => AlterPartitionReassignmentsResponseDataJsonConverter.write(res.data, version) case res: AlterReplicaLogDirsResponse => AlterReplicaLogDirsResponseDataJsonConverter.write(res.data, version) case res: AlterUserScramCredentialsResponse => AlterUserScramCredentialsResponseDataJsonConverter.write(res.data, version) diff --git a/core/src/main/scala/kafka/server/AlterIsrManager.scala b/core/src/main/scala/kafka/server/AlterIsrManager.scala index 64441ad7b9b0..99ab7378f7c5 100644 --- a/core/src/main/scala/kafka/server/AlterIsrManager.scala +++ b/core/src/main/scala/kafka/server/AlterIsrManager.scala @@ -20,6 +20,8 @@ import java.util import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, TimeUnit} +import kafka.api.ApiVersion +import kafka.api.KAFKA_3_2_IV0 import kafka.api.LeaderAndIsr import kafka.metrics.KafkaMetricsGroup import kafka.utils.{KafkaScheduler, Logging, Scheduler} @@ -27,18 +29,20 @@ import kafka.zk.KafkaZkClient import org.apache.kafka.clients.ClientResponse import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.OperationNotAttemptedException -import org.apache.kafka.common.message.AlterIsrRequestData +import org.apache.kafka.common.message.AlterPartitionRequestData import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.{AlterIsrRequest, AlterIsrResponse} +import org.apache.kafka.common.requests.{AlterPartitionRequest, AlterPartitionResponse} import org.apache.kafka.common.utils.Time +import org.apache.kafka.metadata.LeaderRecoveryState import scala.collection.mutable import scala.collection.mutable.ListBuffer +import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ /** - * Handles updating the ISR by sending AlterIsr requests to the controller (as of 2.7) or by updating ZK directly + * Handles updating the ISR by sending AlterPartition requests to the controller (as of 2.7) or by updating ZK directly * (prior to 2.7). Updating the ISR is an asynchronous operation, so partitions will learn about the result of their * request through a callback. * @@ -65,7 +69,7 @@ case class AlterIsrItem(topicPartition: TopicPartition, object AlterIsrManager { /** - * Factory to AlterIsr based implementation, used when IBP >= 2.7-IV2 + * Factory to AlterPartition based implementation, used when IBP >= 2.7-IV2 */ def apply( config: KafkaConfig, @@ -74,8 +78,7 @@ object AlterIsrManager { time: Time, metrics: Metrics, threadNamePrefix: Option[String], - brokerEpochSupplier: () => Long, - brokerId: Int + brokerEpochSupplier: () => Long ): AlterIsrManager = { val nodeProvider = MetadataCacheControllerNodeProvider(config, metadataCache) @@ -84,7 +87,7 @@ object AlterIsrManager { time = time, metrics = metrics, config = config, - channelName = "alterIsr", + channelName = "alterPartition", threadNamePrefix = threadNamePrefix, retryTimeoutMs = Long.MaxValue ) @@ -92,8 +95,9 @@ object AlterIsrManager { controllerChannelManager = channelManager, scheduler = scheduler, time = time, - brokerId = brokerId, - brokerEpochSupplier = brokerEpochSupplier + brokerId = config.brokerId, + brokerEpochSupplier = brokerEpochSupplier, + ibpVersion = config.interBrokerProtocolVersion ) } @@ -115,7 +119,8 @@ class DefaultAlterIsrManager( val scheduler: Scheduler, val time: Time, val brokerId: Int, - val brokerEpochSupplier: () => Long + val brokerEpochSupplier: () => Long, + ibpVersion: ApiVersion ) extends AlterIsrManager with Logging with KafkaMetricsGroup { // Used to allow only one pending ISR update per partition (visible for testing) @@ -161,21 +166,21 @@ class DefaultAlterIsrManager( private[server] def clearInFlightRequest(): Unit = { if (!inflightRequest.compareAndSet(true, false)) { - warn("Attempting to clear AlterIsr in-flight flag when no apparent request is in-flight") + warn("Attempting to clear AlterPartition in-flight flag when no apparent request is in-flight") } } private def sendRequest(inflightAlterIsrItems: Seq[AlterIsrItem]): Unit = { val message = buildRequest(inflightAlterIsrItems) - debug(s"Sending AlterIsr to controller $message") + debug(s"Sending AlterPartition to controller $message") - // We will not timeout AlterISR request, instead letting it retry indefinitely + // We will not timeout AlterPartition request, instead letting it retry indefinitely // until a response is received, or a new LeaderAndIsr overwrites the existing isrState // which causes the response for those partitions to be ignored. - controllerChannelManager.sendRequest(new AlterIsrRequest.Builder(message), + controllerChannelManager.sendRequest(new AlterPartitionRequest.Builder(message), new ControllerRequestCompletionHandler { override def onComplete(response: ClientResponse): Unit = { - debug(s"Received AlterIsr response $response") + debug(s"Received AlterPartition response $response") val error = try { if (response.authenticationException != null) { // For now we treat authentication errors as retriable. We use the @@ -186,8 +191,8 @@ class DefaultAlterIsrManager( } else if (response.versionMismatch != null) { Errors.UNSUPPORTED_VERSION } else { - val body = response.responseBody().asInstanceOf[AlterIsrResponse] - handleAlterIsrResponse(body, message.brokerEpoch, inflightAlterIsrItems) + val body = response.responseBody().asInstanceOf[AlterPartitionResponse] + handleAlterPartitionResponse(body, message.brokerEpoch, inflightAlterIsrItems) } } finally { // clear the flag so future requests can proceed @@ -201,48 +206,56 @@ class DefaultAlterIsrManager( maybePropagateIsrChanges() case _ => // If we received a top-level error from the controller, retry the request in the near future - scheduler.schedule("send-alter-isr", () => maybePropagateIsrChanges(), 50, -1, TimeUnit.MILLISECONDS) + scheduler.schedule("send-alter-partition", () => maybePropagateIsrChanges(), 50, -1, TimeUnit.MILLISECONDS) } } override def onTimeout(): Unit = { - throw new IllegalStateException("Encountered unexpected timeout when sending AlterIsr to the controller") + throw new IllegalStateException("Encountered unexpected timeout when sending AlterPartition to the controller") } }) } - private def buildRequest(inflightAlterIsrItems: Seq[AlterIsrItem]): AlterIsrRequestData = { - val message = new AlterIsrRequestData() + private def buildRequest(inflightAlterIsrItems: Seq[AlterIsrItem]): AlterPartitionRequestData = { + val message = new AlterPartitionRequestData() .setBrokerId(brokerId) .setBrokerEpoch(brokerEpochSupplier.apply()) - inflightAlterIsrItems.groupBy(_.topicPartition.topic).foreach { case (topic, items) => - val topicData = new AlterIsrRequestData.TopicData().setName(topic) + inflightAlterIsrItems.groupBy(_.topicPartition.topic).foreach { case (topic, items) => + val topicData = new AlterPartitionRequestData.TopicData() + .setName(topic) message.topics.add(topicData) - items.foreach { item => - topicData.partitions.add(new AlterIsrRequestData.PartitionData() + items.foreach { item => + val partitionData = new AlterPartitionRequestData.PartitionData() .setPartitionIndex(item.topicPartition.partition) .setLeaderEpoch(item.leaderAndIsr.leaderEpoch) .setNewIsr(item.leaderAndIsr.isr.map(Integer.valueOf).asJava) - .setCurrentIsrVersion(item.leaderAndIsr.zkVersion) - ) + .setPartitionEpoch(item.leaderAndIsr.zkVersion) + + if (ibpVersion >= KAFKA_3_2_IV0) { + partitionData.setLeaderRecoveryState(item.leaderAndIsr.leaderRecoveryState.value) + } + + topicData.partitions.add(partitionData) } } message } - def handleAlterIsrResponse(alterIsrResponse: AlterIsrResponse, - sentBrokerEpoch: Long, - inflightAlterIsrItems: Seq[AlterIsrItem]): Errors = { - val data = alterIsrResponse.data + def handleAlterPartitionResponse( + alterPartitionResp: AlterPartitionResponse, + sentBrokerEpoch: Long, + inflightAlterIsrItems: Seq[AlterIsrItem] + ): Errors = { + val data = alterPartitionResp.data Errors.forCode(data.errorCode) match { case Errors.STALE_BROKER_EPOCH => warn(s"Broker had a stale broker epoch ($sentBrokerEpoch), retrying.") case Errors.CLUSTER_AUTHORIZATION_FAILED => - error(s"Broker is not authorized to send AlterIsr to controller", - Errors.CLUSTER_AUTHORIZATION_FAILED.exception("Broker is not authorized to send AlterIsr to controller")) + error(s"Broker is not authorized to send AlterPartition to controller", + Errors.CLUSTER_AUTHORIZATION_FAILED.exception("Broker is not authorized to send AlterPartition to controller")) case Errors.NONE => // Collect partition-level responses to pass to the callbacks @@ -250,21 +263,34 @@ class DefaultAlterIsrManager( data.topics.forEach { topic => topic.partitions.forEach { partition => val tp = new TopicPartition(topic.name, partition.partitionIndex) - val error = Errors.forCode(partition.errorCode) - debug(s"Controller successfully handled AlterIsr request for $tp: $partition") - if (error == Errors.NONE) { - val newLeaderAndIsr = new LeaderAndIsr(partition.leaderId, partition.leaderEpoch, - partition.isr.asScala.toList.map(_.toInt), partition.currentIsrVersion) - partitionResponses(tp) = Right(newLeaderAndIsr) + val apiError = Errors.forCode(partition.errorCode) + debug(s"Controller successfully handled AlterPartition request for $tp: $partition") + if (apiError == Errors.NONE) { + LeaderRecoveryState.optionalOf(partition.leaderRecoveryState).asScala match { + case Some(leaderRecoveryState) => + partitionResponses(tp) = Right( + LeaderAndIsr( + partition.leaderId, + partition.leaderEpoch, + partition.isr.asScala.toList.map(_.toInt), + leaderRecoveryState, + partition.partitionEpoch + ) + ) + + case None => + error(s"Controller returned an invalid leader recovery state (${partition.leaderRecoveryState}) for $tp: $partition") + partitionResponses(tp) = Left(Errors.UNKNOWN_SERVER_ERROR) + } } else { - partitionResponses(tp) = Left(error) + partitionResponses(tp) = Left(apiError) } } } // Iterate across the items we sent rather than what we received to ensure we run the callback even if a // partition was somehow erroneously excluded from the response. Note that these callbacks are run from - // the leaderIsrUpdateLock write lock in Partition#sendAlterIsrRequest + // the leaderIsrUpdateLock write lock in Partition#sendAlterPartitionRequest inflightAlterIsrItems.foreach { inflightAlterIsr => partitionResponses.get(inflightAlterIsr.topicPartition) match { case Some(leaderAndIsrOrError) => @@ -284,7 +310,7 @@ class DefaultAlterIsrManager( } case e => - warn(s"Controller returned an unexpected top-level error when handling AlterIsr request: $e") + warn(s"Controller returned an unexpected top-level error when handling AlterPartition request: $e") } Errors.forCode(data.errorCode) diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 602ee95268b7..56cd2412fe59 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -252,7 +252,8 @@ class BrokerServer( scheduler = kafkaScheduler, time = time, brokerId = config.nodeId, - brokerEpochSupplier = () => lifecycleManager.brokerEpoch + brokerEpochSupplier = () => lifecycleManager.brokerEpoch, + ibpVersion = config.interBrokerProtocolVersion ) alterIsrManager.start() diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala index e6d302cdb2db..c31b205530ea 100644 --- a/core/src/main/scala/kafka/server/ControllerApis.scala +++ b/core/src/main/scala/kafka/server/ControllerApis.scala @@ -91,7 +91,7 @@ class ControllerApis(val requestChannel: RequestChannel, case ApiKeys.BEGIN_QUORUM_EPOCH => handleBeginQuorumEpoch(request) case ApiKeys.END_QUORUM_EPOCH => handleEndQuorumEpoch(request) case ApiKeys.DESCRIBE_QUORUM => handleDescribeQuorum(request) - case ApiKeys.ALTER_ISR => handleAlterIsrRequest(request) + case ApiKeys.ALTER_PARTITION => handleAlterPartitionRequest(request) case ApiKeys.BROKER_REGISTRATION => handleBrokerRegistration(request) case ApiKeys.BROKER_HEARTBEAT => handleBrokerHeartBeatRequest(request) case ApiKeys.UNREGISTER_BROKER => handleUnregisterBroker(request) @@ -510,15 +510,15 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleAlterIsrRequest(request: RequestChannel.Request): Unit = { - val alterIsrRequest = request.body[AlterIsrRequest] + def handleAlterPartitionRequest(request: RequestChannel.Request): Unit = { + val alterPartitionRequest = request.body[AlterPartitionRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - val future = controller.alterIsr(alterIsrRequest.data) + val future = controller.alterPartition(alterPartitionRequest.data) future.whenComplete { (result, exception) => val response = if (exception != null) { - alterIsrRequest.getErrorResponse(exception) + alterPartitionRequest.getErrorResponse(exception) } else { - new AlterIsrResponse(result) + new AlterPartitionResponse(result) } requestHelper.sendResponseExemptThrottle(request, response) } diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index e26b25d63048..a384926b0b5e 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -21,6 +21,7 @@ import java.util import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.{CompletableFuture, TimeUnit} +import kafka.api.KAFKA_3_2_IV0 import kafka.cluster.Broker.ServerInfo import kafka.metrics.{KafkaMetricsGroup, KafkaYammerMetrics, LinuxIoMetricsCollector} import kafka.network.{DataPlaneAcceptor, SocketServer} @@ -165,6 +166,7 @@ class ControllerServer( setRaftClient(raftManager.client). setDefaultReplicationFactor(config.defaultReplicationFactor.toShort). setDefaultNumPartitions(config.numPartitions.intValue()). + setIsLeaderRecoverySupported(config.interBrokerProtocolVersion >= KAFKA_3_2_IV0). setSessionTimeoutNs(TimeUnit.NANOSECONDS.convert(config.brokerSessionTimeoutMs.longValue(), TimeUnit.MILLISECONDS)). setSnapshotMaxNewRecordBytes(config.metadataSnapshotMaxNewRecordBytes). diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 2bc28e40c295..86295816f12a 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -221,7 +221,7 @@ class KafkaApis(val requestChannel: RequestChannel, case ApiKeys.ALTER_CLIENT_QUOTAS => maybeForwardToController(request, handleAlterClientQuotasRequest) case ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS => handleDescribeUserScramCredentialsRequest(request) case ApiKeys.ALTER_USER_SCRAM_CREDENTIALS => maybeForwardToController(request, handleAlterUserScramCredentialsRequest) - case ApiKeys.ALTER_ISR => handleAlterIsrRequest(request) + case ApiKeys.ALTER_PARTITION => handleAlterPartitionRequest(request) case ApiKeys.UPDATE_FEATURES => maybeForwardToController(request, handleUpdateFeatures) case ApiKeys.ENVELOPE => handleEnvelope(request, requestLocal) case ApiKeys.DESCRIBE_CLUSTER => handleDescribeCluster(request) @@ -3289,17 +3289,17 @@ class KafkaApis(val requestChannel: RequestChannel, } } - def handleAlterIsrRequest(request: RequestChannel.Request): Unit = { + def handleAlterPartitionRequest(request: RequestChannel.Request): Unit = { val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldNeverReceive(request)) - val alterIsrRequest = request.body[AlterIsrRequest] + val alterPartitionRequest = request.body[AlterPartitionRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) if (!zkSupport.controller.isActive) - requestHelper.sendResponseExemptThrottle(request, alterIsrRequest.getErrorResponse( + requestHelper.sendResponseExemptThrottle(request, alterPartitionRequest.getErrorResponse( AbstractResponse.DEFAULT_THROTTLE_TIME, Errors.NOT_CONTROLLER.exception)) else - zkSupport.controller.alterIsrs(alterIsrRequest.data, alterIsrResp => - requestHelper.sendResponseExemptThrottle(request, new AlterIsrResponse(alterIsrResp)) + zkSupport.controller.alterPartitions(alterPartitionRequest.data, alterPartitionResp => + requestHelper.sendResponseExemptThrottle(request, new AlterPartitionResponse(alterPartitionResp)) ) } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index afacaa162fd1..67013d339193 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -308,7 +308,7 @@ class KafkaServer( socketServer = new SocketServer(config, metrics, time, credentialProvider, apiVersionManager) socketServer.startup(startProcessingRequests = false) - /* start replica manager */ + // Start alter partition manager based on the IBP version alterIsrManager = if (config.interBrokerProtocolVersion.isAlterIsrSupported) { AlterIsrManager( config = config, @@ -317,14 +317,14 @@ class KafkaServer( time = time, metrics = metrics, threadNamePrefix = threadNamePrefix, - brokerEpochSupplier = () => kafkaController.brokerEpoch, - config.brokerId + brokerEpochSupplier = () => kafkaController.brokerEpoch ) } else { AlterIsrManager(kafkaScheduler, time, zkClient) } alterIsrManager.start() + /* start replica manager */ _replicaManager = createReplicaManager(isShuttingDown) replicaManager.startup() diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 22f2755ba58f..42124aaa3db9 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -1662,7 +1662,6 @@ class ReplicaManager(val config: KafkaConfig, val partitionsToMakeFollower: mutable.Set[Partition] = mutable.Set() try { - // TODO: Delete leaders from LeaderAndIsrRequest partitionStates.forKeyValue { (partition, partitionState) => val newLeaderBrokerId = partitionState.leader try { diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index baed563094f0..c8e1861bbd2e 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -30,23 +30,24 @@ import kafka.security.authorizer.AclEntry import kafka.server.{ConfigType, DelegationTokenManager} import kafka.utils.Json import kafka.utils.json.JsonObject -import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} import org.apache.kafka.common.errors.UnsupportedVersionException -import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange} import org.apache.kafka.common.feature.Features._ +import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType} import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.utils.{SecurityUtils, Time} +import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} +import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.server.common.ProducerIdsBlock import org.apache.zookeeper.ZooDefs import org.apache.zookeeper.data.{ACL, Stat} import scala.beans.BeanProperty -import scala.jdk.CollectionConverters._ import scala.collection.mutable.ArrayBuffer import scala.collection.{Map, Seq, immutable, mutable} +import scala.jdk.CollectionConverters._ import scala.util.{Failure, Success, Try} // This file contains objects for encoding/decoding data stored in ZooKeeper nodes (znodes). @@ -349,21 +350,39 @@ object TopicPartitionZNode { object TopicPartitionStateZNode { def path(partition: TopicPartition) = s"${TopicPartitionZNode.path(partition)}/state" + def encode(leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch): Array[Byte] = { val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr val controllerEpoch = leaderIsrAndControllerEpoch.controllerEpoch - Json.encodeAsBytes(Map("version" -> 1, "leader" -> leaderAndIsr.leader, "leader_epoch" -> leaderAndIsr.leaderEpoch, - "controller_epoch" -> controllerEpoch, "isr" -> leaderAndIsr.isr.asJava).asJava) + var partitionState = Map( + "version" -> 1, + "leader" -> leaderAndIsr.leader, + "leader_epoch" -> leaderAndIsr.leaderEpoch, + "controller_epoch" -> controllerEpoch, + "isr" -> leaderAndIsr.isr.asJava + ) + + if (leaderAndIsr.leaderRecoveryState != LeaderRecoveryState.RECOVERED) { + partitionState = partitionState ++ Seq("leader_recovery_state" -> leaderAndIsr.leaderRecoveryState.value.toInt) + } + + Json.encodeAsBytes(partitionState.asJava) } + def decode(bytes: Array[Byte], stat: Stat): Option[LeaderIsrAndControllerEpoch] = { Json.parseBytes(bytes).map { js => val leaderIsrAndEpochInfo = js.asJsonObject val leader = leaderIsrAndEpochInfo("leader").to[Int] val epoch = leaderIsrAndEpochInfo("leader_epoch").to[Int] val isr = leaderIsrAndEpochInfo("isr").to[List[Int]] + val recovery = leaderIsrAndEpochInfo + .get("leader_recovery_state") + .map(jsonValue => LeaderRecoveryState.of(jsonValue.to[Int].toByte)) + .getOrElse(LeaderRecoveryState.RECOVERED) val controllerEpoch = leaderIsrAndEpochInfo("controller_epoch").to[Int] + val zkPathVersion = stat.getVersion - LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, epoch, isr, zkPathVersion), controllerEpoch) + LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, epoch, isr, recovery, zkPathVersion), controllerEpoch) } } } diff --git a/core/src/test/java/kafka/test/MockController.java b/core/src/test/java/kafka/test/MockController.java index c163b2df49a5..acb6f90e7fdc 100644 --- a/core/src/test/java/kafka/test/MockController.java +++ b/core/src/test/java/kafka/test/MockController.java @@ -25,8 +25,8 @@ import org.apache.kafka.common.errors.NotControllerException; import org.apache.kafka.common.message.AllocateProducerIdsRequestData; import org.apache.kafka.common.message.AllocateProducerIdsResponseData; -import org.apache.kafka.common.message.AlterIsrRequestData; -import org.apache.kafka.common.message.AlterIsrResponseData; +import org.apache.kafka.common.message.AlterPartitionRequestData; +import org.apache.kafka.common.message.AlterPartitionResponseData; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; import org.apache.kafka.common.message.BrokerHeartbeatRequestData; @@ -107,7 +107,7 @@ private MockController(Collection initialTopics) { } @Override - public CompletableFuture alterIsr(AlterIsrRequestData request) { + public CompletableFuture alterPartition(AlterPartitionRequestData request) { throw new UnsupportedOperationException(); } diff --git a/core/src/test/scala/kafka/api/LeaderAndIsrTest.scala b/core/src/test/scala/kafka/api/LeaderAndIsrTest.scala new file mode 100644 index 000000000000..bcb48f748ed1 --- /dev/null +++ b/core/src/test/scala/kafka/api/LeaderAndIsrTest.scala @@ -0,0 +1,34 @@ +/** + * 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 kafka.api + +import org.apache.kafka.metadata.LeaderRecoveryState +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Test + +final class LeaderAndIsrTest { + @Test + def testRecoveringLeaderAndIsr(): Unit = { + val leaderAndIsr = LeaderAndIsr(1, List(1, 2)) + val recoveringLeaderAndIsr = leaderAndIsr.newRecoveringLeaderAndIsr(3, List(3)) + + assertEquals(3, recoveringLeaderAndIsr.leader) + assertEquals(List(3), recoveringLeaderAndIsr.isr) + assertEquals(LeaderRecoveryState.RECOVERING, recoveringLeaderAndIsr.leaderRecoveryState) + } +} diff --git a/core/src/test/scala/kafka/zk/TopicPartitionStateZNodeTest.scala b/core/src/test/scala/kafka/zk/TopicPartitionStateZNodeTest.scala new file mode 100644 index 000000000000..90d770f58dcf --- /dev/null +++ b/core/src/test/scala/kafka/zk/TopicPartitionStateZNodeTest.scala @@ -0,0 +1,74 @@ +/** + * 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 kafka.zk + +import TopicPartitionStateZNode.decode +import TopicPartitionStateZNode.encode +import kafka.api.LeaderAndIsr +import kafka.controller.LeaderIsrAndControllerEpoch +import kafka.utils.Json +import org.apache.kafka.metadata.LeaderRecoveryState +import org.apache.zookeeper.data.Stat +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Test +import org.mockito.Mockito.mock +import org.mockito.Mockito.when +import scala.jdk.CollectionConverters._ + +final class TopicPartitionStateZNodeTest { + + @Test + def testEncodeDecodeRecovering(): Unit = { + val zkVersion = 5 + val stat = mock(classOf[Stat]) + when(stat.getVersion).thenReturn(zkVersion) + + val expected = LeaderIsrAndControllerEpoch(LeaderAndIsr(1, 6, List(1), LeaderRecoveryState.RECOVERING, zkVersion), 10) + + assertEquals(Some(expected), decode(encode(expected), stat)) + } + + @Test + def testEncodeDecodeRecovered(): Unit = { + val zkVersion = 5 + val stat = mock(classOf[Stat]) + when(stat.getVersion).thenReturn(zkVersion) + + val expected = LeaderIsrAndControllerEpoch(LeaderAndIsr(1, 6, List(1), LeaderRecoveryState.RECOVERED, zkVersion), 10) + + assertEquals(Some(expected), decode(encode(expected), stat)) + } + + @Test + def testDecodeOldValue(): Unit = { + val zkVersion = 5 + val stat = mock(classOf[Stat]) + when(stat.getVersion).thenReturn(zkVersion) + + val expected = LeaderIsrAndControllerEpoch(LeaderAndIsr(1, 6, List(1), LeaderRecoveryState.RECOVERED, zkVersion), 10) + + val partitionState = Map( + "version" -> 1, + "leader" -> expected.leaderAndIsr.leader, + "leader_epoch" -> expected.leaderAndIsr.leaderEpoch, + "controller_epoch" -> expected.controllerEpoch, + "isr" -> expected.leaderAndIsr.isr.asJava + ) + + assertEquals(Some(expected), decode(Json.encodeAsBytes(partitionState.asJava), stat)) + } +} diff --git a/core/src/test/scala/unit/kafka/api/ApiVersionTest.scala b/core/src/test/scala/unit/kafka/api/ApiVersionTest.scala index 75dd68280ad7..efdc93f36ce3 100644 --- a/core/src/test/scala/unit/kafka/api/ApiVersionTest.scala +++ b/core/src/test/scala/unit/kafka/api/ApiVersionTest.scala @@ -126,6 +126,9 @@ class ApiVersionTest { assertEquals(KAFKA_3_1_IV0, ApiVersion("3.1")) assertEquals(KAFKA_3_1_IV0, ApiVersion("3.1-IV0")) + + assertEquals(KAFKA_3_2_IV0, ApiVersion("3.2")) + assertEquals(KAFKA_3_2_IV0, ApiVersion("3.2-IV0")) } @Test @@ -178,6 +181,7 @@ class ApiVersionTest { assertEquals("3.0", KAFKA_3_0_IV0.shortVersion) assertEquals("3.0", KAFKA_3_0_IV1.shortVersion) assertEquals("3.1", KAFKA_3_1_IV0.shortVersion) + assertEquals("3.2", KAFKA_3_2_IV0.shortVersion) } @Test diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 972d9d5f72d3..a5e791800695 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -34,6 +34,7 @@ import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.ListOffsetsRequest import org.apache.kafka.common.utils.SystemTime import org.apache.kafka.common.{IsolationLevel, TopicPartition, Uuid} +import org.apache.kafka.metadata.LeaderRecoveryState import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.mockito.ArgumentMatchers @@ -575,7 +576,7 @@ class PartitionTest extends AbstractPartitionTest { assertTrue(partition.makeLeader(leaderState, offsetCheckpoints, None), "Expected first makeLeader() to return 'leader changed'") assertEquals(leaderEpoch, partition.getLeaderEpoch, "Current leader epoch") - assertEquals(Set[Integer](leader, follower2), partition.isrState.isr, "ISR") + assertEquals(Set[Integer](leader, follower2), partition.partitionState.isr, "ISR") val requestLocal = RequestLocal.withThreadConfinedCaching // after makeLeader(() call, partition should know about all the replicas @@ -904,7 +905,7 @@ class PartitionTest extends AbstractPartitionTest { .setIsNew(true) assertTrue(partition.makeLeader(leaderState, offsetCheckpoints, None), "Expected first makeLeader() to return 'leader changed'") assertEquals(leaderEpoch, partition.getLeaderEpoch, "Current leader epoch") - assertEquals(Set[Integer](leader, follower2), partition.isrState.isr, "ISR") + assertEquals(Set[Integer](leader, follower2), partition.partitionState.isr, "ISR") val requestLocal = RequestLocal.withThreadConfinedCaching @@ -958,15 +959,15 @@ class PartitionTest extends AbstractPartitionTest { // fetch from follower not in ISR from log start offset should not add this follower to ISR updateFollowerFetchState(follower1, LogOffsetMetadata(0)) updateFollowerFetchState(follower1, LogOffsetMetadata(lastOffsetOfFirstBatch)) - assertEquals(Set[Integer](leader, follower2), partition.isrState.isr, "ISR") + assertEquals(Set[Integer](leader, follower2), partition.partitionState.isr, "ISR") // fetch from the follower not in ISR from start offset of the current leader epoch should // add this follower to ISR updateFollowerFetchState(follower1, LogOffsetMetadata(currentLeaderEpochStartOffset)) // Expansion does not affect the ISR - assertEquals(Set[Integer](leader, follower2), partition.isrState.isr, "ISR") - assertEquals(Set[Integer](leader, follower1, follower2), partition.isrState.maximalIsr, "ISR") + assertEquals(Set[Integer](leader, follower2), partition.partitionState.isr, "ISR") + assertEquals(Set[Integer](leader, follower1, follower2), partition.partitionState.maximalIsr, "ISR") assertEquals(alterIsrManager.isrUpdates.head.leaderAndIsr.isr.toSet, Set(leader, follower1, follower2), "AlterIsr") } @@ -1077,6 +1078,59 @@ class PartitionTest extends AbstractPartitionTest { } + @Test + def testInvalidAlterPartitionRequestsAreNotRetried(): Unit = { + val log = logManager.getOrCreateLog(topicPartition, topicId = None) + seedLogData(log, numRecords = 10, leaderEpoch = 4) + + val controllerEpoch = 0 + val leaderEpoch = 5 + val remoteBrokerId = brokerId + 1 + val replicas = List[Integer](brokerId, remoteBrokerId).asJava + val isr = List[Integer](brokerId).asJava + + partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None) + assertTrue(partition.makeLeader( + new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas) + .setIsNew(true), + offsetCheckpoints, None), "Expected become leader transition to succeed") + assertEquals(Set(brokerId), partition.partitionState.isr) + + val remoteReplica = partition.getReplica(remoteBrokerId).get + assertEquals(LogOffsetMetadata.UnknownOffsetMetadata.messageOffset, remoteReplica.logEndOffset) + assertEquals(UnifiedLog.UnknownOffset, remoteReplica.logStartOffset) + + partition.updateFollowerFetchState(remoteBrokerId, + followerFetchOffsetMetadata = LogOffsetMetadata(10), + followerStartOffset = 0L, + followerFetchTimeMs = time.milliseconds(), + leaderEndOffset = 10L) + + // Check that the isr didn't change and alter update is scheduled + assertEquals(Set(brokerId), partition.inSyncReplicaIds) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.maximalIsr) + assertEquals(1, alterIsrManager.isrUpdates.size) + assertEquals(Set(brokerId, remoteBrokerId), alterIsrManager.isrUpdates.head.leaderAndIsr.isr.toSet) + + // Simulate invalid request failure + alterIsrManager.failIsrUpdate(Errors.INVALID_REQUEST) + + // Still no ISR change and no retry + assertEquals(Set(brokerId), partition.inSyncReplicaIds) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.maximalIsr) + assertEquals(0, alterIsrManager.isrUpdates.size) + + assertEquals(0, isrChangeListener.expands.get) + assertEquals(0, isrChangeListener.shrinks.get) + assertEquals(1, isrChangeListener.failures.get) + } + @Test def testIsrExpansion(): Unit = { val log = logManager.getOrCreateLog(topicPartition, topicId = None) @@ -1099,7 +1153,7 @@ class PartitionTest extends AbstractPartitionTest { .setReplicas(replicas.map(Int.box).asJava) .setIsNew(true), offsetCheckpoints, None), "Expected become leader transition to succeed") - assertEquals(Set(brokerId), partition.isrState.isr) + assertEquals(Set(brokerId), partition.partitionState.isr) val remoteReplica = partition.getReplica(remoteBrokerId).get assertEquals(LogOffsetMetadata.UnknownOffsetMetadata.messageOffset, remoteReplica.logEndOffset) @@ -1111,7 +1165,7 @@ class PartitionTest extends AbstractPartitionTest { followerFetchTimeMs = time.milliseconds(), leaderEndOffset = 6L) - assertEquals(Set(brokerId), partition.isrState.isr) + assertEquals(Set(brokerId), partition.partitionState.isr) assertEquals(3L, remoteReplica.logEndOffset) assertEquals(0L, remoteReplica.logStartOffset) @@ -1124,14 +1178,14 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(alterIsrManager.isrUpdates.size, 1) val isrItem = alterIsrManager.isrUpdates.head assertEquals(isrItem.leaderAndIsr.isr, List(brokerId, remoteBrokerId)) - assertEquals(Set(brokerId), partition.isrState.isr) - assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.maximalIsr) + assertEquals(Set(brokerId), partition.partitionState.isr) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.maximalIsr) assertEquals(10L, remoteReplica.logEndOffset) assertEquals(0L, remoteReplica.logStartOffset) // Complete the ISR expansion alterIsrManager.completeIsrUpdate(2) - assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.isr) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.isr) assertEquals(isrChangeListener.expands.get, 1) assertEquals(isrChangeListener.shrinks.get, 0) @@ -1160,7 +1214,7 @@ class PartitionTest extends AbstractPartitionTest { .setReplicas(replicas) .setIsNew(true), offsetCheckpoints, None), "Expected become leader transition to succeed") - assertEquals(Set(brokerId), partition.isrState.isr) + assertEquals(Set(brokerId), partition.partitionState.isr) val remoteReplica = partition.getReplica(remoteBrokerId).get assertEquals(LogOffsetMetadata.UnknownOffsetMetadata.messageOffset, remoteReplica.logEndOffset) @@ -1174,7 +1228,7 @@ class PartitionTest extends AbstractPartitionTest { // Follower state is updated, but the ISR has not expanded assertEquals(Set(brokerId), partition.inSyncReplicaIds) - assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.maximalIsr) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.maximalIsr) assertEquals(alterIsrManager.isrUpdates.size, 1) assertEquals(10L, remoteReplica.logEndOffset) assertEquals(0L, remoteReplica.logStartOffset) @@ -1182,11 +1236,10 @@ class PartitionTest extends AbstractPartitionTest { // Simulate failure callback alterIsrManager.failIsrUpdate(Errors.INVALID_UPDATE_VERSION) - // Still no ISR change + // Still no ISR change and it doesn't retry assertEquals(Set(brokerId), partition.inSyncReplicaIds) - assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.maximalIsr) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.maximalIsr) assertEquals(alterIsrManager.isrUpdates.size, 0) - assertEquals(isrChangeListener.expands.get, 0) assertEquals(isrChangeListener.shrinks.get, 0) assertEquals(isrChangeListener.failures.get, 1) @@ -1222,8 +1275,8 @@ class PartitionTest extends AbstractPartitionTest { partition.maybeShrinkIsr() assertEquals(alterIsrManager.isrUpdates.size, 1) assertEquals(alterIsrManager.isrUpdates.head.leaderAndIsr.isr, List(brokerId)) - assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.isr) - assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.maximalIsr) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.isr) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.maximalIsr) // The shrink fails and we retry alterIsrManager.failIsrUpdate(Errors.NETWORK_EXCEPTION) @@ -1231,8 +1284,8 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(1, isrChangeListener.failures.get) assertEquals(1, partition.getZkVersion) assertEquals(alterIsrManager.isrUpdates.size, 1) - assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.isr) - assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.maximalIsr) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.isr) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.maximalIsr) assertEquals(0L, partition.localLogOrException.highWatermark) // The shrink succeeds after retrying @@ -1240,8 +1293,8 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(1, isrChangeListener.shrinks.get) assertEquals(2, partition.getZkVersion) assertEquals(alterIsrManager.isrUpdates.size, 0) - assertEquals(Set(brokerId), partition.isrState.isr) - assertEquals(Set(brokerId), partition.isrState.maximalIsr) + assertEquals(Set(brokerId), partition.partitionState.isr) + assertEquals(Set(brokerId), partition.partitionState.maximalIsr) assertEquals(log.logEndOffset, partition.localLogOrException.highWatermark) } @@ -1275,7 +1328,7 @@ class PartitionTest extends AbstractPartitionTest { // On initialization, the replica is considered caught up and should not be removed partition.maybeShrinkIsr() - assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.isr) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.isr) // If enough time passes without a fetch update, the ISR should shrink time.sleep(partition.replicaLagTimeMaxMs + 1) @@ -1285,8 +1338,8 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(0, isrChangeListener.shrinks.get) assertEquals(alterIsrManager.isrUpdates.size, 1) assertEquals(alterIsrManager.isrUpdates.head.leaderAndIsr.isr, List(brokerId)) - assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.isr) - assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.maximalIsr) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.isr) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.maximalIsr) assertEquals(0L, partition.localLogOrException.highWatermark) // After the ISR shrink completes, the ISR state should be updated and the @@ -1295,8 +1348,8 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(1, isrChangeListener.shrinks.get) assertEquals(2, partition.getZkVersion) assertEquals(alterIsrManager.isrUpdates.size, 0) - assertEquals(Set(brokerId), partition.isrState.isr) - assertEquals(Set(brokerId), partition.isrState.maximalIsr) + assertEquals(Set(brokerId), partition.partitionState.isr) + assertEquals(Set(brokerId), partition.partitionState.maximalIsr) assertEquals(log.logEndOffset, partition.localLogOrException.highWatermark) } @@ -1331,7 +1384,7 @@ class PartitionTest extends AbstractPartitionTest { // Shrink the ISR time.sleep(partition.replicaLagTimeMaxMs + 1) partition.maybeShrinkIsr() - assertTrue(partition.isrState.isInflight) + assertTrue(partition.partitionState.isInflight) // Become leader again, reset the ISR state assertFalse(makeLeader( @@ -1344,17 +1397,17 @@ class PartitionTest extends AbstractPartitionTest { isNew = false )) assertEquals(0L, partition.localLogOrException.highWatermark) - assertFalse(partition.isrState.isInflight, "ISR should be committed and not inflight") + assertFalse(partition.partitionState.isInflight, "ISR should be committed and not inflight") // Try the shrink again, should not submit until AlterIsr response arrives time.sleep(partition.replicaLagTimeMaxMs + 1) partition.maybeShrinkIsr() - assertFalse(partition.isrState.isInflight, "ISR should still be committed and not inflight") + assertFalse(partition.partitionState.isInflight, "ISR should still be committed and not inflight") // Complete the AlterIsr update and now we can make modifications again alterIsrManager.completeIsrUpdate(10) partition.maybeShrinkIsr() - assertTrue(partition.isrState.isInflight, "ISR should be pending a shrink") + assertTrue(partition.partitionState.isInflight, "ISR should be pending a shrink") } @Test @@ -1415,7 +1468,7 @@ class PartitionTest extends AbstractPartitionTest { // The ISR should not be shrunk because the follower has caught up with the leader at the // time of the first fetch. partition.maybeShrinkIsr() - assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.isr) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.isr) assertEquals(alterIsrManager.isrUpdates.size, 0) } @@ -1463,7 +1516,7 @@ class PartitionTest extends AbstractPartitionTest { // The ISR should not be shrunk because the follower is caught up to the leader's log end partition.maybeShrinkIsr() - assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.isr) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.isr) assertEquals(alterIsrManager.isrUpdates.size, 0) } @@ -1507,7 +1560,7 @@ class PartitionTest extends AbstractPartitionTest { alterIsrManager.failIsrUpdate(Errors.INVALID_UPDATE_VERSION) // Ensure ISR hasn't changed - assertEquals(partition.isrState.getClass, classOf[PendingShrinkIsr]) + assertEquals(partition.partitionState.getClass, classOf[PendingShrinkIsr]) assertEquals(Set(brokerId, remoteBrokerId), partition.inSyncReplicaIds) assertEquals(alterIsrManager.isrUpdates.size, 0) assertEquals(0L, partition.localLogOrException.highWatermark) @@ -1517,8 +1570,8 @@ class PartitionTest extends AbstractPartitionTest { def testAlterIsrUnknownTopic(): Unit = { handleAlterIsrFailure(Errors.UNKNOWN_TOPIC_OR_PARTITION, (brokerId: Int, remoteBrokerId: Int, partition: Partition) => { - assertEquals(partition.isrState.isr, Set(brokerId)) - assertEquals(partition.isrState.maximalIsr, Set(brokerId, remoteBrokerId)) + assertEquals(partition.partitionState.isr, Set(brokerId)) + assertEquals(partition.partitionState.maximalIsr, Set(brokerId, remoteBrokerId)) assertEquals(alterIsrManager.isrUpdates.size, 0) }) } @@ -1527,8 +1580,8 @@ class PartitionTest extends AbstractPartitionTest { def testAlterIsrInvalidVersion(): Unit = { handleAlterIsrFailure(Errors.INVALID_UPDATE_VERSION, (brokerId: Int, remoteBrokerId: Int, partition: Partition) => { - assertEquals(partition.isrState.isr, Set(brokerId)) - assertEquals(partition.isrState.maximalIsr, Set(brokerId, remoteBrokerId)) + assertEquals(partition.partitionState.isr, Set(brokerId)) + assertEquals(partition.partitionState.maximalIsr, Set(brokerId, remoteBrokerId)) assertEquals(alterIsrManager.isrUpdates.size, 0) }) } @@ -1538,8 +1591,8 @@ class PartitionTest extends AbstractPartitionTest { handleAlterIsrFailure(Errors.UNKNOWN_SERVER_ERROR, (brokerId: Int, remoteBrokerId: Int, partition: Partition) => { // We retry these - assertEquals(partition.isrState.isr, Set(brokerId)) - assertEquals(partition.isrState.maximalIsr, Set(brokerId, remoteBrokerId)) + assertEquals(partition.partitionState.isr, Set(brokerId)) + assertEquals(partition.partitionState.maximalIsr, Set(brokerId, remoteBrokerId)) assertEquals(alterIsrManager.isrUpdates.size, 1) }) } @@ -1578,7 +1631,7 @@ class PartitionTest extends AbstractPartitionTest { // Follower state is updated, but the ISR has not expanded assertEquals(Set(brokerId), partition.inSyncReplicaIds) - assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.maximalIsr) + assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.maximalIsr) assertEquals(alterIsrManager.isrUpdates.size, 1) assertEquals(10L, remoteReplica.logEndOffset) assertEquals(0L, remoteReplica.logStartOffset) @@ -1622,8 +1675,8 @@ class PartitionTest extends AbstractPartitionTest { followerFetchTimeMs = time.milliseconds(), leaderEndOffset = 10 ) - assertEquals(Set(brokerId, follower1, follower2), partition.isrState.isr) - assertEquals(Set(brokerId, follower1, follower2, follower3), partition.isrState.maximalIsr) + assertEquals(Set(brokerId, follower1, follower2), partition.partitionState.isr) + assertEquals(Set(brokerId, follower1, follower2, follower3), partition.partitionState.maximalIsr) // One AlterIsr request in-flight assertEquals(alterIsrManager.isrUpdates.size, 1) @@ -1694,10 +1747,10 @@ class PartitionTest extends AbstractPartitionTest { ) // Try avoiding a race - TestUtils.waitUntilTrue(() => !partition.isrState.isInflight, "Expected ISR state to be committed", 100) + TestUtils.waitUntilTrue(() => !partition.partitionState.isInflight, "Expected ISR state to be committed", 100) - partition.isrState match { - case committed: CommittedIsr => assertEquals(Set(brokerId, follower1, follower2, follower3), committed.isr) + partition.partitionState match { + case CommittedPartitionState(isr, _) => assertEquals(Set(brokerId, follower1, follower2, follower3), isr) case _ => fail("Expected a committed ISR following Zk expansion") } @@ -1883,11 +1936,11 @@ class PartitionTest extends AbstractPartitionTest { val removing = Seq(1, 2) // Test with ongoing reassignment - partition.updateAssignmentAndIsr(replicas, isr, adding, removing) + partition.updateAssignmentAndIsr(replicas, isr, adding, removing, LeaderRecoveryState.RECOVERED) assertTrue(partition.assignmentState.isInstanceOf[OngoingReassignmentState], "The assignmentState is not OngoingReassignmentState") assertEquals(replicas, partition.assignmentState.replicas) - assertEquals(isr, partition.isrState.isr) + assertEquals(isr, partition.partitionState.isr) assertEquals(adding, partition.assignmentState.asInstanceOf[OngoingReassignmentState].addingReplicas) assertEquals(removing, partition.assignmentState.asInstanceOf[OngoingReassignmentState].removingReplicas) assertEquals(Seq(1, 2, 3), partition.remoteReplicas.map(_.brokerId)) @@ -1895,11 +1948,11 @@ class PartitionTest extends AbstractPartitionTest { // Test with simple assignment val replicas2 = Seq(0, 3, 4, 5) val isr2 = Set(0, 3, 4, 5) - partition.updateAssignmentAndIsr(replicas2, isr2, Seq.empty, Seq.empty) + partition.updateAssignmentAndIsr(replicas2, isr2, Seq.empty, Seq.empty, LeaderRecoveryState.RECOVERED) assertTrue(partition.assignmentState.isInstanceOf[SimpleAssignmentState], "The assignmentState is not SimpleAssignmentState") assertEquals(replicas2, partition.assignmentState.replicas) - assertEquals(isr2, partition.isrState.isr) + assertEquals(isr2, partition.partitionState.isr) assertEquals(Seq(3, 4, 5), partition.remoteReplicas.map(_.brokerId)) } @@ -2044,11 +2097,11 @@ class PartitionTest extends AbstractPartitionTest { topicId ) assertTrue(partition.isLeader) - assertFalse(partition.isrState.isInflight) + assertFalse(partition.partitionState.isInflight) assertEquals(topicId, partition.topicId) assertEquals(leaderEpoch, partition.getLeaderEpoch) - assertEquals(isr.toSet, partition.isrState.isr) - assertEquals(isr.toSet, partition.isrState.maximalIsr) + assertEquals(isr.toSet, partition.partitionState.isr) + assertEquals(isr.toSet, partition.partitionState.maximalIsr) assertEquals(zkVersion, partition.getZkVersion) newLeader } diff --git a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala index 495f819df143..f9a1642cd019 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala @@ -17,26 +17,26 @@ package kafka.controller import java.util.Properties -import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, KAFKA_0_10_2_IV0, KAFKA_0_9_0, KAFKA_1_0_IV0, KAFKA_2_2_IV0, KAFKA_2_4_IV0, KAFKA_2_4_IV1, KAFKA_2_6_IV0, KAFKA_2_8_IV1, LeaderAndIsr} +import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, KAFKA_0_10_2_IV0, KAFKA_0_9_0, KAFKA_1_0_IV0, KAFKA_2_2_IV0, KAFKA_2_4_IV0, KAFKA_2_4_IV1, KAFKA_2_6_IV0, KAFKA_2_8_IV1, KAFKA_3_2_IV0, LeaderAndIsr} import kafka.cluster.{Broker, EndPoint} import kafka.server.KafkaConfig import kafka.utils.TestUtils -import org.apache.kafka.common.{TopicPartition, Uuid} -import org.apache.kafka.common.message.{LeaderAndIsrResponseData, StopReplicaResponseData, UpdateMetadataResponseData} import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError +import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrTopicError import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaPartitionState import org.apache.kafka.common.message.StopReplicaResponseData.StopReplicaPartitionError +import org.apache.kafka.common.message.{LeaderAndIsrResponseData, StopReplicaResponseData, UpdateMetadataResponseData} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{AbstractControlRequest, AbstractResponse, LeaderAndIsrRequest, LeaderAndIsrResponse, StopReplicaRequest, StopReplicaResponse, UpdateMetadataRequest, UpdateMetadataResponse} -import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrTopicError import org.apache.kafka.common.security.auth.SecurityProtocol +import org.apache.kafka.common.{TopicPartition, Uuid} +import org.apache.kafka.metadata.LeaderRecoveryState import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test - -import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.collection.mutable.ListBuffer +import scala.jdk.CollectionConverters._ class ControllerChannelManagerTest { private val controllerId = 1 @@ -161,7 +161,8 @@ class ControllerChannelManagerTest { for (apiVersion <- ApiVersion.allVersions) { val leaderAndIsrRequestVersion: Short = - if (apiVersion >= KAFKA_2_8_IV1) 5 + if (apiVersion >= KAFKA_3_2_IV0) 6 + else if (apiVersion >= KAFKA_2_8_IV1) 5 else if (apiVersion >= KAFKA_2_4_IV1) 4 else if (apiVersion >= KAFKA_2_4_IV0) 3 else if (apiVersion >= KAFKA_2_2_IV0) 2 @@ -179,7 +180,10 @@ class ControllerChannelManagerTest { val batch = new MockControllerBrokerRequestBatch(context, config) val partition = new TopicPartition("foo", 0) - val leaderAndIsr = LeaderAndIsr(1, List(1, 2)) + var leaderAndIsr = LeaderAndIsr(1, List(1, 2)) + if (interBrokerProtocolVersion >= KAFKA_3_2_IV0) { + leaderAndIsr = leaderAndIsr.copy(leaderRecoveryState = LeaderRecoveryState.RECOVERING) + } val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) context.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) @@ -192,11 +196,23 @@ class ControllerChannelManagerTest { assertEquals(1, leaderAndIsrRequests.size) assertEquals(expectedLeaderAndIsrVersion, leaderAndIsrRequests.head.version, s"IBP $interBrokerProtocolVersion should use version $expectedLeaderAndIsrVersion") - + val request = leaderAndIsrRequests.head val byteBuffer = request.serialize val deserializedRequest = LeaderAndIsrRequest.parse(byteBuffer, expectedLeaderAndIsrVersion) - + + val expectedRecovery = if (interBrokerProtocolVersion >= KAFKA_3_2_IV0) { + LeaderRecoveryState.RECOVERING + } else { + LeaderRecoveryState.RECOVERED + } + + Seq(request, deserializedRequest).foreach { request => + request.partitionStates.forEach { state => + assertEquals(expectedRecovery , LeaderRecoveryState.of(state.leaderRecoveryState())) + } + } + if (interBrokerProtocolVersion >= KAFKA_2_8_IV1) { assertFalse(request.topicIds().get("foo").equals(Uuid.ZERO_UUID)) assertFalse(deserializedRequest.topicIds().get("foo").equals(Uuid.ZERO_UUID)) diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 300db0047b44..9d539fa7f608 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -21,7 +21,7 @@ import java.util.Properties import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingQueue, TimeUnit} import com.yammer.metrics.core.Timer import kafka.api.{ApiVersion, KAFKA_2_6_IV0, KAFKA_2_7_IV0, LeaderAndIsr} -import kafka.controller.KafkaController.AlterIsrCallback +import kafka.controller.KafkaController.AlterPartitionCallback import kafka.metrics.KafkaYammerMetrics import kafka.server.{KafkaConfig, KafkaServer, QuorumTestHarness} import kafka.utils.{LogCaptureAppender, TestUtils} @@ -31,6 +31,7 @@ import org.apache.kafka.common.feature.Features import org.apache.kafka.common.metrics.KafkaMetric import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.{ElectionType, TopicPartition, Uuid} +import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.log4j.Level import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -874,57 +875,69 @@ class ControllerIntegrationTest extends QuorumTestHarness { val brokerEpoch = controller.controllerContext.liveBrokerIdAndEpochs.get(otherBroker.config.brokerId).get // When re-sending the current ISR, we should not get and error or any ISR changes - controller.eventManager.put(AlterIsrReceived(otherBroker.config.brokerId, brokerEpoch, Map(tp -> newLeaderAndIsr), callback)) + controller.eventManager.put(AlterPartitionReceived(otherBroker.config.brokerId, brokerEpoch, Map(tp -> newLeaderAndIsr), callback)) latch.await() } @Test def testAlterIsrErrors(): Unit = { - servers = makeServers(1) + servers = makeServers(2) val controllerId = TestUtils.waitUntilControllerElected(zkClient) val tp = new TopicPartition("t", 0) - val assignment = Map(tp.partition -> Seq(controllerId)) + val replicas = controllerId :: servers.map(_.config.nodeId).filter(_ != controllerId).take(1).toList + val assignment = Map(tp.partition -> replicas) + TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) val controller = getController().kafkaController var future = captureAlterIsrError(controllerId, controller.brokerEpoch - 1, - Map(tp -> LeaderAndIsr(controllerId, List(controllerId)))) + Map(tp -> LeaderAndIsr(controllerId, replicas))) var capturedError = future.get(5, TimeUnit.SECONDS) assertEquals(Errors.STALE_BROKER_EPOCH, capturedError) future = captureAlterIsrError(99, controller.brokerEpoch, - Map(tp -> LeaderAndIsr(controllerId, List(controllerId)))) + Map(tp -> LeaderAndIsr(controllerId, replicas))) capturedError = future.get(5, TimeUnit.SECONDS) assertEquals(Errors.STALE_BROKER_EPOCH, capturedError) val unknownTopicPartition = new TopicPartition("unknown", 99) future = captureAlterIsrPartitionError(controllerId, controller.brokerEpoch, - Map(unknownTopicPartition -> LeaderAndIsr(controllerId, List(controllerId))), unknownTopicPartition) + Map(unknownTopicPartition -> LeaderAndIsr(controllerId, replicas)), unknownTopicPartition) capturedError = future.get(5, TimeUnit.SECONDS) assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, capturedError) future = captureAlterIsrPartitionError(controllerId, controller.brokerEpoch, - Map(tp -> LeaderAndIsr(controllerId, 1, List(controllerId), 99)), tp) + Map(tp -> LeaderAndIsr(controllerId, 1, replicas, LeaderRecoveryState.RECOVERED, 99)), tp) capturedError = future.get(5, TimeUnit.SECONDS) assertEquals(Errors.INVALID_UPDATE_VERSION, capturedError) + + future = captureAlterIsrPartitionError(controllerId, controller.brokerEpoch, + Map(tp -> LeaderAndIsr(controllerId, 1, replicas, LeaderRecoveryState.RECOVERING, 1)), tp) + capturedError = future.get(5, TimeUnit.SECONDS) + assertEquals(Errors.INVALID_REQUEST, capturedError) + + future = captureAlterIsrPartitionError(controllerId, controller.brokerEpoch, + Map(tp -> LeaderAndIsr(controllerId, 1, List(controllerId), LeaderRecoveryState.RECOVERING, 1)), tp) + capturedError = future.get(5, TimeUnit.SECONDS) + assertEquals(Errors.INVALID_REQUEST, capturedError) } def captureAlterIsrError(brokerId: Int, brokerEpoch: Long, isrsToAlter: Map[TopicPartition, LeaderAndIsr]): CompletableFuture[Errors] = { val future = new CompletableFuture[Errors]() val controller = getController().kafkaController - val callback: AlterIsrCallback = { + val callback: AlterPartitionCallback = { case Left(_: Map[TopicPartition, Either[Errors, LeaderAndIsr]]) => future.completeExceptionally(new AssertionError(s"Should have seen top-level error")) case Right(error: Errors) => future.complete(error) } - controller.eventManager.put(AlterIsrReceived(brokerId, brokerEpoch, isrsToAlter, callback)) + controller.eventManager.put(AlterPartitionReceived(brokerId, brokerEpoch, isrsToAlter, callback)) future } def captureAlterIsrPartitionError(brokerId: Int, brokerEpoch: Long, isrsToAlter: Map[TopicPartition, LeaderAndIsr], tp: TopicPartition): CompletableFuture[Errors] = { val future = new CompletableFuture[Errors]() val controller = getController().kafkaController - val callback: AlterIsrCallback = { + val callback: AlterPartitionCallback = { case Left(partitionResults: Map[TopicPartition, Either[Errors, LeaderAndIsr]]) => partitionResults.get(tp) match { case Some(Left(error: Errors)) => future.complete(error) @@ -934,7 +947,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { case Right(_: Errors) => future.completeExceptionally(new AssertionError(s"Should not seen top-level error")) } - controller.eventManager.put(AlterIsrReceived(brokerId, brokerEpoch, isrsToAlter, callback)) + controller.eventManager.put(AlterPartitionReceived(brokerId, brokerEpoch, isrsToAlter, callback)) future } @@ -1376,7 +1389,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { controlPlaneListenerName : Option[String] = None, interBrokerProtocolVersion: Option[ApiVersion] = None, logDirCount: Int = 1, - startingIdNumber: Int = 0) = { + startingIdNumber: Int = 0): Seq[KafkaServer] = { val configs = TestUtils.createBrokerConfigs(numConfigs, zkConnect, enableControlledShutdown = enableControlledShutdown, logDirCount = logDirCount, startingIdNumber = startingIdNumber) configs.foreach { config => config.setProperty(KafkaConfig.AutoLeaderRebalanceEnableProp, autoLeaderRebalanceEnable.toString) diff --git a/core/src/test/scala/unit/kafka/controller/MockPartitionStateMachine.scala b/core/src/test/scala/unit/kafka/controller/MockPartitionStateMachine.scala index b9a4d04198da..9bc6e3cd6344 100644 --- a/core/src/test/scala/unit/kafka/controller/MockPartitionStateMachine.scala +++ b/core/src/test/scala/unit/kafka/controller/MockPartitionStateMachine.scala @@ -23,9 +23,11 @@ import org.apache.kafka.common.TopicPartition import scala.collection.{Seq, mutable} -class MockPartitionStateMachine(controllerContext: ControllerContext, - uncleanLeaderElectionEnabled: Boolean) - extends PartitionStateMachine(controllerContext) { +class MockPartitionStateMachine( + controllerContext: ControllerContext, + uncleanLeaderElectionEnabled: Boolean, + isLeaderRecoverySupported: Boolean +) extends PartitionStateMachine(controllerContext) { var stateChangesByTargetState = mutable.Map.empty[PartitionState, Int].withDefaultValue(0) @@ -101,7 +103,11 @@ class MockPartitionStateMachine(controllerContext: ControllerContext, val partitionsWithUncleanLeaderElectionState = validLeaderAndIsrs.map { case (partition, leaderAndIsr) => (partition, Some(leaderAndIsr), isUnclean || uncleanLeaderElectionEnabled) } - leaderForOffline(controllerContext, partitionsWithUncleanLeaderElectionState) + leaderForOffline( + controllerContext, + isLeaderRecoverySupported, + partitionsWithUncleanLeaderElectionState + ) case ReassignPartitionLeaderElectionStrategy => leaderForReassign(controllerContext, validLeaderAndIsrs) case PreferredReplicaPartitionLeaderElectionStrategy => diff --git a/core/src/test/scala/unit/kafka/controller/PartitionStateMachineTest.scala b/core/src/test/scala/unit/kafka/controller/PartitionStateMachineTest.scala index 174b9f190e8d..a527774d8ce4 100644 --- a/core/src/test/scala/unit/kafka/controller/PartitionStateMachineTest.scala +++ b/core/src/test/scala/unit/kafka/controller/PartitionStateMachineTest.scala @@ -16,6 +16,8 @@ */ package kafka.controller +import kafka.api.KAFKA_3_1_IV0 +import kafka.api.KAFKA_3_2_IV0 import kafka.api.LeaderAndIsr import kafka.log.LogConfig import kafka.server.KafkaConfig @@ -28,6 +30,8 @@ import org.apache.zookeeper.KeeperException.Code import org.apache.zookeeper.data.Stat import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, Test} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import org.mockito.ArgumentMatchers.{any, anyInt} import org.mockito.Mockito.{mock, verify, when} @@ -275,8 +279,11 @@ class PartitionStateMachineTest { assertEquals(OnlinePartition, partitionState(partition)) } - @Test - def testOfflinePartitionToUncleanOnlinePartitionTransition(): Unit = { + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testOfflinePartitionToUncleanOnlinePartitionTransition( + isLeaderRecoverySupported: Boolean + ): Unit = { /* Starting scenario: Leader: X, Isr: [X], Replicas: [X, Y], LiveBrokers: [Y] * Ending scenario: Leader: Y, Isr: [Y], Replicas: [X, Y], LiverBrokers: [Y] * @@ -284,6 +291,22 @@ class PartitionStateMachineTest { * election on the offline partition results on the first live broker getting * elected. */ + + + val partitionStateMachine = { + val apiVersion = if (isLeaderRecoverySupported) KAFKA_3_2_IV0 else KAFKA_3_1_IV0 + val properties = TestUtils.createBrokerConfig(brokerId, "zkConnect") + + properties.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, apiVersion.toString) + + new ZkPartitionStateMachine( + KafkaConfig.fromProps(properties), + new StateChangeLogger(brokerId, true, None), + controllerContext, + mockZkClient, + mockControllerBrokerRequestBatch + ) + } val leaderBrokerId = brokerId + 1 controllerContext.setLiveBrokers(Map(TestUtils.createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.updatePartitionFullReplicaAssignment( @@ -309,7 +332,11 @@ class PartitionStateMachineTest { ) ) - val leaderAndIsrAfterElection = leaderAndIsr.newLeaderAndIsr(brokerId, List(brokerId)) + val leaderAndIsrAfterElection = if (isLeaderRecoverySupported) { + leaderAndIsr.newRecoveringLeaderAndIsr(brokerId, List(brokerId)) + } else { + leaderAndIsr.newLeaderAndIsr(brokerId, List(brokerId)) + } val updatedLeaderAndIsr = leaderAndIsrAfterElection.withZkVersion(2) when(mockZkClient.updateLeaderAndIsr(Map(partition -> leaderAndIsrAfterElection), controllerEpoch, controllerContext.epochZkVersion)) .thenReturn(UpdateLeaderAndIsrResult(Map(partition -> Right(updatedLeaderAndIsr)), Seq.empty)) @@ -484,7 +511,11 @@ class PartitionStateMachineTest { controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) } - val partitionStateMachine = new MockPartitionStateMachine(controllerContext, uncleanLeaderElectionEnabled = false) + val partitionStateMachine = new MockPartitionStateMachine( + controllerContext, + uncleanLeaderElectionEnabled = false, + isLeaderRecoverySupported = true + ) val replicaStateMachine = new MockReplicaStateMachine(controllerContext) val deletionClient = mock(classOf[DeletionClient]) val topicDeletionManager = new TopicDeletionManager(config, controllerContext, diff --git a/core/src/test/scala/unit/kafka/controller/TopicDeletionManagerTest.scala b/core/src/test/scala/unit/kafka/controller/TopicDeletionManagerTest.scala index ec2339dea1e4..a2ee8bfbd800 100644 --- a/core/src/test/scala/unit/kafka/controller/TopicDeletionManagerTest.scala +++ b/core/src/test/scala/unit/kafka/controller/TopicDeletionManagerTest.scala @@ -43,7 +43,11 @@ class TopicDeletionManagerTest { val replicaStateMachine = new MockReplicaStateMachine(controllerContext) replicaStateMachine.startup() - val partitionStateMachine = new MockPartitionStateMachine(controllerContext, uncleanLeaderElectionEnabled = false) + val partitionStateMachine = new MockPartitionStateMachine( + controllerContext, + uncleanLeaderElectionEnabled = false, + isLeaderRecoverySupported = true + ) partitionStateMachine.startup() val deletionManager = new TopicDeletionManager(config, controllerContext, replicaStateMachine, @@ -66,7 +70,11 @@ class TopicDeletionManagerTest { val replicaStateMachine = new MockReplicaStateMachine(controllerContext) replicaStateMachine.startup() - val partitionStateMachine = new MockPartitionStateMachine(controllerContext, uncleanLeaderElectionEnabled = false) + val partitionStateMachine = new MockPartitionStateMachine( + controllerContext, + uncleanLeaderElectionEnabled = false, + isLeaderRecoverySupported = true + ) partitionStateMachine.startup() val deletionManager = new TopicDeletionManager(config, controllerContext, replicaStateMachine, @@ -126,7 +134,11 @@ class TopicDeletionManagerTest { val replicaStateMachine = new MockReplicaStateMachine(controllerContext) replicaStateMachine.startup() - val partitionStateMachine = new MockPartitionStateMachine(controllerContext, uncleanLeaderElectionEnabled = false) + val partitionStateMachine = new MockPartitionStateMachine( + controllerContext, + uncleanLeaderElectionEnabled = false, + isLeaderRecoverySupported = true + ) partitionStateMachine.startup() val deletionManager = new TopicDeletionManager(config, controllerContext, replicaStateMachine, @@ -194,7 +206,11 @@ class TopicDeletionManagerTest { val replicaStateMachine = new MockReplicaStateMachine(controllerContext) replicaStateMachine.startup() - val partitionStateMachine = new MockPartitionStateMachine(controllerContext, uncleanLeaderElectionEnabled = false) + val partitionStateMachine = new MockPartitionStateMachine( + controllerContext, + uncleanLeaderElectionEnabled = false, + isLeaderRecoverySupported = true + ) partitionStateMachine.startup() val deletionManager = new TopicDeletionManager(config, controllerContext, replicaStateMachine, diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index 56e14d114efa..09dceac15102 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -80,7 +80,7 @@ class RequestConvertToJsonTest { @Test def testRequestHeaderNode(): Unit = { - val alterIsrRequest = new AlterIsrRequest(new AlterIsrRequestData(), 0) + val alterIsrRequest = new AlterPartitionRequest(new AlterPartitionRequestData(), 0) val req = request(alterIsrRequest) val header = req.header @@ -107,7 +107,7 @@ class RequestConvertToJsonTest { @Test def testRequestDesc(): Unit = { - val alterIsrRequest = new AlterIsrRequest(new AlterIsrRequestData(), 0) + val alterIsrRequest = new AlterPartitionRequest(new AlterPartitionRequestData(), 0) val req = request(alterIsrRequest) val expectedNode = new ObjectNode(JsonNodeFactory.instance) @@ -122,7 +122,7 @@ class RequestConvertToJsonTest { @Test def testRequestDescMetrics(): Unit = { - val alterIsrRequest = new AlterIsrRequest(new AlterIsrRequestData(), 0) + val alterIsrRequest = new AlterPartitionRequest(new AlterPartitionRequestData(), 0) val req = request(alterIsrRequest) val send = new NetworkSend(req.context.connectionId, alterIsrRequest.toSend(req.header)) val headerLog = RequestConvertToJson.requestHeaderNode(req.header) diff --git a/core/src/test/scala/unit/kafka/server/AlterIsrManagerTest.scala b/core/src/test/scala/unit/kafka/server/AlterIsrManagerTest.scala index 40b1b5933a20..483a5347e4e2 100644 --- a/core/src/test/scala/unit/kafka/server/AlterIsrManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterIsrManagerTest.scala @@ -18,24 +18,31 @@ package kafka.server import java.util.Collections - +import java.util.stream.{Stream => JStream} +import kafka.api.ApiVersion +import kafka.api.KAFKA_2_7_IV2 +import kafka.api.KAFKA_3_2_IV0 import kafka.api.LeaderAndIsr import kafka.utils.{MockScheduler, MockTime} import kafka.zk.KafkaZkClient import org.apache.kafka.clients.ClientResponse import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{AuthenticationException, InvalidUpdateVersionException, OperationNotAttemptedException, UnknownServerException, UnsupportedVersionException} -import org.apache.kafka.common.message.AlterIsrResponseData +import org.apache.kafka.common.message.AlterPartitionResponseData import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import org.apache.kafka.common.requests.{AbstractRequest, AlterIsrRequest, AlterIsrResponse} +import org.apache.kafka.common.requests.{AbstractRequest, AlterPartitionRequest, AlterPartitionResponse} +import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.test.TestUtils.assertFutureThrows import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{BeforeEach, Test} +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.Test +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.Arguments +import org.junit.jupiter.params.provider.MethodSource import org.mockito.ArgumentMatchers.{any, anyString} import org.mockito.Mockito.{mock, reset, times, verify} import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito} - import scala.jdk.CollectionConverters._ class AlterIsrManagerTest { @@ -56,43 +63,64 @@ class AlterIsrManagerTest { brokerToController = mock(classOf[BrokerToControllerChannelManager]) } - @Test - def testBasic(): Unit = { + @ParameterizedTest + @MethodSource(Array("provideApiVersions")) + def testBasic(apiVersion: ApiVersion): Unit = { val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => 2) + val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => 2, apiVersion) alterIsrManager.start() - alterIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1,2,3), 10), 0) + alterIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) verify(brokerToController).start() verify(brokerToController).sendRequest(any(), any()) + } + + @ParameterizedTest + @MethodSource(Array("provideLeaderRecoveryState")) + def testBasicSentLeaderRecoveryState( + apiVersion: ApiVersion, + leaderRecoveryState: LeaderRecoveryState + ): Unit = { + val requestCapture = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]]) + + val scheduler = new MockScheduler(time) + val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => 2, apiVersion) + alterIsrManager.start() + alterIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1), leaderRecoveryState, 10), 0) + verify(brokerToController).start() + verify(brokerToController).sendRequest(requestCapture.capture(), any()) + val request = requestCapture.getValue.build() + val expectedLeaderRecoveryState = if (apiVersion >= KAFKA_3_2_IV0) leaderRecoveryState else LeaderRecoveryState.RECOVERED + assertEquals(expectedLeaderRecoveryState.value, request.data.topics.get(0).partitions.get(0).leaderRecoveryState()) } - @Test - def testOverwriteWithinBatch(): Unit = { - val capture: ArgumentCaptor[AbstractRequest.Builder[AlterIsrRequest]] = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterIsrRequest]]) + @ParameterizedTest + @MethodSource(Array("provideApiVersions")) + def testOverwriteWithinBatch(apiVersion: ApiVersion): Unit = { + val capture: ArgumentCaptor[AbstractRequest.Builder[AlterPartitionRequest]] = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]]) val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => 2) + val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => 2, apiVersion) alterIsrManager.start() // Only send one ISR update for a given topic+partition - val firstSubmitFuture = alterIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1,2,3), 10), 0) + val firstSubmitFuture = alterIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) assertFalse(firstSubmitFuture.isDone) - val failedSubmitFuture = alterIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1,2), 10), 0) + val failedSubmitFuture = alterIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2), LeaderRecoveryState.RECOVERED, 10), 0) assertTrue(failedSubmitFuture.isCompletedExceptionally) assertFutureThrows(failedSubmitFuture, classOf[OperationNotAttemptedException]) // Simulate response - val alterIsrResp = partitionResponse(tp0, Errors.NONE) + val alterPartitionResp = partitionResponse(tp0, Errors.NONE) val resp = new ClientResponse(null, null, "", 0L, 0L, - false, null, null, alterIsrResp) + false, null, null, alterPartitionResp) verify(brokerToController).sendRequest(capture.capture(), callbackCapture.capture()) callbackCapture.getValue.onComplete(resp) // Now we can submit this partition again - val newSubmitFuture = alterIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1), 10), 0) + val newSubmitFuture = alterIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1), LeaderRecoveryState.RECOVERED, 10), 0) assertFalse(newSubmitFuture.isDone) verify(brokerToController).start() @@ -104,29 +132,30 @@ class AlterIsrManagerTest { assertEquals(request.data().topics().get(0).partitions().get(0).newIsr().size(), 1) } - @Test - def testSingleBatch(): Unit = { - val capture: ArgumentCaptor[AbstractRequest.Builder[AlterIsrRequest]] = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterIsrRequest]]) + @ParameterizedTest + @MethodSource(Array("provideApiVersions")) + def testSingleBatch(apiVersion: ApiVersion): Unit = { + val capture: ArgumentCaptor[AbstractRequest.Builder[AlterPartitionRequest]] = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]]) val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => 2) + val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => 2, apiVersion) alterIsrManager.start() // First request will send batch of one alterIsrManager.submit(new TopicPartition(topic, 0), - new LeaderAndIsr(1, 1, List(1,2,3), 10), 0) + LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) // Other submissions will queue up until a response for (i <- 1 to 9) { alterIsrManager.submit(new TopicPartition(topic, i), - new LeaderAndIsr(1, 1, List(1,2,3), 10), 0) + LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) } // Simulate response, omitting partition 0 will allow it to stay in unsent queue - val alterIsrResp = new AlterIsrResponse(new AlterIsrResponseData()) + val alterPartitionResp = new AlterPartitionResponse(new AlterPartitionResponseData()) val resp = new ClientResponse(null, null, "", 0L, 0L, - false, null, null, alterIsrResp) + false, null, null, alterPartitionResp) // On the callback, we check for unsent items and send another request verify(brokerToController).sendRequest(capture.capture(), callbackCapture.capture()) @@ -169,18 +198,18 @@ class AlterIsrManagerTest { } private def testRetryOnTopLevelError(error: Errors): Unit = { - val alterIsrResp = new AlterIsrResponse(new AlterIsrResponseData().setErrorCode(error.code)) + val alterPartitionResp = new AlterPartitionResponse(new AlterPartitionResponseData().setErrorCode(error.code)) val response = new ClientResponse(null, null, "", 0L, 0L, - false, null, null, alterIsrResp) + false, null, null, alterPartitionResp) testRetryOnErrorResponse(response) } private def testRetryOnErrorResponse(response: ClientResponse): Unit = { - val leaderAndIsr = new LeaderAndIsr(1, 1, List(1,2,3), 10) + val leaderAndIsr = new LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10) val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => 2) + val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => 2, KAFKA_3_2_IV0) alterIsrManager.start() alterIsrManager.submit(tp0, leaderAndIsr, 0) @@ -198,9 +227,9 @@ class AlterIsrManagerTest { scheduler.tick() // After a successful response, we can submit another AlterIsrItem - val retryAlterIsrResponse = partitionResponse(tp0, Errors.NONE) + val retryAlterPartitionResponse = partitionResponse(tp0, Errors.NONE) val retryResponse = new ClientResponse(null, null, "", 0L, 0L, - false, null, null, retryAlterIsrResponse) + false, null, null, retryAlterPartitionResponse) verify(brokerToController).sendRequest(any(), callbackCapture.capture()) callbackCapture.getValue.onComplete(retryResponse) @@ -223,10 +252,15 @@ class AlterIsrManagerTest { checkPartitionError(Errors.NOT_LEADER_OR_FOLLOWER) } + @Test + def testInvalidRequest(): Unit = { + checkPartitionError(Errors.INVALID_REQUEST) + } + private def checkPartitionError(error: Errors): Unit = { val alterIsrManager = testPartitionError(tp0, error) // Any partition-level error should clear the item from the pending queue allowing for future updates - val future = alterIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1,2,3), 10), 0) + val future = alterIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) assertFalse(future.isDone) } @@ -235,38 +269,39 @@ class AlterIsrManagerTest { reset(brokerToController) val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => 2) + val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => 2, KAFKA_3_2_IV0) alterIsrManager.start() - val future = alterIsrManager.submit(tp, new LeaderAndIsr(1, 1, List(1,2,3), 10), 0) + val future = alterIsrManager.submit(tp, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) verify(brokerToController).start() verify(brokerToController).sendRequest(any(), callbackCapture.capture()) reset(brokerToController) - val alterIsrResp = partitionResponse(tp, error) + val alterPartitionResp = partitionResponse(tp, error) val resp = new ClientResponse(null, null, "", 0L, 0L, - false, null, null, alterIsrResp) + false, null, null, alterPartitionResp) callbackCapture.getValue.onComplete(resp) assertTrue(future.isCompletedExceptionally) assertFutureThrows(future, error.exception.getClass) alterIsrManager } - @Test - def testOneInFlight(): Unit = { + @ParameterizedTest + @MethodSource(Array("provideApiVersions")) + def testOneInFlight(apiVersion: ApiVersion): Unit = { val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => 2) + val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => 2, apiVersion) alterIsrManager.start() // First submit will send the request - alterIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1,2,3), 10), 0) + alterIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) // These will become pending unsent items - alterIsrManager.submit(tp1, new LeaderAndIsr(1, 1, List(1,2,3), 10), 0) - alterIsrManager.submit(tp2, new LeaderAndIsr(1, 1, List(1,2,3), 10), 0) + alterIsrManager.submit(tp1, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) + alterIsrManager.submit(tp2, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) verify(brokerToController).start() verify(brokerToController).sendRequest(any(), callbackCapture.capture()) @@ -274,27 +309,28 @@ class AlterIsrManagerTest { // Once the callback runs, another request will be sent reset(brokerToController) - val alterIsrResp = new AlterIsrResponse(new AlterIsrResponseData()) + val alterPartitionResp = new AlterPartitionResponse(new AlterPartitionResponseData()) val resp = new ClientResponse(null, null, "", 0L, 0L, - false, null, null, alterIsrResp) + false, null, null, alterPartitionResp) callbackCapture.getValue.onComplete(resp) } - @Test - def testPartitionMissingInResponse(): Unit = { + @ParameterizedTest + @MethodSource(Array("provideApiVersions")) + def testPartitionMissingInResponse(apiVersion: ApiVersion): Unit = { brokerToController = Mockito.mock(classOf[BrokerToControllerChannelManager]) val brokerEpoch = 2 val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => brokerEpoch) + val alterIsrManager = new DefaultAlterIsrManager(brokerToController, scheduler, time, brokerId, () => brokerEpoch, apiVersion) alterIsrManager.start() def matchesAlterIsr(topicPartitions: Set[TopicPartition]): AbstractRequest.Builder[_ <: AbstractRequest] = { ArgumentMatchers.argThat[AbstractRequest.Builder[_ <: AbstractRequest]] { request => - assertEquals(ApiKeys.ALTER_ISR, request.apiKey()) - val alterIsrRequest = request.asInstanceOf[AlterIsrRequest.Builder].build() + assertEquals(ApiKeys.ALTER_PARTITION, request.apiKey()) + val alterPartitionRequest = request.asInstanceOf[AlterPartitionRequest.Builder].build() - val requestTopicPartitions = alterIsrRequest.data.topics.asScala.flatMap { topicData => + val requestTopicPartitions = alterPartitionRequest.data.topics.asScala.flatMap { topicData => val topic = topicData.name topicData.partitions.asScala.map(partitionData => new TopicPartition(topic, partitionData.partitionIndex)) }.toSet @@ -321,12 +357,12 @@ class AlterIsrManagerTest { } // The first `submit` will send the `AlterIsr` request - val future1 = alterIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1,2,3), 10), 0) + val future1 = alterIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) val callback1 = verifySendAlterIsr(Set(tp0)) // Additional calls while the `AlterIsr` request is inflight will be queued - val future2 = alterIsrManager.submit(tp1, new LeaderAndIsr(1, 1, List(1,2,3), 10), 0) - val future3 = alterIsrManager.submit(tp2, new LeaderAndIsr(1, 1, List(1,2,3), 10), 0) + val future2 = alterIsrManager.submit(tp1, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) + val future3 = alterIsrManager.submit(tp2, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) // Respond to the first request, which will also allow the next request to get sent callback1.onComplete(clientResponse(tp0, Errors.UNKNOWN_SERVER_ERROR)) @@ -363,24 +399,45 @@ class AlterIsrManagerTest { zkIsrManager.start() // Correct ZK version - val future1 = zkIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1,2,3), 1), 0) + val future1 = zkIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 1), 0) assertTrue(future1.isDone) - assertEquals(new LeaderAndIsr(1, 1, List(1,2,3), 2), future1.get) + assertEquals(LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 2), future1.get) // Wrong ZK version - val future2 = zkIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1,2,3), 3), 0) + val future2 = zkIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 3), 0) assertTrue(future2.isCompletedExceptionally) assertFutureThrows(future2, classOf[InvalidUpdateVersionException]) } - private def partitionResponse(tp: TopicPartition, error: Errors): AlterIsrResponse = { - new AlterIsrResponse(new AlterIsrResponseData() + private def partitionResponse(tp: TopicPartition, error: Errors): AlterPartitionResponse = { + new AlterPartitionResponse(new AlterPartitionResponseData() .setTopics(Collections.singletonList( - new AlterIsrResponseData.TopicData() + new AlterPartitionResponseData.TopicData() .setName(tp.topic()) .setPartitions(Collections.singletonList( - new AlterIsrResponseData.PartitionData() + new AlterPartitionResponseData.PartitionData() .setPartitionIndex(tp.partition()) .setErrorCode(error.code)))))) } } + +object AlterIsrManagerTest { + def provideApiVersions(): JStream[ApiVersion] = { + JStream.of( + // Supports KIP-704: unclean leader recovery + KAFKA_3_2_IV0, + // Supports KIP-497: alter partition + KAFKA_2_7_IV2 + ) + } + + def provideLeaderRecoveryState(): JStream[Arguments] = { + // Multiply apiVersions by leaderRecoveryState + provideApiVersions().flatMap { apiVersion => + JStream.of( + Arguments.of(apiVersion, LeaderRecoveryState.RECOVERED), + Arguments.of(apiVersion, LeaderRecoveryState.RECOVERING) + ) + } + } +} diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 479bf8e83155..c8ce3a160517 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -293,11 +293,11 @@ class ControllerApisTest { } @Test - def testUnauthorizedHandleAlterIsrRequest(): Unit = { + def testUnauthorizedHandleAlterPartitionRequest(): Unit = { assertThrows(classOf[ClusterAuthorizationException], () => createControllerApis( Some(createDenyAllAuthorizer()), new MockController.Builder().build()). - handleAlterIsrRequest(buildRequest(new AlterIsrRequest.Builder( - new AlterIsrRequestData()).build(0)))) + handleAlterPartitionRequest(buildRequest(new AlterPartitionRequest.Builder( + new AlterPartitionRequestData()).build(0)))) } @Test diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index 221fd9a42656..e7c5372d6a3b 100755 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -21,6 +21,7 @@ import java.io.File import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.Utils +import org.apache.kafka.metadata.LeaderRecoveryState import org.junit.jupiter.api._ import org.junit.jupiter.api.Assertions._ import kafka.utils.{KafkaScheduler, MockTime, TestUtils} @@ -87,7 +88,8 @@ class HighwatermarkPersistenceTest { assignment = Seq(configs.head.brokerId, configs.last.brokerId), isr = Set(configs.head.brokerId), addingReplicas = Seq.empty, - removingReplicas = Seq.empty + removingReplicas = Seq.empty, + leaderRecoveryState = LeaderRecoveryState.RECOVERED ) replicaManager.checkpointHighWatermarks() diff --git a/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala b/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala index f81e301db6cc..6a68217f94d6 100644 --- a/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala @@ -27,6 +27,7 @@ import kafka.utils._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.Time +import org.apache.kafka.metadata.LeaderRecoveryState import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.Mockito.{atLeastOnce, mock, verify, when} @@ -228,7 +229,8 @@ class IsrExpirationTest { assignment = configs.map(_.brokerId), isr = configs.map(_.brokerId).toSet, addingReplicas = Seq.empty, - removingReplicas = Seq.empty + removingReplicas = Seq.empty, + leaderRecoveryState = LeaderRecoveryState.RECOVERED ) // set lastCaughtUpTime to current time diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index e0ea465ae696..2668303cfe81 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -4003,9 +4003,9 @@ class KafkaApisTest { } @Test - def testRaftShouldNeverHandleAlterIsrRequest(): Unit = { + def testRaftShouldNeverHandleAlterPartitionRequest(): Unit = { metadataCache = MetadataCache.kRaftMetadataCache(brokerId) - verifyShouldNeverHandleErrorMessage(createKafkaApis(raftSupport = true).handleAlterIsrRequest) + verifyShouldNeverHandleErrorMessage(createKafkaApis(raftSupport = true).handleAlterPartitionRequest) } @Test diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala index f0003f42a2ea..776c0b06803c 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala @@ -36,6 +36,7 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{IsolationLevel, TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.image.{MetadataDelta, MetadataImage} +import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.PartitionRegistration import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} @@ -76,7 +77,8 @@ class ReplicaManagerConcurrencyTest { val initialPartitionRegistration = registration( replicaIds = Seq(localId, remoteId), isr = Seq(localId), - leader = localId + leader = localId, + LeaderRecoveryState.RECOVERED ) val topicModel = new TopicModel(Uuid.randomUuid(), "foo", Map(0 -> initialPartitionRegistration)) @@ -439,8 +441,9 @@ class ReplicaManagerConcurrencyTest { replicaIds: Seq[Int], isr: Seq[Int], leader: Int, + leaderRecoveryState: LeaderRecoveryState, leaderEpoch: Int = 0, - version: Int = 0 + partitionEpoch: Int = 0 ): PartitionRegistration = { new PartitionRegistration( replicaIds.toArray, @@ -448,8 +451,9 @@ class ReplicaManagerConcurrencyTest { Array.empty[Int], Array.empty[Int], leader, + leaderRecoveryState, leaderEpoch, - version + partitionEpoch ) } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala index 5a2b4d08dca6..658575aca67f 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala @@ -22,11 +22,12 @@ import kafka.cluster.Partition import kafka.log.{LogManager, LogOffsetSnapshot, UnifiedLog} import kafka.server.QuotaFactory.QuotaManagers import kafka.utils._ -import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} -import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.FetchRequest +import org.apache.kafka.common.requests.FetchRequest.PartitionData +import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} +import org.apache.kafka.metadata.LeaderRecoveryState import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} import org.mockito.{AdditionalMatchers, ArgumentMatchers} @@ -333,7 +334,8 @@ class ReplicaManagerQuotasTest { assignment = Seq(leaderBrokerId, configs.last.brokerId), isr = if (bothReplicasInSync) Set(leaderBrokerId, configs.last.brokerId) else Set(leaderBrokerId), addingReplicas = Seq.empty, - removingReplicas = Seq.empty + removingReplicas = Seq.empty, + leaderRecoveryState = LeaderRecoveryState.RECOVERED ) } } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index b606f5bcbba4..dd644c8edf2d 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -3580,7 +3580,7 @@ class ReplicaManagerTest { brokerId = 0, aliveBrokersIds) try { val tp = new TopicPartition(topic, 0) - val leaderAndIsr = new LeaderAndIsr(1, 0, aliveBrokersIds.toList, 0) + val leaderAndIsr = LeaderAndIsr(1, aliveBrokersIds.toList) // This test either starts with a topic ID in the PartitionFetchState and removes it on the next request (startsWithTopicId) // or does not start with a topic ID in the PartitionFetchState and adds one on the next request (!startsWithTopicId) diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index ddbc98770f00..43ba5bae4853 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -598,8 +598,8 @@ class RequestQuotaTest extends BaseRequestTest { new EndQuorumEpochRequest.Builder(EndQuorumEpochRequest.singletonRequest( tp, 10, 5, Collections.singletonList(3))) - case ApiKeys.ALTER_ISR => - new AlterIsrRequest.Builder(new AlterIsrRequestData()) + case ApiKeys.ALTER_PARTITION => + new AlterPartitionRequest.Builder(new AlterPartitionRequestData()) case ApiKeys.UPDATE_FEATURES => new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData()) 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 a8c5002a05d9..9482ae27be38 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala @@ -21,12 +21,11 @@ import kafka.log.UnifiedLog import kafka.server.metadata.BrokerMetadataPublisher 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.Test import org.junit.jupiter.api.Assertions.assertEquals - +import org.junit.jupiter.api.Test import org.mockito.Mockito - import scala.jdk.CollectionConverters._ class BrokerMetadataPublisherTest { @@ -127,6 +126,7 @@ class BrokerMetadataPublisherTest { Array.empty[Int], Array.empty[Int], replicas.head, + LeaderRecoveryState.RECOVERED, 0, 0 ) diff --git a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala index a610956d7ca9..ea3d369d87b4 100644 --- a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala @@ -22,6 +22,7 @@ import kafka.controller.LeaderIsrAndControllerEpoch import kafka.server.QuorumTestHarness import kafka.zk._ import org.apache.kafka.common.TopicPartition +import org.apache.kafka.metadata.LeaderRecoveryState import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} @@ -39,7 +40,7 @@ class ReplicationUtilsTest extends QuorumTestHarness { super.setUp(testInfo) zkClient.makeSurePersistentPathExists(TopicZNode.path(topic)) val topicPartition = new TopicPartition(topic, partition) - val leaderAndIsr = LeaderAndIsr(leader, leaderEpoch, isr, 1) + val leaderAndIsr = LeaderAndIsr(leader, leaderEpoch, isr, LeaderRecoveryState.RECOVERED, 1) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) zkClient.createTopicPartitionStatesRaw(Map(topicPartition -> leaderIsrAndControllerEpoch), ZkVersion.MatchAnyVersion) } @@ -51,14 +52,14 @@ class ReplicationUtilsTest extends QuorumTestHarness { val replicas = List(0, 1) // regular update - val newLeaderAndIsr1 = new LeaderAndIsr(leader, leaderEpoch, replicas, 0) + val newLeaderAndIsr1 = LeaderAndIsr(leader, leaderEpoch, replicas, LeaderRecoveryState.RECOVERED, 0) val (updateSucceeded1, newZkVersion1) = ReplicationUtils.updateLeaderAndIsr(zkClient, new TopicPartition(topic, partition), newLeaderAndIsr1, controllerEpoch) assertTrue(updateSucceeded1) assertEquals(newZkVersion1, 1) // mismatched zkVersion with the same data - val newLeaderAndIsr2 = new LeaderAndIsr(leader, leaderEpoch, replicas, zkVersion + 1) + val newLeaderAndIsr2 = LeaderAndIsr(leader, leaderEpoch, replicas, LeaderRecoveryState.RECOVERED, zkVersion + 1) val (updateSucceeded2, newZkVersion2) = ReplicationUtils.updateLeaderAndIsr(zkClient, new TopicPartition(topic, partition), newLeaderAndIsr2, controllerEpoch) assertTrue(updateSucceeded2) @@ -66,7 +67,7 @@ class ReplicationUtilsTest extends QuorumTestHarness { assertEquals(newZkVersion2, 1) // mismatched zkVersion and leaderEpoch - val newLeaderAndIsr3 = new LeaderAndIsr(leader, leaderEpoch + 1, replicas, zkVersion + 1) + val newLeaderAndIsr3 = LeaderAndIsr(leader, leaderEpoch + 1, replicas, LeaderRecoveryState.RECOVERED, zkVersion + 1) val (updateSucceeded3, newZkVersion3) = ReplicationUtils.updateLeaderAndIsr(zkClient, new TopicPartition(topic, partition), newLeaderAndIsr3, controllerEpoch) assertFalse(updateSucceeded3) diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala index 6be954d5e7ad..e8dfdca3a87d 100644 --- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala @@ -16,45 +16,45 @@ */ package kafka.zk -import java.util.{Collections, Properties} import java.nio.charset.StandardCharsets.UTF_8 import java.util.concurrent.{CountDownLatch, TimeUnit} +import java.util.{Collections, Properties} import kafka.api.{ApiVersion, LeaderAndIsr} import kafka.cluster.{Broker, EndPoint} +import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment} import kafka.log.LogConfig +import kafka.security.authorizer.AclEntry import kafka.server.{ConfigType, KafkaConfig, QuorumTestHarness} import kafka.utils.CoreUtils -import org.apache.kafka.common.{TopicPartition, Uuid} -import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} -import org.apache.kafka.common.security.token.delegation.TokenInformation -import org.apache.kafka.common.utils.{SecurityUtils, Time} -import org.apache.zookeeper.KeeperException.{Code, NoAuthException, NoNodeException, NodeExistsException} -import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} - -import scala.jdk.CollectionConverters._ -import scala.collection.mutable.ArrayBuffer -import scala.collection.{Seq, mutable} -import scala.util.Random -import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment} -import kafka.security.authorizer.AclEntry import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult import kafka.zookeeper._ import org.apache.kafka.common.acl.AclOperation.READ import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY} import org.apache.kafka.common.errors.ControllerMovedException -import org.apache.kafka.common.feature.{Features, SupportedVersionRange} import org.apache.kafka.common.feature.Features._ +import org.apache.kafka.common.feature.{Features, SupportedVersionRange} +import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.resource.ResourcePattern import org.apache.kafka.common.resource.ResourceType.{GROUP, TOPIC} import org.apache.kafka.common.security.JaasUtils +import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} +import org.apache.kafka.common.security.token.delegation.TokenInformation +import org.apache.kafka.common.utils.{SecurityUtils, Time} +import org.apache.kafka.common.{TopicPartition, Uuid} +import org.apache.kafka.metadata.LeaderRecoveryState +import org.apache.zookeeper.KeeperException.{Code, NoAuthException, NoNodeException, NodeExistsException} import org.apache.zookeeper.ZooDefs import org.apache.zookeeper.client.ZKClientConfig import org.apache.zookeeper.common.ZKConfig import org.apache.zookeeper.data.Stat +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource +import scala.collection.mutable.ArrayBuffer +import scala.collection.{Seq, mutable} +import scala.jdk.CollectionConverters._ +import scala.util.Random class KafkaZkClientTest extends QuorumTestHarness { @@ -922,10 +922,10 @@ class KafkaZkClientTest extends QuorumTestHarness { private def leaderIsrAndControllerEpochs(state: Int, zkVersion: Int): Map[TopicPartition, LeaderIsrAndControllerEpoch] = Map( topicPartition10 -> LeaderIsrAndControllerEpoch( - LeaderAndIsr(leader = 1, leaderEpoch = state, isr = List(2 + state, 3 + state), zkVersion = zkVersion), + LeaderAndIsr(leader = 1, leaderEpoch = state, isr = List(2 + state, 3 + state), LeaderRecoveryState.RECOVERED, zkVersion = zkVersion), controllerEpoch = 4), topicPartition11 -> LeaderIsrAndControllerEpoch( - LeaderAndIsr(leader = 0, leaderEpoch = state + 1, isr = List(1 + state, 2 + state), zkVersion = zkVersion), + LeaderAndIsr(leader = 0, leaderEpoch = state + 1, isr = List(1 + state, 2 + state), LeaderRecoveryState.RECOVERED, zkVersion = zkVersion), controllerEpoch = 4)) val initialLeaderIsrAndControllerEpochs: Map[TopicPartition, LeaderIsrAndControllerEpoch] = @@ -1014,9 +1014,9 @@ class KafkaZkClientTest extends QuorumTestHarness { // Trigger successful, to be retried and failed partitions in same call val mixedState = Map( - topicPartition10 -> LeaderAndIsr(leader = 1, leaderEpoch = 2, isr = List(4, 5), zkVersion = 1), - topicPartition11 -> LeaderAndIsr(leader = 0, leaderEpoch = 2, isr = List(3, 4), zkVersion = 0), - topicPartition20 -> LeaderAndIsr(leader = 0, leaderEpoch = 2, isr = List(3, 4), zkVersion = 0)) + topicPartition10 -> LeaderAndIsr(leader = 1, leaderEpoch = 2, isr = List(4, 5), LeaderRecoveryState.RECOVERED, zkVersion = 1), + topicPartition11 -> LeaderAndIsr(leader = 0, leaderEpoch = 2, isr = List(3, 4), LeaderRecoveryState.RECOVERED, zkVersion = 0), + topicPartition20 -> LeaderAndIsr(leader = 0, leaderEpoch = 2, isr = List(3, 4), LeaderRecoveryState.RECOVERED, zkVersion = 0)) checkUpdateLeaderAndIsrResult( leaderIsrs(state = 2, zkVersion = 2).filter { case (tp, _) => tp == topicPartition10 }, diff --git a/metadata/src/main/java/org/apache/kafka/controller/Controller.java b/metadata/src/main/java/org/apache/kafka/controller/Controller.java index eafea6cdcc34..3d91f67b349e 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/Controller.java +++ b/metadata/src/main/java/org/apache/kafka/controller/Controller.java @@ -22,8 +22,8 @@ import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.message.AllocateProducerIdsRequestData; import org.apache.kafka.common.message.AllocateProducerIdsResponseData; -import org.apache.kafka.common.message.AlterIsrRequestData; -import org.apache.kafka.common.message.AlterIsrResponseData; +import org.apache.kafka.common.message.AlterPartitionRequestData; +import org.apache.kafka.common.message.AlterPartitionResponseData; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; import org.apache.kafka.common.message.BrokerHeartbeatRequestData; @@ -52,13 +52,13 @@ public interface Controller extends AclMutator, AutoCloseable { /** - * Change partition ISRs. + * Change partition information. * - * @param request The AlterIsrRequest data. + * @param request The AlterPartitionRequest data. * * @return A future yielding the response. */ - CompletableFuture alterIsr(AlterIsrRequestData request); + CompletableFuture alterPartition(AlterPartitionRequestData request); /** * Create a batch of topics. diff --git a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java index cf0f6bfd609a..b872a5eb23ee 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java +++ b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.metadata.PartitionChangeRecord; +import org.apache.kafka.metadata.LeaderRecoveryState; import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.metadata.Replicas; import org.apache.kafka.server.common.ApiMessageAndVersion; @@ -56,26 +57,31 @@ public static boolean changeRecordIsNoOp(PartitionChangeRecord record) { private final int partitionId; private final Function isAcceptableLeader; private final Supplier uncleanElectionOk; + private final boolean isLeaderRecoverySupported; private List targetIsr; private List targetReplicas; private List targetRemoving; private List targetAdding; + private LeaderRecoveryState targetLeaderRecoveryState; private boolean alwaysElectPreferredIfPossible; public PartitionChangeBuilder(PartitionRegistration partition, Uuid topicId, int partitionId, Function isAcceptableLeader, - Supplier uncleanElectionOk) { + Supplier uncleanElectionOk, + boolean isLeaderRecoverySupported) { this.partition = partition; this.topicId = topicId; this.partitionId = partitionId; this.isAcceptableLeader = isAcceptableLeader; this.uncleanElectionOk = uncleanElectionOk; + this.isLeaderRecoverySupported = isLeaderRecoverySupported; this.targetIsr = Replicas.toList(partition.isr); this.targetReplicas = Replicas.toList(partition.replicas); this.targetRemoving = Replicas.toList(partition.removingReplicas); this.targetAdding = Replicas.toList(partition.addingReplicas); + this.targetLeaderRecoveryState = partition.leaderRecoveryState; this.alwaysElectPreferredIfPossible = false; } @@ -104,6 +110,11 @@ public PartitionChangeBuilder setTargetAdding(List targetAdding) { return this; } + public PartitionChangeBuilder setTargetLeaderRecoveryState(LeaderRecoveryState targetLeaderRecoveryState) { + this.targetLeaderRecoveryState = targetLeaderRecoveryState; + return this; + } + boolean shouldTryElection() { // If the new isr doesn't have the current leader, we need to try to elect a new // one. Note: this also handles the case where the current leader is NO_LEADER, @@ -151,6 +162,11 @@ private void tryElection(PartitionChangeRecord record) { // If the election was unclean, we have to forcibly set the ISR to just the // new leader. This can result in data loss! record.setIsr(Collections.singletonList(bestLeader.node)); + if (partition.leaderRecoveryState != LeaderRecoveryState.RECOVERING && + isLeaderRecoverySupported) { + // And mark the leader recovery state as RECOVERING + record.setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value()); + } } } else { log.debug("Failed to find a new leader with current state: {}", this); @@ -240,6 +256,10 @@ public Optional build() { if (!targetAdding.equals(Replicas.toList(partition.addingReplicas))) { record.setAddingReplicas(targetAdding); } + if (targetLeaderRecoveryState != partition.leaderRecoveryState) { + record.setLeaderRecoveryState(targetLeaderRecoveryState.value()); + } + if (changeRecordIsNoOp(record)) { return Optional.empty(); } else { @@ -260,6 +280,7 @@ public String toString() { ", targetReplicas=" + targetReplicas + ", targetRemoving=" + targetRemoving + ", targetAdding=" + targetAdding + + ", targetLeaderRecoveryState=" + targetLeaderRecoveryState + ", alwaysElectPreferredIfPossible=" + alwaysElectPreferredIfPossible + ')'; } 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 5b52c2d2fd47..1138a3cd1a33 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -30,8 +30,8 @@ import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.message.AllocateProducerIdsRequestData; import org.apache.kafka.common.message.AllocateProducerIdsResponseData; -import org.apache.kafka.common.message.AlterIsrRequestData; -import org.apache.kafka.common.message.AlterIsrResponseData; +import org.apache.kafka.common.message.AlterPartitionRequestData; +import org.apache.kafka.common.message.AlterPartitionResponseData; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; import org.apache.kafka.common.message.BrokerHeartbeatRequestData; @@ -145,6 +145,7 @@ static public class Builder { private Map supportedFeatures = Collections.emptyMap(); private short defaultReplicationFactor = 3; private int defaultNumPartitions = 1; + private boolean isLeaderRecoverySupported = false; private ReplicaPlacer replicaPlacer = new StripedReplicaPlacer(new Random()); private long snapshotMaxNewRecordBytes = Long.MAX_VALUE; private long sessionTimeoutNs = NANOSECONDS.convert(18, TimeUnit.SECONDS); @@ -199,6 +200,11 @@ public Builder setDefaultNumPartitions(int defaultNumPartitions) { return this; } + public Builder setIsLeaderRecoverySupported(boolean isLeaderRecoverySupported) { + this.isLeaderRecoverySupported = isLeaderRecoverySupported; + return this; + } + public Builder setReplicaPlacer(ReplicaPlacer replicaPlacer) { this.replicaPlacer = replicaPlacer; return this; @@ -259,8 +265,8 @@ public QuorumController build() throws Exception { queue = new KafkaEventQueue(time, logContext, threadNamePrefix + "QuorumController"); return new QuorumController(logContext, nodeId, clusterId, queue, time, configSchema, raftClient, supportedFeatures, defaultReplicationFactor, - defaultNumPartitions, replicaPlacer, snapshotMaxNewRecordBytes, - sessionTimeoutNs, controllerMetrics, createTopicPolicy, + defaultNumPartitions, isLeaderRecoverySupported, replicaPlacer, + snapshotMaxNewRecordBytes, sessionTimeoutNs, controllerMetrics, createTopicPolicy, alterConfigPolicy, configurationValidator, authorizer); } catch (Exception e) { Utils.closeQuietly(queue, "event queue"); @@ -1207,6 +1213,7 @@ private QuorumController(LogContext logContext, Map supportedFeatures, short defaultReplicationFactor, int defaultNumPartitions, + boolean isLeaderRecoverySupported, ReplicaPlacer replicaPlacer, long snapshotMaxNewRecordBytes, long sessionTimeoutNs, @@ -1238,7 +1245,7 @@ private QuorumController(LogContext logContext, this.producerIdControlManager = new ProducerIdControlManager(clusterControl, snapshotRegistry); this.snapshotMaxNewRecordBytes = snapshotMaxNewRecordBytes; this.replicationControl = new ReplicationControlManager(snapshotRegistry, - logContext, defaultReplicationFactor, defaultNumPartitions, + logContext, defaultReplicationFactor, defaultNumPartitions, isLeaderRecoverySupported, configurationControl, clusterControl, controllerMetrics, createTopicPolicy); this.authorizer = authorizer; authorizer.ifPresent(a -> a.setAclMutator(this)); @@ -1254,12 +1261,12 @@ private QuorumController(LogContext logContext, } @Override - public CompletableFuture alterIsr(AlterIsrRequestData request) { + public CompletableFuture alterPartition(AlterPartitionRequestData request) { if (request.topics().isEmpty()) { - return CompletableFuture.completedFuture(new AlterIsrResponseData()); + return CompletableFuture.completedFuture(new AlterPartitionResponseData()); } - return appendWriteEvent("alterIsr", () -> - replicationControl.alterIsr(request)); + return appendWriteEvent("alterPartition", () -> + replicationControl.alterPartition(request)); } @Override diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index c492ed9af079..877a144ac2e4 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -34,8 +34,8 @@ import org.apache.kafka.common.errors.UnknownTopicIdException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.internals.Topic; -import org.apache.kafka.common.message.AlterIsrRequestData; -import org.apache.kafka.common.message.AlterIsrResponseData; +import org.apache.kafka.common.message.AlterPartitionRequestData; +import org.apache.kafka.common.message.AlterPartitionResponseData; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData.ReassignablePartition; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData.ReassignableTopic; @@ -75,6 +75,7 @@ import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistration; +import org.apache.kafka.metadata.LeaderRecoveryState; import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.metadata.Replicas; import org.apache.kafka.server.policy.CreateTopicPolicy; @@ -160,6 +161,11 @@ public Uuid topicId() { */ private final int defaultNumPartitions; + /** + * If the cluster supports leader recovery state from KIP-704. + */ + private final boolean isLeaderRecoverySupported; + /** * A count of the total number of partitions in the cluster. */ @@ -214,6 +220,7 @@ public Uuid topicId() { LogContext logContext, short defaultReplicationFactor, int defaultNumPartitions, + boolean isLeaderRecoverySupported, ConfigurationControlManager configurationControl, ClusterControlManager clusterControl, ControllerMetrics controllerMetrics, @@ -222,6 +229,7 @@ public Uuid topicId() { this.log = logContext.logger(ReplicationControlManager.class); this.defaultReplicationFactor = defaultReplicationFactor; this.defaultNumPartitions = defaultNumPartitions; + this.isLeaderRecoverySupported = isLeaderRecoverySupported; this.configurationControl = configurationControl; this.controllerMetrics = controllerMetrics; this.createTopicPolicy = createTopicPolicy; @@ -464,7 +472,7 @@ private ApiError createTopic(CreatableTopic topic, } newParts.put(assignment.partitionIndex(), new PartitionRegistration( Replicas.toArray(assignment.brokerIds()), Replicas.toArray(isr), - Replicas.NONE, Replicas.NONE, isr.get(0), 0, 0)); + Replicas.NONE, Replicas.NONE, isr.get(0), LeaderRecoveryState.RECOVERED, 0, 0)); } ApiError error = maybeCheckCreateTopicPolicy(() -> { Map> assignments = new HashMap<>(); @@ -493,7 +501,7 @@ private ApiError createTopic(CreatableTopic topic, for (int partitionId = 0; partitionId < replicas.size(); partitionId++) { int[] r = Replicas.toArray(replicas.get(partitionId)); newParts.put(partitionId, - new PartitionRegistration(r, r, Replicas.NONE, Replicas.NONE, r[0], 0, 0)); + new PartitionRegistration(r, r, Replicas.NONE, Replicas.NONE, r[0], LeaderRecoveryState.RECOVERED, 0, 0)); } } catch (InvalidReplicationFactorException e) { return new ApiError(Errors.INVALID_REPLICATION_FACTOR, @@ -661,94 +669,52 @@ BrokersToIsrs brokersToIsrs() { return brokersToIsrs; } - ControllerResult alterIsr(AlterIsrRequestData request) { + ControllerResult alterPartition(AlterPartitionRequestData request) { clusterControl.checkBrokerEpoch(request.brokerId(), request.brokerEpoch()); - AlterIsrResponseData response = new AlterIsrResponseData(); + AlterPartitionResponseData response = new AlterPartitionResponseData(); List records = new ArrayList<>(); - for (AlterIsrRequestData.TopicData topicData : request.topics()) { - AlterIsrResponseData.TopicData responseTopicData = - new AlterIsrResponseData.TopicData().setName(topicData.name()); + for (AlterPartitionRequestData.TopicData topicData : request.topics()) { + AlterPartitionResponseData.TopicData responseTopicData = + new AlterPartitionResponseData.TopicData().setName(topicData.name()); response.topics().add(responseTopicData); Uuid topicId = topicsByName.get(topicData.name()); if (topicId == null || !topics.containsKey(topicId)) { - for (AlterIsrRequestData.PartitionData partitionData : topicData.partitions()) { - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). + for (AlterPartitionRequestData.PartitionData partitionData : topicData.partitions()) { + responseTopicData.partitions().add(new AlterPartitionResponseData.PartitionData(). setPartitionIndex(partitionData.partitionIndex()). setErrorCode(UNKNOWN_TOPIC_OR_PARTITION.code())); } - log.info("Rejecting alterIsr request for unknown topic ID {}.", topicId); + log.info("Rejecting AlterPartition request for unknown topic ID {}.", topicId); continue; } + TopicControlInfo topic = topics.get(topicId); - for (AlterIsrRequestData.PartitionData partitionData : topicData.partitions()) { + for (AlterPartitionRequestData.PartitionData partitionData : topicData.partitions()) { int partitionId = partitionData.partitionIndex(); PartitionRegistration partition = topic.parts.get(partitionId); - if (partition == null) { - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). - setPartitionIndex(partitionId). - setErrorCode(UNKNOWN_TOPIC_OR_PARTITION.code())); - log.info("Rejecting alterIsr request for unknown partition {}-{}.", - topic.name, partitionId); - continue; - } - if (partitionData.leaderEpoch() != partition.leaderEpoch) { - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). - setPartitionIndex(partitionId). - setErrorCode(FENCED_LEADER_EPOCH.code())); - log.debug("Rejecting alterIsr request from node {} for {}-{} because " + - "the current leader epoch is {}, not {}.", request.brokerId(), topic.name, - partitionId, partition.leaderEpoch, partitionData.leaderEpoch()); - continue; - } - if (request.brokerId() != partition.leader) { - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). - setPartitionIndex(partitionId). - setErrorCode(INVALID_REQUEST.code())); - log.info("Rejecting alterIsr request from node {} for {}-{} because " + - "the current leader is {}.", request.brokerId(), topic.name, - partitionId, partition.leader); - continue; - } - if (partitionData.currentIsrVersion() != partition.partitionEpoch) { - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). - setPartitionIndex(partitionId). - setErrorCode(INVALID_UPDATE_VERSION.code())); - log.info("Rejecting alterIsr request from node {} for {}-{} because " + - "the current partition epoch is {}, not {}.", request.brokerId(), - topic.name, partitionId, partition.partitionEpoch, - partitionData.currentIsrVersion()); - continue; - } - int[] newIsr = Replicas.toArray(partitionData.newIsr()); - if (!Replicas.validateIsr(partition.replicas, newIsr)) { - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). - setPartitionIndex(partitionId). - setErrorCode(INVALID_REQUEST.code())); - log.error("Rejecting alterIsr request from node {} for {}-{} because " + - "it specified an invalid ISR {}.", request.brokerId(), - topic.name, partitionId, partitionData.newIsr()); - continue; - } - if (!Replicas.contains(newIsr, partition.leader)) { - // An alterIsr request can't ask for the current leader to be removed. - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). - setPartitionIndex(partitionId). - setErrorCode(INVALID_REQUEST.code())); - log.error("Rejecting alterIsr request from node {} for {}-{} because " + - "it specified an invalid ISR {} that doesn't include itself.", - request.brokerId(), topic.name, partitionId, partitionData.newIsr()); + + Errors validationError = validateAlterPartitionData(request.brokerId(), topic, partitionId, partition, partitionData); + if (validationError != Errors.NONE) { + responseTopicData.partitions().add( + new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(partitionId) + .setErrorCode(validationError.code()) + ); + continue; } - // At this point, we have decided to perform the ISR change. We use - // PartitionChangeBuilder to find out what its effect will be. - PartitionChangeBuilder builder = new PartitionChangeBuilder(partition, + + PartitionChangeBuilder builder = new PartitionChangeBuilder( + partition, topic.id, partitionId, r -> clusterControl.unfenced(r), - () -> configurationControl.uncleanLeaderElectionEnabledForTopic(topicData.name())); + () -> configurationControl.uncleanLeaderElectionEnabledForTopic(topicData.name()), + isLeaderRecoverySupported); builder.setTargetIsr(partitionData.newIsr()); + builder.setTargetLeaderRecoveryState( + LeaderRecoveryState.of(partitionData.leaderRecoveryState())); Optional record = builder.build(); - Errors result = Errors.NONE; if (record.isPresent()) { records.add(record.get()); PartitionChangeRecord change = (PartitionChangeRecord) record.get().message(); @@ -759,44 +725,137 @@ ControllerResult alterIsr(AlterIsrRequestData request) { } if (change.leader() != request.brokerId() && change.leader() != NO_LEADER_CHANGE) { - // Normally, an alterIsr request, which is made by the partition + // Normally, an AlterPartition request, which is made by the partition // leader itself, is not allowed to modify the partition leader. // However, if there is an ongoing partition reassignment and the // ISR change completes it, then the leader may change as part of // the changes made during reassignment cleanup. // // In this case, we report back FENCED_LEADER_EPOCH to the leader - // which made the alterIsr request. This lets it know that it must + // which made the AlterPartition request. This lets it know that it must // fetch new metadata before trying again. This return code is // unusual because we both return an error and generate a new // metadata record. We usually only do one or the other. - log.info("AlterIsr request from node {} for {}-{} completed " + + log.info("AlterPartition request from node {} for {}-{} completed " + "the ongoing partition reassignment and triggered a " + - "leadership change. Reutrning FENCED_LEADER_EPOCH.", + "leadership change. Returning FENCED_LEADER_EPOCH.", request.brokerId(), topic.name, partitionId); - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). + responseTopicData.partitions().add(new AlterPartitionResponseData.PartitionData(). setPartitionIndex(partitionId). setErrorCode(FENCED_LEADER_EPOCH.code())); continue; } else if (change.removingReplicas() != null || change.addingReplicas() != null) { - log.info("AlterIsr request from node {} for {}-{} completed " + + log.info("AlterPartition request from node {} for {}-{} completed " + "the ongoing partition reassignment.", request.brokerId(), topic.name, partitionId); } } - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). + + /* Setting the LeaderRecoveryState field is always safe because it will always be the + * same as the value set in the request. For version 0, that is always the default + * RECOVERED which is ignored when serializing to version 0. For any other version, the + * LeaderRecoveryState field is supported. + */ + responseTopicData.partitions().add(new AlterPartitionResponseData.PartitionData(). setPartitionIndex(partitionId). - setErrorCode(result.code()). + setErrorCode(Errors.NONE.code()). setLeaderId(partition.leader). + setIsr(Replicas.toList(partition.isr)). + setLeaderRecoveryState(partition.leaderRecoveryState.value()). setLeaderEpoch(partition.leaderEpoch). - setCurrentIsrVersion(partition.partitionEpoch). - setIsr(Replicas.toList(partition.isr))); + setPartitionEpoch(partition.partitionEpoch)); } } + return ControllerResult.of(records, response); } + /** + * Validate the partition information included in the alter partition request. + * + * @param brokerId id of the broker requesting the alter partition + * @param topic current topic information store by the replication manager + * @param partitionId partition id being altered + * @param partition current partition registration for the partition being altered + * @param partitionData partition data from the alter partition request + * + * @return Errors.NONE for valid alter partition data; otherwise the validation error + */ + private Errors validateAlterPartitionData( + int brokerId, + TopicControlInfo topic, + int partitionId, + PartitionRegistration partition, + AlterPartitionRequestData.PartitionData partitionData + ) { + if (partition == null) { + log.info("Rejecting AlterPartition request for unknown partition {}-{}.", + topic.name, partitionId); + + return UNKNOWN_TOPIC_OR_PARTITION; + } + if (partitionData.leaderEpoch() != partition.leaderEpoch) { + log.debug("Rejecting AlterPartition request from node {} for {}-{} because " + + "the current leader epoch is {}, not {}.", brokerId, topic.name, + partitionId, partition.leaderEpoch, partitionData.leaderEpoch()); + + return FENCED_LEADER_EPOCH; + } + if (brokerId != partition.leader) { + log.info("Rejecting AlterPartition request from node {} for {}-{} because " + + "the current leader is {}.", brokerId, topic.name, + partitionId, partition.leader); + + return INVALID_REQUEST; + } + if (partitionData.partitionEpoch() != partition.partitionEpoch) { + log.info("Rejecting AlterPartition request from node {} for {}-{} because " + + "the current partition epoch is {}, not {}.", brokerId, + topic.name, partitionId, partition.partitionEpoch, + partitionData.partitionEpoch()); + + return INVALID_UPDATE_VERSION; + } + int[] newIsr = Replicas.toArray(partitionData.newIsr()); + if (!Replicas.validateIsr(partition.replicas, newIsr)) { + log.error("Rejecting AlterPartition request from node {} for {}-{} because " + + "it specified an invalid ISR {}.", brokerId, + topic.name, partitionId, partitionData.newIsr()); + + return INVALID_REQUEST; + } + if (!Replicas.contains(newIsr, partition.leader)) { + // The ISR must always include the current leader. + log.error("Rejecting AlterPartition request from node {} for {}-{} because " + + "it specified an invalid ISR {} that doesn't include itself.", + brokerId, topic.name, partitionId, partitionData.newIsr()); + + return INVALID_REQUEST; + } + LeaderRecoveryState leaderRecoveryState = LeaderRecoveryState.of(partitionData.leaderRecoveryState()); + if (leaderRecoveryState == LeaderRecoveryState.RECOVERING && newIsr.length > 1) { + log.info("Rejecting AlterPartition request from node {} for {}-{} because " + + "the ISR {} had more than one replica while the leader was still " + + "recovering from an unlcean leader election {}.", + brokerId, topic.name, partitionId, partitionData.newIsr(), + leaderRecoveryState); + + return INVALID_REQUEST; + } + if (partition.leaderRecoveryState == LeaderRecoveryState.RECOVERED && + leaderRecoveryState == LeaderRecoveryState.RECOVERING) { + + log.info("Rejecting AlterPartition request from node {} for {}-{} because " + + "the leader recovery state cannot change from RECOVERED to RECOVERING.", + brokerId, topic.name, partitionId); + + return INVALID_REQUEST; + } + + return Errors.NONE; + } + /** * Generate the appropriate records to handle a broker being fenced. * @@ -938,7 +997,8 @@ ApiError electLeader(String topic, int partitionId, ElectionType electionType, topicId, partitionId, r -> clusterControl.unfenced(r), - () -> electionType == ElectionType.UNCLEAN); + () -> electionType == ElectionType.UNCLEAN, + isLeaderRecoverySupported); builder.setAlwaysElectPreferredIfPossible(electionType == ElectionType.PREFERRED); Optional record = builder.build(); @@ -1115,6 +1175,7 @@ void createPartitions(CreatePartitionsTopic topic, setTopicId(topicId). setReplicas(placement). setIsr(isr). + setLeaderRecoveryState(LeaderRecoveryState.RECOVERED.value()). setRemovingReplicas(Collections.emptyList()). setAddingReplicas(Collections.emptyList()). setLeader(isr.get(0)). @@ -1205,7 +1266,8 @@ void generateLeaderAndIsrUpdates(String context, topicIdPart.topicId(), topicIdPart.partitionId(), isAcceptableLeader, - () -> configurationControl.uncleanLeaderElectionEnabledForTopic(topic.name)); + () -> configurationControl.uncleanLeaderElectionEnabledForTopic(topic.name), + isLeaderRecoverySupported); // Note: if brokerToRemove was passed as NO_LEADER, this is a no-op (the new // target ISR will be the same as the old one). @@ -1312,7 +1374,8 @@ Optional cancelPartitionReassignment(String topicName, tp.topicId(), tp.partitionId(), r -> clusterControl.unfenced(r), - () -> configurationControl.uncleanLeaderElectionEnabledForTopic(topicName)); + () -> configurationControl.uncleanLeaderElectionEnabledForTopic(topicName), + isLeaderRecoverySupported); builder.setTargetIsr(revert.isr()). setTargetReplicas(revert.replicas()). setTargetRemoving(Collections.emptyList()). @@ -1361,7 +1424,8 @@ Optional changePartitionReassignment(TopicIdPartition tp, tp.topicId(), tp.partitionId(), r -> clusterControl.unfenced(r), - () -> false); + () -> false, + isLeaderRecoverySupported); if (!reassignment.merged().equals(currentReplicas)) { builder.setTargetReplicas(reassignment.merged()); } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/LeaderRecoveryState.java b/metadata/src/main/java/org/apache/kafka/metadata/LeaderRecoveryState.java new file mode 100644 index 000000000000..4724e990abbb --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/metadata/LeaderRecoveryState.java @@ -0,0 +1,74 @@ +/* + * 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.metadata; + +import java.util.Optional; + +public enum LeaderRecoveryState { + /** + * Represent that the election for the partition was either an ISR election or the + * leader recovered from an unclean leader election. + */ + RECOVERED((byte) 0), + + /** + * Represent that the election for the partition was an unclean leader election and + * that the leader is recovering from it. + */ + RECOVERING((byte) 1); + + /** + * A special value used to represent that the LeaderRecoveryState field of a + * PartitionChangeRecord didn't change. + */ + private static final byte NO_CHANGE = (byte) -1; + + public static LeaderRecoveryState of(byte value) { + return optionalOf(value) + .orElseThrow(() -> new IllegalArgumentException(String.format("Value %s is not a valid leader recovery state", value))); + } + + public static Optional optionalOf(byte value) { + if (value == RECOVERED.value()) { + return Optional.of(RECOVERED); + } + if (value == RECOVERING.value()) { + return Optional.of(RECOVERING); + } + + return Optional.empty(); + } + + private final byte value; + + private LeaderRecoveryState(byte value) { + this.value = value; + } + + public byte value() { + return value; + } + + public LeaderRecoveryState changeTo(byte value) { + if (value == NO_CHANGE) { + return this; + } + + return of(value); + } +} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/PartitionRegistration.java b/metadata/src/main/java/org/apache/kafka/metadata/PartitionRegistration.java index 933bda95cad6..8fd1ad411f6e 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/PartitionRegistration.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/PartitionRegistration.java @@ -39,6 +39,7 @@ public class PartitionRegistration { public final int[] removingReplicas; public final int[] addingReplicas; public final int leader; + public final LeaderRecoveryState leaderRecoveryState; public final int leaderEpoch; public final int partitionEpoch; @@ -52,18 +53,20 @@ public PartitionRegistration(PartitionRecord record) { Replicas.toArray(record.removingReplicas()), Replicas.toArray(record.addingReplicas()), record.leader(), + LeaderRecoveryState.of(record.leaderRecoveryState()), record.leaderEpoch(), record.partitionEpoch()); } public PartitionRegistration(int[] replicas, int[] isr, int[] removingReplicas, - int[] addingReplicas, int leader, int leaderEpoch, - int partitionEpoch) { + int[] addingReplicas, int leader, LeaderRecoveryState leaderRecoveryState, + int leaderEpoch, int partitionEpoch) { this.replicas = replicas; this.isr = isr; this.removingReplicas = removingReplicas; this.addingReplicas = addingReplicas; this.leader = leader; + this.leaderRecoveryState = leaderRecoveryState; this.leaderEpoch = leaderEpoch; this.partitionEpoch = partitionEpoch; } @@ -76,6 +79,7 @@ public PartitionRegistration merge(PartitionChangeRecord record) { removingReplicas : Replicas.toArray(record.removingReplicas()); int[] newAddingReplicas = (record.addingReplicas() == null) ? addingReplicas : Replicas.toArray(record.addingReplicas()); + int newLeader; int newLeaderEpoch; if (record.leader() == NO_LEADER_CHANGE) { @@ -85,11 +89,15 @@ public PartitionRegistration merge(PartitionChangeRecord record) { newLeader = record.leader(); newLeaderEpoch = leaderEpoch + 1; } + + LeaderRecoveryState newLeaderRecoveryState = leaderRecoveryState.changeTo(record.leaderRecoveryState()); + return new PartitionRegistration(newReplicas, newIsr, newRemovingReplicas, newAddingReplicas, newLeader, + newLeaderRecoveryState, newLeaderEpoch, partitionEpoch + 1); } @@ -126,6 +134,11 @@ public String diff(PartitionRegistration prev) { append(prev.leader).append(" -> ").append(leader); prefix = ", "; } + if (leaderRecoveryState != prev.leaderRecoveryState) { + builder.append(prefix).append("leaderRecoveryState: "). + append(prev.leaderRecoveryState).append(" -> ").append(leaderRecoveryState); + prefix = ", "; + } if (leaderEpoch != prev.leaderEpoch) { builder.append(prefix).append("leaderEpoch: "). append(prev.leaderEpoch).append(" -> ").append(leaderEpoch); @@ -167,6 +180,7 @@ public ApiMessageAndVersion toRecord(Uuid topicId, int partitionId) { setRemovingReplicas(Replicas.toList(removingReplicas)). setAddingReplicas(Replicas.toList(addingReplicas)). setLeader(leader). + setLeaderRecoveryState(leaderRecoveryState.value()). setLeaderEpoch(leaderEpoch). setPartitionEpoch(partitionEpoch), PARTITION_RECORD.highestSupportedVersion()); } @@ -184,6 +198,7 @@ public LeaderAndIsrPartitionState toLeaderAndIsrPartitionState(TopicPartition tp setReplicas(Replicas.toList(replicas)). setAddingReplicas(Replicas.toList(addingReplicas)). setRemovingReplicas(Replicas.toList(removingReplicas)). + setLeaderRecoveryState(leaderRecoveryState.value()). setIsNew(isNew); } @@ -196,7 +211,7 @@ public boolean isReassigning() { @Override public int hashCode() { - return Objects.hash(replicas, isr, removingReplicas, addingReplicas, leader, + return Objects.hash(replicas, isr, removingReplicas, addingReplicas, leader, leaderRecoveryState, leaderEpoch, partitionEpoch); } @@ -209,6 +224,7 @@ public boolean equals(Object o) { Arrays.equals(removingReplicas, other.removingReplicas) && Arrays.equals(addingReplicas, other.addingReplicas) && leader == other.leader && + leaderRecoveryState == other.leaderRecoveryState && leaderEpoch == other.leaderEpoch && partitionEpoch == other.partitionEpoch; } @@ -221,6 +237,7 @@ public String toString() { builder.append(", removingReplicas=").append(Arrays.toString(removingReplicas)); builder.append(", addingReplicas=").append(Arrays.toString(addingReplicas)); builder.append(", leader=").append(leader); + builder.append(", leaderRecoveryState=").append(leaderRecoveryState); builder.append(", leaderEpoch=").append(leaderEpoch); builder.append(", partitionEpoch=").append(partitionEpoch); builder.append(")"); diff --git a/metadata/src/main/resources/common/metadata/PartitionChangeRecord.json b/metadata/src/main/resources/common/metadata/PartitionChangeRecord.json index 7afaa425b739..587e512d575a 100644 --- a/metadata/src/main/resources/common/metadata/PartitionChangeRecord.json +++ b/metadata/src/main/resources/common/metadata/PartitionChangeRecord.json @@ -38,6 +38,8 @@ "about": "null if the removing replicas didn't change; the new removing replicas otherwise." }, { "name": "AddingReplicas", "type": "[]int32", "default": "null", "entityType": "brokerId", "versions": "0+", "nullableVersions": "0+", "taggedVersions": "0+", "tag": 4, - "about": "null if the adding replicas didn't change; the new adding replicas otherwise." } + "about": "null if the adding replicas didn't change; the new adding replicas otherwise." }, + { "name": "LeaderRecoveryState", "type": "int8", "default": "-1", "versions": "0+", "taggedVersions": "0+", "tag": 5, + "about": "-1 if it didn't change; 0 if the leader was elected from the ISR or recovered from an unclean election; 1 if the leader that was elected using unclean leader election and it is still recovering." } ] } diff --git a/metadata/src/main/resources/common/metadata/PartitionRecord.json b/metadata/src/main/resources/common/metadata/PartitionRecord.json index 66a13e2a0670..fdd05f8a5ca8 100644 --- a/metadata/src/main/resources/common/metadata/PartitionRecord.json +++ b/metadata/src/main/resources/common/metadata/PartitionRecord.json @@ -34,6 +34,8 @@ "about": "The replicas that we are in the process of adding." }, { "name": "Leader", "type": "int32", "versions": "0+", "default": "-1", "entityType": "brokerId", "about": "The lead replica, or -1 if there is no leader." }, + { "name": "LeaderRecoveryState", "type": "int8", "default": "0", "versions": "0+", "taggedVersions": "0+", "tag": 0, + "about": "1 if the partition is recovering from an unclean leader election; 0 otherwise." }, { "name": "LeaderEpoch", "type": "int32", "versions": "0+", "default": "-1", "about": "The epoch of the partition leader." }, { "name": "PartitionEpoch", "type": "int32", "versions": "0+", "default": "-1", diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java index f935a808f0b6..cf3bb5160f04 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java @@ -20,11 +20,14 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.metadata.PartitionChangeRecord; import org.apache.kafka.controller.PartitionChangeBuilder.BestLeader; +import org.apache.kafka.metadata.LeaderRecoveryState; import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.metadata.Replicas; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.util.Arrays; import java.util.Collections; @@ -55,22 +58,22 @@ public void testChangeRecordIsNoOp() { private final static PartitionRegistration FOO = new PartitionRegistration( new int[] {2, 1, 3}, new int[] {2, 1, 3}, Replicas.NONE, Replicas.NONE, - 1, 100, 200); + 1, LeaderRecoveryState.RECOVERED, 100, 200); private final static Uuid FOO_ID = Uuid.fromString("FbrrdcfiR-KC2CPSTHaJrg"); private static PartitionChangeBuilder createFooBuilder(boolean allowUnclean) { - return new PartitionChangeBuilder(FOO, FOO_ID, 0, r -> r != 3, () -> allowUnclean); + return new PartitionChangeBuilder(FOO, FOO_ID, 0, r -> r != 3, () -> allowUnclean, true); } private final static PartitionRegistration BAR = new PartitionRegistration( new int[] {1, 2, 3, 4}, new int[] {1, 2, 3}, new int[] {1}, new int[] {4}, - 1, 100, 200); + 1, LeaderRecoveryState.RECOVERED, 100, 200); private final static Uuid BAR_ID = Uuid.fromString("LKfUsCBnQKekvL9O5dY9nw"); private static PartitionChangeBuilder createBarBuilder(boolean allowUnclean) { - return new PartitionChangeBuilder(BAR, BAR_ID, 0, r -> r != 3, () -> allowUnclean); + return new PartitionChangeBuilder(BAR, BAR_ID, 0, r -> r != 3, () -> allowUnclean, true); } private static void assertBestLeaderEquals(PartitionChangeBuilder builder, @@ -252,4 +255,122 @@ public void testAddingReplicaReassignment() { setTargetAdding(replicas.adding()). build()); } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testChangeInLeadershipDoesNotChangeRecoveryState(boolean isLeaderRecoverySupported) { + final byte noChange = (byte) -1; + int leaderId = 1; + LeaderRecoveryState recoveryState = LeaderRecoveryState.RECOVERING; + PartitionRegistration registration = new PartitionRegistration( + new int[] {leaderId, leaderId + 1, leaderId + 2}, + new int[] {leaderId}, + Replicas.NONE, + Replicas.NONE, + leaderId, + recoveryState, + 100, + 200 + ); + + // Change the partition so that there is no leader + PartitionChangeBuilder offlineBuilder = new PartitionChangeBuilder( + registration, + FOO_ID, + 0, + brokerId -> false, + () -> false, + isLeaderRecoverySupported + ); + // Set the target ISR to empty to indicate that the last leader is offline + offlineBuilder.setTargetIsr(Collections.emptyList()); + + // The partition should stay as recovering + PartitionChangeRecord changeRecord = (PartitionChangeRecord) offlineBuilder + .build() + .get() + .message(); + assertEquals(noChange, changeRecord.leaderRecoveryState()); + assertEquals(NO_LEADER, changeRecord.leader()); + + registration = registration.merge(changeRecord); + + assertEquals(NO_LEADER, registration.leader); + assertEquals(leaderId, registration.isr[0]); + assertEquals(recoveryState, registration.leaderRecoveryState); + + // Bring the leader back online + PartitionChangeBuilder onlineBuilder = new PartitionChangeBuilder( + registration, + FOO_ID, + 0, + brokerId -> true, + () -> false, + isLeaderRecoverySupported + ); + + // The only broker in the ISR is elected leader and stays in the recovering + changeRecord = (PartitionChangeRecord) onlineBuilder.build().get().message(); + assertEquals(noChange, changeRecord.leaderRecoveryState()); + + registration = registration.merge(changeRecord); + + assertEquals(leaderId, registration.leader); + assertEquals(leaderId, registration.isr[0]); + assertEquals(recoveryState, registration.leaderRecoveryState); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testUncleanSetsLeaderRecoveringState(boolean isLeaderRecoverySupported) { + final byte noChange = (byte) -1; + int leaderId = 1; + PartitionRegistration registration = new PartitionRegistration( + new int[] {leaderId, leaderId + 1, leaderId + 2}, + new int[] {leaderId + 1, leaderId + 2}, + Replicas.NONE, + Replicas.NONE, + NO_LEADER, + LeaderRecoveryState.RECOVERED, + 100, + 200 + ); + + // Change the partition using unclean leader election + PartitionChangeBuilder onlineBuilder = new PartitionChangeBuilder( + registration, + FOO_ID, + 0, + brokerId -> brokerId == leaderId, + () -> true, + isLeaderRecoverySupported + ); + + // The partition should stay as recovering + PartitionChangeRecord changeRecord = (PartitionChangeRecord) onlineBuilder + .build() + .get() + .message(); + + byte expectedRecoveryChange = noChange; + if (isLeaderRecoverySupported) { + expectedRecoveryChange = LeaderRecoveryState.RECOVERING.value(); + } + + assertEquals(expectedRecoveryChange, changeRecord.leaderRecoveryState()); + assertEquals(leaderId, changeRecord.leader()); + assertEquals(1, changeRecord.isr().size()); + assertEquals(leaderId, changeRecord.isr().get(0)); + + registration = registration.merge(changeRecord); + + LeaderRecoveryState expectedRecovery = LeaderRecoveryState.RECOVERED; + if (isLeaderRecoverySupported) { + expectedRecovery = LeaderRecoveryState.RECOVERING; + } + + assertEquals(leaderId, registration.leader); + assertEquals(leaderId, registration.isr[0]); + assertEquals(expectedRecovery, registration.leaderRecoveryState); + } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentRevertTest.java b/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentRevertTest.java index 26120be8fb82..6d32b2eb7dbd 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentRevertTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentRevertTest.java @@ -19,6 +19,7 @@ import java.util.Arrays; +import org.apache.kafka.metadata.LeaderRecoveryState; import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.metadata.Replicas; import org.junit.jupiter.api.Test; @@ -35,7 +36,7 @@ public class PartitionReassignmentRevertTest { public void testNoneAddedOrRemoved() { PartitionRegistration registration = new PartitionRegistration( new int[] {3, 2, 1}, new int[] {3, 2}, - Replicas.NONE, Replicas.NONE, 3, 100, 200); + Replicas.NONE, Replicas.NONE, 3, LeaderRecoveryState.RECOVERED, 100, 200); PartitionReassignmentRevert revert = new PartitionReassignmentRevert(registration); assertEquals(Arrays.asList(3, 2, 1), revert.replicas()); assertEquals(Arrays.asList(3, 2), revert.isr()); @@ -46,7 +47,7 @@ public void testNoneAddedOrRemoved() { public void testSomeRemoving() { PartitionRegistration registration = new PartitionRegistration( new int[] {3, 2, 1}, new int[] {3, 2}, - new int[] {2, 1}, Replicas.NONE, 3, 100, 200); + new int[] {2, 1}, Replicas.NONE, 3, LeaderRecoveryState.RECOVERED, 100, 200); PartitionReassignmentRevert revert = new PartitionReassignmentRevert(registration); assertEquals(Arrays.asList(3, 2, 1), revert.replicas()); assertEquals(Arrays.asList(3, 2), revert.isr()); @@ -57,7 +58,7 @@ public void testSomeRemoving() { public void testSomeAdding() { PartitionRegistration registration = new PartitionRegistration( new int[] {4, 5, 3, 2, 1}, new int[] {4, 5, 2}, - Replicas.NONE, new int[] {4, 5}, 3, 100, 200); + Replicas.NONE, new int[] {4, 5}, 3, LeaderRecoveryState.RECOVERED, 100, 200); PartitionReassignmentRevert revert = new PartitionReassignmentRevert(registration); assertEquals(Arrays.asList(3, 2, 1), revert.replicas()); assertEquals(Arrays.asList(2), revert.isr()); @@ -68,7 +69,7 @@ public void testSomeAdding() { public void testSomeRemovingAndAdding() { PartitionRegistration registration = new PartitionRegistration( new int[] {4, 5, 3, 2, 1}, new int[] {4, 5, 2}, - new int[] {2}, new int[] {4, 5}, 3, 100, 200); + new int[] {2}, new int[] {4, 5}, 3, LeaderRecoveryState.RECOVERED, 100, 200); PartitionReassignmentRevert revert = new PartitionReassignmentRevert(registration); assertEquals(Arrays.asList(3, 2, 1), revert.replicas()); assertEquals(Arrays.asList(2), revert.isr()); @@ -79,7 +80,7 @@ public void testSomeRemovingAndAdding() { public void testIsrSpecialCase() { PartitionRegistration registration = new PartitionRegistration( new int[] {4, 5, 3, 2, 1}, new int[] {4, 5}, - new int[] {2}, new int[] {4, 5}, 3, 100, 200); + new int[] {2}, new int[] {4, 5}, 3, LeaderRecoveryState.RECOVERED, 100, 200); PartitionReassignmentRevert revert = new PartitionReassignmentRevert(registration); assertEquals(Arrays.asList(3, 2, 1), revert.replicas()); assertEquals(Arrays.asList(3), revert.isr()); 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 26300d5a0637..439b200b350a 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -41,7 +41,7 @@ import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.message.AllocateProducerIdsRequestData; -import org.apache.kafka.common.message.AlterIsrRequestData; +import org.apache.kafka.common.message.AlterPartitionRequestData; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData.ReassignableTopic; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; @@ -756,7 +756,7 @@ public void testMissingInMemorySnapshot() throws Exception { ).get().topics().find(topicName).topicId(); // Create a lot of alter isr - List alterIsrs = IntStream + List alterPartitions = IntStream .range(0, numPartitions) .mapToObj(partitionIndex -> { PartitionRegistration partitionRegistration = controller.replicationControl().getPartition( @@ -764,30 +764,30 @@ public void testMissingInMemorySnapshot() throws Exception { partitionIndex ); - return new AlterIsrRequestData.PartitionData() + return new AlterPartitionRequestData.PartitionData() .setPartitionIndex(partitionIndex) .setLeaderEpoch(partitionRegistration.leaderEpoch) - .setCurrentIsrVersion(partitionRegistration.partitionEpoch) + .setPartitionEpoch(partitionRegistration.partitionEpoch) .setNewIsr(Arrays.asList(0, 1)); }) .collect(Collectors.toList()); - AlterIsrRequestData.TopicData topicData = new AlterIsrRequestData.TopicData() + AlterPartitionRequestData.TopicData topicData = new AlterPartitionRequestData.TopicData() .setName(topicName); - topicData.partitions().addAll(alterIsrs); + topicData.partitions().addAll(alterPartitions); int leaderId = 0; - AlterIsrRequestData alterIsrRequest = new AlterIsrRequestData() + AlterPartitionRequestData alterPartitionRequest = new AlterPartitionRequestData() .setBrokerId(leaderId) .setBrokerEpoch(brokerEpochs.get(leaderId)); - alterIsrRequest.topics().add(topicData); + alterPartitionRequest.topics().add(topicData); logEnv.logManagers().get(0).resignAfterNonAtomicCommit(); int oldClaimEpoch = controller.curClaimEpoch(); assertThrows( ExecutionException.class, - () -> controller.alterIsr(alterIsrRequest).get() + () -> controller.alterPartition(alterPartitionRequest).get() ); // Wait for the controller to become active again @@ -797,7 +797,7 @@ public void testMissingInMemorySnapshot() throws Exception { String.format("oldClaimEpoch = %s, newClaimEpoch = %s", oldClaimEpoch, controller.curClaimEpoch()) ); - // Since the alterIsr partially failed we expect to see + // Since the alterPartition partially failed we expect to see // some partitions to still have 2 in the ISR. int partitionsWithReplica2 = Utils.toList( controller diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index a6c8c1e66171..1bb171c9af2c 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -24,10 +24,10 @@ import org.apache.kafka.common.errors.InvalidReplicaAssignmentException; import org.apache.kafka.common.errors.PolicyViolationException; import org.apache.kafka.common.errors.StaleBrokerEpochException; -import org.apache.kafka.common.message.AlterIsrRequestData; -import org.apache.kafka.common.message.AlterIsrRequestData.PartitionData; -import org.apache.kafka.common.message.AlterIsrRequestData.TopicData; -import org.apache.kafka.common.message.AlterIsrResponseData; +import org.apache.kafka.common.message.AlterPartitionRequestData; +import org.apache.kafka.common.message.AlterPartitionRequestData.PartitionData; +import org.apache.kafka.common.message.AlterPartitionRequestData.TopicData; +import org.apache.kafka.common.message.AlterPartitionResponseData; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData.ReassignablePartition; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData.ReassignableTopic; @@ -68,6 +68,7 @@ import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistration; import org.apache.kafka.metadata.KafkaConfigSchema; +import org.apache.kafka.metadata.LeaderRecoveryState; import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.metadata.Replicas; @@ -166,6 +167,7 @@ void replay(List records) throws Exception { new LogContext(), (short) 3, 1, + true, configurationControl, clusterControl, metrics, @@ -261,10 +263,11 @@ void registerBrokers(Integer... brokerIds) throws Exception { } } - void alterIsr( + void alterPartition( TopicIdPartition topicIdPartition, int leaderId, - List isr + List isr, + LeaderRecoveryState leaderRecoveryState ) throws Exception { BrokerRegistration registration = clusterControl.brokerRegistrations().get(leaderId); assertFalse(registration.fenced()); @@ -278,8 +281,9 @@ void alterIsr( PartitionData partitionData = new PartitionData() .setPartitionIndex(topicIdPartition.partitionId()) - .setCurrentIsrVersion(partition.partitionEpoch) + .setPartitionEpoch(partition.partitionEpoch) .setLeaderEpoch(partition.leaderEpoch) + .setLeaderRecoveryState(leaderRecoveryState.value()) .setNewIsr(isr); String topicName = replicationControl.getTopic(topicIdPartition.topicId()).name(); @@ -287,12 +291,12 @@ void alterIsr( .setName(topicName) .setPartitions(singletonList(partitionData)); - ControllerResult alterIsr = replicationControl.alterIsr( - new AlterIsrRequestData() + ControllerResult alterPartition = replicationControl.alterPartition( + new AlterPartitionRequestData() .setBrokerId(leaderId) .setBrokerEpoch(registration.epoch()) .setTopics(singletonList(topicData))); - replay(alterIsr.records()); + replay(alterPartition.records()); } void unfenceBrokers(Integer... brokerIds) throws Exception { @@ -418,7 +422,7 @@ public void testCreateTopics() throws Exception { assertEquals(expectedResponse2, result2.response()); ctx.replay(result2.records()); assertEquals(new PartitionRegistration(new int[] {1, 2, 0}, - new int[] {1, 2, 0}, Replicas.NONE, Replicas.NONE, 1, 0, 0), + new int[] {1, 2, 0}, Replicas.NONE, Replicas.NONE, 1, LeaderRecoveryState.RECOVERED, 0, 0), replicationControl.getPartition( ((TopicRecord) result2.records().get(0).message()).topicId(), 0)); ControllerResult result3 = @@ -688,24 +692,24 @@ public void testShrinkAndExpandIsr() throws Exception { assertEquals(OptionalInt.of(0), ctx.currentLeader(topicIdPartition)); long brokerEpoch = ctx.currentBrokerEpoch(0); PartitionData shrinkIsrRequest = newAlterIsrPartition( - replicationControl, topicIdPartition, asList(0, 1)); - ControllerResult shrinkIsrResult = sendAlterIsr( + replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERED); + ControllerResult shrinkIsrResult = sendAlterIsr( replicationControl, 0, brokerEpoch, "foo", shrinkIsrRequest); - AlterIsrResponseData.PartitionData shrinkIsrResponse = assertAlterIsrResponse( + AlterPartitionResponseData.PartitionData shrinkIsrResponse = assertAlterPartitionResponse( shrinkIsrResult, topicPartition, NONE); - assertConsistentAlterIsrResponse(replicationControl, topicIdPartition, shrinkIsrResponse); + assertConsistentAlterPartitionResponse(replicationControl, topicIdPartition, shrinkIsrResponse); PartitionData expandIsrRequest = newAlterIsrPartition( - replicationControl, topicIdPartition, asList(0, 1, 2)); - ControllerResult expandIsrResult = sendAlterIsr( + replicationControl, topicIdPartition, asList(0, 1, 2), LeaderRecoveryState.RECOVERED); + ControllerResult expandIsrResult = sendAlterIsr( replicationControl, 0, brokerEpoch, "foo", expandIsrRequest); - AlterIsrResponseData.PartitionData expandIsrResponse = assertAlterIsrResponse( + AlterPartitionResponseData.PartitionData expandIsrResponse = assertAlterPartitionResponse( expandIsrResult, topicPartition, NONE); - assertConsistentAlterIsrResponse(replicationControl, topicIdPartition, expandIsrResponse); + assertConsistentAlterPartitionResponse(replicationControl, topicIdPartition, expandIsrResponse); } @Test - public void testInvalidAlterIsrRequests() throws Exception { + public void testInvalidAlterPartitionRequests() throws Exception { ReplicationControlTestContext ctx = new ReplicationControlTestContext(); ReplicationControlManager replicationControl = ctx.replicationControl; ctx.registerBrokers(0, 1, 2); @@ -715,114 +719,132 @@ public void testInvalidAlterIsrRequests() throws Exception { TopicIdPartition topicIdPartition = new TopicIdPartition(createTopicResult.topicId(), 0); TopicPartition topicPartition = new TopicPartition("foo", 0); - assertEquals(OptionalInt.of(0), ctx.currentLeader(topicIdPartition)); + int leaderId = 0; + int notLeaderId = 1; + assertEquals(OptionalInt.of(leaderId), ctx.currentLeader(topicIdPartition)); long brokerEpoch = ctx.currentBrokerEpoch(0); // Invalid leader PartitionData invalidLeaderRequest = newAlterIsrPartition( - replicationControl, topicIdPartition, asList(0, 1)); - ControllerResult invalidLeaderResult = sendAlterIsr( - replicationControl, 1, ctx.currentBrokerEpoch(1), + replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERED); + ControllerResult invalidLeaderResult = sendAlterIsr( + replicationControl, notLeaderId, ctx.currentBrokerEpoch(notLeaderId), "foo", invalidLeaderRequest); - assertAlterIsrResponse(invalidLeaderResult, topicPartition, Errors.INVALID_REQUEST); + assertAlterPartitionResponse(invalidLeaderResult, topicPartition, Errors.INVALID_REQUEST); // Stale broker epoch PartitionData invalidBrokerEpochRequest = newAlterIsrPartition( - replicationControl, topicIdPartition, asList(0, 1)); + replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERED); assertThrows(StaleBrokerEpochException.class, () -> sendAlterIsr( - replicationControl, 0, brokerEpoch - 1, "foo", invalidBrokerEpochRequest)); + replicationControl, leaderId, brokerEpoch - 1, "foo", invalidBrokerEpochRequest)); // Invalid leader epoch PartitionData invalidLeaderEpochRequest = newAlterIsrPartition( - replicationControl, topicIdPartition, asList(0, 1)); + replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERED); invalidLeaderEpochRequest.setLeaderEpoch(500); - ControllerResult invalidLeaderEpochResult = sendAlterIsr( - replicationControl, 1, ctx.currentBrokerEpoch(1), + ControllerResult invalidLeaderEpochResult = sendAlterIsr( + replicationControl, leaderId, ctx.currentBrokerEpoch(leaderId), "foo", invalidLeaderEpochRequest); - assertAlterIsrResponse(invalidLeaderEpochResult, topicPartition, FENCED_LEADER_EPOCH); + assertAlterPartitionResponse(invalidLeaderEpochResult, topicPartition, FENCED_LEADER_EPOCH); // Invalid ISR (3 is not a valid replica) PartitionData invalidIsrRequest1 = newAlterIsrPartition( - replicationControl, topicIdPartition, asList(0, 1)); - invalidIsrRequest1.setNewIsr(asList(0, 1, 3)); - ControllerResult invalidIsrResult1 = sendAlterIsr( - replicationControl, 1, ctx.currentBrokerEpoch(1), + replicationControl, topicIdPartition, asList(0, 1, 3), LeaderRecoveryState.RECOVERED); + ControllerResult invalidIsrResult1 = sendAlterIsr( + replicationControl, leaderId, ctx.currentBrokerEpoch(leaderId), "foo", invalidIsrRequest1); - assertAlterIsrResponse(invalidIsrResult1, topicPartition, Errors.INVALID_REQUEST); + assertAlterPartitionResponse(invalidIsrResult1, topicPartition, Errors.INVALID_REQUEST); // Invalid ISR (does not include leader 0) PartitionData invalidIsrRequest2 = newAlterIsrPartition( - replicationControl, topicIdPartition, asList(0, 1)); - invalidIsrRequest2.setNewIsr(asList(1, 2)); - ControllerResult invalidIsrResult2 = sendAlterIsr( - replicationControl, 1, ctx.currentBrokerEpoch(1), + replicationControl, topicIdPartition, asList(1, 2), LeaderRecoveryState.RECOVERED); + ControllerResult invalidIsrResult2 = sendAlterIsr( + replicationControl, leaderId, ctx.currentBrokerEpoch(leaderId), "foo", invalidIsrRequest2); - assertAlterIsrResponse(invalidIsrResult2, topicPartition, Errors.INVALID_REQUEST); + assertAlterPartitionResponse(invalidIsrResult2, topicPartition, Errors.INVALID_REQUEST); + + // Invalid ISR length and recovery state + PartitionData invalidIsrRecoveryRequest = newAlterIsrPartition( + replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERING); + ControllerResult invalidIsrRecoveryResult = sendAlterIsr( + replicationControl, leaderId, ctx.currentBrokerEpoch(leaderId), + "foo", invalidIsrRecoveryRequest); + assertAlterPartitionResponse(invalidIsrRecoveryResult, topicPartition, Errors.INVALID_REQUEST); + + // Invalid recovery state transtion from RECOVERED to RECOVERING + PartitionData invalidRecoveryRequest = newAlterIsrPartition( + replicationControl, topicIdPartition, asList(0), LeaderRecoveryState.RECOVERING); + ControllerResult invalidRecoveryResult = sendAlterIsr( + replicationControl, leaderId, ctx.currentBrokerEpoch(leaderId), + "foo", invalidRecoveryRequest); + assertAlterPartitionResponse(invalidRecoveryResult, topicPartition, Errors.INVALID_REQUEST); } private PartitionData newAlterIsrPartition( ReplicationControlManager replicationControl, TopicIdPartition topicIdPartition, - List newIsr + List newIsr, + LeaderRecoveryState leaderRecoveryState ) { PartitionRegistration partitionControl = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId()); - return new AlterIsrRequestData.PartitionData() + return new AlterPartitionRequestData.PartitionData() .setPartitionIndex(0) .setLeaderEpoch(partitionControl.leaderEpoch) - .setCurrentIsrVersion(partitionControl.partitionEpoch) - .setNewIsr(newIsr); + .setPartitionEpoch(partitionControl.partitionEpoch) + .setNewIsr(newIsr) + .setLeaderRecoveryState(leaderRecoveryState.value()); } - private ControllerResult sendAlterIsr( + private ControllerResult sendAlterIsr( ReplicationControlManager replicationControl, int brokerId, long brokerEpoch, String topic, - AlterIsrRequestData.PartitionData partitionData + AlterPartitionRequestData.PartitionData partitionData ) throws Exception { - AlterIsrRequestData request = new AlterIsrRequestData() + AlterPartitionRequestData request = new AlterPartitionRequestData() .setBrokerId(brokerId) .setBrokerEpoch(brokerEpoch); - AlterIsrRequestData.TopicData topicData = new AlterIsrRequestData.TopicData() + AlterPartitionRequestData.TopicData topicData = new AlterPartitionRequestData.TopicData() .setName(topic); request.topics().add(topicData); topicData.partitions().add(partitionData); - ControllerResult result = replicationControl.alterIsr(request); + ControllerResult result = replicationControl.alterPartition(request); RecordTestUtils.replayAll(replicationControl, result.records()); return result; } - private AlterIsrResponseData.PartitionData assertAlterIsrResponse( - ControllerResult alterIsrResult, + private AlterPartitionResponseData.PartitionData assertAlterPartitionResponse( + ControllerResult alterPartitionResult, TopicPartition topicPartition, Errors expectedError ) { - AlterIsrResponseData response = alterIsrResult.response(); + AlterPartitionResponseData response = alterPartitionResult.response(); assertEquals(1, response.topics().size()); - AlterIsrResponseData.TopicData topicData = response.topics().get(0); + AlterPartitionResponseData.TopicData topicData = response.topics().get(0); assertEquals(topicPartition.topic(), topicData.name()); assertEquals(1, topicData.partitions().size()); - AlterIsrResponseData.PartitionData partitionData = topicData.partitions().get(0); + AlterPartitionResponseData.PartitionData partitionData = topicData.partitions().get(0); assertEquals(topicPartition.partition(), partitionData.partitionIndex()); assertEquals(expectedError, Errors.forCode(partitionData.errorCode())); return partitionData; } - private void assertConsistentAlterIsrResponse( + private void assertConsistentAlterPartitionResponse( ReplicationControlManager replicationControl, TopicIdPartition topicIdPartition, - AlterIsrResponseData.PartitionData partitionData + AlterPartitionResponseData.PartitionData partitionData ) { PartitionRegistration partitionControl = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId()); assertEquals(partitionControl.leader, partitionData.leaderId()); assertEquals(partitionControl.leaderEpoch, partitionData.leaderEpoch()); - assertEquals(partitionControl.partitionEpoch, partitionData.currentIsrVersion()); + assertEquals(partitionControl.partitionEpoch, partitionData.partitionEpoch()); List expectedIsr = IntStream.of(partitionControl.isr).boxed().collect(Collectors.toList()); assertEquals(expectedIsr, partitionData.isr()); } @@ -1134,19 +1156,19 @@ public void testReassignPartitions() throws Exception { setErrorCode(NO_REASSIGNMENT_IN_PROGRESS.code()). setErrorMessage(null)))))), cancelResult); - log.info("running final alterIsr..."); - ControllerResult alterIsrResult = replication.alterIsr( - new AlterIsrRequestData().setBrokerId(3).setBrokerEpoch(103). + log.info("running final alterPartition..."); + ControllerResult alterPartitionResult = replication.alterPartition( + new AlterPartitionRequestData().setBrokerId(3).setBrokerEpoch(103). setTopics(asList(new TopicData().setName("foo").setPartitions(asList( - new PartitionData().setPartitionIndex(1).setCurrentIsrVersion(1). + new PartitionData().setPartitionIndex(1).setPartitionEpoch(1). setLeaderEpoch(0).setNewIsr(asList(3, 0, 2, 1))))))); - assertEquals(new AlterIsrResponseData().setTopics(asList( - new AlterIsrResponseData.TopicData().setName("foo").setPartitions(asList( - new AlterIsrResponseData.PartitionData(). + assertEquals(new AlterPartitionResponseData().setTopics(asList( + new AlterPartitionResponseData.TopicData().setName("foo").setPartitions(asList( + new AlterPartitionResponseData.PartitionData(). setPartitionIndex(1). setErrorCode(FENCED_LEADER_EPOCH.code()))))), - alterIsrResult.response()); - ctx.replay(alterIsrResult.records()); + alterPartitionResult.response()); + ctx.replay(alterPartitionResult.records()); assertEquals(NONE_REASSIGNING, replication.listPartitionReassignments(null)); } @@ -1166,7 +1188,7 @@ public void testCancelReassignPartitions() throws Exception { replication.handleBrokerFenced(3, fenceRecords); ctx.replay(fenceRecords); assertEquals(new PartitionRegistration(new int[] {1, 2, 3, 4}, new int[] {1, 2, 4}, - new int[] {}, new int[] {}, 1, 1, 1), replication.getPartition(fooId, 0)); + new int[] {}, new int[] {}, 1, LeaderRecoveryState.RECOVERED, 1, 1), replication.getPartition(fooId, 0)); ControllerResult alterResult = replication.alterPartitionReassignments( new AlterPartitionReassignmentsRequestData().setTopics(asList( @@ -1203,11 +1225,11 @@ public void testCancelReassignPartitions() throws Exception { alterResult.response()); ctx.replay(alterResult.records()); assertEquals(new PartitionRegistration(new int[] {1, 2, 3}, new int[] {1, 2}, - new int[] {}, new int[] {}, 1, 2, 2), replication.getPartition(fooId, 0)); + new int[] {}, new int[] {}, 1, LeaderRecoveryState.RECOVERED, 2, 2), replication.getPartition(fooId, 0)); assertEquals(new PartitionRegistration(new int[] {1, 2, 3, 0}, new int[] {0, 1, 2}, - new int[] {}, new int[] {}, 0, 1, 2), replication.getPartition(fooId, 1)); + new int[] {}, new int[] {}, 0, LeaderRecoveryState.RECOVERED, 1, 2), replication.getPartition(fooId, 1)); assertEquals(new PartitionRegistration(new int[] {1, 2, 3, 4, 0}, new int[] {4, 2}, - new int[] {}, new int[] {0, 1}, 4, 1, 2), replication.getPartition(barId, 0)); + new int[] {}, new int[] {0, 1}, 4, LeaderRecoveryState.RECOVERED, 1, 2), replication.getPartition(barId, 0)); ListPartitionReassignmentsResponseData currentReassigning = new ListPartitionReassignmentsResponseData().setErrorMessage(null). setTopics(asList(new OngoingTopicReassignment(). @@ -1223,21 +1245,21 @@ public void testCancelReassignPartitions() throws Exception { assertEquals(currentReassigning, replication.listPartitionReassignments(asList( new ListPartitionReassignmentsTopics().setName("bar"). setPartitionIndexes(asList(0, 1, 2))))); - ControllerResult alterIsrResult = replication.alterIsr( - new AlterIsrRequestData().setBrokerId(4).setBrokerEpoch(104). + ControllerResult alterPartitionResult = replication.alterPartition( + new AlterPartitionRequestData().setBrokerId(4).setBrokerEpoch(104). setTopics(asList(new TopicData().setName("bar").setPartitions(asList( - new PartitionData().setPartitionIndex(0).setCurrentIsrVersion(2). + new PartitionData().setPartitionIndex(0).setPartitionEpoch(2). setLeaderEpoch(1).setNewIsr(asList(4, 1, 2, 3, 0))))))); - assertEquals(new AlterIsrResponseData().setTopics(asList( - new AlterIsrResponseData.TopicData().setName("bar").setPartitions(asList( - new AlterIsrResponseData.PartitionData(). + assertEquals(new AlterPartitionResponseData().setTopics(asList( + new AlterPartitionResponseData.TopicData().setName("bar").setPartitions(asList( + new AlterPartitionResponseData.PartitionData(). setPartitionIndex(0). setLeaderId(4). setLeaderEpoch(1). setIsr(asList(4, 1, 2, 3, 0)). - setCurrentIsrVersion(3). + setPartitionEpoch(3). setErrorCode(NONE.code()))))), - alterIsrResult.response()); + alterPartitionResult.response()); ControllerResult cancelResult = replication.alterPartitionReassignments( new AlterPartitionReassignmentsRequestData().setTopics(asList( @@ -1265,7 +1287,7 @@ public void testCancelReassignPartitions() throws Exception { ctx.replay(cancelResult.records()); assertEquals(NONE_REASSIGNING, replication.listPartitionReassignments(null)); assertEquals(new PartitionRegistration(new int[] {2, 3, 4}, new int[] {4, 2}, - new int[] {}, new int[] {}, 4, 2, 3), replication.getPartition(barId, 0)); + new int[] {}, new int[] {}, 4, LeaderRecoveryState.RECOVERED, 2, 3), replication.getPartition(barId, 0)); } @Test @@ -1286,7 +1308,7 @@ public void testCreatePartitionsFailsWithManualAssignmentWithAllFenced() throws INVALID_REPLICA_ASSIGNMENT.code()); ctx.createPartitions(2, "foo", new int[][] {new int[] {2, 4, 5}}, NONE.code()); assertEquals(new PartitionRegistration(new int[] {2, 4, 5}, - new int[] {2}, Replicas.NONE, Replicas.NONE, 2, 0, 0), + new int[] {2}, Replicas.NONE, Replicas.NONE, 2, LeaderRecoveryState.RECOVERED, 0, 0), ctx.replicationControl.getPartition(fooId, 1)); } @@ -1356,7 +1378,7 @@ public void testElectUncleanLeaders(boolean electAllPartitions) throws Exception // Bring 2 back into the ISR for partition 1. This allows us to verify that // preferred election does not occur as a result of the unclean election request. - ctx.alterIsr(partition1, 4, asList(2, 4)); + ctx.alterPartition(partition1, 4, asList(2, 4), LeaderRecoveryState.RECOVERED); ControllerResult result = replication.electLeaders(request); assertEquals(1, result.records().size()); @@ -1513,22 +1535,22 @@ public void testElectPreferredLeaders() throws Exception { assertEquals(Collections.emptyList(), election1Result.records()); ctx.unfenceBrokers(0, 1); - ControllerResult alterIsrResult = replication.alterIsr( - new AlterIsrRequestData().setBrokerId(2).setBrokerEpoch(102). - setTopics(asList(new AlterIsrRequestData.TopicData().setName("foo"). - setPartitions(asList(new AlterIsrRequestData.PartitionData(). - setPartitionIndex(0).setCurrentIsrVersion(0). + ControllerResult alterPartitionResult = replication.alterPartition( + new AlterPartitionRequestData().setBrokerId(2).setBrokerEpoch(102). + setTopics(asList(new AlterPartitionRequestData.TopicData().setName("foo"). + setPartitions(asList(new AlterPartitionRequestData.PartitionData(). + setPartitionIndex(0).setPartitionEpoch(0). setLeaderEpoch(0).setNewIsr(asList(1, 2, 3))))))); - assertEquals(new AlterIsrResponseData().setTopics(asList( - new AlterIsrResponseData.TopicData().setName("foo").setPartitions(asList( - new AlterIsrResponseData.PartitionData(). + assertEquals(new AlterPartitionResponseData().setTopics(asList( + new AlterPartitionResponseData.TopicData().setName("foo").setPartitions(asList( + new AlterPartitionResponseData.PartitionData(). setPartitionIndex(0). setLeaderId(2). setLeaderEpoch(0). setIsr(asList(1, 2, 3)). - setCurrentIsrVersion(1). + setPartitionEpoch(1). setErrorCode(NONE.code()))))), - alterIsrResult.response()); + alterPartitionResult.response()); ElectLeadersResponseData expectedResponse2 = buildElectLeadersResponse(NONE, false, Utils.mkMap( Utils.mkEntry( @@ -1549,7 +1571,7 @@ public void testElectPreferredLeaders() throws Exception { ) )); - ctx.replay(alterIsrResult.records()); + ctx.replay(alterPartitionResult.records()); ControllerResult election2Result = replication.electLeaders(request1); assertElectLeadersResponse(expectedResponse2, election2Result.response()); diff --git a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java index e417fb220236..3f6dece390e2 100644 --- a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.metadata.PartitionRecord; import org.apache.kafka.common.metadata.RemoveTopicRecord; import org.apache.kafka.common.metadata.TopicRecord; +import org.apache.kafka.metadata.LeaderRecoveryState; import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.metadata.Replicas; @@ -96,14 +97,14 @@ private static Map newTopicsByNameMap(Collection TOPIC_IMAGES1 = Arrays.asList( newTopicImage("foo", FOO_UUID, new PartitionRegistration(new int[] {2, 3, 4}, - new int[] {2, 3}, Replicas.NONE, Replicas.NONE, 2, 1, 345), + new int[] {2, 3}, Replicas.NONE, Replicas.NONE, 2, LeaderRecoveryState.RECOVERED, 1, 345), new PartitionRegistration(new int[] {3, 4, 5}, - new int[] {3, 4, 5}, Replicas.NONE, Replicas.NONE, 3, 4, 684), + new int[] {3, 4, 5}, Replicas.NONE, Replicas.NONE, 3, LeaderRecoveryState.RECOVERED, 4, 684), new PartitionRegistration(new int[] {2, 4, 5}, - new int[] {2, 4, 5}, Replicas.NONE, Replicas.NONE, 2, 10, 84)), + new int[] {2, 4, 5}, Replicas.NONE, Replicas.NONE, 2, LeaderRecoveryState.RECOVERED, 10, 84)), newTopicImage("bar", BAR_UUID, new PartitionRegistration(new int[] {0, 1, 2, 3, 4}, - new int[] {0, 1, 2, 3}, new int[] {1}, new int[] {3, 4}, 0, 1, 345))); + new int[] {0, 1, 2, 3}, new int[] {1}, new int[] {3, 4}, 0, LeaderRecoveryState.RECOVERED, 1, 345))); IMAGE1 = new TopicsImage(newTopicsByIdMap(TOPIC_IMAGES1), newTopicsByNameMap(TOPIC_IMAGES1)); @@ -135,10 +136,10 @@ private static Map newTopicsByNameMap(Collection List topics2 = Arrays.asList( newTopicImage("bar", BAR_UUID, new PartitionRegistration(new int[] {0, 1, 2, 3, 4}, - new int[] {0, 1, 2, 3}, new int[] {1}, new int[] {3, 4}, 1, 2, 346)), + new int[] {0, 1, 2, 3}, new int[] {1}, new int[] {3, 4}, 1, LeaderRecoveryState.RECOVERED, 2, 346)), newTopicImage("baz", BAZ_UUID, new PartitionRegistration(new int[] {1, 2, 3, 4}, - new int[] {3, 4}, new int[] {2}, new int[] {1}, 3, 2, 1))); + new int[] {3, 4}, new int[] {2}, new int[] {1}, 3, LeaderRecoveryState.RECOVERED, 2, 1))); IMAGE2 = new TopicsImage(newTopicsByIdMap(topics2), newTopicsByNameMap(topics2)); } @@ -157,7 +158,7 @@ private ApiMessageAndVersion newPartitionRecord(Uuid topicId, int partitionId, L } private PartitionRegistration newPartition(int[] replicas) { - return new PartitionRegistration(replicas, replicas, Replicas.NONE, Replicas.NONE, replicas[0], 1, 1); + return new PartitionRegistration(replicas, replicas, Replicas.NONE, Replicas.NONE, replicas[0], LeaderRecoveryState.RECOVERED, 1, 1); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/metadata/LeaderRecoveryStateTest.java b/metadata/src/test/java/org/apache/kafka/metadata/LeaderRecoveryStateTest.java new file mode 100644 index 000000000000..45268dfbc3d5 --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/metadata/LeaderRecoveryStateTest.java @@ -0,0 +1,87 @@ +/* + * 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.metadata; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +final public class LeaderRecoveryStateTest { + private static final byte NO_CHANGE = (byte) -1; + + @Test + void testUniqueValues() { + Set set = new HashSet<>(); + for (LeaderRecoveryState recovery : LeaderRecoveryState.values()) { + assertTrue( + set.add(recovery.value()), + String.format("Value %s for election state %s has already been used", recovery.value(), recovery) + ); + } + } + + @Test + void testDoesNotContainNoChange() { + for (LeaderRecoveryState recovery : LeaderRecoveryState.values()) { + assertNotEquals(NO_CHANGE, recovery.value()); + } + } + + @Test + void testByteToLeaderRecoveryState() { + assertEquals(LeaderRecoveryState.RECOVERED, LeaderRecoveryState.of((byte) 0)); + assertEquals(LeaderRecoveryState.RECOVERING, LeaderRecoveryState.of((byte) 1)); + } + + @Test + void testLeaderRecoveryStateValue() { + assertEquals(0, LeaderRecoveryState.RECOVERED.value()); + assertEquals(1, LeaderRecoveryState.RECOVERING.value()); + } + + @Test + void testInvalidValue() { + assertThrows( + IllegalArgumentException.class, + () -> LeaderRecoveryState.of(NO_CHANGE) + ); + assertThrows(IllegalArgumentException.class, () -> LeaderRecoveryState.of((byte) 2)); + } + + @Test + void testOptionalInvalidValue() { + assertEquals(Optional.empty(), LeaderRecoveryState.optionalOf(NO_CHANGE)); + assertEquals(Optional.empty(), LeaderRecoveryState.optionalOf((byte) 2)); + } + + @Test + void testChangeTo() { + LeaderRecoveryState state = LeaderRecoveryState.RECOVERED; + assertEquals(LeaderRecoveryState.RECOVERED, state.changeTo(NO_CHANGE)); + state = state.changeTo(LeaderRecoveryState.RECOVERING.value()); + assertEquals(LeaderRecoveryState.RECOVERING, state); + assertEquals(LeaderRecoveryState.RECOVERING, state.changeTo(NO_CHANGE)); + state = state.changeTo(LeaderRecoveryState.RECOVERED.value()); + assertEquals(LeaderRecoveryState.RECOVERED, state); + } +} diff --git a/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java b/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java index 9b1be5d4b3d2..93f167268e34 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java @@ -47,11 +47,11 @@ public void testElectionWasClean() { @Test public void testPartitionControlInfoMergeAndDiff() { PartitionRegistration a = new PartitionRegistration( - new int[]{1, 2, 3}, new int[]{1, 2}, Replicas.NONE, Replicas.NONE, 1, 0, 0); + new int[]{1, 2, 3}, new int[]{1, 2}, Replicas.NONE, Replicas.NONE, 1, LeaderRecoveryState.RECOVERED, 0, 0); PartitionRegistration b = new PartitionRegistration( - new int[]{1, 2, 3}, new int[]{3}, Replicas.NONE, Replicas.NONE, 3, 1, 1); + new int[]{1, 2, 3}, new int[]{3}, Replicas.NONE, Replicas.NONE, 3, LeaderRecoveryState.RECOVERED, 1, 1); PartitionRegistration c = new PartitionRegistration( - new int[]{1, 2, 3}, new int[]{1}, Replicas.NONE, Replicas.NONE, 1, 0, 1); + new int[]{1, 2, 3}, new int[]{1}, Replicas.NONE, Replicas.NONE, 1, LeaderRecoveryState.RECOVERED, 0, 1); assertEquals(b, a.merge(new PartitionChangeRecord(). setLeader(3).setIsr(Arrays.asList(3)))); assertEquals("isr: [1, 2] -> [3], leader: 1 -> 3, leaderEpoch: 0 -> 1, partitionEpoch: 0 -> 1", @@ -63,7 +63,7 @@ public void testPartitionControlInfoMergeAndDiff() { @Test public void testRecordRoundTrip() { PartitionRegistration registrationA = new PartitionRegistration( - new int[]{1, 2, 3}, new int[]{1, 2}, new int[]{1}, Replicas.NONE, 1, 0, 0); + new int[]{1, 2, 3}, new int[]{1, 2}, new int[]{1}, Replicas.NONE, 1, LeaderRecoveryState.RECOVERED, 0, 0); Uuid topicId = Uuid.fromString("OGdAI5nxT_m-ds3rJMqPLA"); int partitionId = 4; ApiMessageAndVersion record = registrationA.toRecord(topicId, partitionId); @@ -75,9 +75,9 @@ public void testRecordRoundTrip() { @Test public void testToLeaderAndIsrPartitionState() { PartitionRegistration a = new PartitionRegistration( - new int[]{1, 2, 3}, new int[]{1, 2}, Replicas.NONE, Replicas.NONE, 1, 123, 456); + new int[]{1, 2, 3}, new int[]{1, 2}, Replicas.NONE, Replicas.NONE, 1, LeaderRecoveryState.RECOVERED, 123, 456); PartitionRegistration b = new PartitionRegistration( - new int[]{2, 3, 4}, new int[]{2, 3, 4}, Replicas.NONE, Replicas.NONE, 2, 234, 567); + new int[]{2, 3, 4}, new int[]{2, 3, 4}, Replicas.NONE, Replicas.NONE, 2, LeaderRecoveryState.RECOVERED, 234, 567); assertEquals(new LeaderAndIsrPartitionState(). setTopicName("foo"). setPartitionIndex(1). @@ -109,20 +109,20 @@ public void testToLeaderAndIsrPartitionState() { @Test public void testMergePartitionChangeRecordWithReassignmentData() { PartitionRegistration partition0 = new PartitionRegistration(new int[] {1, 2, 3}, - new int[] {1, 2, 3}, Replicas.NONE, Replicas.NONE, 1, 100, 200); + new int[] {1, 2, 3}, Replicas.NONE, Replicas.NONE, 1, LeaderRecoveryState.RECOVERED, 100, 200); PartitionRegistration partition1 = partition0.merge(new PartitionChangeRecord(). setRemovingReplicas(Collections.singletonList(3)). setAddingReplicas(Collections.singletonList(4)). setReplicas(Arrays.asList(1, 2, 3, 4))); assertEquals(new PartitionRegistration(new int[] {1, 2, 3, 4}, - new int[] {1, 2, 3}, new int[] {3}, new int[] {4}, 1, 100, 201), partition1); + new int[] {1, 2, 3}, new int[] {3}, new int[] {4}, 1, LeaderRecoveryState.RECOVERED, 100, 201), partition1); PartitionRegistration partition2 = partition1.merge(new PartitionChangeRecord(). setIsr(Arrays.asList(1, 2, 4)). setRemovingReplicas(Collections.emptyList()). setAddingReplicas(Collections.emptyList()). setReplicas(Arrays.asList(1, 2, 4))); assertEquals(new PartitionRegistration(new int[] {1, 2, 4}, - new int[] {1, 2, 4}, Replicas.NONE, Replicas.NONE, 1, 100, 202), partition2); + new int[] {1, 2, 4}, Replicas.NONE, Replicas.NONE, 1, LeaderRecoveryState.RECOVERED, 100, 202), partition2); assertFalse(partition2.isReassigning()); } } From 8d6968e8322d74ebb0fde513113d42bef69fb72b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 18 Mar 2022 14:30:52 -0700 Subject: [PATCH 026/447] KAFKA-13682; KRaft Controller auto preferred leader election (#11893) Implement auto leader rebalance for KRaft by keeping track of the set of topic partitions which have a leader that is not the preferred replica. If this set is non-empty then schedule a leader balance event for the replica control manager. When applying PartitionRecords and PartitionChangeRecords to the ReplicationControlManager, if the elected leader is not the preferred replica then remember this topic partition in the set of imbalancedPartitions. Anytime the quorum controller processes a ControllerWriteEvent it schedules a rebalance operation if the there are no pending rebalance operations, the feature is enabled and there are imbalance partitions. This KRaft implementation only supports the configurations properties auto.leader.rebalance.enable and leader.imbalance.check.interval.seconds. The configuration property leader.imbalance.per.broker.percentage is not supported and ignored. Reviewers: Jun Rao , David Arthur --- .../scala/kafka/server/ControllerServer.scala | 40 +++-- .../main/scala/kafka/server/KafkaConfig.scala | 4 +- .../kafka/controller/BrokersToIsrs.java | 36 ---- .../controller/PartitionChangeBuilder.java | 160 +++++++++++------ .../kafka/controller/QuorumController.java | 128 ++++++++++++-- .../controller/ReplicationControlManager.java | 138 ++++++++++++--- .../kafka/controller/TopicIdPartition.java | 56 ++++++ .../kafka/controller/BrokersToIsrsTest.java | 1 - .../PartitionChangeBuilderTest.java | 161 +++++++++++------- .../controller/QuorumControllerTest.java | 147 +++++++++++++++- .../controller/QuorumControllerTestEnv.java | 15 +- .../ReplicationControlManagerTest.java | 99 +++++++++-- 12 files changed, 754 insertions(+), 231 deletions(-) create mode 100644 metadata/src/main/java/org/apache/kafka/controller/TopicIdPartition.java diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index a384926b0b5e..8c3502984787 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -18,6 +18,7 @@ package kafka.server import java.util +import java.util.OptionalLong import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.{CompletableFuture, TimeUnit} @@ -159,21 +160,30 @@ class ControllerServer( alterConfigPolicy = Option(config. getConfiguredInstance(AlterConfigPolicyClassNameProp, classOf[AlterConfigPolicy])) - val controllerBuilder = new QuorumController.Builder(config.nodeId, metaProperties.clusterId). - setTime(time). - setThreadNamePrefix(threadNamePrefixAsString). - setConfigSchema(configSchema). - setRaftClient(raftManager.client). - setDefaultReplicationFactor(config.defaultReplicationFactor.toShort). - setDefaultNumPartitions(config.numPartitions.intValue()). - setIsLeaderRecoverySupported(config.interBrokerProtocolVersion >= KAFKA_3_2_IV0). - setSessionTimeoutNs(TimeUnit.NANOSECONDS.convert(config.brokerSessionTimeoutMs.longValue(), - TimeUnit.MILLISECONDS)). - setSnapshotMaxNewRecordBytes(config.metadataSnapshotMaxNewRecordBytes). - setMetrics(new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry())). - setCreateTopicPolicy(createTopicPolicy.asJava). - setAlterConfigPolicy(alterConfigPolicy.asJava). - setConfigurationValidator(new ControllerConfigurationValidator()) + val controllerBuilder = { + val leaderImbalanceCheckIntervalNs = if (config.autoLeaderRebalanceEnable) { + OptionalLong.of(TimeUnit.NANOSECONDS.convert(config.leaderImbalanceCheckIntervalSeconds, TimeUnit.SECONDS)) + } else { + OptionalLong.empty() + } + + new QuorumController.Builder(config.nodeId, metaProperties.clusterId). + setTime(time). + setThreadNamePrefix(threadNamePrefixAsString). + setConfigSchema(configSchema). + setRaftClient(raftManager.client). + setDefaultReplicationFactor(config.defaultReplicationFactor.toShort). + setDefaultNumPartitions(config.numPartitions.intValue()). + setIsLeaderRecoverySupported(config.interBrokerProtocolVersion >= KAFKA_3_2_IV0). + setSessionTimeoutNs(TimeUnit.NANOSECONDS.convert(config.brokerSessionTimeoutMs.longValue(), + TimeUnit.MILLISECONDS)). + setSnapshotMaxNewRecordBytes(config.metadataSnapshotMaxNewRecordBytes). + setLeaderImbalanceCheckIntervalNs(leaderImbalanceCheckIntervalNs). + setMetrics(new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry())). + setCreateTopicPolicy(createTopicPolicy.asJava). + setAlterConfigPolicy(alterConfigPolicy.asJava). + setConfigurationValidator(new ControllerConfigurationValidator()) + } authorizer match { case Some(a: ClusterMetadataAuthorizer) => controllerBuilder.setAuthorizer(a) case _ => // nothing to do diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 4b697fcf1bff..3282ef6c57c8 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -1226,7 +1226,7 @@ object KafkaConfig { .define(DeleteRecordsPurgatoryPurgeIntervalRequestsProp, INT, Defaults.DeleteRecordsPurgatoryPurgeIntervalRequests, MEDIUM, DeleteRecordsPurgatoryPurgeIntervalRequestsDoc) .define(AutoLeaderRebalanceEnableProp, BOOLEAN, Defaults.AutoLeaderRebalanceEnable, HIGH, AutoLeaderRebalanceEnableDoc) .define(LeaderImbalancePerBrokerPercentageProp, INT, Defaults.LeaderImbalancePerBrokerPercentage, HIGH, LeaderImbalancePerBrokerPercentageDoc) - .define(LeaderImbalanceCheckIntervalSecondsProp, LONG, Defaults.LeaderImbalanceCheckIntervalSeconds, HIGH, LeaderImbalanceCheckIntervalSecondsDoc) + .define(LeaderImbalanceCheckIntervalSecondsProp, LONG, Defaults.LeaderImbalanceCheckIntervalSeconds, atLeast(1), HIGH, LeaderImbalanceCheckIntervalSecondsDoc) .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, HIGH, UncleanLeaderElectionEnableDoc) .define(InterBrokerSecurityProtocolProp, STRING, Defaults.InterBrokerSecurityProtocol, MEDIUM, InterBrokerSecurityProtocolDoc) .define(InterBrokerProtocolVersionProp, STRING, Defaults.InterBrokerProtocolVersion, ApiVersionValidator, MEDIUM, InterBrokerProtocolVersionDoc) @@ -1745,7 +1745,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val deleteRecordsPurgatoryPurgeIntervalRequests = getInt(KafkaConfig.DeleteRecordsPurgatoryPurgeIntervalRequestsProp) val autoLeaderRebalanceEnable = getBoolean(KafkaConfig.AutoLeaderRebalanceEnableProp) val leaderImbalancePerBrokerPercentage = getInt(KafkaConfig.LeaderImbalancePerBrokerPercentageProp) - val leaderImbalanceCheckIntervalSeconds = getLong(KafkaConfig.LeaderImbalanceCheckIntervalSecondsProp) + val leaderImbalanceCheckIntervalSeconds: Long = getLong(KafkaConfig.LeaderImbalanceCheckIntervalSecondsProp) def uncleanLeaderElectionEnable: java.lang.Boolean = getBoolean(KafkaConfig.UncleanLeaderElectionEnableProp) // We keep the user-provided String as `ApiVersion.apply` can choose a slightly different version (eg if `0.10.0` diff --git a/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java b/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java index aceb6ddae3a3..d12e6634061a 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java +++ b/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java @@ -29,7 +29,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.NoSuchElementException; -import java.util.Objects; import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER; import static org.apache.kafka.metadata.Replicas.NONE; @@ -55,41 +54,6 @@ public class BrokersToIsrs { private final static int REPLICA_MASK = 0x7fff_ffff; - static class TopicIdPartition { - private final Uuid topicId; - private final int partitionId; - - TopicIdPartition(Uuid topicId, int partitionId) { - this.topicId = topicId; - this.partitionId = partitionId; - } - - public Uuid topicId() { - return topicId; - } - - public int partitionId() { - return partitionId; - } - - @Override - public boolean equals(Object o) { - if (!(o instanceof TopicIdPartition)) return false; - TopicIdPartition other = (TopicIdPartition) o; - return other.topicId.equals(topicId) && other.partitionId == partitionId; - } - - @Override - public int hashCode() { - return Objects.hash(topicId, partitionId); - } - - @Override - public String toString() { - return topicId + ":" + partitionId; - } - } - static class PartitionsOnReplicaIterator implements Iterator { private final Iterator> iterator; private final boolean leaderOnly; diff --git a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java index b872a5eb23ee..1bbef97c1143 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java +++ b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java @@ -31,7 +31,6 @@ import java.util.List; import java.util.Optional; import java.util.function.Function; -import java.util.function.Supplier; import static org.apache.kafka.common.metadata.MetadataRecordType.PARTITION_CHANGE_RECORD; import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER; @@ -52,37 +51,51 @@ public static boolean changeRecordIsNoOp(PartitionChangeRecord record) { return true; } + /** + * Election types. + */ + public enum Election { + /** + * Perform leader election to keep the partition online. Elect the preferred replica if it is in the ISR. + */ + PREFERRED, + /** + * Perform leader election from the ISR to keep the partition online. + */ + ONLINE, + /** + * Prefer replicas in the ISR but keep the partition online even if it requires picking a leader that is not in the ISR. + */ + UNCLEAN + } + private final PartitionRegistration partition; private final Uuid topicId; private final int partitionId; private final Function isAcceptableLeader; - private final Supplier uncleanElectionOk; private final boolean isLeaderRecoverySupported; private List targetIsr; private List targetReplicas; private List targetRemoving; private List targetAdding; + private Election election = Election.ONLINE; private LeaderRecoveryState targetLeaderRecoveryState; - private boolean alwaysElectPreferredIfPossible; public PartitionChangeBuilder(PartitionRegistration partition, Uuid topicId, int partitionId, Function isAcceptableLeader, - Supplier uncleanElectionOk, boolean isLeaderRecoverySupported) { this.partition = partition; this.topicId = topicId; this.partitionId = partitionId; this.isAcceptableLeader = isAcceptableLeader; - this.uncleanElectionOk = uncleanElectionOk; this.isLeaderRecoverySupported = isLeaderRecoverySupported; this.targetIsr = Replicas.toList(partition.isr); this.targetReplicas = Replicas.toList(partition.replicas); this.targetRemoving = Replicas.toList(partition.removingReplicas); this.targetAdding = Replicas.toList(partition.addingReplicas); this.targetLeaderRecoveryState = partition.leaderRecoveryState; - this.alwaysElectPreferredIfPossible = false; } public PartitionChangeBuilder setTargetIsr(List targetIsr) { @@ -95,8 +108,8 @@ public PartitionChangeBuilder setTargetReplicas(List targetReplicas) { return this; } - public PartitionChangeBuilder setAlwaysElectPreferredIfPossible(boolean alwaysElectPreferredIfPossible) { - this.alwaysElectPreferredIfPossible = alwaysElectPreferredIfPossible; + public PartitionChangeBuilder setElection(Election election) { + this.election = election; return this; } @@ -115,53 +128,104 @@ public PartitionChangeBuilder setTargetLeaderRecoveryState(LeaderRecoveryState t return this; } - boolean shouldTryElection() { - // If the new isr doesn't have the current leader, we need to try to elect a new - // one. Note: this also handles the case where the current leader is NO_LEADER, - // since that value cannot appear in targetIsr. - if (!targetIsr.contains(partition.leader)) return true; + // VisibleForTesting + static class ElectionResult { + final int node; + final boolean unclean; - // Check if we want to try to get away from a non-preferred leader. - if (alwaysElectPreferredIfPossible && !partition.hasPreferredLeader()) return true; + private ElectionResult(int node, boolean unclean) { + this.node = node; + this.unclean = unclean; + } + } - return false; + // VisibleForTesting + /** + * Perform leader election based on the partition state and leader election type. + * + * See documentation for the Election type to see more details on the election types supported. + */ + ElectionResult electLeader() { + if (election == Election.PREFERRED) { + return electPreferredLeader(); + } + + return electAnyLeader(); } - class BestLeader { - final int node; - final boolean unclean; + /** + * Assumes that the election type is Election.PREFERRED + */ + private ElectionResult electPreferredLeader() { + int preferredReplica = targetReplicas.get(0); + if (isValidNewLeader(preferredReplica)) { + return new ElectionResult(preferredReplica, false); + } - BestLeader() { - for (int replica : targetReplicas) { - if (targetIsr.contains(replica) && isAcceptableLeader.apply(replica)) { - this.node = replica; - this.unclean = false; - return; - } - } - if (uncleanElectionOk.get()) { - for (int replica : targetReplicas) { - if (isAcceptableLeader.apply(replica)) { - this.node = replica; - this.unclean = true; - return; - } - } + if (isValidNewLeader(partition.leader)) { + // Don't consider a new leader since the current leader meets all the constraints + return new ElectionResult(partition.leader, false); + } + + Optional onlineLeader = targetReplicas.stream() + .skip(1) + .filter(this::isValidNewLeader) + .findFirst(); + if (onlineLeader.isPresent()) { + return new ElectionResult(onlineLeader.get(), false); + } + + return new ElectionResult(NO_LEADER, false); + } + + /** + * Assumes that the election type is either Election.ONLINE or Election.UNCLEAN + */ + private ElectionResult electAnyLeader() { + if (isValidNewLeader(partition.leader)) { + // Don't consider a new leader since the current leader meets all the constraints + return new ElectionResult(partition.leader, false); + } + + Optional onlineLeader = targetReplicas.stream() + .filter(this::isValidNewLeader) + .findFirst(); + if (onlineLeader.isPresent()) { + return new ElectionResult(onlineLeader.get(), false); + } + + if (election == Election.UNCLEAN) { + // Attempt unclean leader election + Optional uncleanLeader = targetReplicas.stream() + .filter(replica -> isAcceptableLeader.apply(replica)) + .findFirst(); + if (uncleanLeader.isPresent()) { + return new ElectionResult(uncleanLeader.get(), true); } - this.node = NO_LEADER; - this.unclean = false; } + + return new ElectionResult(NO_LEADER, false); + } + + private boolean isValidNewLeader(int replica) { + return targetIsr.contains(replica) && isAcceptableLeader.apply(replica); } private void tryElection(PartitionChangeRecord record) { - BestLeader bestLeader = new BestLeader(); - if (bestLeader.node != partition.leader) { - log.debug("Setting new leader for topicId {}, partition {} to {}", topicId, partitionId, bestLeader.node); - record.setLeader(bestLeader.node); - if (bestLeader.unclean) { + ElectionResult electionResult = electLeader(); + if (electionResult.node != partition.leader) { + log.debug( + "Setting new leader for topicId {}, partition {} to {} using {} election", + topicId, + partitionId, + electionResult.node, + electionResult.unclean ? "an unclean" : "a clean" + ); + record.setLeader(electionResult.node); + if (electionResult.unclean) { // If the election was unclean, we have to forcibly set the ISR to just the // new leader. This can result in data loss! - record.setIsr(Collections.singletonList(bestLeader.node)); + record.setIsr(Collections.singletonList(electionResult.node)); if (partition.leaderRecoveryState != LeaderRecoveryState.RECOVERING && isLeaderRecoverySupported) { // And mark the leader recovery state as RECOVERING @@ -238,13 +302,12 @@ public Optional build() { completeReassignmentIfNeeded(); - if (shouldTryElection()) { - tryElection(record); - } + tryElection(record); triggerLeaderEpochBumpIfNeeded(record); - if (!targetIsr.isEmpty() && !targetIsr.equals(Replicas.toList(partition.isr))) { + if (record.isr() == null && !targetIsr.isEmpty() && !targetIsr.equals(Replicas.toList(partition.isr))) { + // Set the new ISR if it is different from the current ISR and unclean leader election didn't already set it. record.setIsr(targetIsr); } if (!targetReplicas.isEmpty() && !targetReplicas.equals(Replicas.toList(partition.replicas))) { @@ -275,13 +338,12 @@ public String toString() { ", topicId=" + topicId + ", partitionId=" + partitionId + ", isAcceptableLeader=" + isAcceptableLeader + - ", uncleanElectionOk=" + uncleanElectionOk + ", targetIsr=" + targetIsr + ", targetReplicas=" + targetReplicas + ", targetRemoving=" + targetRemoving + ", targetAdding=" + targetAdding + + ", election=" + election + ", targetLeaderRecoveryState=" + targetLeaderRecoveryState + - ", alwaysElectPreferredIfPossible=" + alwaysElectPreferredIfPossible + ')'; } } 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 1138a3cd1a33..966cfa1dd3ea 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -148,6 +148,7 @@ static public class Builder { private boolean isLeaderRecoverySupported = false; private ReplicaPlacer replicaPlacer = new StripedReplicaPlacer(new Random()); private long snapshotMaxNewRecordBytes = Long.MAX_VALUE; + private OptionalLong leaderImbalanceCheckIntervalNs = OptionalLong.empty(); private long sessionTimeoutNs = NANOSECONDS.convert(18, TimeUnit.SECONDS); private ControllerMetrics controllerMetrics = null; private Optional createTopicPolicy = Optional.empty(); @@ -215,6 +216,11 @@ public Builder setSnapshotMaxNewRecordBytes(long value) { return this; } + public Builder setLeaderImbalanceCheckIntervalNs(OptionalLong value) { + this.leaderImbalanceCheckIntervalNs = value; + return this; + } + public Builder setSessionTimeoutNs(long sessionTimeoutNs) { this.sessionTimeoutNs = sessionTimeoutNs; return this; @@ -265,9 +271,9 @@ public QuorumController build() throws Exception { queue = new KafkaEventQueue(time, logContext, threadNamePrefix + "QuorumController"); return new QuorumController(logContext, nodeId, clusterId, queue, time, configSchema, raftClient, supportedFeatures, defaultReplicationFactor, - defaultNumPartitions, isLeaderRecoverySupported, replicaPlacer, - snapshotMaxNewRecordBytes, sessionTimeoutNs, controllerMetrics, createTopicPolicy, - alterConfigPolicy, configurationValidator, authorizer); + defaultNumPartitions, isLeaderRecoverySupported, replicaPlacer, snapshotMaxNewRecordBytes, + leaderImbalanceCheckIntervalNs, sessionTimeoutNs, controllerMetrics, + createTopicPolicy, alterConfigPolicy, configurationValidator, authorizer); } catch (Exception e) { Utils.closeQuietly(queue, "event queue"); throw e; @@ -351,7 +357,7 @@ private Throwable handleEventException(String name, OptionalLong startProcessingTimeNs, Throwable exception) { if (!startProcessingTimeNs.isPresent()) { - log.info("unable to start processing {} because of {}.", name, + log.error("{}: unable to start processing because of {}.", name, exception.getClass().getSimpleName()); if (exception instanceof ApiException) { return exception; @@ -369,7 +375,7 @@ private Throwable handleEventException(String name, } log.warn("{}: failed with unknown server exception {} at epoch {} in {} us. " + "Reverting to last committed offset {}.", - this, exception.getClass().getSimpleName(), curClaimEpoch, deltaUs, + name, exception.getClass().getSimpleName(), curClaimEpoch, deltaUs, lastCommittedOffset, exception); raftClient.resign(curClaimEpoch); renounce(); @@ -649,18 +655,18 @@ public void run() throws Exception { OptionalLong maybeOffset = purgatory.highestPendingOffset(); if (!maybeOffset.isPresent()) { // If the purgatory is empty, there are no pending operations and no - // uncommitted state. We can return immediately. + // uncommitted state. We can complete immediately. resultAndOffset = ControllerResultAndOffset.of(-1, result); log.debug("Completing read-only operation {} immediately because " + "the purgatory is empty.", this); complete(null); - return; + } else { + // If there are operations in the purgatory, we want to wait for the latest + // one to complete before returning our result to the user. + resultAndOffset = ControllerResultAndOffset.of(maybeOffset.getAsLong(), result); + log.debug("Read-only operation {} will be completed when the log " + + "reaches offset {}", this, resultAndOffset.offset()); } - // If there are operations in the purgatory, we want to wait for the latest - // one to complete before returning our result to the user. - resultAndOffset = ControllerResultAndOffset.of(maybeOffset.getAsLong(), result); - log.debug("Read-only operation {} will be completed when the log " + - "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 @@ -679,10 +685,19 @@ public void run() throws Exception { replay(message.message(), Optional.empty(), offset); } snapshotRegistry.getOrCreateSnapshot(offset); + log.debug("Read-write operation {} will be completed when the log " + "reaches offset {}.", this, resultAndOffset.offset()); } - purgatory.add(resultAndOffset.offset(), this); + + // After every controller write event, schedule a leader rebalance if there are any topic partition + // with leader that is not the preferred leader. + maybeScheduleNextBalancePartitionLeaders(); + + // Remember the latest offset and future if it is not already completed + if (!future.isDone()) { + purgatory.add(resultAndOffset.offset(), this); + } } @Override @@ -749,7 +764,6 @@ public void handleCommit(BatchReader reader) { // otherwise, we should delete up to the current committed offset. snapshotRegistry.deleteSnapshotsUpTo( snapshotGeneratorManager.snapshotLastOffsetFromLog().orElse(offset)); - } else { // If the controller is a standby, replay the records that were // created by the active controller. @@ -867,6 +881,10 @@ public void handleLeaderChange(LeaderAndEpoch newLeader) { // required because the active controller assumes that there is always an in-memory snapshot at the // last committed offset. snapshotRegistry.getOrCreateSnapshot(lastCommittedOffset); + + // When becoming the active controller, schedule a leader rebalance if there are any topic partition + // with leader that is not the preferred leader. + maybeScheduleNextBalancePartitionLeaders(); }); } else if (curClaimEpoch != -1) { appendRaftEvent("handleRenounce[" + curClaimEpoch + "]", () -> { @@ -913,6 +931,7 @@ private void renounce() { writeOffset = -1; clusterControl.deactivate(); cancelMaybeFenceReplicas(); + cancelMaybeBalancePartitionLeaders(); } private void scheduleDeferredWriteEvent(String name, long deadlineNs, @@ -959,6 +978,62 @@ private void cancelMaybeFenceReplicas() { queue.cancelDeferred(MAYBE_FENCE_REPLICAS); } + private static final String MAYBE_BALANCE_PARTITION_LEADERS = "maybeBalancePartitionLeaders"; + + private static final int MAX_ELECTIONS_PER_IMBALANCE = 1_000; + + private void maybeScheduleNextBalancePartitionLeaders() { + if (imbalancedScheduled != ImbalanceSchedule.SCHEDULED && + leaderImbalanceCheckIntervalNs.isPresent() && + replicationControl.arePartitionLeadersImbalanced()) { + + log.debug( + "Scheduling write event for {} because scheduled ({}), checkIntervalNs ({}) and isImbalanced ({})", + MAYBE_BALANCE_PARTITION_LEADERS, + imbalancedScheduled, + leaderImbalanceCheckIntervalNs, + replicationControl.arePartitionLeadersImbalanced() + ); + + ControllerWriteEvent event = new ControllerWriteEvent<>(MAYBE_BALANCE_PARTITION_LEADERS, () -> { + ControllerResult result = replicationControl.maybeBalancePartitionLeaders(); + + // reschedule the operation after the leaderImbalanceCheckIntervalNs interval. + // Mark the imbalance event as completed and reschedule if necessary + if (result.response()) { + imbalancedScheduled = ImbalanceSchedule.IMMEDIATELY; + } else { + imbalancedScheduled = ImbalanceSchedule.DEFERRED; + } + + // Note that rescheduling this event here is not required because MAYBE_BALANCE_PARTITION_LEADERS + // is a ControllerWriteEvent. ControllerWriteEvent always calls this method after the records + // generated by a ControllerWriteEvent have been applied. + + return result; + }); + + long delayNs = time.nanoseconds(); + if (imbalancedScheduled == ImbalanceSchedule.DEFERRED) { + delayNs += leaderImbalanceCheckIntervalNs.getAsLong(); + } else { + // The current implementation of KafkaEventQueue always picks from the deferred collection of operations + // before picking from the non-deferred collection of operations. This can result in some unfairness if + // deferred operation are scheduled for immediate execution. This delays them by a small amount of time. + delayNs += NANOSECONDS.convert(10, TimeUnit.MILLISECONDS); + } + + queue.scheduleDeferred(MAYBE_BALANCE_PARTITION_LEADERS, new EarliestDeadlineFunction(delayNs), event); + + imbalancedScheduled = ImbalanceSchedule.SCHEDULED; + } + } + + private void cancelMaybeBalancePartitionLeaders() { + imbalancedScheduled = ImbalanceSchedule.DEFERRED; + queue.cancelDeferred(MAYBE_BALANCE_PARTITION_LEADERS); + } + @SuppressWarnings("unchecked") private void replay(ApiMessage message, Optional snapshotId, long offset) { try { @@ -1203,6 +1278,25 @@ private void resetState() { */ private long newBytesSinceLastSnapshot = 0; + /** + * How long to delay partition leader balancing operations. + */ + private final OptionalLong leaderImbalanceCheckIntervalNs; + + private enum ImbalanceSchedule { + // The leader balancing operation has been scheduled + SCHEDULED, + // If the leader balancing operation should be scheduled, schedule it with a delay + DEFERRED, + // If the leader balancing operation should be scheduled, schedule it immediately + IMMEDIATELY + } + + /** + * Tracks the scheduling state for partition leader balancing operations. + */ + private ImbalanceSchedule imbalancedScheduled = ImbalanceSchedule.DEFERRED; + private QuorumController(LogContext logContext, int nodeId, String clusterId, @@ -1216,6 +1310,7 @@ private QuorumController(LogContext logContext, boolean isLeaderRecoverySupported, ReplicaPlacer replicaPlacer, long snapshotMaxNewRecordBytes, + OptionalLong leaderImbalanceCheckIntervalNs, long sessionTimeoutNs, ControllerMetrics controllerMetrics, Optional createTopicPolicy, @@ -1244,9 +1339,10 @@ private QuorumController(LogContext logContext, this.featureControl = new FeatureControlManager(supportedFeatures, snapshotRegistry); this.producerIdControlManager = new ProducerIdControlManager(clusterControl, snapshotRegistry); this.snapshotMaxNewRecordBytes = snapshotMaxNewRecordBytes; + this.leaderImbalanceCheckIntervalNs = leaderImbalanceCheckIntervalNs; this.replicationControl = new ReplicationControlManager(snapshotRegistry, - logContext, defaultReplicationFactor, defaultNumPartitions, isLeaderRecoverySupported, - configurationControl, clusterControl, controllerMetrics, createTopicPolicy); + logContext, defaultReplicationFactor, defaultNumPartitions, MAX_ELECTIONS_PER_IMBALANCE, + isLeaderRecoverySupported, configurationControl, clusterControl, controllerMetrics, createTopicPolicy); this.authorizer = authorizer; authorizer.ifPresent(a -> a.setAclMutator(this)); this.aclControlManager = new AclControlManager(snapshotRegistry, authorizer); diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index 877a144ac2e4..edf627eb51e3 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -71,7 +71,6 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.controller.BrokersToIsrs.TopicIdPartition; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistration; @@ -81,6 +80,7 @@ import org.apache.kafka.server.policy.CreateTopicPolicy; import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.TimelineHashMap; +import org.apache.kafka.timeline.TimelineHashSet; import org.apache.kafka.timeline.TimelineInteger; import org.slf4j.Logger; @@ -88,16 +88,18 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Optional; -import java.util.function.Function; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.ListIterator; -import java.util.Map; import java.util.Map.Entry; +import java.util.Map; import java.util.NoSuchElementException; +import java.util.Optional; import java.util.OptionalInt; +import java.util.Set; +import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -167,14 +169,14 @@ public Uuid topicId() { private final boolean isLeaderRecoverySupported; /** - * A count of the total number of partitions in the cluster. + * Maximum number of leader elections to perform during one partition leader balancing operation. */ - private final TimelineInteger globalPartitionCount; + private final int maxElectionsPerImbalance; /** - * A count of the number of partitions that do not have their first replica as a leader. + * A count of the total number of partitions in the cluster. */ - private final TimelineInteger preferredReplicaImbalanceCount; + private final TimelineInteger globalPartitionCount; /** * A reference to the controller's configuration control manager. @@ -216,10 +218,16 @@ public Uuid topicId() { */ private final TimelineHashMap reassigningTopics; + /** + * The set of topic partitions for which the leader is not the preferred leader. + */ + private final TimelineHashSet imbalancedPartitions; + ReplicationControlManager(SnapshotRegistry snapshotRegistry, LogContext logContext, short defaultReplicationFactor, int defaultNumPartitions, + int maxElectionsPerImbalance, boolean isLeaderRecoverySupported, ConfigurationControlManager configurationControl, ClusterControlManager clusterControl, @@ -229,17 +237,18 @@ public Uuid topicId() { this.log = logContext.logger(ReplicationControlManager.class); this.defaultReplicationFactor = defaultReplicationFactor; this.defaultNumPartitions = defaultNumPartitions; + this.maxElectionsPerImbalance = maxElectionsPerImbalance; this.isLeaderRecoverySupported = isLeaderRecoverySupported; this.configurationControl = configurationControl; this.controllerMetrics = controllerMetrics; this.createTopicPolicy = createTopicPolicy; this.clusterControl = clusterControl; this.globalPartitionCount = new TimelineInteger(snapshotRegistry); - this.preferredReplicaImbalanceCount = new TimelineInteger(snapshotRegistry); this.topicsByName = new TimelineHashMap<>(snapshotRegistry, 0); this.topics = new TimelineHashMap<>(snapshotRegistry, 0); this.brokersToIsrs = new BrokersToIsrs(snapshotRegistry); this.reassigningTopics = new TimelineHashMap<>(snapshotRegistry, 0); + this.imbalancedPartitions = new TimelineHashSet<>(snapshotRegistry, 0); } public void replay(TopicRecord record) { @@ -277,11 +286,15 @@ public void replay(PartitionRecord record) { updateReassigningTopicsIfNeeded(record.topicId(), record.partitionId(), prevPartInfo.isReassigning(), newPartInfo.isReassigning()); } - if (newPartInfo.leader != newPartInfo.preferredReplica()) { - preferredReplicaImbalanceCount.increment(); + + if (newPartInfo.hasPreferredLeader()) { + imbalancedPartitions.remove(new TopicIdPartition(record.topicId(), record.partitionId())); + } else { + imbalancedPartitions.add(new TopicIdPartition(record.topicId(), record.partitionId())); } + controllerMetrics.setOfflinePartitionCount(brokersToIsrs.offlinePartitionCount()); - controllerMetrics.setPreferredReplicaImbalanceCount(preferredReplicaImbalanceCount.get()); + controllerMetrics.setPreferredReplicaImbalanceCount(imbalancedPartitions.size()); } private void updateReassigningTopicsIfNeeded(Uuid topicId, int partitionId, @@ -323,11 +336,16 @@ public void replay(PartitionChangeRecord record) { String topicPart = topicInfo.name + "-" + record.partitionId() + " with topic ID " + record.topicId(); newPartitionInfo.maybeLogPartitionChange(log, topicPart, prevPartitionInfo); - if (!newPartitionInfo.hasPreferredLeader() && prevPartitionInfo.hasPreferredLeader()) { - preferredReplicaImbalanceCount.increment(); + + if (newPartitionInfo.hasPreferredLeader()) { + imbalancedPartitions.remove(new TopicIdPartition(record.topicId(), record.partitionId())); + } else { + imbalancedPartitions.add(new TopicIdPartition(record.topicId(), record.partitionId())); } + controllerMetrics.setOfflinePartitionCount(brokersToIsrs.offlinePartitionCount()); - controllerMetrics.setPreferredReplicaImbalanceCount(preferredReplicaImbalanceCount.get()); + controllerMetrics.setPreferredReplicaImbalanceCount(imbalancedPartitions.size()); + if (record.removingReplicas() != null || record.addingReplicas() != null) { log.info("Replayed partition assignment change {} for topic {}", record, topicInfo.name); } else if (log.isTraceEnabled()) { @@ -348,14 +366,17 @@ public void replay(RemoveTopicRecord record) { // Delete the configurations associated with this topic. configurationControl.deleteTopicConfigs(topic.name); - // Remove the entries for this topic in brokersToIsrs. - for (PartitionRegistration partition : topic.parts.values()) { + for (Map.Entry entry : topic.parts.entrySet()) { + int partitionId = entry.getKey(); + PartitionRegistration partition = entry.getValue(); + + // Remove the entries for this topic in brokersToIsrs. for (int i = 0; i < partition.isr.length; i++) { brokersToIsrs.removeTopicEntryForBroker(topic.id, partition.isr[i]); } - if (partition.leader != partition.preferredReplica()) { - preferredReplicaImbalanceCount.decrement(); - } + + imbalancedPartitions.remove(new TopicIdPartition(record.topicId(), partitionId)); + globalPartitionCount.decrement(); } brokersToIsrs.removeTopicEntryForBroker(topic.id, NO_LEADER); @@ -363,7 +384,7 @@ public void replay(RemoveTopicRecord record) { controllerMetrics.setGlobalTopicsCount(topics.size()); controllerMetrics.setGlobalPartitionCount(globalPartitionCount.get()); controllerMetrics.setOfflinePartitionCount(brokersToIsrs.offlinePartitionCount()); - controllerMetrics.setPreferredReplicaImbalanceCount(preferredReplicaImbalanceCount.get()); + controllerMetrics.setPreferredReplicaImbalanceCount(imbalancedPartitions.size()); log.info("Removed topic {} with ID {}.", topic.name, record.topicId()); } @@ -669,6 +690,11 @@ BrokersToIsrs brokersToIsrs() { return brokersToIsrs; } + // VisibleForTesting + Set imbalancedPartitions() { + return new HashSet<>(imbalancedPartitions); + } + ControllerResult alterPartition(AlterPartitionRequestData request) { clusterControl.checkBrokerEpoch(request.brokerId(), request.brokerEpoch()); AlterPartitionResponseData response = new AlterPartitionResponseData(); @@ -709,8 +735,10 @@ ControllerResult alterPartition(AlterPartitionReques topic.id, partitionId, r -> clusterControl.unfenced(r), - () -> configurationControl.uncleanLeaderElectionEnabledForTopic(topicData.name()), isLeaderRecoverySupported); + if (configurationControl.uncleanLeaderElectionEnabledForTopic(topicData.name())) { + builder.setElection(PartitionChangeBuilder.Election.UNCLEAN); + } builder.setTargetIsr(partitionData.newIsr()); builder.setTargetLeaderRecoveryState( LeaderRecoveryState.of(partitionData.leaderRecoveryState())); @@ -993,14 +1021,16 @@ ApiError electLeader(String topic, int partitionId, ElectionType electionType, return new ApiError(Errors.ELECTION_NOT_NEEDED); } + PartitionChangeBuilder.Election election = PartitionChangeBuilder.Election.PREFERRED; + if (electionType == ElectionType.UNCLEAN) { + election = PartitionChangeBuilder.Election.UNCLEAN; + } PartitionChangeBuilder builder = new PartitionChangeBuilder(partition, topicId, partitionId, r -> clusterControl.unfenced(r), - () -> electionType == ElectionType.UNCLEAN, isLeaderRecoverySupported); - - builder.setAlwaysElectPreferredIfPossible(electionType == ElectionType.PREFERRED); + builder.setElection(election); Optional record = builder.build(); if (!record.isPresent()) { if (electionType == ElectionType.PREFERRED) { @@ -1075,6 +1105,55 @@ ControllerResult maybeFenceOneStaleBroker() { return ControllerResult.of(records, null); } + boolean arePartitionLeadersImbalanced() { + return !imbalancedPartitions.isEmpty(); + } + + /** + * Attempt to elect a preferred leader for all topic partitions which have a leader that is not the preferred replica. + * + * The response() method in the return object is true if this method returned without electing all possible preferred replicas. + * The quorum controlller should reschedule this operation immediately if it is true. + * + * @return All of the election records and if there may be more available preferred replicas to elect as leader + */ + ControllerResult maybeBalancePartitionLeaders() { + List records = new ArrayList<>(); + + boolean rescheduleImmidiately = false; + for (TopicIdPartition topicPartition : imbalancedPartitions) { + if (records.size() >= maxElectionsPerImbalance) { + rescheduleImmidiately = true; + break; + } + + TopicControlInfo topic = topics.get(topicPartition.topicId()); + if (topic == null) { + log.error("Skipping unknown imbalanced topic {}", topicPartition); + continue; + } + + PartitionRegistration partition = topic.parts.get(topicPartition.partitionId()); + if (partition == null) { + log.error("Skipping unknown imbalanced partition {}", topicPartition); + continue; + } + + // Attempt to perform a preferred leader election + PartitionChangeBuilder builder = new PartitionChangeBuilder( + partition, + topicPartition.topicId(), + topicPartition.partitionId(), + r -> clusterControl.unfenced(r), + isLeaderRecoverySupported + ); + builder.setElection(PartitionChangeBuilder.Election.PREFERRED); + builder.build().ifPresent(records::add); + } + + return ControllerResult.of(records, rescheduleImmidiately); + } + // Visible for testing Boolean isBrokerUnfenced(int brokerId) { return clusterControl.unfenced(brokerId); @@ -1266,8 +1345,10 @@ void generateLeaderAndIsrUpdates(String context, topicIdPart.topicId(), topicIdPart.partitionId(), isAcceptableLeader, - () -> configurationControl.uncleanLeaderElectionEnabledForTopic(topic.name), isLeaderRecoverySupported); + if (configurationControl.uncleanLeaderElectionEnabledForTopic(topic.name)) { + builder.setElection(PartitionChangeBuilder.Election.UNCLEAN); + } // Note: if brokerToRemove was passed as NO_LEADER, this is a no-op (the new // target ISR will be the same as the old one). @@ -1374,8 +1455,10 @@ Optional cancelPartitionReassignment(String topicName, tp.topicId(), tp.partitionId(), r -> clusterControl.unfenced(r), - () -> configurationControl.uncleanLeaderElectionEnabledForTopic(topicName), isLeaderRecoverySupported); + if (configurationControl.uncleanLeaderElectionEnabledForTopic(topicName)) { + builder.setElection(PartitionChangeBuilder.Election.UNCLEAN); + } builder.setTargetIsr(revert.isr()). setTargetReplicas(revert.replicas()). setTargetRemoving(Collections.emptyList()). @@ -1424,7 +1507,6 @@ Optional changePartitionReassignment(TopicIdPartition tp, tp.topicId(), tp.partitionId(), r -> clusterControl.unfenced(r), - () -> false, isLeaderRecoverySupported); if (!reassignment.merged().equals(currentReplicas)) { builder.setTargetReplicas(reassignment.merged()); diff --git a/metadata/src/main/java/org/apache/kafka/controller/TopicIdPartition.java b/metadata/src/main/java/org/apache/kafka/controller/TopicIdPartition.java new file mode 100644 index 000000000000..b594c4a4ec35 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/controller/TopicIdPartition.java @@ -0,0 +1,56 @@ +/* + * 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.controller; + +import java.util.Objects; +import org.apache.kafka.common.Uuid; + +final class TopicIdPartition { + private final Uuid topicId; + private final int partitionId; + + TopicIdPartition(Uuid topicId, int partitionId) { + this.topicId = topicId; + this.partitionId = partitionId; + } + + public Uuid topicId() { + return topicId; + } + + public int partitionId() { + return partitionId; + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof TopicIdPartition)) return false; + TopicIdPartition other = (TopicIdPartition) o; + return other.topicId.equals(topicId) && other.partitionId == partitionId; + } + + @Override + public int hashCode() { + return Objects.hash(topicId, partitionId); + } + + @Override + public String toString() { + return topicId + ":" + partitionId; + } +} diff --git a/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java b/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java index 6510ee5d4b9e..1258c9d5a2e7 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.controller.BrokersToIsrs.PartitionsOnReplicaIterator; -import org.apache.kafka.controller.BrokersToIsrs.TopicIdPartition; import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java index cf3bb5160f04..06c54322ec94 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.metadata.PartitionChangeRecord; -import org.apache.kafka.controller.PartitionChangeBuilder.BestLeader; +import org.apache.kafka.controller.PartitionChangeBuilder.ElectionResult; import org.apache.kafka.metadata.LeaderRecoveryState; import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.metadata.Replicas; @@ -34,6 +34,7 @@ import java.util.Optional; import static org.apache.kafka.common.metadata.MetadataRecordType.PARTITION_CHANGE_RECORD; +import static org.apache.kafka.controller.PartitionChangeBuilder.Election; import static org.apache.kafka.controller.PartitionChangeBuilder.changeRecordIsNoOp; import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER; import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER_CHANGE; @@ -62,8 +63,8 @@ public void testChangeRecordIsNoOp() { private final static Uuid FOO_ID = Uuid.fromString("FbrrdcfiR-KC2CPSTHaJrg"); - private static PartitionChangeBuilder createFooBuilder(boolean allowUnclean) { - return new PartitionChangeBuilder(FOO, FOO_ID, 0, r -> r != 3, () -> allowUnclean, true); + private static PartitionChangeBuilder createFooBuilder() { + return new PartitionChangeBuilder(FOO, FOO_ID, 0, r -> r != 3, true); } private final static PartitionRegistration BAR = new PartitionRegistration( @@ -72,52 +73,56 @@ private static PartitionChangeBuilder createFooBuilder(boolean allowUnclean) { private final static Uuid BAR_ID = Uuid.fromString("LKfUsCBnQKekvL9O5dY9nw"); - private static PartitionChangeBuilder createBarBuilder(boolean allowUnclean) { - return new PartitionChangeBuilder(BAR, BAR_ID, 0, r -> r != 3, () -> allowUnclean, true); + private static PartitionChangeBuilder createBarBuilder() { + return new PartitionChangeBuilder(BAR, BAR_ID, 0, r -> r != 3, true); } - private static void assertBestLeaderEquals(PartitionChangeBuilder builder, + private final static PartitionRegistration BAZ = new PartitionRegistration( + new int[] {2, 1, 3}, new int[] {1, 3}, Replicas.NONE, Replicas.NONE, + 3, LeaderRecoveryState.RECOVERED, 100, 200); + + private final static Uuid BAZ_ID = Uuid.fromString("wQzt5gkSTwuQNXZF5gIw7A"); + + private static PartitionChangeBuilder createBazBuilder() { + return new PartitionChangeBuilder(BAZ, BAZ_ID, 0, __ -> true, true); + } + + private final static PartitionRegistration OFFLINE = new PartitionRegistration( + new int[] {2, 1, 3}, new int[] {3}, Replicas.NONE, Replicas.NONE, + -1, LeaderRecoveryState.RECOVERED, 100, 200); + + private final static Uuid OFFLINE_ID = Uuid.fromString("LKfUsCBnQKekvL9O5dY9nw"); + + private static PartitionChangeBuilder createOfflineBuilder() { + return new PartitionChangeBuilder(OFFLINE, OFFLINE_ID, 0, r -> r == 1, true); + } + + private static void assertElectLeaderEquals(PartitionChangeBuilder builder, int expectedNode, boolean expectedUnclean) { - BestLeader bestLeader = builder.new BestLeader(); - assertEquals(expectedNode, bestLeader.node); - assertEquals(expectedUnclean, bestLeader.unclean); + ElectionResult electionResult = builder.electLeader(); + assertEquals(expectedNode, electionResult.node); + assertEquals(expectedUnclean, electionResult.unclean); } @Test - public void testBestLeader() { - assertBestLeaderEquals(createFooBuilder(false), 2, false); - assertBestLeaderEquals(createFooBuilder(true), 2, false); - assertBestLeaderEquals(createFooBuilder(false). - setTargetIsr(Arrays.asList(1, 3)), 1, false); - assertBestLeaderEquals(createFooBuilder(true). - setTargetIsr(Arrays.asList(1, 3)), 1, false); - assertBestLeaderEquals(createFooBuilder(false). - setTargetIsr(Arrays.asList(3)), NO_LEADER, false); - assertBestLeaderEquals(createFooBuilder(true). - setTargetIsr(Arrays.asList(3)), 2, true); - assertBestLeaderEquals(createFooBuilder(true). - setTargetIsr(Arrays.asList(4)).setTargetReplicas(Arrays.asList(2, 1, 3, 4)), - 4, false); - } + public void testElectLeader() { + assertElectLeaderEquals(createFooBuilder().setElection(Election.PREFERRED), 2, false); + assertElectLeaderEquals(createFooBuilder(), 1, false); + assertElectLeaderEquals(createFooBuilder().setElection(Election.UNCLEAN), 1, false); + assertElectLeaderEquals(createFooBuilder().setTargetIsr(Arrays.asList(1, 3)), 1, false); + assertElectLeaderEquals(createFooBuilder().setElection(Election.UNCLEAN).setTargetIsr(Arrays.asList(1, 3)), 1, false); + assertElectLeaderEquals(createFooBuilder().setTargetIsr(Arrays.asList(3)), NO_LEADER, false); + assertElectLeaderEquals(createFooBuilder().setElection(Election.UNCLEAN).setTargetIsr(Arrays.asList(3)), 2, true); + assertElectLeaderEquals( + createFooBuilder().setElection(Election.UNCLEAN).setTargetIsr(Arrays.asList(4)).setTargetReplicas(Arrays.asList(2, 1, 3, 4)), + 4, + false + ); - @Test - public void testShouldTryElection() { - assertFalse(createFooBuilder(false).shouldTryElection()); - assertTrue(createFooBuilder(false).setAlwaysElectPreferredIfPossible(true). - shouldTryElection()); - assertTrue(createFooBuilder(false).setTargetIsr(Arrays.asList(2, 3)). - shouldTryElection()); - assertFalse(createFooBuilder(false).setTargetIsr(Arrays.asList(2, 1)). - shouldTryElection()); - - assertTrue(createFooBuilder(true) - .setTargetIsr(Arrays.asList(3)) - .shouldTryElection()); - assertTrue(createFooBuilder(true) - .setTargetIsr(Arrays.asList(4)) - .setTargetReplicas(Arrays.asList(2, 1, 3, 4)) - .shouldTryElection()); + assertElectLeaderEquals(createBazBuilder().setElection(Election.PREFERRED), 3, false); + assertElectLeaderEquals(createBazBuilder(), 3, false); + assertElectLeaderEquals(createBazBuilder().setElection(Election.UNCLEAN), 3, false); } private static void testTriggerLeaderEpochBumpIfNeededLeader(PartitionChangeBuilder builder, @@ -129,27 +134,28 @@ private static void testTriggerLeaderEpochBumpIfNeededLeader(PartitionChangeBuil @Test public void testTriggerLeaderEpochBumpIfNeeded() { - testTriggerLeaderEpochBumpIfNeededLeader(createFooBuilder(false), + testTriggerLeaderEpochBumpIfNeededLeader(createFooBuilder(), new PartitionChangeRecord(), NO_LEADER_CHANGE); - testTriggerLeaderEpochBumpIfNeededLeader(createFooBuilder(false). + testTriggerLeaderEpochBumpIfNeededLeader(createFooBuilder(). setTargetIsr(Arrays.asList(2, 1)), new PartitionChangeRecord(), 1); - testTriggerLeaderEpochBumpIfNeededLeader(createFooBuilder(false). + testTriggerLeaderEpochBumpIfNeededLeader(createFooBuilder(). setTargetIsr(Arrays.asList(2, 1, 3, 4)), new PartitionChangeRecord(), NO_LEADER_CHANGE); - testTriggerLeaderEpochBumpIfNeededLeader(createFooBuilder(false). + testTriggerLeaderEpochBumpIfNeededLeader(createFooBuilder(). setTargetReplicas(Arrays.asList(2, 1, 3, 4)), new PartitionChangeRecord(), NO_LEADER_CHANGE); - testTriggerLeaderEpochBumpIfNeededLeader(createFooBuilder(false). + testTriggerLeaderEpochBumpIfNeededLeader(createFooBuilder(). setTargetReplicas(Arrays.asList(2, 1, 3, 4)), new PartitionChangeRecord().setLeader(2), 2); } @Test public void testNoChange() { - assertEquals(Optional.empty(), createFooBuilder(false).build()); - assertEquals(Optional.empty(), createFooBuilder(true).build()); - assertEquals(Optional.empty(), createBarBuilder(false).build()); - assertEquals(Optional.empty(), createBarBuilder(true).build()); + assertEquals(Optional.empty(), createFooBuilder().build()); + assertEquals(Optional.empty(), createFooBuilder().setElection(Election.UNCLEAN).build()); + assertEquals(Optional.empty(), createBarBuilder().build()); + assertEquals(Optional.empty(), createBarBuilder().setElection(Election.UNCLEAN).build()); + assertEquals(Optional.empty(), createBazBuilder().setElection(Election.PREFERRED).build()); } @Test @@ -159,7 +165,7 @@ public void testIsrChangeAndLeaderBump() { setPartitionId(0). setIsr(Arrays.asList(2, 1)). setLeader(1), PARTITION_CHANGE_RECORD.highestSupportedVersion())), - createFooBuilder(false).setTargetIsr(Arrays.asList(2, 1)).build()); + createFooBuilder().setTargetIsr(Arrays.asList(2, 1)).build()); } @Test @@ -169,7 +175,7 @@ public void testIsrChangeAndLeaderChange() { setPartitionId(0). setIsr(Arrays.asList(2, 3)). setLeader(2), PARTITION_CHANGE_RECORD.highestSupportedVersion())), - createFooBuilder(false).setTargetIsr(Arrays.asList(2, 3)).build()); + createFooBuilder().setTargetIsr(Arrays.asList(2, 3)).build()); } @Test @@ -179,7 +185,7 @@ public void testReassignmentRearrangesReplicas() { setPartitionId(0). setReplicas(Arrays.asList(3, 2, 1)), PARTITION_CHANGE_RECORD.highestSupportedVersion())), - createFooBuilder(false).setTargetReplicas(Arrays.asList(3, 2, 1)).build()); + createFooBuilder().setTargetReplicas(Arrays.asList(3, 2, 1)).build()); } @Test @@ -193,7 +199,7 @@ public void testIsrEnlargementCompletesReassignment() { setRemovingReplicas(Collections.emptyList()). setAddingReplicas(Collections.emptyList()), PARTITION_CHANGE_RECORD.highestSupportedVersion())), - createBarBuilder(false).setTargetIsr(Arrays.asList(1, 2, 3, 4)).build()); + createBarBuilder().setTargetIsr(Arrays.asList(1, 2, 3, 4)).build()); } @Test @@ -209,7 +215,7 @@ public void testRevertReassignment() { setRemovingReplicas(Collections.emptyList()). setAddingReplicas(Collections.emptyList()), PARTITION_CHANGE_RECORD.highestSupportedVersion())), - createBarBuilder(false). + createBarBuilder(). setTargetReplicas(revert.replicas()). setTargetIsr(revert.isr()). setTargetRemoving(Collections.emptyList()). @@ -231,7 +237,7 @@ public void testRemovingReplicaReassignment() { setIsr(Arrays.asList(2, 1)). setLeader(1), PARTITION_CHANGE_RECORD.highestSupportedVersion())), - createFooBuilder(false). + createFooBuilder(). setTargetReplicas(replicas.merged()). setTargetRemoving(replicas.removing()). build()); @@ -250,12 +256,47 @@ public void testAddingReplicaReassignment() { setReplicas(Arrays.asList(1, 2, 3, 4)). setAddingReplicas(Collections.singletonList(4)), PARTITION_CHANGE_RECORD.highestSupportedVersion())), - createFooBuilder(false). + createFooBuilder(). setTargetReplicas(replicas.merged()). setTargetAdding(replicas.adding()). build()); } + @Test + public void testUncleanLeaderElection() { + ApiMessageAndVersion expectedRecord = new ApiMessageAndVersion( + new PartitionChangeRecord() + .setTopicId(FOO_ID) + .setPartitionId(0) + .setIsr(Arrays.asList(2)) + .setLeader(2) + .setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value()), + PARTITION_CHANGE_RECORD.highestSupportedVersion() + ); + assertEquals( + Optional.of(expectedRecord), + createFooBuilder().setElection(Election.UNCLEAN).setTargetIsr(Arrays.asList(3)).build() + ); + + expectedRecord = new ApiMessageAndVersion( + new PartitionChangeRecord() + .setTopicId(OFFLINE_ID) + .setPartitionId(0) + .setIsr(Arrays.asList(1)) + .setLeader(1) + .setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value()), + PARTITION_CHANGE_RECORD.highestSupportedVersion() + ); + assertEquals( + Optional.of(expectedRecord), + createOfflineBuilder().setElection(Election.UNCLEAN).build() + ); + assertEquals( + Optional.of(expectedRecord), + createOfflineBuilder().setElection(Election.UNCLEAN).setTargetIsr(Arrays.asList(2)).build() + ); + } + @ParameterizedTest @ValueSource(booleans = {true, false}) public void testChangeInLeadershipDoesNotChangeRecoveryState(boolean isLeaderRecoverySupported) { @@ -279,7 +320,6 @@ public void testChangeInLeadershipDoesNotChangeRecoveryState(boolean isLeaderRec FOO_ID, 0, brokerId -> false, - () -> false, isLeaderRecoverySupported ); // Set the target ISR to empty to indicate that the last leader is offline @@ -305,7 +345,6 @@ public void testChangeInLeadershipDoesNotChangeRecoveryState(boolean isLeaderRec FOO_ID, 0, brokerId -> true, - () -> false, isLeaderRecoverySupported ); @@ -342,9 +381,9 @@ void testUncleanSetsLeaderRecoveringState(boolean isLeaderRecoverySupported) { FOO_ID, 0, brokerId -> brokerId == leaderId, - () -> true, isLeaderRecoverySupported - ); + ).setElection(Election.UNCLEAN); + // The partition should stay as recovering PartitionChangeRecord changeRecord = (PartitionChangeRecord) onlineBuilder 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 439b200b350a..963c99eeff53 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -24,15 +24,19 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.OptionalLong; +import java.util.Set; import java.util.Spliterator; import java.util.Spliterators; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; -import java.util.stream.StreamSupport; import java.util.stream.IntStream; +import java.util.stream.StreamSupport; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.BrokerIdNotRegisteredException; @@ -70,7 +74,6 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.common.utils.BufferSupplier; -import org.apache.kafka.controller.BrokersToIsrs.TopicIdPartition; import org.apache.kafka.controller.QuorumController.ConfigResourceExistenceChecker; import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistrationReply; @@ -193,14 +196,15 @@ public void testFenceMultipleBrokers() throws Throwable { List allBrokers = Arrays.asList(1, 2, 3, 4, 5); List brokersToKeepUnfenced = Arrays.asList(1); List brokersToFence = Arrays.asList(2, 3, 4, 5); - short replicationFactor = 5; + short replicationFactor = (short) allBrokers.size(); + short numberOfPartitions = (short) allBrokers.size(); long sessionTimeoutMillis = 1000; try ( LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(1, Optional.empty()); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv(logEnv, b -> { b.setConfigSchema(SCHEMA); - }, Optional.of(sessionTimeoutMillis)); + }, OptionalLong.of(sessionTimeoutMillis), OptionalLong.empty()); ) { ListenerCollection listeners = new ListenerCollection(); listeners.add(new Listener().setName("PLAINTEXT").setHost("localhost").setPort(9092)); @@ -227,7 +231,7 @@ public void testFenceMultipleBrokers() throws Throwable { sendBrokerheartbeat(active, allBrokers, brokerEpochs); CreateTopicsRequestData createTopicsRequestData = new CreateTopicsRequestData().setTopics( new CreatableTopicCollection(Collections.singleton( - new CreatableTopic().setName("foo").setNumPartitions(1). + new CreatableTopic().setName("foo").setNumPartitions(numberOfPartitions). setReplicationFactor(replicationFactor)).iterator())); CreateTopicsResponseData createTopicsResponseData = active.createTopics(createTopicsRequestData).get(); assertEquals(Errors.NONE, Errors.forCode(createTopicsResponseData.topics().find("foo").errorCode())); @@ -269,6 +273,139 @@ public void testFenceMultipleBrokers() throws Throwable { int fooLeader = active.replicationControl().getPartition(topicIdFoo, 0).leader; assertEquals(expectedIsr[0], fooLeader); + + // Check that there are imbalaned partitions + assertTrue(active.replicationControl().arePartitionLeadersImbalanced()); + } + } + + @Test + public void testBalancePartitionLeaders() throws Throwable { + List allBrokers = Arrays.asList(1, 2, 3); + List brokersToKeepUnfenced = Arrays.asList(1, 2); + List brokersToFence = Arrays.asList(3); + short replicationFactor = (short) allBrokers.size(); + short numberOfPartitions = (short) allBrokers.size(); + long sessionTimeoutMillis = 1000; + long leaderImbalanceCheckIntervalNs = 1_000_000_000; + + try ( + LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(1, Optional.empty()); + QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv(logEnv, b -> { + b.setConfigSchema(SCHEMA); + }, OptionalLong.of(sessionTimeoutMillis), OptionalLong.of(leaderImbalanceCheckIntervalNs)); + ) { + ListenerCollection listeners = new ListenerCollection(); + listeners.add(new Listener().setName("PLAINTEXT").setHost("localhost").setPort(9092)); + QuorumController active = controlEnv.activeController(); + Map brokerEpochs = new HashMap<>(); + + for (Integer brokerId : allBrokers) { + CompletableFuture reply = active.registerBroker( + new BrokerRegistrationRequestData(). + setBrokerId(brokerId). + setClusterId(active.clusterId()). + setIncarnationId(Uuid.randomUuid()). + setListeners(listeners)); + brokerEpochs.put(brokerId, reply.get().epoch()); + } + + // Brokers are only registered and should still be fenced + allBrokers.forEach(brokerId -> { + assertFalse(active.replicationControl().isBrokerUnfenced(brokerId), + "Broker " + brokerId + " should have been fenced"); + }); + + // Unfence all brokers and create a topic foo + sendBrokerheartbeat(active, allBrokers, brokerEpochs); + CreateTopicsRequestData createTopicsRequestData = new CreateTopicsRequestData().setTopics( + new CreatableTopicCollection(Collections.singleton( + new CreatableTopic().setName("foo").setNumPartitions(numberOfPartitions). + setReplicationFactor(replicationFactor)).iterator())); + CreateTopicsResponseData createTopicsResponseData = active.createTopics(createTopicsRequestData).get(); + assertEquals(Errors.NONE, Errors.forCode(createTopicsResponseData.topics().find("foo").errorCode())); + Uuid topicIdFoo = createTopicsResponseData.topics().find("foo").topicId(); + + // Fence some of the brokers + TestUtils.waitForCondition( + () -> { + sendBrokerheartbeat(active, brokersToKeepUnfenced, brokerEpochs); + for (Integer brokerId : brokersToFence) { + if (active.replicationControl().isBrokerUnfenced(brokerId)) { + return false; + } + } + return true; + }, + sessionTimeoutMillis * 3, + "Fencing of brokers did not process within expected time" + ); + + // Send another heartbeat to the brokers we want to keep alive + sendBrokerheartbeat(active, brokersToKeepUnfenced, brokerEpochs); + + // At this point only the brokers we want fenced should be fenced. + brokersToKeepUnfenced.forEach(brokerId -> { + assertTrue(active.replicationControl().isBrokerUnfenced(brokerId), + "Broker " + brokerId + " should have been unfenced"); + }); + brokersToFence.forEach(brokerId -> { + assertFalse(active.replicationControl().isBrokerUnfenced(brokerId), + "Broker " + brokerId + " should have been fenced"); + }); + + // Check that there are imbalaned partitions + assertTrue(active.replicationControl().arePartitionLeadersImbalanced()); + + // Re-register all fenced brokers + for (Integer brokerId : brokersToFence) { + CompletableFuture reply = active.registerBroker( + new BrokerRegistrationRequestData(). + setBrokerId(brokerId). + setClusterId(active.clusterId()). + setIncarnationId(Uuid.randomUuid()). + setListeners(listeners)); + brokerEpochs.put(brokerId, reply.get().epoch()); + } + + // Unfence all brokers + sendBrokerheartbeat(active, allBrokers, brokerEpochs); + + // Let the unfenced broker, 3, join the ISR partition 2 + Set imbalancedPartitions = active.replicationControl().imbalancedPartitions(); + assertEquals(1, imbalancedPartitions.size()); + int imbalancedPartitionId = imbalancedPartitions.iterator().next().partitionId(); + PartitionRegistration partitionRegistration = active.replicationControl().getPartition(topicIdFoo, imbalancedPartitionId); + AlterPartitionRequestData.PartitionData partitionData = new AlterPartitionRequestData.PartitionData() + .setPartitionIndex(imbalancedPartitionId) + .setLeaderEpoch(partitionRegistration.leaderEpoch) + .setPartitionEpoch(partitionRegistration.partitionEpoch) + .setNewIsr(Arrays.asList(1, 2, 3)); + + AlterPartitionRequestData.TopicData topicData = new AlterPartitionRequestData.TopicData() + .setName("foo"); + topicData.partitions().add(partitionData); + + AlterPartitionRequestData alterPartitionRequest = new AlterPartitionRequestData() + .setBrokerId(partitionRegistration.leader) + .setBrokerEpoch(brokerEpochs.get(partitionRegistration.leader)); + alterPartitionRequest.topics().add(topicData); + + active.alterPartition(alterPartitionRequest).get(); + + // Check that partitions are balanced + AtomicLong lastHeartbeat = new AtomicLong(active.time().milliseconds()); + TestUtils.waitForCondition( + () -> { + if (active.time().milliseconds() > lastHeartbeat.get() + (sessionTimeoutMillis / 2)) { + lastHeartbeat.set(active.time().milliseconds()); + sendBrokerheartbeat(active, allBrokers, brokerEpochs); + } + return !active.replicationControl().arePartitionLeadersImbalanced(); + }, + TimeUnit.MILLISECONDS.convert(leaderImbalanceCheckIntervalNs * 10, TimeUnit.NANOSECONDS), + "Leaders where not balanced after unfencing all of the brokers" + ); } } 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 f905621f4e69..43346fe1fbd6 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java @@ -19,7 +19,7 @@ import static java.util.concurrent.TimeUnit.NANOSECONDS; -import java.util.Optional; +import java.util.OptionalLong; import java.util.concurrent.TimeUnit; import org.apache.kafka.controller.QuorumController.Builder; import org.apache.kafka.metalog.LocalLogManagerTestEnv; @@ -45,13 +45,14 @@ public QuorumControllerTestEnv( LocalLogManagerTestEnv logEnv, Consumer builderConsumer ) throws Exception { - this(logEnv, builderConsumer, Optional.empty()); + this(logEnv, builderConsumer, OptionalLong.empty(), OptionalLong.empty()); } public QuorumControllerTestEnv( LocalLogManagerTestEnv logEnv, Consumer builderConsumer, - Optional sessionTimeoutMillis + OptionalLong sessionTimeoutMillis, + OptionalLong leaderImbalanceCheckIntervalNs ) throws Exception { this.logEnv = logEnv; int numControllers = logEnv.logManagers().size(); @@ -60,10 +61,10 @@ public QuorumControllerTestEnv( for (int i = 0; i < numControllers; i++) { QuorumController.Builder builder = new QuorumController.Builder(i, logEnv.clusterId()); builder.setRaftClient(logEnv.logManagers().get(i)); - if (sessionTimeoutMillis.isPresent()) { - builder.setSessionTimeoutNs(NANOSECONDS.convert( - sessionTimeoutMillis.get(), TimeUnit.MILLISECONDS)); - } + sessionTimeoutMillis.ifPresent(timeout -> { + builder.setSessionTimeoutNs(NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS)); + }); + builder.setLeaderImbalanceCheckIntervalNs(leaderImbalanceCheckIntervalNs); builderConsumer.accept(builder); this.controllers.add(builder.build()); } diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index 1bb171c9af2c..1712d2044466 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -112,7 +112,6 @@ import static org.apache.kafka.common.protocol.Errors.PREFERRED_LEADER_NOT_AVAILABLE; import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_ID; import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_OR_PARTITION; -import static org.apache.kafka.controller.BrokersToIsrs.TopicIdPartition; import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -167,6 +166,7 @@ void replay(List records) throws Exception { new LogContext(), (short) 3, 1, + Integer.MAX_VALUE, true, configurationControl, clusterControl, @@ -691,7 +691,7 @@ public void testShrinkAndExpandIsr() throws Exception { TopicPartition topicPartition = new TopicPartition("foo", 0); assertEquals(OptionalInt.of(0), ctx.currentLeader(topicIdPartition)); long brokerEpoch = ctx.currentBrokerEpoch(0); - PartitionData shrinkIsrRequest = newAlterIsrPartition( + PartitionData shrinkIsrRequest = newAlterPartition( replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERED); ControllerResult shrinkIsrResult = sendAlterIsr( replicationControl, 0, brokerEpoch, "foo", shrinkIsrRequest); @@ -699,7 +699,7 @@ public void testShrinkAndExpandIsr() throws Exception { shrinkIsrResult, topicPartition, NONE); assertConsistentAlterPartitionResponse(replicationControl, topicIdPartition, shrinkIsrResponse); - PartitionData expandIsrRequest = newAlterIsrPartition( + PartitionData expandIsrRequest = newAlterPartition( replicationControl, topicIdPartition, asList(0, 1, 2), LeaderRecoveryState.RECOVERED); ControllerResult expandIsrResult = sendAlterIsr( replicationControl, 0, brokerEpoch, "foo", expandIsrRequest); @@ -725,7 +725,7 @@ public void testInvalidAlterPartitionRequests() throws Exception { long brokerEpoch = ctx.currentBrokerEpoch(0); // Invalid leader - PartitionData invalidLeaderRequest = newAlterIsrPartition( + PartitionData invalidLeaderRequest = newAlterPartition( replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERED); ControllerResult invalidLeaderResult = sendAlterIsr( replicationControl, notLeaderId, ctx.currentBrokerEpoch(notLeaderId), @@ -733,13 +733,13 @@ public void testInvalidAlterPartitionRequests() throws Exception { assertAlterPartitionResponse(invalidLeaderResult, topicPartition, Errors.INVALID_REQUEST); // Stale broker epoch - PartitionData invalidBrokerEpochRequest = newAlterIsrPartition( + PartitionData invalidBrokerEpochRequest = newAlterPartition( replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERED); assertThrows(StaleBrokerEpochException.class, () -> sendAlterIsr( replicationControl, leaderId, brokerEpoch - 1, "foo", invalidBrokerEpochRequest)); // Invalid leader epoch - PartitionData invalidLeaderEpochRequest = newAlterIsrPartition( + PartitionData invalidLeaderEpochRequest = newAlterPartition( replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERED); invalidLeaderEpochRequest.setLeaderEpoch(500); ControllerResult invalidLeaderEpochResult = sendAlterIsr( @@ -748,7 +748,7 @@ public void testInvalidAlterPartitionRequests() throws Exception { assertAlterPartitionResponse(invalidLeaderEpochResult, topicPartition, FENCED_LEADER_EPOCH); // Invalid ISR (3 is not a valid replica) - PartitionData invalidIsrRequest1 = newAlterIsrPartition( + PartitionData invalidIsrRequest1 = newAlterPartition( replicationControl, topicIdPartition, asList(0, 1, 3), LeaderRecoveryState.RECOVERED); ControllerResult invalidIsrResult1 = sendAlterIsr( replicationControl, leaderId, ctx.currentBrokerEpoch(leaderId), @@ -756,7 +756,7 @@ public void testInvalidAlterPartitionRequests() throws Exception { assertAlterPartitionResponse(invalidIsrResult1, topicPartition, Errors.INVALID_REQUEST); // Invalid ISR (does not include leader 0) - PartitionData invalidIsrRequest2 = newAlterIsrPartition( + PartitionData invalidIsrRequest2 = newAlterPartition( replicationControl, topicIdPartition, asList(1, 2), LeaderRecoveryState.RECOVERED); ControllerResult invalidIsrResult2 = sendAlterIsr( replicationControl, leaderId, ctx.currentBrokerEpoch(leaderId), @@ -764,7 +764,7 @@ public void testInvalidAlterPartitionRequests() throws Exception { assertAlterPartitionResponse(invalidIsrResult2, topicPartition, Errors.INVALID_REQUEST); // Invalid ISR length and recovery state - PartitionData invalidIsrRecoveryRequest = newAlterIsrPartition( + PartitionData invalidIsrRecoveryRequest = newAlterPartition( replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERING); ControllerResult invalidIsrRecoveryResult = sendAlterIsr( replicationControl, leaderId, ctx.currentBrokerEpoch(leaderId), @@ -772,7 +772,7 @@ public void testInvalidAlterPartitionRequests() throws Exception { assertAlterPartitionResponse(invalidIsrRecoveryResult, topicPartition, Errors.INVALID_REQUEST); // Invalid recovery state transtion from RECOVERED to RECOVERING - PartitionData invalidRecoveryRequest = newAlterIsrPartition( + PartitionData invalidRecoveryRequest = newAlterPartition( replicationControl, topicIdPartition, asList(0), LeaderRecoveryState.RECOVERING); ControllerResult invalidRecoveryResult = sendAlterIsr( replicationControl, leaderId, ctx.currentBrokerEpoch(leaderId), @@ -780,7 +780,7 @@ public void testInvalidAlterPartitionRequests() throws Exception { assertAlterPartitionResponse(invalidRecoveryResult, topicPartition, Errors.INVALID_REQUEST); } - private PartitionData newAlterIsrPartition( + private PartitionData newAlterPartition( ReplicationControlManager replicationControl, TopicIdPartition topicIdPartition, List newIsr, @@ -1581,6 +1581,83 @@ public void testElectPreferredLeaders() throws Exception { setLeader(1), (short) 0)), election2Result.records()); } + @Test + public void testBalancePartitionLeaders() throws Exception { + ReplicationControlTestContext ctx = new ReplicationControlTestContext(); + ReplicationControlManager replication = ctx.replicationControl; + ctx.registerBrokers(0, 1, 2, 3, 4); + ctx.unfenceBrokers(2, 3, 4); + Uuid fooId = ctx.createTestTopic("foo", new int[][]{ + new int[]{1, 2, 3}, new int[]{2, 3, 4}, new int[]{0, 2, 1}}).topicId(); + + assertTrue(replication.arePartitionLeadersImbalanced()); + assertEquals(2, ctx.metrics.preferredReplicaImbalanceCount()); + + ctx.unfenceBrokers(1); + + ControllerResult alterPartitionResult = replication.alterPartition( + new AlterPartitionRequestData().setBrokerId(2).setBrokerEpoch(102). + setTopics(asList(new AlterPartitionRequestData.TopicData().setName("foo"). + setPartitions(asList(new AlterPartitionRequestData.PartitionData(). + setPartitionIndex(0).setPartitionEpoch(0). + setLeaderEpoch(0).setNewIsr(asList(1, 2, 3))))))); + assertEquals(new AlterPartitionResponseData().setTopics(asList( + new AlterPartitionResponseData.TopicData().setName("foo").setPartitions(asList( + new AlterPartitionResponseData.PartitionData(). + setPartitionIndex(0). + setLeaderId(2). + setLeaderEpoch(0). + setIsr(asList(1, 2, 3)). + setPartitionEpoch(1). + setErrorCode(NONE.code()))))), + alterPartitionResult.response()); + ctx.replay(alterPartitionResult.records()); + + ControllerResult balanceResult = replication.maybeBalancePartitionLeaders(); + ctx.replay(balanceResult.records()); + + PartitionChangeRecord expectedChangeRecord = new PartitionChangeRecord() + .setPartitionId(0) + .setTopicId(fooId) + .setLeader(1); + assertEquals(asList(new ApiMessageAndVersion(expectedChangeRecord, (short) 0)), balanceResult.records()); + assertTrue(replication.arePartitionLeadersImbalanced()); + assertEquals(1, ctx.metrics.preferredReplicaImbalanceCount()); + assertFalse(balanceResult.response()); + + ctx.unfenceBrokers(0); + + alterPartitionResult = replication.alterPartition( + new AlterPartitionRequestData().setBrokerId(2).setBrokerEpoch(102). + setTopics(asList(new AlterPartitionRequestData.TopicData().setName("foo"). + setPartitions(asList(new AlterPartitionRequestData.PartitionData(). + setPartitionIndex(2).setPartitionEpoch(0). + setLeaderEpoch(0).setNewIsr(asList(0, 2, 1))))))); + assertEquals(new AlterPartitionResponseData().setTopics(asList( + new AlterPartitionResponseData.TopicData().setName("foo").setPartitions(asList( + new AlterPartitionResponseData.PartitionData(). + setPartitionIndex(2). + setLeaderId(2). + setLeaderEpoch(0). + setIsr(asList(0, 2, 1)). + setPartitionEpoch(1). + setErrorCode(NONE.code()))))), + alterPartitionResult.response()); + ctx.replay(alterPartitionResult.records()); + + balanceResult = replication.maybeBalancePartitionLeaders(); + ctx.replay(balanceResult.records()); + + expectedChangeRecord = new PartitionChangeRecord() + .setPartitionId(2) + .setTopicId(fooId) + .setLeader(0); + assertEquals(asList(new ApiMessageAndVersion(expectedChangeRecord, (short) 0)), balanceResult.records()); + assertFalse(replication.arePartitionLeadersImbalanced()); + assertEquals(0, ctx.metrics.preferredReplicaImbalanceCount()); + assertFalse(balanceResult.response()); + } + private void assertElectLeadersResponse( ElectLeadersResponseData expected, ElectLeadersResponseData actual From 6145974fef05cc212ad78a87505f930c47ba26e7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E5=B0=8F=E6=BC=AA?= <644120242@qq.com> Date: Sun, 20 Mar 2022 12:09:28 +0800 Subject: [PATCH 027/447] KAFKA-13728: fix PushHttpMetricsReporter no longer pushes metrics when network failure is recovered. (#11879) The class PushHttpMetricsReporter no longer pushes metrics when network failure is recovered. I debugged the code and found the problem here: when we submit a task to the ScheduledThreadPoolExecutor that needs to be executed periodically, if the task throws an exception and is not swallowed, the task will no longer be scheduled to execute. So when an IO exception occasionally occurs on the network, we should swallow it rather than throw it in task HttpReporter. Reviewers: Guozhang Wang --- .../java/org/apache/kafka/tools/PushHttpMetricsReporter.java | 1 - 1 file changed, 1 deletion(-) diff --git a/tools/src/main/java/org/apache/kafka/tools/PushHttpMetricsReporter.java b/tools/src/main/java/org/apache/kafka/tools/PushHttpMetricsReporter.java index b33b75c71061..e8d590eeb0ee 100644 --- a/tools/src/main/java/org/apache/kafka/tools/PushHttpMetricsReporter.java +++ b/tools/src/main/java/org/apache/kafka/tools/PushHttpMetricsReporter.java @@ -213,7 +213,6 @@ public void run() { } } catch (Throwable t) { log.error("Error reporting metrics", t); - throw new KafkaException("Failed to report current metrics", t); } finally { if (connection != null) { connection.disconnect(); From 3a8f6b17a6a771d99322c78e8455bbbe4bdc6930 Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Mon, 21 Mar 2022 16:51:21 +0800 Subject: [PATCH 028/447] KAFKA-7540: commit offset sync before close (#11898) Reviewers: Guozhang Wang --- .../internals/ConsumerCoordinator.java | 20 ++++++++++++++++++- .../kafka/api/ConsumerBounceTest.scala | 9 ++------- 2 files changed, 21 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 84cf822a43a3..10939b2a0e48 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -970,7 +970,7 @@ public void close(final Timer timer) { // we do not need to re-enable wakeups since we are closing already client.disableWakeups(); try { - maybeAutoCommitOffsetsAsync(); + maybeAutoCommitOffsetsSync(timer); while (pendingAsyncCommits.get() > 0 && timer.notExpired()) { ensureCoordinatorReady(timer); client.poll(timer); @@ -1110,6 +1110,24 @@ public boolean commitOffsetsSync(Map offsets, return false; } + private void maybeAutoCommitOffsetsSync(Timer timer) { + if (autoCommitEnabled) { + Map allConsumedOffsets = subscriptions.allConsumed(); + try { + log.debug("Sending synchronous auto-commit of offsets {}", allConsumedOffsets); + if (!commitOffsetsSync(allConsumedOffsets, timer)) + log.debug("Auto-commit of offsets {} timed out before completion", allConsumedOffsets); + } catch (WakeupException | InterruptException e) { + log.debug("Auto-commit of offsets {} was interrupted before completion", allConsumedOffsets); + // rethrow wakeups since they are triggered by the user + throw e; + } catch (Exception e) { + // consistent with async auto-commit failures, we do not propagate the exception + log.warn("Synchronous auto-commit of offsets {} failed: {}", allConsumedOffsets, e.getMessage()); + } + } + } + public void maybeAutoCommitOffsetsAsync(long now) { if (autoCommitEnabled) { nextAutoCommitTimer.update(now); diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index b3afc5075826..510a98a90089 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -211,7 +211,6 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { } @Test - @Disabled // TODO: To be re-enabled once we can make it less flaky (KAFKA-7540) def testClose(): Unit = { val numRecords = 10 val producer = createProducer() @@ -247,12 +246,8 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { killBroker(findCoordinator(dynamicGroup)) killBroker(findCoordinator(manualGroup)) - val future1 = submitCloseAndValidate(consumer1, Long.MaxValue, None, gracefulCloseTimeMs) - - val future2 = submitCloseAndValidate(consumer2, Long.MaxValue, None, gracefulCloseTimeMs) - - future1.get - future2.get + submitCloseAndValidate(consumer1, Long.MaxValue, None, gracefulCloseTimeMs).get + submitCloseAndValidate(consumer2, Long.MaxValue, None, gracefulCloseTimeMs).get restartDeadBrokers() checkClosedState(dynamicGroup, 0) From e5eb180a6fa6e1ff3ff2c3d1d67b702ca8c9957b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?M=C3=A1rton=20Sigmond?= <5667024+msigmond@users.noreply.github.com> Date: Mon, 21 Mar 2022 05:03:04 -0400 Subject: [PATCH 029/447] MINOR: Pass materialized to the inner KTable instance (#11888) Reviewers: Luke Chen --- .../kafka/streams/scala/kstream/KTable.scala | 2 +- .../streams/scala/kstream/KTableTest.scala | 38 +++++++++++++++++++ 2 files changed, 39 insertions(+), 1 deletion(-) diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KTable.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KTable.scala index 3a405b68a771..9d8fe81f71a8 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KTable.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KTable.scala @@ -249,7 +249,7 @@ class KTable[K, V](val inner: KTableJ[K, V]) { * @see `org.apache.kafka.streams.kstream.KTable#mapValues` */ def mapValues[VR](mapper: (K, V) => VR, materialized: Materialized[K, VR, ByteArrayKeyValueStore]): KTable[K, VR] = - new KTable(inner.mapValues[VR](mapper.asValueMapperWithKey)) + new KTable(inner.mapValues[VR](mapper.asValueMapperWithKey, materialized)) /** * Create a new [[KTable]] by transforming the value of each record in this [[KTable]] into a new value diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KTableTest.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KTableTest.scala index 09a3a7d9087b..9e872601ef13 100644 --- a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KTableTest.scala +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KTableTest.scala @@ -496,4 +496,42 @@ class KTableTest extends TestDriver { assertTrue(joinNodeLeft.name().contains("my-name")) assertTrue(joinNodeRight.name().contains("my-name")) } + + @Test + def testMapValuesWithValueMapperWithMaterialized(): Unit = { + val builder = new StreamsBuilder() + val sourceTopic = "source" + val stateStore = "store" + val materialized = Materialized.as[String, Long, ByteArrayKeyValueStore](stateStore) + + val table = builder.stream[String, String](sourceTopic).toTable + table.mapValues(value => value.length.toLong, materialized) + + val testDriver = createTestDriver(builder) + val testInput = testDriver.createInput[String, String](sourceTopic) + + testInput.pipeInput("1", "topic1value1") + assertEquals(12, testDriver.getKeyValueStore[String, Long](stateStore).get("1")) + + testDriver.close() + } + + @Test + def testMapValuesWithValueMapperWithKeyAndWithMaterialized(): Unit = { + val builder = new StreamsBuilder() + val sourceTopic = "source" + val stateStore = "store" + val materialized = Materialized.as[String, Long, ByteArrayKeyValueStore](stateStore) + + val table = builder.stream[String, String](sourceTopic).toTable + table.mapValues((key, value) => key.length + value.length.toLong, materialized) + + val testDriver = createTestDriver(builder) + val testInput = testDriver.createInput[String, String](sourceTopic) + + testInput.pipeInput("1", "topic1value1") + assertEquals(13, testDriver.getKeyValueStore[String, Long](stateStore).get("1")) + + testDriver.close() + } } From 72558da97654ee88c3fb046eb8aac84ddc375827 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Mon, 21 Mar 2022 11:23:31 +0100 Subject: [PATCH 030/447] MINOR: Small cleanups in the AclAuthorizer (#11921) Reviewers: Manikumar Reddy --- .../security/authorizer/AclAuthorizer.scala | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala b/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala index 88648fd3178c..e2ac9737fc3f 100644 --- a/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala @@ -192,11 +192,12 @@ class AclAuthorizer extends Authorizer with Logging { override def start(serverInfo: AuthorizerServerInfo): util.Map[Endpoint, _ <: CompletionStage[Void]] = { serverInfo.endpoints.asScala.map { endpoint => - endpoint -> CompletableFuture.completedFuture[Void](null) }.toMap.asJava + endpoint -> CompletableFuture.completedFuture[Void](null) + }.toMap.asJava } override def authorize(requestContext: AuthorizableRequestContext, actions: util.List[Action]): util.List[AuthorizationResult] = { - actions.asScala.map { action => authorizeAction(requestContext, action) }.asJava + actions.asScala.map(action => authorizeAction(requestContext, action)).asJava } override def createAcls(requestContext: AuthorizableRequestContext, @@ -440,7 +441,6 @@ class AclAuthorizer extends Authorizer with Logging { false } - private def authorizeAction(requestContext: AuthorizableRequestContext, action: Action): AuthorizationResult = { val resource = action.resourcePattern if (resource.patternType != PatternType.LITERAL) { @@ -547,7 +547,7 @@ class AclAuthorizer extends Authorizer with Logging { private def loadCache(): Unit = { lock synchronized { - ZkAclStore.stores.foreach(store => { + ZkAclStore.stores.foreach { store => val resourceTypes = zkClient.getResourceTypes(store.patternType) for (rType <- resourceTypes) { val resourceType = Try(SecurityUtils.resourceType(rType)) @@ -562,7 +562,7 @@ class AclAuthorizer extends Authorizer with Logging { case Failure(_) => warn(s"Ignoring unknown ResourceType: $rType") } } - }) + } } } @@ -691,14 +691,14 @@ class AclAuthorizer extends Authorizer with Logging { val acesToAdd = newAces.diff(currentAces) val acesToRemove = currentAces.diff(newAces) - acesToAdd.foreach(ace => { + acesToAdd.foreach { ace => val resourceTypeKey = ResourceTypeKey(ace, resource.resourceType(), resource.patternType()) resourceCache.get(resourceTypeKey) match { case Some(resources) => resourceCache += (resourceTypeKey -> (resources + resource.name())) case None => resourceCache += (resourceTypeKey -> immutable.HashSet(resource.name())) } - }) - acesToRemove.foreach(ace => { + } + acesToRemove.foreach { ace => val resourceTypeKey = ResourceTypeKey(ace, resource.resourceType(), resource.patternType()) resourceCache.get(resourceTypeKey) match { case Some(resources) => @@ -710,7 +710,7 @@ class AclAuthorizer extends Authorizer with Logging { } case None => } - }) + } if (versionedAcls.acls.nonEmpty) { aclCache = aclCache.updated(resource, versionedAcls) From 4c8685e701e89e1c65acea720cb3facf63389791 Mon Sep 17 00:00:00 2001 From: Bruno Cadonna Date: Mon, 21 Mar 2022 21:37:05 +0100 Subject: [PATCH 031/447] MINOR: Bump trunk to 3.3.0-SNAPSHOT (#11925) Version bumps on trunk following the creation of the 3.2 release branch. Reviewer: David Jacot --- docs/js/templateData.js | 6 +++--- gradle.properties | 2 +- kafka-merge-pr.py | 2 +- streams/quickstart/java/pom.xml | 2 +- .../java/src/main/resources/archetype-resources/pom.xml | 2 +- streams/quickstart/pom.xml | 2 +- tests/kafkatest/__init__.py | 2 +- tests/kafkatest/version.py | 8 ++++++-- 8 files changed, 15 insertions(+), 11 deletions(-) diff --git a/docs/js/templateData.js b/docs/js/templateData.js index b7b23e8487f8..6b773468f962 100644 --- a/docs/js/templateData.js +++ b/docs/js/templateData.js @@ -17,8 +17,8 @@ limitations under the License. // Define variables for doc templates var context={ - "version": "32", - "dotVersion": "3.2", - "fullDotVersion": "3.2.0", + "version": "33", + "dotVersion": "3.3", + "fullDotVersion": "3.3.0", "scalaVersion": "2.13" }; diff --git a/gradle.properties b/gradle.properties index 753c3e6af33c..4b672a3a964d 100644 --- a/gradle.properties +++ b/gradle.properties @@ -20,7 +20,7 @@ group=org.apache.kafka # - tests/kafkatest/__init__.py # - tests/kafkatest/version.py (variable DEV_VERSION) # - kafka-merge-pr.py -version=3.2.0-SNAPSHOT +version=3.3.0-SNAPSHOT scalaVersion=2.13.6 task=build org.gradle.jvmargs=-Xmx2g -Xss4m -XX:+UseParallelGC diff --git a/kafka-merge-pr.py b/kafka-merge-pr.py index 89b756e04aba..75a2fc4ba975 100755 --- a/kafka-merge-pr.py +++ b/kafka-merge-pr.py @@ -70,7 +70,7 @@ DEV_BRANCH_NAME = "trunk" -DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "3.2.0") +DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "3.3.0") ORIGINAL_HEAD = "" diff --git a/streams/quickstart/java/pom.xml b/streams/quickstart/java/pom.xml index ba32c037259c..1c4c6913ba3b 100644 --- a/streams/quickstart/java/pom.xml +++ b/streams/quickstart/java/pom.xml @@ -26,7 +26,7 @@ org.apache.kafka streams-quickstart - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT .. diff --git a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml index b9fa16c5bb55..aeb3a71a829d 100644 --- a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml +++ b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml @@ -29,7 +29,7 @@ UTF-8 - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT 1.7.7 1.2.17 diff --git a/streams/quickstart/pom.xml b/streams/quickstart/pom.xml index 2804d0ad7144..56812c33e7ac 100644 --- a/streams/quickstart/pom.xml +++ b/streams/quickstart/pom.xml @@ -22,7 +22,7 @@ org.apache.kafka streams-quickstart pom - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT Kafka Streams :: Quickstart diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py index 69101d87630b..7520e0d15771 100644 --- a/tests/kafkatest/__init__.py +++ b/tests/kafkatest/__init__.py @@ -22,4 +22,4 @@ # Instead, in development branches, the version should have a suffix of the form ".devN" # # For example, when Kafka is at version 1.0.0-SNAPSHOT, this should be something like "1.0.0.dev0" -__version__ = '3.2.0.dev0' +__version__ = '3.3.0.dev0' diff --git a/tests/kafkatest/version.py b/tests/kafkatest/version.py index 2d7fe0377e01..76ea5cc4989a 100644 --- a/tests/kafkatest/version.py +++ b/tests/kafkatest/version.py @@ -116,7 +116,7 @@ def get_version(node=None): return DEV_BRANCH DEV_BRANCH = KafkaVersion("dev") -DEV_VERSION = KafkaVersion("3.2.0-SNAPSHOT") +DEV_VERSION = KafkaVersion("3.3.0-SNAPSHOT") # 0.8.2.x versions V_0_8_2_1 = KafkaVersion("0.8.2.1") @@ -223,4 +223,8 @@ def get_version(node=None): # 3.2.x versions V_3_2_0 = KafkaVersion("3.2.0") -LATEST_3_2 = V_3_2_0 \ No newline at end of file +LATEST_3_2 = V_3_2_0 + +# 3.3.x versions +V_3_3_0 = KafkaVersion("3.3.0") +LATEST_3_3 = V_3_3_0 From d449f850e1934680a96780c01ac95bfef52eeac7 Mon Sep 17 00:00:00 2001 From: dengziming Date: Tue, 22 Mar 2022 05:33:51 +0800 Subject: [PATCH 032/447] MINOR: show LogRecoveryState in MetadataShell and fix log message Show the LeaderRecoveryState in MetadataShell. Fix a case where we were comparing a Byte type with an enum type. Reviewers: Colin P. McCabe --- core/src/main/scala/kafka/cluster/Partition.scala | 2 +- .../org/apache/kafka/metadata/LeaderRecoveryState.java | 2 +- .../java/org/apache/kafka/shell/MetadataNodeManager.java | 5 +++++ .../org/apache/kafka/shell/MetadataNodeManagerTest.java | 7 +++++++ 4 files changed, 14 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 429e73b4d834..e692db83ea80 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -549,7 +549,7 @@ class Partition(val topicPartition: TopicPartition, val addingReplicas = partitionState.addingReplicas.asScala.map(_.toInt) val removingReplicas = partitionState.removingReplicas.asScala.map(_.toInt) - if (partitionState.leaderRecoveryState == LeaderRecoveryState.RECOVERING) { + if (partitionState.leaderRecoveryState == LeaderRecoveryState.RECOVERING.value()) { stateChangeLogger.info( s"The topic partition $topicPartition was marked as RECOVERING. Leader log recovery is not implemented. " + "Marking the topic partition as RECOVERED." diff --git a/metadata/src/main/java/org/apache/kafka/metadata/LeaderRecoveryState.java b/metadata/src/main/java/org/apache/kafka/metadata/LeaderRecoveryState.java index 4724e990abbb..08086751b709 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/LeaderRecoveryState.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/LeaderRecoveryState.java @@ -36,7 +36,7 @@ public enum LeaderRecoveryState { * A special value used to represent that the LeaderRecoveryState field of a * PartitionChangeRecord didn't change. */ - private static final byte NO_CHANGE = (byte) -1; + public static final byte NO_CHANGE = (byte) -1; public static LeaderRecoveryState of(byte value) { return optionalOf(value) diff --git a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java index f7b867a6b01c..9d4941f8020b 100644 --- a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java +++ b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java @@ -59,6 +59,8 @@ import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; +import static org.apache.kafka.metadata.LeaderRecoveryState.NO_CHANGE; + /** * Maintains the in-memory metadata for the metadata tool. */ @@ -280,6 +282,9 @@ private void handleCommitImpl(MetadataRecordType type, ApiMessage message) partition.setLeader(record.leader()); partition.setLeaderEpoch(partition.leaderEpoch() + 1); } + if (record.leaderRecoveryState() != NO_CHANGE) { + partition.setLeaderRecoveryState(record.leaderRecoveryState()); + } partition.setPartitionEpoch(partition.partitionEpoch() + 1); file.setContents(PartitionRecordJsonConverter.write(partition, PartitionRecord.HIGHEST_SUPPORTED_VERSION).toPrettyString()); diff --git a/shell/src/test/java/org/apache/kafka/shell/MetadataNodeManagerTest.java b/shell/src/test/java/org/apache/kafka/shell/MetadataNodeManagerTest.java index f0cfffb28178..c580e1d5c27d 100644 --- a/shell/src/test/java/org/apache/kafka/shell/MetadataNodeManagerTest.java +++ b/shell/src/test/java/org/apache/kafka/shell/MetadataNodeManagerTest.java @@ -31,6 +31,7 @@ import org.apache.kafka.common.metadata.TopicRecord; import org.apache.kafka.common.metadata.UnfenceBrokerRecord; import org.apache.kafka.common.metadata.UnregisterBrokerRecord; +import org.apache.kafka.metadata.LeaderRecoveryState; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -209,6 +210,12 @@ public void testPartitionChangeRecord() { partitionChangeRecord.duplicate().setLeader(1), newPartitionRecord.duplicate().setLeader(1).setLeaderEpoch(1) ); + + // Change leader recovery state + checkPartitionChangeRecord( + oldPartitionRecord, + partitionChangeRecord.duplicate().setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value()), + newPartitionRecord.duplicate().setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value())); } private void checkPartitionChangeRecord(PartitionRecord oldPartitionRecord, From c9c03dd7ef9ff4edf2596e905cabececc72a9e9d Mon Sep 17 00:00:00 2001 From: dengziming Date: Tue, 22 Mar 2022 05:56:25 +0800 Subject: [PATCH 033/447] MINOR: Remove scala KafkaException (#11913) Use the standard org.apache.kafka.common.KafkaException instead of kafka.common.KafkaException. Reviewers: Colin P. McCabe , Ismael Juma --- .../scala/kafka/common/KafkaException.scala | 27 ------------------- .../scala/kafka/log/LogCleanerManager.scala | 6 ++--- .../server/BrokerMetadataCheckpoint.scala | 3 ++- .../scala/kafka/server/KafkaRaftServer.scala | 6 ++--- .../kafka/server/KafkaRaftServerTest.scala | 5 ++-- 5 files changed, 11 insertions(+), 36 deletions(-) delete mode 100644 core/src/main/scala/kafka/common/KafkaException.scala diff --git a/core/src/main/scala/kafka/common/KafkaException.scala b/core/src/main/scala/kafka/common/KafkaException.scala deleted file mode 100644 index 9c34dd9bd78b..000000000000 --- a/core/src/main/scala/kafka/common/KafkaException.scala +++ /dev/null @@ -1,27 +0,0 @@ -/** - * 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 kafka.common - -/** - * Usage of this class is discouraged. Use org.apache.kafka.common.KafkaException instead. - * - * This class will be removed once kafka.security.auth classes are removed. - */ -class KafkaException(message: String, t: Throwable) extends RuntimeException(message, t) { - def this(message: String) = this(message, null) - def this(t: Throwable) = this("", t) -} diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index 8b6926b0d46d..89d4686e2852 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -21,15 +21,15 @@ import java.io.File import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock -import kafka.common.{KafkaException, LogCleaningAbortedException} +import kafka.common.LogCleaningAbortedException import kafka.metrics.KafkaMetricsGroup import kafka.server.LogDirFailureChannel import kafka.server.checkpoints.OffsetCheckpointFile import kafka.utils.CoreUtils._ import kafka.utils.{Logging, Pool} -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.utils.Time +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.KafkaStorageException +import org.apache.kafka.common.utils.Time import scala.collection.{Iterable, Seq, mutable} diff --git a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala index 0a9bfbda535f..67d27cb05293 100755 --- a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala +++ b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala @@ -21,9 +21,10 @@ import java.io._ import java.nio.file.{Files, NoSuchFileException} import java.util.Properties -import kafka.common.{InconsistentBrokerMetadataException, KafkaException} +import kafka.common.InconsistentBrokerMetadataException import kafka.server.RawMetaProperties._ import kafka.utils._ +import org.apache.kafka.common.KafkaException import org.apache.kafka.common.utils.Utils import scala.collection.mutable diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index 876925c9797f..20c87ed9e51a 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -19,15 +19,15 @@ package kafka.server import java.io.File import java.util.concurrent.CompletableFuture -import kafka.common.{InconsistentNodeIdException, KafkaException} +import kafka.common.InconsistentNodeIdException import kafka.log.{LogConfig, UnifiedLog} import kafka.metrics.{KafkaMetricsReporter, KafkaYammerMetrics} import kafka.raft.KafkaRaftManager import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole} import kafka.utils.{CoreUtils, Logging, Mx4jLoader, VerifiableProperties} -import org.apache.kafka.common.utils.{AppInfoParser, Time} -import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.config.{ConfigDef, ConfigResource} +import org.apache.kafka.common.utils.{AppInfoParser, Time} +import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} import org.apache.kafka.metadata.{KafkaConfigSchema, MetadataRecordSerde} import org.apache.kafka.raft.RaftConfig import org.apache.kafka.server.common.ApiMessageAndVersion diff --git a/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala b/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala index 82ad5427a6a4..b4dac22c6c58 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala @@ -19,9 +19,10 @@ package kafka.server import java.io.File import java.nio.file.Files import java.util.Properties -import kafka.common.{InconsistentBrokerMetadataException, InconsistentNodeIdException, KafkaException} + +import kafka.common.{InconsistentBrokerMetadataException, InconsistentNodeIdException} import kafka.log.UnifiedLog -import org.apache.kafka.common.Uuid +import org.apache.kafka.common.{KafkaException, Uuid} import org.apache.kafka.common.utils.Utils import org.apache.kafka.test.TestUtils import org.junit.jupiter.api.Assertions._ From 0924fd3f9f75c446310ed1e97b44bbc3f33c6c31 Mon Sep 17 00:00:00 2001 From: vamossagar12 Date: Tue, 22 Mar 2022 05:46:00 +0530 Subject: [PATCH 034/447] KAFKA-13152: Replace "buffered.records.per.partition" with "input.buffer.max.bytes" (#11796) Implements KIP-770 Reviewers: Guozhang Wang --- .../examples/pageview/PageViewTypedDemo.java | 2 +- .../pageview/PageViewUntypedDemo.java | 2 +- .../examples/temperature/TemperatureDemo.java | 2 +- .../examples/wordcount/WordCountDemo.java | 2 +- .../wordcount/WordCountProcessorDemo.java | 2 +- .../wordcount/WordCountTransformerDemo.java | 2 +- .../apache/kafka/streams/KafkaStreams.java | 76 +++--- .../apache/kafka/streams/StreamsConfig.java | 46 ++++ .../apache/kafka/streams/TopologyConfig.java | 43 +++- .../streams/kstream/CogroupedKStream.java | 8 +- .../kafka/streams/kstream/KGroupedStream.java | 20 +- .../kafka/streams/kstream/KGroupedTable.java | 22 +- .../SessionWindowedCogroupedKStream.java | 8 +- .../kstream/SessionWindowedKStream.java | 24 +- .../kstream/TimeWindowedCogroupedKStream.java | 8 +- .../streams/kstream/TimeWindowedKStream.java | 24 +- .../processor/internals/PartitionGroup.java | 81 ++++-- .../processor/internals/RecordQueue.java | 37 +++ .../processor/internals/StreamTask.java | 15 +- .../processor/internals/StreamThread.java | 40 ++- .../streams/processor/internals/Task.java | 1 + .../processor/internals/TaskManager.java | 19 ++ .../internals/metrics/TaskMetrics.java | 37 +++ .../streams/state/internals/NamedCache.java | 10 + .../kafka/streams/KafkaStreamsTest.java | 9 +- .../kafka/streams/StreamsConfigTest.java | 30 +++ .../AbstractJoinIntegrationTest.java | 2 +- .../AbstractResetIntegrationTest.java | 2 +- .../AdjustStreamThreadCountTest.java | 89 ++++++- .../EmitOnChangeIntegrationTest.java | 175 +++++++++++++ .../integration/EosIntegrationTest.java | 6 +- .../EosV2UpgradeIntegrationTest.java | 2 +- .../ErrorHandlingIntegrationTest.java | 161 ++++++++++++ .../FineGrainedAutoResetIntegrationTest.java | 4 +- .../GlobalKTableEOSIntegrationTest.java | 2 +- .../GlobalKTableIntegrationTest.java | 2 +- .../GlobalThreadShutDownOrderTest.java | 2 +- .../InternalTopicIntegrationTest.java | 2 +- ...StreamAggregationDedupIntegrationTest.java | 2 +- .../KStreamAggregationIntegrationTest.java | 2 +- .../KStreamRepartitionIntegrationTest.java | 2 +- ...rJoinCustomPartitionerIntegrationTest.java | 1 + ...reignKeyInnerJoinMultiIntegrationTest.java | 2 +- ...ableSourceTopicRestartIntegrationTest.java | 2 +- .../integration/MetricsIntegrationTest.java | 6 +- .../NamedTopologyIntegrationTest.java | 2 +- .../OptimizedKTableIntegrationTest.java | 2 +- .../QueryableStateIntegrationTest.java | 2 +- .../RegexSourceIntegrationTest.java | 2 +- .../integration/RestoreIntegrationTest.java | 2 +- .../RocksDBMetricsIntegrationTest.java | 2 +- .../StandbyTaskEOSIntegrationTest.java | 2 +- .../StoreUpgradeIntegrationTest.java | 2 +- ...inTopologyOptimizationIntegrationTest.java | 2 +- .../kstream/internals/KTableFilterTest.java | 4 +- .../SessionWindowedKStreamImplTest.java | 2 +- .../InternalTopologyBuilderTest.java | 18 +- .../internals/PartitionGroupTest.java | 101 ++++++++ .../internals/RepartitionOptimizingTest.java | 2 +- .../RepartitionWithMergeOptimizingTest.java | 2 +- .../processor/internals/StandbyTaskTest.java | 1 - .../processor/internals/StreamTaskTest.java | 1 + .../processor/internals/StreamThreadTest.java | 232 +++++++++++++++++- .../internals/metrics/TaskMetricsTest.java | 41 ++++ .../state/internals/NamedCacheTest.java | 12 +- .../tests/BrokerCompatibilityTest.java | 2 +- .../kafka/streams/tests/EosTestClient.java | 2 +- .../tests/StreamsNamedRepartitionTest.java | 2 +- .../streams/tests/StreamsOptimizedTest.java | 2 +- .../tests/StreamsStandByReplicaTest.java | 2 +- .../kafka/streams/TopologyTestDriver.java | 5 +- 71 files changed, 1287 insertions(+), 198 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/ErrorHandlingIntegrationTest.java diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java index a5086de8c66a..be54bafca2b4 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java @@ -179,7 +179,7 @@ public static void main(final String[] args) { props.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class); props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, JSONSerde.class); props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, JSONSerde.class); - props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java index cdb36394a98a..8fc874488abe 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java @@ -60,7 +60,7 @@ public static void main(final String[] args) throws Exception { props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-untyped"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class); - props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java index 3dc8eda25f10..6e40fa03066d 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java @@ -77,7 +77,7 @@ public static void main(final String[] args) { props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); final Duration duration24Hours = Duration.ofHours(24); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java index 4ca5d73f1d84..d290c660bbf5 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java @@ -61,7 +61,7 @@ static Properties getStreamsConfig(final String[] args) throws IOException { } props.putIfAbsent(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount"); props.putIfAbsent(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - props.putIfAbsent(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + props.putIfAbsent(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); props.putIfAbsent(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); props.putIfAbsent(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java index 014923fb64cb..6204c422bc0a 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java @@ -108,7 +108,7 @@ public static void main(final String[] args) throws IOException { props.putIfAbsent(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount-processor"); props.putIfAbsent(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - props.putIfAbsent(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + props.putIfAbsent(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); props.putIfAbsent(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); props.putIfAbsent(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountTransformerDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountTransformerDemo.java index 028d317df5b4..8e80e929aef4 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountTransformerDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountTransformerDemo.java @@ -131,7 +131,7 @@ public static void main(final String[] args) throws IOException { } props.putIfAbsent(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount-transformer"); props.putIfAbsent(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - props.putIfAbsent(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + props.putIfAbsent(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); props.putIfAbsent(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); props.putIfAbsent(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 039fdaf2ec34..369bc560f80e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -174,6 +174,7 @@ public class KafkaStreams implements AutoCloseable { protected final Admin adminClient; private final StreamsMetricsImpl streamsMetrics; private final long totalCacheSize; + private final long inputBufferMaxBytes; private final StreamStateListener streamStateListener; private final StateRestoreListener delegatingStateRestoreListener; private final Map threadState; @@ -938,9 +939,9 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, streamsUncaughtExceptionHandler = this::defaultStreamsUncaughtExceptionHandler; delegatingStateRestoreListener = new DelegatingStateRestoreListener(); - totalCacheSize = applicationConfigs.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG); + totalCacheSize = applicationConfigs.getTotalCacheSize(); + inputBufferMaxBytes = applicationConfigs.getLong(StreamsConfig.INPUT_BUFFER_MAX_BYTES_CONFIG); final int numStreamThreads = topologyMetadata.getNumStreamThreads(applicationConfigs); - final long cacheSizePerThread = getCacheSizePerThread(numStreamThreads); GlobalStreamThread.State globalThreadState = null; if (hasGlobalTopology) { @@ -950,7 +951,7 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, applicationConfigs, clientSupplier.getGlobalConsumer(applicationConfigs.getGlobalConsumerConfigs(clientId)), stateDirectory, - cacheSizePerThread, + 0L, streamsMetrics, time, globalThreadId, @@ -971,14 +972,16 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, queryableStoreProvider = new QueryableStoreProvider(globalStateStoreProvider); for (int i = 1; i <= numStreamThreads; i++) { - createAndAddStreamThread(cacheSizePerThread, i); + createAndAddStreamThread(0L, 0L, i); } + // Initially, all Stream Threads are created with 0 cache size and max buffer size and then resized here. + resizeThreadCacheAndBufferMemory(numStreamThreads); stateDirCleaner = setupStateDirCleaner(); rocksDBMetricsRecordingService = maybeCreateRocksDBMetricsRecordingService(clientId, applicationConfigs); } - private StreamThread createAndAddStreamThread(final long cacheSizePerThread, final int threadIdx) { + private StreamThread createAndAddStreamThread(final long cacheSizePerThread, final long maxBufferSizePerThread, final int threadIdx) { final StreamThread streamThread = StreamThread.create( topologyMetadata, applicationConfigs, @@ -990,7 +993,7 @@ private StreamThread createAndAddStreamThread(final long cacheSizePerThread, fin time, streamsMetadataState, cacheSizePerThread, - + maxBufferSizePerThread, stateDirectory, delegatingStateRestoreListener, threadIdx, @@ -1027,7 +1030,7 @@ private static Metrics getMetrics(final StreamsConfig config, final Time time, f * Since the number of stream threads increases, the sizes of the caches in the new stream thread * and the existing stream threads are adapted so that the sum of the cache sizes over all stream * threads does not exceed the total cache size specified in configuration - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG}. + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG}. *

* Stream threads can only be added if this Kafka Streams client is in state RUNNING or REBALANCING. * @@ -1038,14 +1041,15 @@ public Optional addStreamThread() { final StreamThread streamThread; synchronized (changeThreadCount) { final int threadIdx = getNextThreadIndex(); - final int numLiveThreads = getNumLiveStreamThreads(); - final long cacheSizePerThread = getCacheSizePerThread(numLiveThreads + 1); - log.info("Adding StreamThread-{}, there will now be {} live threads and the new cache size per thread is {}", - threadIdx, numLiveThreads + 1, cacheSizePerThread); - resizeThreadCache(cacheSizePerThread); // Creating thread should hold the lock in order to avoid duplicate thread index. // If the duplicate index happen, the metadata of thread may be duplicate too. - streamThread = createAndAddStreamThread(cacheSizePerThread, threadIdx); + // Also, we create the new thread with initial values of cache size and max buffer size as 0 + // and then resize them later + streamThread = createAndAddStreamThread(0L, 0L, threadIdx); + final int numLiveThreads = getNumLiveStreamThreads(); + resizeThreadCacheAndBufferMemory(numLiveThreads + 1); + log.info("Adding StreamThread-{}, there are now {} threads with cache size/max buffer size values as {} per thread.", + threadIdx, numLiveThreads + 1, getThreadCacheAndBufferMemoryString()); } synchronized (stateLock) { @@ -1056,9 +1060,9 @@ public Optional addStreamThread() { log.warn("Terminating the new thread because the Kafka Streams client is in state {}", state); streamThread.shutdown(); threads.remove(streamThread); - final long cacheSizePerThread = getCacheSizePerThread(getNumLiveStreamThreads()); - log.info("Resizing thread cache due to terminating added thread, new cache size per thread is {}", cacheSizePerThread); - resizeThreadCache(cacheSizePerThread); + resizeThreadCacheAndBufferMemory(getNumLiveStreamThreads()); + log.info("Resizing thread cache and max buffer size per thread since new thread can not be " + + "started, cache size/max buffer size per thread is {}", getThreadCacheAndBufferMemoryString()); return Optional.empty(); } } @@ -1076,7 +1080,7 @@ public Optional addStreamThread() { *

* Since the number of stream threads decreases, the sizes of the caches in the remaining stream * threads are adapted so that the sum of the cache sizes over all stream threads equals the total - * cache size specified in configuration {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG}. + * cache size specified in configuration {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG}. * * @return name of the removed stream thread or empty if a stream thread could not be removed because * no stream threads are alive @@ -1091,9 +1095,10 @@ public Optional removeStreamThread() { * The removed stream thread is gracefully shut down. This method does not specify which stream * thread is shut down. *

- * Since the number of stream threads decreases, the sizes of the caches in the remaining stream - * threads are adapted so that the sum of the cache sizes over all stream threads equals the total - * cache size specified in configuration {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG}. + * Since the number of stream threads decreases, the sizes of the caches and buffer bytes in the remaining stream + * threads are adapted so that the sum of the cache sizes and buffer bytes over all stream threads equals the total + * cache size specified in configuration {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG} and + * {@link StreamsConfig#INPUT_BUFFER_MAX_BYTES_CONFIG} respectively. * * @param timeout The length of time to wait for the thread to shutdown * @throws org.apache.kafka.common.errors.TimeoutException if the thread does not stop in time @@ -1133,16 +1138,14 @@ private Optional removeStreamThread(final long timeoutMs) throws Timeout } } else { log.info("{} is the last remaining thread and must remove itself, therefore we cannot wait " - + "for it to complete shutdown as this will result in deadlock.", streamThread.getName()); + + "for it to complete shutdown as this will result in deadlock.", streamThread.getName()); } - - final long cacheSizePerThread = getCacheSizePerThread(getNumLiveStreamThreads()); - log.info("Resizing thread cache due to thread removal, new cache size per thread is {}", cacheSizePerThread); - resizeThreadCache(cacheSizePerThread); + resizeThreadCacheAndBufferMemory(getNumLiveStreamThreads()); + log.info("Resizing thread cache/max buffer size due to removal of thread {}, new cache size/max buffer size per thread is {}", streamThread.getName(), getThreadCacheAndBufferMemoryString()); if (groupInstanceID.isPresent() && callingThreadIsNotCurrentStreamThread) { final MemberToRemove memberToRemove = new MemberToRemove(groupInstanceID.get()); final Collection membersToRemove = Collections.singletonList(memberToRemove); - final RemoveMembersFromConsumerGroupResult removeMembersFromConsumerGroupResult = + final RemoveMembersFromConsumerGroupResult removeMembersFromConsumerGroupResult = adminClient.removeMembersFromConsumerGroup( applicationConfigs.getString(StreamsConfig.APPLICATION_ID_CONFIG), new RemoveMembersFromConsumerGroupOptions(membersToRemove) @@ -1241,15 +1244,22 @@ private int getNextThreadIndex() { } } - private long getCacheSizePerThread(final int numStreamThreads) { - if (numStreamThreads == 0) { - return totalCacheSize; - } - return totalCacheSize / (numStreamThreads + (topologyMetadata.hasGlobalTopology() ? 1 : 0)); + private String getThreadCacheAndBufferMemoryString() { + final StreamThread streamThread = threads.get(0); + return streamThread.getCacheSize() + "/" + streamThread.getMaxBufferSize(); } - private void resizeThreadCache(final long cacheSizePerThread) { - processStreamThread(thread -> thread.resizeCache(cacheSizePerThread)); + private void resizeThreadCacheAndBufferMemory(final int numStreamThreads) { + final long cacheSizePerThread; + final long inputBufferMaxBytesPerThread; + if (numStreamThreads == 0) { + cacheSizePerThread = totalCacheSize; + inputBufferMaxBytesPerThread = inputBufferMaxBytes; + } else { + cacheSizePerThread = totalCacheSize / (numStreamThreads + (topologyMetadata.hasGlobalTopology() ? 1 : 0)); + inputBufferMaxBytesPerThread = inputBufferMaxBytes / (numStreamThreads + (topologyMetadata.hasGlobalTopology() ? 1 : 0)); + } + processStreamThread(thread -> thread.resizeCacheAndBufferMemory(cacheSizePerThread, inputBufferMaxBytesPerThread)); if (globalStreamThread != null) { globalStreamThread.resize(cacheSizePerThread); } diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 58ee252abf0a..e9e0cca05a87 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -369,18 +369,30 @@ public class StreamsConfig extends AbstractConfig { /** {@code buffered.records.per.partition} */ @SuppressWarnings("WeakerAccess") + @Deprecated public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition"; public static final String BUFFERED_RECORDS_PER_PARTITION_DOC = "Maximum number of records to buffer per partition."; + /** {@code input.buffer.max.bytes} */ + @SuppressWarnings("WeakerAccess") + public static final String INPUT_BUFFER_MAX_BYTES_CONFIG = "input.buffer.max.bytes"; + public static final String INPUT_BUFFER_MAX_BYTES_DOC = "Maximum bytes of records to buffer across all threads"; + /** {@code built.in.metrics.version} */ public static final String BUILT_IN_METRICS_VERSION_CONFIG = "built.in.metrics.version"; private static final String BUILT_IN_METRICS_VERSION_DOC = "Version of the built-in metrics to use."; /** {@code cache.max.bytes.buffering} */ @SuppressWarnings("WeakerAccess") + @Deprecated public static final String CACHE_MAX_BYTES_BUFFERING_CONFIG = "cache.max.bytes.buffering"; public static final String CACHE_MAX_BYTES_BUFFERING_DOC = "Maximum number of memory bytes to be used for buffering across all threads"; + /** {@statestore.cache.max.bytes} */ + @SuppressWarnings("WeakerAccess") + public static final String STATESTORE_CACHE_MAX_BYTES_CONFIG = "statestore.cache.max.bytes"; + public static final String STATESTORE_CACHE_MAX_BYTES_DOC = "Maximum number of memory bytes to be used for statestore cache across all threads"; + /** {@code client.id} */ @SuppressWarnings("WeakerAccess") public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; @@ -688,6 +700,12 @@ public class StreamsConfig extends AbstractConfig { atLeast(0), Importance.MEDIUM, CACHE_MAX_BYTES_BUFFERING_DOC) + .define(STATESTORE_CACHE_MAX_BYTES_CONFIG, + Type.LONG, + 10 * 1024 * 1024L, + atLeast(0), + Importance.MEDIUM, + STATESTORE_CACHE_MAX_BYTES_DOC) .define(CLIENT_ID_CONFIG, Type.STRING, "", @@ -788,6 +806,11 @@ public class StreamsConfig extends AbstractConfig { in(NO_OPTIMIZATION, OPTIMIZE), Importance.MEDIUM, TOPOLOGY_OPTIMIZATION_DOC) + .define(INPUT_BUFFER_MAX_BYTES_CONFIG, + Type.LONG, + 512 * 1024 * 1024, + Importance.MEDIUM, + INPUT_BUFFER_MAX_BYTES_DOC) // LOW @@ -1513,6 +1536,29 @@ public Map getProducerConfigs(final String clientId) { return props; } + public long getTotalCacheSize() { + // both deprecated and new config set. Warn and use the new one. + if (originals().containsKey(CACHE_MAX_BYTES_BUFFERING_CONFIG) && originals().containsKey(STATESTORE_CACHE_MAX_BYTES_CONFIG)) { + if (!getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG).equals(getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG))) { + log.warn("Both deprecated config {} and the new config {} are set, hence {} is ignored and {} is used instead.", + CACHE_MAX_BYTES_BUFFERING_CONFIG, + STATESTORE_CACHE_MAX_BYTES_CONFIG, + CACHE_MAX_BYTES_BUFFERING_CONFIG, + STATESTORE_CACHE_MAX_BYTES_CONFIG); + } + return getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG); + } else if (originals().containsKey(CACHE_MAX_BYTES_BUFFERING_CONFIG)) { + // only deprecated config set. + log.warn("Deprecated config {} is set, and will be used; we suggest setting the new config {} instead as deprecated {} would be removed in the future.", + CACHE_MAX_BYTES_BUFFERING_CONFIG, + STATESTORE_CACHE_MAX_BYTES_CONFIG, + CACHE_MAX_BYTES_BUFFERING_CONFIG); + return getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG); + } + // only new or no config set. Use default or user specified value. + return getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG); + } + /** * Get the configs for the {@link Admin admin client}. * @param clientId clientId diff --git a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java index f046e71c8e9c..de8aec94ce4a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java @@ -32,9 +32,10 @@ import java.util.function.Supplier; import static org.apache.kafka.common.config.ConfigDef.ValidString.in; +import static org.apache.kafka.streams.StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG; import static org.apache.kafka.streams.StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG; import static org.apache.kafka.streams.StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_DOC; -import static org.apache.kafka.streams.StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG; +import static org.apache.kafka.streams.StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG; import static org.apache.kafka.streams.StreamsConfig.CACHE_MAX_BYTES_BUFFERING_DOC; import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG; import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC; @@ -55,15 +56,21 @@ * determine the defaults, which can then be overridden for specific topologies by passing them in when creating the * topology builders via the {@link org.apache.kafka.streams.StreamsBuilder()} method. */ +@SuppressWarnings("deprecation") public class TopologyConfig extends AbstractConfig { private static final ConfigDef CONFIG; static { CONFIG = new ConfigDef() - .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, - Type.INT, + .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, + Type.INT, + null, + Importance.LOW, + BUFFERED_RECORDS_PER_PARTITION_DOC) + .define(STATESTORE_CACHE_MAX_BYTES_CONFIG, + Type.LONG, null, - Importance.LOW, - BUFFERED_RECORDS_PER_PARTITION_DOC) + Importance.MEDIUM, + CACHE_MAX_BYTES_BUFFERING_DOC) .define(CACHE_MAX_BYTES_BUFFERING_CONFIG, Type.LONG, null, @@ -129,14 +136,32 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo maxBufferedSize = getInt(BUFFERED_RECORDS_PER_PARTITION_CONFIG); log.info("Topology {} is overriding {} to {}", topologyName, BUFFERED_RECORDS_PER_PARTITION_CONFIG, maxBufferedSize); } else { - maxBufferedSize = globalAppConfigs.getInt(BUFFERED_RECORDS_PER_PARTITION_CONFIG); + maxBufferedSize = globalAppConfigs.originals().containsKey(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG) + ? globalAppConfigs.getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG) : -1; } - if (isTopologyOverride(CACHE_MAX_BYTES_BUFFERING_CONFIG, topologyOverrides)) { + if (isTopologyOverride(STATESTORE_CACHE_MAX_BYTES_CONFIG, topologyOverrides) && isTopologyOverride(CACHE_MAX_BYTES_BUFFERING_CONFIG, topologyOverrides)) { + cacheSize = getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG); + log.info("Topology {} is using both deprecated config {} and new config {}, hence {} is ignored and the new config {} (value {}) is used", + topologyName, + CACHE_MAX_BYTES_BUFFERING_CONFIG, + STATESTORE_CACHE_MAX_BYTES_CONFIG, + CACHE_MAX_BYTES_BUFFERING_CONFIG, + STATESTORE_CACHE_MAX_BYTES_CONFIG, + cacheSize); + } else if (isTopologyOverride(CACHE_MAX_BYTES_BUFFERING_CONFIG, topologyOverrides)) { cacheSize = getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG); - log.info("Topology {} is overriding {} to {}", topologyName, CACHE_MAX_BYTES_BUFFERING_CONFIG, cacheSize); + log.info("Topology {} is using only deprecated config {}, and will be used to set cache size to {}; " + + "we suggest setting the new config {} instead as deprecated {} would be removed in the future.", + topologyName, + CACHE_MAX_BYTES_BUFFERING_CONFIG, + cacheSize, + STATESTORE_CACHE_MAX_BYTES_CONFIG, + CACHE_MAX_BYTES_BUFFERING_CONFIG); + } else if (isTopologyOverride(STATESTORE_CACHE_MAX_BYTES_CONFIG, topologyOverrides)) { + cacheSize = getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG); } else { - cacheSize = globalAppConfigs.getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG); + cacheSize = globalAppConfigs.getTotalCacheSize(); } if (isTopologyOverride(MAX_TASK_IDLE_MS_CONFIG, topologyOverrides)) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/CogroupedKStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/CogroupedKStream.java index 051396fbc94f..b0f1deca1cbd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/CogroupedKStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/CogroupedKStream.java @@ -78,7 +78,7 @@ CogroupedKStream cogroup(final KGroupedStream groupedStre * same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via @@ -128,7 +128,7 @@ CogroupedKStream cogroup(final KGroupedStream groupedStre * same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via @@ -179,7 +179,7 @@ KTable aggregate(final Initializer initializer, * same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via @@ -232,7 +232,7 @@ KTable aggregate(final Initializer initializer, * same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link org.apache.kafka.streams.state.ReadOnlyKeyValueStore} it must be obtained via diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java index 072558cf6ed3..513d94dae65f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java @@ -53,7 +53,7 @@ public interface KGroupedStream { * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store (which always will be of type {@link TimestampedKeyValueStore}) will be backed by @@ -81,7 +81,7 @@ public interface KGroupedStream { * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store (which always will be of type {@link TimestampedKeyValueStore}) will be backed by @@ -112,7 +112,7 @@ public interface KGroupedStream { * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via @@ -158,7 +158,7 @@ public interface KGroupedStream { * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via @@ -211,7 +211,7 @@ KTable count(final Named named, * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. * *

@@ -262,7 +262,7 @@ KTable count(final Named named, * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via @@ -326,7 +326,7 @@ KTable reduce(final Reducer reducer, * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via @@ -385,7 +385,7 @@ KTable reduce(final Reducer reducer, * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. * *

@@ -431,7 +431,7 @@ KTable aggregate(final Initializer initializer, * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via @@ -490,7 +490,7 @@ KTable aggregate(final Initializer initializer, * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java index 06d12e1d4ce8..5733aef319ce 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java @@ -52,7 +52,7 @@ public interface KGroupedTable { * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via @@ -95,7 +95,7 @@ public interface KGroupedTable { * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via @@ -138,7 +138,7 @@ public interface KGroupedTable { * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. @@ -167,7 +167,7 @@ public interface KGroupedTable { * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. @@ -223,7 +223,7 @@ public interface KGroupedTable { * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via @@ -296,7 +296,7 @@ KTable reduce(final Reducer adder, * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via @@ -368,7 +368,7 @@ KTable reduce(final Reducer adder, * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. @@ -434,7 +434,7 @@ KTable reduce(final Reducer adder, * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via @@ -518,7 +518,7 @@ KTable aggregate(final Initializer initializer, * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via @@ -604,7 +604,7 @@ KTable aggregate(final Initializer initializer, * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is @@ -674,7 +674,7 @@ KTable aggregate(final Initializer initializer, * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindowedCogroupedKStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindowedCogroupedKStream.java index b7e3b07e3718..eeeb3e1a0364 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindowedCogroupedKStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindowedCogroupedKStream.java @@ -77,7 +77,7 @@ public interface SessionWindowedCogroupedKStream { * the same window and key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. @@ -122,7 +122,7 @@ KTable, V> aggregate(final Initializer initializer, * the same window and key. * The rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. @@ -166,7 +166,7 @@ KTable, V> aggregate(final Initializer initializer, * the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link SessionStore} it must be obtained via @@ -226,7 +226,7 @@ KTable, V> aggregate(final Initializer initializer, * to the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link SessionStore} it must be obtained via diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindowedKStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindowedKStream.java index 1b7a363d3c20..c561b62abf47 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindowedKStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindowedKStream.java @@ -65,7 +65,7 @@ public interface SessionWindowedKStream { * the same session and key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. @@ -95,7 +95,7 @@ public interface SessionWindowedKStream { * the same session and key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. @@ -126,7 +126,7 @@ public interface SessionWindowedKStream { * to the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link SessionStore} it must be obtained via @@ -172,7 +172,7 @@ public interface SessionWindowedKStream { * to the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link SessionStore} it must be obtained via @@ -233,7 +233,7 @@ KTable, Long> count(final Named named, * the same window and key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. @@ -282,7 +282,7 @@ KTable, VR> aggregate(final Initializer initializer, * the same window and key. * The rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. @@ -330,7 +330,7 @@ KTable, VR> aggregate(final Initializer initializer, * the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link SessionStore} it must be obtained via @@ -391,7 +391,7 @@ KTable, VR> aggregate(final Initializer initializer, * to the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link SessionStore} it must be obtained via @@ -459,7 +459,7 @@ KTable, VR> aggregate(final Initializer initializer, * the same window and key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. @@ -504,7 +504,7 @@ KTable, VR> aggregate(final Initializer initializer, * the same window and key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. @@ -549,7 +549,7 @@ KTable, VR> aggregate(final Initializer initializer, * to the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link SessionStore} it must be obtained via @@ -609,7 +609,7 @@ KTable, V> reduce(final Reducer reducer, * to the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link SessionStore} it must be obtained via diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindowedCogroupedKStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindowedCogroupedKStream.java index e4178bc9e3d0..a46da057a08a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindowedCogroupedKStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindowedCogroupedKStream.java @@ -75,7 +75,7 @@ public interface TimeWindowedCogroupedKStream { * the same window and key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by @@ -115,7 +115,7 @@ public interface TimeWindowedCogroupedKStream { * the same window and key. * The rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by @@ -156,7 +156,7 @@ KTable, V> aggregate(final Initializer initializer, * the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyWindowStore} it must be obtained via @@ -213,7 +213,7 @@ KTable, V> aggregate(final Initializer initializer, * to the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyWindowStore} it must be obtained via diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindowedKStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindowedKStream.java index c015e79032a2..8bce98a586cf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindowedKStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindowedKStream.java @@ -65,7 +65,7 @@ public interface TimeWindowedKStream { * the same window and key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by @@ -95,7 +95,7 @@ public interface TimeWindowedKStream { * the same window and key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by @@ -126,7 +126,7 @@ public interface TimeWindowedKStream { * to the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyWindowStore} it must be obtained via @@ -175,7 +175,7 @@ public interface TimeWindowedKStream { * to the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval} *

* To query the local {@link ReadOnlyWindowStore} it must be obtained via @@ -236,7 +236,7 @@ KTable, Long> count(final Named named, * the same window and key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by @@ -281,7 +281,7 @@ KTable, VR> aggregate(final Initializer initializer, * the same window and key. * The rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by @@ -326,7 +326,7 @@ KTable, VR> aggregate(final Initializer initializer, * the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyWindowStore} it must be obtained via @@ -387,7 +387,7 @@ KTable, VR> aggregate(final Initializer initializer, * to the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval} *

* To query the local {@link ReadOnlyWindowStore} it must be obtained via @@ -457,7 +457,7 @@ KTable, VR> aggregate(final Initializer initializer, * the same window and key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by @@ -502,7 +502,7 @@ KTable, VR> aggregate(final Initializer initializer, * the same window and key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by @@ -547,7 +547,7 @@ KTable, VR> aggregate(final Initializer initializer, * to the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyWindowStore} it must be obtained via @@ -610,7 +610,7 @@ KTable, V> reduce(final Reducer reducer, * to the same window and key if caching is enabled on the {@link Materialized} instance. * When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct * keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} - * parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyWindowStore} it must be obtained via diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java index 199bc0e6456c..dd1257f17984 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java @@ -64,10 +64,12 @@ public class PartitionGroup { private final Sensor enforcedProcessingSensor; private final long maxTaskIdleMs; private final Sensor recordLatenessSensor; + private final Sensor totalBytesSensor; private final PriorityQueue nonEmptyQueuesByTime; private long streamTime; private int totalBuffered; + private long totalBytesBuffered; private boolean allBuffered; private final Map idlePartitionDeadlines = new HashMap<>(); @@ -92,6 +94,7 @@ RecordQueue queue() { final Function lagProvider, final Sensor recordLatenessSensor, final Sensor enforcedProcessingSensor, + final Sensor totalBytesSensor, final long maxTaskIdleMs) { this.logger = logContext.logger(PartitionGroup.class); nonEmptyQueuesByTime = new PriorityQueue<>(partitionQueues.size(), Comparator.comparingLong(RecordQueue::headRecordTimestamp)); @@ -100,6 +103,7 @@ RecordQueue queue() { this.enforcedProcessingSensor = enforcedProcessingSensor; this.maxTaskIdleMs = maxTaskIdleMs; this.recordLatenessSensor = recordLatenessSensor; + this.totalBytesSensor = totalBytesSensor; totalBuffered = 0; allBuffered = false; streamTime = RecordQueue.UNKNOWN; @@ -118,11 +122,11 @@ public boolean readyToProcess(final long wallClockTime) { } } logger.trace("Ready for processing because max.task.idle.ms is disabled." + - "\n\tThere may be out-of-order processing for this task as a result." + - "\n\tBuffered partitions: {}" + - "\n\tNon-buffered partitions: {}", - bufferedPartitions, - emptyPartitions); + "\n\tThere may be out-of-order processing for this task as a result." + + "\n\tBuffered partitions: {}" + + "\n\tNon-buffered partitions: {}", + bufferedPartitions, + emptyPartitions); } return true; } @@ -151,9 +155,9 @@ public boolean readyToProcess(final long wallClockTime) { // must wait to poll the data we know to be on the broker idlePartitionDeadlines.remove(partition); logger.trace( - "Lag for {} is currently {}, but no data is buffered locally. Waiting to buffer some records.", - partition, - fetchedLag.getAsLong() + "Lag for {} is currently {}, but no data is buffered locally. Waiting to buffer some records.", + partition, + fetchedLag.getAsLong() ); return false; } else { @@ -167,11 +171,11 @@ public boolean readyToProcess(final long wallClockTime) { final long deadline = idlePartitionDeadlines.get(partition); if (wallClockTime < deadline) { logger.trace( - "Lag for {} is currently 0 and current time is {}. Waiting for new data to be produced for configured idle time {} (deadline is {}).", - partition, - wallClockTime, - maxTaskIdleMs, - deadline + "Lag for {} is currently 0 and current time is {}. Waiting for new data to be produced for configured idle time {} (deadline is {}).", + partition, + wallClockTime, + maxTaskIdleMs, + deadline ); return false; } else { @@ -193,15 +197,15 @@ public boolean readyToProcess(final long wallClockTime) { } else { enforcedProcessingSensor.record(1.0d, wallClockTime); logger.trace("Continuing to process although some partitions are empty on the broker." + - "\n\tThere may be out-of-order processing for this task as a result." + - "\n\tPartitions with local data: {}." + - "\n\tPartitions we gave up waiting for, with their corresponding deadlines: {}." + - "\n\tConfigured max.task.idle.ms: {}." + - "\n\tCurrent wall-clock time: {}.", - queued, - enforced, - maxTaskIdleMs, - wallClockTime); + "\n\tThere may be out-of-order processing for this task as a result." + + "\n\tPartitions with local data: {}." + + "\n\tPartitions we gave up waiting for, with their corresponding deadlines: {}." + + "\n\tConfigured max.task.idle.ms: {}." + + "\n\tCurrent wall-clock time: {}.", + queued, + enforced, + maxTaskIdleMs, + wallClockTime); return true; } } @@ -225,6 +229,7 @@ void updatePartitions(final Set newInputPartitions, final Functi if (!newInputPartitions.contains(topicPartition)) { // if partition is removed should delete its queue totalBuffered -= queueEntry.getValue().size(); + totalBytesBuffered -= queueEntry.getValue().getTotalBytesBuffered(); queuesIterator.remove(); removedPartitions.add(topicPartition); } @@ -260,12 +265,17 @@ StampedRecord nextRecord(final RecordInfo info, final long wallClockTime) { info.queue = queue; if (queue != null) { + // get the buffer size of queue before poll + final long oldBufferSize = queue.getTotalBytesBuffered(); // get the first record from this queue. record = queue.poll(); + // After polling, the buffer size would have reduced. + final long newBufferSize = queue.getTotalBytesBuffered(); if (record != null) { --totalBuffered; - + totalBytesBuffered -= oldBufferSize - newBufferSize; + totalBytesSensor.record(totalBytesBuffered); if (queue.isEmpty()) { // if a certain queue has been drained, reset the flag allBuffered = false; @@ -301,7 +311,9 @@ int addRawRecords(final TopicPartition partition, final Iterable 0) { @@ -316,7 +328,8 @@ int addRawRecords(final TopicPartition partition, final Iterable getNonEmptyTopicPartitions() { + final Set nonEmptyTopicPartitions = new HashSet<>(); + for (final RecordQueue recordQueue : nonEmptyQueuesByTime) { + nonEmptyTopicPartitions.add(recordQueue.partition()); + } + return nonEmptyTopicPartitions; + } + int numBuffered() { return totalBuffered; } - boolean allPartitionsBufferedLocally() { - return allBuffered; + long totalBytesBuffered() { + return totalBytesBuffered; } void clear() { @@ -370,4 +391,10 @@ void clear() { totalBuffered = 0; streamTime = RecordQueue.UNKNOWN; } -} + + // Below methods are for only testing. + + boolean allPartitionsBufferedLocally() { + return allBuffered; + } +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java index 1c0196635cb2..90d67a7b0fca 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -18,8 +18,10 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.Header; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; @@ -50,6 +52,8 @@ public class RecordQueue { private long partitionTime = UNKNOWN; private final Sensor droppedRecordsSensor; + private long totalBytesBuffered; + private long headRecordSizeInBytes; RecordQueue(final TopicPartition partition, final SourceNode source, @@ -74,6 +78,8 @@ public class RecordQueue { droppedRecordsSensor ); this.log = logContext.logger(RecordQueue.class); + this.totalBytesBuffered = 0L; + this.headRecordSizeInBytes = 0L; } void setPartitionTime(final long partitionTime) { @@ -98,6 +104,25 @@ public TopicPartition partition() { return partition; } + private long sizeInBytes(final ConsumerRecord record) { + long headerSizeInBytes = 0L; + + for (final Header header: record.headers().toArray()) { + headerSizeInBytes += Utils.utf8(header.key()).length; + if (header.value() != null) { + headerSizeInBytes += header.value().length; + } + } + + return record.serializedKeySize() + + record.serializedValueSize() + + 8L + // timestamp + 8L + // offset + Utils.utf8(record.topic()).length + + 4L + // partition + headerSizeInBytes; + } + /** * Add a batch of {@link ConsumerRecord} into the queue * @@ -107,6 +132,7 @@ public TopicPartition partition() { int addRawRecords(final Iterable> rawRecords) { for (final ConsumerRecord rawRecord : rawRecords) { fifoQueue.addLast(rawRecord); + this.totalBytesBuffered += sizeInBytes(rawRecord); } updateHead(); @@ -121,7 +147,9 @@ int addRawRecords(final Iterable> rawRecords) { */ public StampedRecord poll() { final StampedRecord recordToReturn = headRecord; + totalBytesBuffered -= headRecordSizeInBytes; headRecord = null; + headRecordSizeInBytes = 0L; partitionTime = Math.max(partitionTime, recordToReturn.timestamp); updateHead(); @@ -167,6 +195,7 @@ public Long headRecordOffset() { public void clear() { fifoQueue.clear(); headRecord = null; + headRecordSizeInBytes = 0L; partitionTime = UNKNOWN; } @@ -206,6 +235,7 @@ private void updateHead() { continue; } headRecord = new StampedRecord(deserialized, timestamp); + headRecordSizeInBytes = sizeInBytes(raw); } // if all records in the FIFO queue are corrupted, make the last one the headRecord @@ -221,4 +251,11 @@ private void updateHead() { long partitionTime() { return partitionTime; } + + /** + * @return the total bytes buffered for this particular RecordQueue + */ + long getTotalBytesBuffered() { + return totalBytesBuffered; + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 4185fcff47c4..b2df5ce096d5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -189,6 +189,7 @@ public StreamTask(final TaskId id, createPartitionQueues(), mainConsumer::currentLag, TaskMetrics.recordLatenessSensor(threadId, taskId, streamsMetrics), + TaskMetrics.totalBytesSensor(threadId, taskId, streamsMetrics), enforcedProcessingSensor, maxTaskIdleMs ); @@ -717,7 +718,8 @@ record = partitionGroup.nextRecord(recordInfo, wallClockTime); // after processing this record, if its partition queue's buffered size has been // decreased to the threshold, we can then resume the consumption on this partition - if (recordInfo.queue().size() == maxBufferedSize) { + // TODO maxBufferedSize != -1 would be removed once the deprecated config buffered.records.per.partition is removed + if (maxBufferedSize != -1 && recordInfo.queue().size() == maxBufferedSize) { mainConsumer.resume(singleton(partition)); } @@ -971,7 +973,8 @@ public void addRecords(final TopicPartition partition, final Iterable maxBufferedSize) { + // We do this only if the deprecated config buffered.records.per.partition is set + if (maxBufferedSize != -1 && newQueueSize > maxBufferedSize) { mainConsumer.pause(singleton(partition)); } } @@ -1252,6 +1255,14 @@ RecordCollector recordCollector() { return recordCollector; } + Set getNonEmptyTopicPartitions() { + return this.partitionGroup.getNonEmptyTopicPartitions(); + } + + long totalBytesBuffered() { + return partitionGroup.totalBytesBuffered(); + } + // below are visible for testing only int numBuffered() { return partitionGroup.numBuffered(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 7401e539c4f7..ab4b09469688 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -318,6 +318,7 @@ public boolean isRunning() { // These are used to signal from outside the stream thread, but the variables themselves are internal to the thread private final AtomicLong cacheResizeSize = new AtomicLong(-1L); private final AtomicBoolean leaveGroupRequested = new AtomicBoolean(false); + private final AtomicLong maxBufferSizeBytes = new AtomicLong(-1L); private final boolean eosEnabled; public static StreamThread create(final TopologyMetadata topologyMetadata, @@ -330,6 +331,7 @@ public static StreamThread create(final TopologyMetadata topologyMetadata, final Time time, final StreamsMetadataState streamsMetadataState, final long cacheSizeBytes, + final long maxBufferSizeBytes, final StateDirectory stateDirectory, final StateRestoreListener userStateRestoreListener, final int threadIdx, @@ -432,7 +434,8 @@ public static StreamThread create(final TopologyMetadata topologyMetadata, referenceContainer.nonFatalExceptionsToHandle, shutdownErrorHook, streamsUncaughtExceptionHandler, - cache::resize + cache::resize, + maxBufferSizeBytes ); return streamThread.updateThreadMetadata(getSharedAdminClientId(clientId)); @@ -455,7 +458,8 @@ public StreamThread(final Time time, final Queue nonFatalExceptionsToHandle, final Runnable shutdownErrorHook, final BiConsumer streamsUncaughtExceptionHandler, - final java.util.function.Consumer cacheResizer) { + final java.util.function.Consumer cacheResizer, + final long maxBufferSizeBytes) { super(threadId); this.stateLock = new Object(); this.adminClient = adminClient; @@ -524,6 +528,7 @@ public StreamThread(final Time time, this.numIterations = 1; this.eosEnabled = eosEnabled(config); + this.maxBufferSizeBytes.set(maxBufferSizeBytes); } private static final class InternalConsumerConfig extends ConsumerConfig { @@ -706,8 +711,17 @@ private void subscribeConsumer() { } } - public void resizeCache(final long size) { - cacheResizeSize.set(size); + public void resizeCacheAndBufferMemory(final long cacheSize, final long maxBufferSize) { + cacheResizeSize.set(cacheSize); + maxBufferSizeBytes.set(maxBufferSize); + } + + public long getCacheSize() { + return cacheResizeSize.get(); + } + + public long getMaxBufferSize() { + return maxBufferSizeBytes.get(); } /** @@ -782,6 +796,10 @@ void runOnce() { totalProcessed += processed; totalRecordsProcessedSinceLastSummary += processed; + final long bufferSize = taskManager.getInputBufferSizeInBytes(); + if (bufferSize <= maxBufferSizeBytes.get()) { + mainConsumer.resume(mainConsumer.paused()); + } } log.debug("Processed {} records with {} iterations; invoking punctuators if necessary", @@ -899,7 +917,8 @@ private void checkForTopologyUpdates() { } } - private long pollPhase() { + // Visible for testing + long pollPhase() { final ConsumerRecords records; log.debug("Invoking poll on main Consumer"); @@ -945,6 +964,17 @@ private long pollPhase() { if (!records.isEmpty()) { pollRecordsSensor.record(numRecords, now); taskManager.addRecordsToTasks(records); + // Check buffer size after adding records to tasks + final long bufferSize = taskManager.getInputBufferSizeInBytes(); + // Pausing partitions as the buffer size now exceeds max buffer size + if (bufferSize > maxBufferSizeBytes.get()) { + log.info("Buffered records size {} bytes exceeds {}. Pausing the consumer", bufferSize, maxBufferSizeBytes.get()); + // Only non-empty partitions are paused here. Reason is that, if a task has multiple partitions with + // some of them empty, then in that case pausing even empty partitions would sacrifice ordered processing + // and even lead to temporal deadlock. More explanation can be found here: + // https://issues.apache.org/jira/browse/KAFKA-13152?focusedCommentId=17400647&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17400647 + mainConsumer.pause(taskManager.nonEmptyPartitions()); + } } while (!nonFatalExceptionsToHandle.isEmpty()) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java index 3549ba2b18cf..fc3e6cb1a82d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java @@ -246,4 +246,5 @@ default boolean commitRequested() { * @return This returns the time the task started idling. If it is not idling it returns empty. */ Optional timeCurrentIdlingStarted(); + } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index f1585633cad9..4bc8e4335c1b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -1066,6 +1066,17 @@ void addRecordsToTasks(final ConsumerRecords records) { } } + /** + * Fetch all non-empty partitions for pausing + */ + Set nonEmptyPartitions() { + final Set nonEmptyPartitions = new HashSet<>(); + for (final Task task : activeTaskIterable()) { + nonEmptyPartitions.addAll(((StreamTask) task).getNonEmptyTopicPartitions()); + } + return nonEmptyPartitions; + } + /** * @throws TaskMigratedException if committing offsets failed (non-EOS) * or if the task producer got fenced (EOS) @@ -1169,6 +1180,14 @@ void maybeCloseTasksFromRemovedTopologies(final Set currentNamedTopologi } } + long getInputBufferSizeInBytes() { + long bytesBuffered = 0L; + for (final Task task : activeTaskIterable()) { + bytesBuffered += ((StreamTask) task).totalBytesBuffered(); + } + return bytesBuffered; + } + /** * @throws TaskMigratedException if the task producer got fenced (EOS only) * @throws StreamsException if any task threw an exception while processing diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/TaskMetrics.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/TaskMetrics.java index cfa1ac6333a0..f173bac40399 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/TaskMetrics.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/TaskMetrics.java @@ -84,6 +84,11 @@ private TaskMetrics() {} private static final String NUM_BUFFERED_RECORDS_DESCRIPTION = "The count of buffered records that are polled " + "from consumer and not yet processed for this active task"; + private static final String INPUT_BUFFER_BYTES_TOTAL = "input-buffer-bytes-total"; + private static final String INPUT_BUFFER_BYTES_TOTAL_DESCRIPTION = "The total number of bytes accumulated in this task's input buffer"; + private static final String CACHE_SIZE_BYTES_TOTAL = "cache-size-bytes-total"; + private static final String CACHE_SIZE_BYTES_TOTAL_DESCRIPTION = "The total size in bytes of this task's cache."; + public static Sensor processLatencySensor(final String threadId, final String taskId, final StreamsMetricsImpl streamsMetrics) { @@ -128,6 +133,38 @@ public static Sensor activeBufferedRecordsSensor(final String threadId, return sensor; } + public static Sensor totalBytesSensor(final String threadId, + final String taskId, + final StreamsMetricsImpl streamsMetrics) { + final String name = INPUT_BUFFER_BYTES_TOTAL; + final Sensor sensor = streamsMetrics.taskLevelSensor(threadId, taskId, name, RecordingLevel.INFO); + + addValueMetricToSensor( + sensor, + TASK_LEVEL_GROUP, + streamsMetrics.taskLevelTagMap(threadId, taskId), + name, + INPUT_BUFFER_BYTES_TOTAL_DESCRIPTION + ); + return sensor; + } + + public static Sensor totalCacheSizeBytesSensor(final String threadId, + final String taskId, + final StreamsMetricsImpl streamsMetrics) { + final String name = CACHE_SIZE_BYTES_TOTAL; + final Sensor sensor = streamsMetrics.taskLevelSensor(threadId, taskId, name, Sensor.RecordingLevel.INFO); + + addValueMetricToSensor( + sensor, + TASK_LEVEL_GROUP, + streamsMetrics.taskLevelTagMap(threadId, taskId), + name, + CACHE_SIZE_BYTES_TOTAL_DESCRIPTION + ); + return sensor; + } + public static Sensor punctuateSensor(final String threadId, final String taskId, final StreamsMetricsImpl streamsMetrics) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java index ecf063b78345..71fdd4f356c7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.state.internals.metrics.NamedCacheMetrics; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,6 +49,7 @@ class NamedCache { private final StreamsMetricsImpl streamsMetrics; private final Sensor hitRatioSensor; + private final Sensor totalCacheSizeSensor; // internal stats private long numReadHits = 0; @@ -66,6 +68,11 @@ class NamedCache { taskName, storeName ); + totalCacheSizeSensor = TaskMetrics.totalCacheSizeBytesSensor( + Thread.currentThread().getName(), + taskName, + streamsMetrics + ); } synchronized final String name() { @@ -182,6 +189,7 @@ synchronized void put(final Bytes key, final LRUCacheEntry value) { dirtyKeys.add(key); } currentSizeBytes += node.size(); + totalCacheSizeSensor.record(currentSizeBytes); } synchronized long sizeInBytes() { @@ -243,6 +251,7 @@ synchronized void evict() { if (eldest.entry.isDirty()) { flush(eldest); } + totalCacheSizeSensor.record(currentSizeBytes); } synchronized LRUCacheEntry putIfAbsent(final Bytes key, final LRUCacheEntry value) { @@ -269,6 +278,7 @@ synchronized LRUCacheEntry delete(final Bytes key) { remove(node); dirtyKeys.remove(key); currentSizeBytes -= node.size(); + totalCacheSizeSensor.record(currentSizeBytes); return node.entry(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java index 9b9a671b471b..b5620d5c13ed 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -229,6 +229,7 @@ private void prepareStreams() throws Exception { anyObject(Time.class), anyObject(StreamsMetadataState.class), anyLong(), + anyLong(), anyObject(StateDirectory.class), anyObject(StateRestoreListener.class), anyInt(), @@ -241,6 +242,10 @@ private void prepareStreams() throws Exception { EasyMock.expect(StreamsConfigUtils.eosEnabled(anyObject(StreamsConfig.class))).andReturn(false).anyTimes(); EasyMock.expect(streamThreadOne.getId()).andReturn(1L).anyTimes(); EasyMock.expect(streamThreadTwo.getId()).andReturn(2L).anyTimes(); + EasyMock.expect(streamThreadOne.getCacheSize()).andReturn(10485760L).anyTimes(); + EasyMock.expect(streamThreadOne.getMaxBufferSize()).andReturn(536870912L).anyTimes(); + EasyMock.expect(streamThreadTwo.getCacheSize()).andReturn(10485760L).anyTimes(); + EasyMock.expect(streamThreadTwo.getMaxBufferSize()).andReturn(536870912L).anyTimes(); prepareStreamThread(streamThreadOne, 1, true); prepareStreamThread(streamThreadTwo, 2, false); @@ -289,6 +294,8 @@ private void prepareStreams() throws Exception { EasyMock.expect(globalStreamThread.stillRunning()).andReturn(globalThreadState.get() == GlobalStreamThread.State.RUNNING).anyTimes(); globalStreamThread.join(); EasyMock.expectLastCall().anyTimes(); + globalStreamThread.resize(EasyMock.anyLong()); + EasyMock.expectLastCall().anyTimes(); PowerMock.replay( StreamThread.class, @@ -345,7 +352,7 @@ private void prepareStreamThread(final StreamThread thread, ).anyTimes(); EasyMock.expect(thread.waitOnThreadState(EasyMock.isA(StreamThread.State.class), anyLong())).andStubReturn(true); EasyMock.expect(thread.isAlive()).andReturn(true).times(0, 1); - thread.resizeCache(EasyMock.anyLong()); + thread.resizeCacheAndBufferMemory(EasyMock.anyLong(), EasyMock.anyLong()); EasyMock.expectLastCall().anyTimes(); thread.requestLeaveGroupDuringShutdown(); EasyMock.expectLastCall().anyTimes(); diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index e569727ca315..243a4741677b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -1252,6 +1252,36 @@ public void shouldThrowExceptionWhenClientTagValueExceedMaxLimit() { ); } + @Test + @SuppressWarnings("deprecation") + public void shouldUseStateStoreCacheMaxBytesWhenBothOldAndNewConfigsAreSet() { + props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 100); + props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10); + final StreamsConfig config = new StreamsConfig(props); + assertEquals(config.getTotalCacheSize(), 100); + } + + @Test + @SuppressWarnings("deprecation") + public void shouldUseCacheMaxBytesBufferingConfigWhenOnlyDeprecatedConfigIsSet() { + props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10); + final StreamsConfig config = new StreamsConfig(props); + assertEquals(config.getTotalCacheSize(), 10); + } + + @Test + public void shouldUseStateStoreCacheMaxBytesWhenNewConfigIsSet() { + props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 10); + final StreamsConfig config = new StreamsConfig(props); + assertEquals(config.getTotalCacheSize(), 10); + } + + @Test + public void shouldUseDefaultStateStoreCacheMaxBytesConfigWhenNoConfigIsSet() { + final StreamsConfig config = new StreamsConfig(props); + assertEquals(config.getTotalCacheSize(), 10 * 1024 * 1024); + } + static class MisconfiguredSerde implements Serde { @Override public void configure(final Map configs, final boolean isKey) { diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java index d41cec044061..5f717e336f6c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java @@ -123,7 +123,7 @@ public static void setupConfigsAndUtils() { void prepareEnvironment() throws InterruptedException { if (!cacheEnabled) { - STREAMS_CONFIG.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + STREAMS_CONFIG.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); } STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot().getPath()); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java index fd5da124bb97..70bcef9cd6c9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java @@ -147,7 +147,7 @@ private void prepareConfigs(final String appID) { streamsConfig.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot().getPath()); streamsConfig.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Long().getClass()); streamsConfig.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); - streamsConfig.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfig.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); streamsConfig.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 100); streamsConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java index 26edd69a1c18..1119818b9272 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java @@ -375,7 +375,7 @@ public void shouldResizeCacheAfterThreadRemovalTimesOut() throws InterruptedExce final Properties props = new Properties(); props.putAll(properties); props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2); - props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, totalCacheBytes); + props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, totalCacheBytes); try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), props)) { addStreamStateChangeListener(kafkaStreams); @@ -386,7 +386,32 @@ public void shouldResizeCacheAfterThreadRemovalTimesOut() throws InterruptedExce for (final String log : appender.getMessages()) { // all 10 bytes should be available for remaining thread - if (log.endsWith("Resizing thread cache due to thread removal, new cache size per thread is 10")) { + if (log.contains("Resizing thread cache/max buffer size due to removal of thread ") && log.contains(", new cache size/max buffer size per thread is 10/536870912")) { + return; + } + } + } + } + fail(); + } + + @Test + public void shouldResizeMaxBufferAfterThreadRemovalTimesOut() throws InterruptedException { + final long maxBufferBytes = 10L; + final Properties props = new Properties(); + props.putAll(properties); + props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2); + props.put(StreamsConfig.INPUT_BUFFER_MAX_BYTES_CONFIG, maxBufferBytes); + + try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), props)) { + addStreamStateChangeListener(kafkaStreams); + startStreamsAndWaitForRunning(kafkaStreams); + + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KafkaStreams.class)) { + assertThrows(TimeoutException.class, () -> kafkaStreams.removeStreamThread(Duration.ofSeconds(0))); + for (final String log : appender.getMessages()) { + // all 10 bytes should be available for remaining thread + if (log.contains("Resizing thread cache/max buffer size due to removal of thread ") && log.contains(", new cache size/max buffer size per thread is 10485760/10")) { return; } } @@ -401,7 +426,7 @@ public void shouldResizeCacheAfterThreadReplacement() throws InterruptedExceptio final Properties props = new Properties(); props.putAll(properties); props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2); - props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, totalCacheBytes); + props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, totalCacheBytes); final AtomicBoolean injectError = new AtomicBoolean(false); @@ -442,7 +467,63 @@ public void close() { for (final String log : appender.getMessages()) { // after we replace the thread there should be two remaining threads with 5 bytes each - if (log.endsWith("Adding StreamThread-3, there will now be 2 live threads and the new cache size per thread is 5")) { + if (log.endsWith("Adding StreamThread-3, there are now 3 threads with cache size/max buffer size values as 3/178956970 per thread.")) { + return; + } + } + } + } + fail(); + } + + @Test + public void shouldResizeMaxBufferAfterThreadReplacement() throws InterruptedException { + final long totalCacheBytes = 10L; + final Properties props = new Properties(); + props.putAll(properties); + props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2); + props.put(StreamsConfig.INPUT_BUFFER_MAX_BYTES_CONFIG, totalCacheBytes); + + final AtomicBoolean injectError = new AtomicBoolean(false); + + final StreamsBuilder builder = new StreamsBuilder(); + final KStream stream = builder.stream(inputTopic); + stream.transform(() -> new Transformer>() { + @Override + public void init(final ProcessorContext context) { + context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, timestamp -> { + if (Thread.currentThread().getName().endsWith("StreamThread-1") && injectError.get()) { + injectError.set(false); + throw new RuntimeException("BOOM"); + } + }); + } + + @Override + public KeyValue transform(final String key, final String value) { + return new KeyValue<>(key, value); + } + + @Override + public void close() { + } + }); + + try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), props)) { + addStreamStateChangeListener(kafkaStreams); + kafkaStreams.setUncaughtExceptionHandler(e -> StreamThreadExceptionResponse.REPLACE_THREAD); + startStreamsAndWaitForRunning(kafkaStreams); + + stateTransitionHistory.clear(); + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister()) { + injectError.set(true); + waitForCondition(() -> !injectError.get(), "StreamThread did not hit and reset the injected error"); + + waitForTransitionFromRebalancingToRunning(); + + for (final String log : appender.getMessages()) { + // after we replace the thread there should be two remaining threads with 5 bytes each + if (log.endsWith("Adding StreamThread-3, there are now 3 threads with cache size/max buffer size values as 3495253/3 per thread.")) { return; } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java new file mode 100644 index 000000000000..e0fcd4daa311 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java @@ -0,0 +1,175 @@ +/* + * 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.streams.integration; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.StreamsTestUtils; +import org.apache.kafka.test.TestUtils; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.common.utils.Utils.mkObjectProperties; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; + +@Category(IntegrationTest.class) +public class EmitOnChangeIntegrationTest { + + private static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); + + @BeforeClass + public static void startCluster() throws IOException { + CLUSTER.start(); + } + + @AfterClass + public static void closeCluster() { + CLUSTER.stop(); + } + + @Rule + public TestName testName = new TestName(); + + private static String inputTopic; + private static String inputTopic2; + private static String outputTopic; + private static String outputTopic2; + private static String appId = ""; + + @Before + public void setup() { + final String testId = safeUniqueTestName(getClass(), testName); + appId = "appId_" + testId; + inputTopic = "input" + testId; + inputTopic2 = "input2" + testId; + outputTopic = "output" + testId; + outputTopic2 = "output2" + testId; + IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic, outputTopic, inputTopic2, outputTopic2); + } + + @Test + public void shouldEmitSameRecordAfterFailover() throws Exception { + final Properties properties = mkObjectProperties( + mkMap( + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()), + mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId), + mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()), + mkEntry(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1), + mkEntry(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0), + mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 300000L), + mkEntry(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class), + mkEntry(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class), + mkEntry(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 10000) + ) + ); + + final AtomicBoolean shouldThrow = new AtomicBoolean(true); + final StreamsBuilder builder = new StreamsBuilder(); + builder.table(inputTopic, Materialized.as("test-store")) + .toStream() + .map((key, value) -> { + if (shouldThrow.compareAndSet(true, false)) { + throw new RuntimeException("Kaboom"); + } else { + return new KeyValue<>(key, value); + } + }) + .to(outputTopic); + builder.stream(inputTopic2).to(outputTopic2); + + try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), properties)) { + kafkaStreams.setUncaughtExceptionHandler(exception -> StreamThreadExceptionResponse.REPLACE_THREAD); + StreamsTestUtils.startKafkaStreamsAndWaitForRunningState(kafkaStreams); + + IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp( + inputTopic, + Arrays.asList( + new KeyValue<>(1, "A"), + new KeyValue<>(1, "B") + ), + TestUtils.producerConfig( + CLUSTER.bootstrapServers(), + IntegerSerializer.class, + StringSerializer.class, + new Properties()), + 0L); + + IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp( + inputTopic2, + Arrays.asList( + new KeyValue<>(1, "A"), + new KeyValue<>(1, "B") + ), + TestUtils.producerConfig( + CLUSTER.bootstrapServers(), + IntegerSerializer.class, + StringSerializer.class, + new Properties()), + 0L); + + IntegrationTestUtils.waitUntilFinalKeyValueRecordsReceived( + TestUtils.consumerConfig( + CLUSTER.bootstrapServers(), + IntegerDeserializer.class, + StringDeserializer.class + ), + outputTopic, + Arrays.asList( + new KeyValue<>(1, "A"), + new KeyValue<>(1, "B") + ) + ); + IntegrationTestUtils.waitUntilFinalKeyValueRecordsReceived( + TestUtils.consumerConfig( + CLUSTER.bootstrapServers(), + IntegerDeserializer.class, + StringDeserializer.class + ), + outputTopic2, + Arrays.asList( + new KeyValue<>(1, "A"), + new KeyValue<>(1, "B") + ) + ); + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java index 5a97579e70da..82765f73ea99 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java @@ -245,7 +245,7 @@ private void runSimpleCopyTest(final int numberOfRestarts, final Properties properties = new Properties(); properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig); - properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + properties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), 1); properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.METADATA_MAX_AGE_CONFIG), "1000"); @@ -322,7 +322,7 @@ public void shouldBeAbleToPerformMultipleTransactions() throws Exception { final Properties properties = new Properties(); properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig); - properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + properties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); properties.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000"); properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); @@ -936,7 +936,7 @@ public void close() { } properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG), maxPollIntervalMs); properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), maxPollIntervalMs - 1); properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), maxPollIntervalMs); - properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + properties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); properties.put(StreamsConfig.STATE_DIR_CONFIG, stateTmpDir + appDir); properties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, dummyHostName + ":2142"); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java index b6aab860eac8..9108bbfa2648 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java @@ -944,7 +944,7 @@ public void close() {} properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), MAX_POLL_INTERVAL_MS); properties.put(StreamsConfig.producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG), (int) commitInterval); properties.put(StreamsConfig.producerPrefix(ProducerConfig.PARTITIONER_CLASS_CONFIG), KeyPartitioner.class); - properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + properties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); properties.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath() + File.separator + appDir); properties.put(InternalConfig.ASSIGNMENT_LISTENER, assignmentListener); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/ErrorHandlingIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ErrorHandlingIntegrationTest.java new file mode 100644 index 000000000000..b3c61248254e --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ErrorHandlingIntegrationTest.java @@ -0,0 +1,161 @@ +/* + * 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.streams.integration; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper; +import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopologyBuilder; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.StreamsTestUtils; +import org.apache.kafka.test.TestUtils; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicInteger; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.common.utils.Utils.mkObjectProperties; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; + +@Category(IntegrationTest.class) +public class ErrorHandlingIntegrationTest { + + private static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); + + @BeforeClass + public static void startCluster() throws IOException { + CLUSTER.start(); + } + + @AfterClass + public static void closeCluster() { + CLUSTER.stop(); + } + + @Rule + public TestName testName = new TestName(); + + private final String testId = safeUniqueTestName(getClass(), testName); + private final String appId = "appId_" + testId; + private final Properties properties = props(); + + // Task 0 + private final String inputTopic = "input" + testId; + private final String outputTopic = "output" + testId; + // Task 1 + private final String errorInputTopic = "error-input" + testId; + private final String errorOutputTopic = "error-output" + testId; + + @Before + public void setup() { + IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, errorInputTopic, errorOutputTopic, inputTopic, outputTopic); + } + + private Properties props() { + return mkObjectProperties( + mkMap( + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()), + mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId), + mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath()), + mkEntry(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0), + mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 15000L), + mkEntry(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class), + mkEntry(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class)) + ); + } + + @Test + public void shouldBackOffTaskAndEmitDataWithinSameTopology() throws Exception { + final AtomicInteger noOutputExpected = new AtomicInteger(0); + final AtomicInteger outputExpected = new AtomicInteger(0); + + try (final KafkaStreamsNamedTopologyWrapper kafkaStreams = new KafkaStreamsNamedTopologyWrapper(properties)) { + kafkaStreams.setUncaughtExceptionHandler(exception -> StreamThreadExceptionResponse.REPLACE_THREAD); + + final NamedTopologyBuilder builder = kafkaStreams.newNamedTopologyBuilder("topology_A"); + builder.stream(inputTopic).peek((k, v) -> outputExpected.incrementAndGet()).to(outputTopic); + builder.stream(errorInputTopic) + .peek((k, v) -> { + throw new RuntimeException("Kaboom"); + }) + .peek((k, v) -> noOutputExpected.incrementAndGet()) + .to(errorOutputTopic); + + kafkaStreams.addNamedTopology(builder.build()); + + StreamsTestUtils.startKafkaStreamsAndWaitForRunningState(kafkaStreams); + IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp( + errorInputTopic, + Arrays.asList( + new KeyValue<>(1, "A") + ), + TestUtils.producerConfig( + CLUSTER.bootstrapServers(), + IntegerSerializer.class, + StringSerializer.class, + new Properties()), + 0L); + IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp( + inputTopic, + Arrays.asList( + new KeyValue<>(1, "A"), + new KeyValue<>(1, "B") + ), + TestUtils.producerConfig( + CLUSTER.bootstrapServers(), + IntegerSerializer.class, + StringSerializer.class, + new Properties()), + 0L); + IntegrationTestUtils.waitUntilFinalKeyValueRecordsReceived( + TestUtils.consumerConfig( + CLUSTER.bootstrapServers(), + IntegerDeserializer.class, + StringDeserializer.class + ), + outputTopic, + Arrays.asList( + new KeyValue<>(1, "A"), + new KeyValue<>(1, "B") + ) + ); + assertThat(noOutputExpected.get(), equalTo(0)); + assertThat(outputExpected.get(), equalTo(2)); + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java index baaf06c573a1..c94066c397df 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java @@ -144,7 +144,7 @@ public static void closeCluster() { public void setUp() throws IOException { final Properties props = new Properties(); - props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); props.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000"); props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); @@ -283,7 +283,7 @@ public void shouldThrowExceptionOverlappingTopic() { @Test public void shouldThrowStreamsExceptionNoResetSpecified() throws InterruptedException { final Properties props = new Properties(); - props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); props.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000"); props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none"); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java index 097a79ff48a2..2a1fedfd5e32 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java @@ -132,7 +132,7 @@ public void before() throws Exception { streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); - streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0L); + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0L); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); streamsConfiguration.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig); streamsConfiguration.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1L); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java index 90dc9e73e2e5..4cdd1727218c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java @@ -110,7 +110,7 @@ public void before() throws Exception { streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); - streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); globalTable = builder.globalTable(globalTableTopic, Consumed.with(Serdes.Long(), Serdes.String()), Materialized.>as(globalStore) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java index 98dec8706c9c..31658bbf3d20 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java @@ -119,7 +119,7 @@ public void before() throws Exception { streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); - streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); final Consumed stringLongConsumed = Consumed.with(Serdes.String(), Serdes.Long()); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java index 29c61ec76477..3abe088216ff 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java @@ -104,7 +104,7 @@ public void before() { streamsProp.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); streamsProp.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsProp.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); - streamsProp.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsProp.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsProp.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java index 4fe35a67daf8..79a697553bf9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java @@ -107,7 +107,7 @@ public void before() throws InterruptedException { streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, COMMIT_INTERVAL_MS); - streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 * 1024L); + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 10 * 1024 * 1024L); final KeyValueMapper mapper = MockMapper.selectValueMapper(); stream = builder.stream(streamOneInput, Consumed.with(Serdes.Integer(), Serdes.String())); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java index e58190354446..3990c67a7b4f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java @@ -142,7 +142,7 @@ public void before() throws InterruptedException { streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); - streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionIntegrationTest.java index 1e7f685debbf..4730f5a83958 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionIntegrationTest.java @@ -138,7 +138,7 @@ public void before() throws InterruptedException { streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); - streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest.java index c83bbaee62d1..e0bed5bf686b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest.java @@ -181,6 +181,7 @@ private void verifyKTableKTableJoin(final Set> expected assertEquals(expectedResult, result); } + @SuppressWarnings("deprecation") private static Properties getStreamsConfig() { final Properties streamsConfig = new Properties(); streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, "KTable-FKJ-Partitioner"); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java index 0788b52e30c6..ed48cf8dbf83 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java @@ -208,7 +208,7 @@ private static Properties getStreamsConfig() { streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, "KTable-FKJ-Multi"); streamsConfig.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); streamsConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - streamsConfig.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfig.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); return streamsConfig; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java index 6d50ea99cc4a..8d4ac2c6e1d4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java @@ -69,7 +69,7 @@ public static void startCluster() throws IOException { STREAMS_CONFIG.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); STREAMS_CONFIG.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); - STREAMS_CONFIG.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + STREAMS_CONFIG.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); STREAMS_CONFIG.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 5L); STREAMS_CONFIG.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class); STREAMS_CONFIG.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java index 9ada60f40367..b2c2e4d5e7bc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java @@ -195,6 +195,8 @@ public static void closeCluster() { private static final String THREAD_START_TIME = "thread-start-time"; private static final String ACTIVE_PROCESS_RATIO = "active-process-ratio"; private static final String ACTIVE_BUFFER_COUNT = "active-buffer-count"; + private static final String INPUT_BUFFER_BYTES_TOTAL = "input-buffer-bytes-total"; + private static final String CACHE_SIZE_BYTES_TOTAL = "cache-size-bytes-total"; private static final String SKIPPED_RECORDS_RATE = "skipped-records-rate"; private static final String SKIPPED_RECORDS_TOTAL = "skipped-records-total"; private static final String RECORD_LATENESS_AVG = "record-lateness-avg"; @@ -251,7 +253,7 @@ public void before() throws InterruptedException { streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); streamsConfiguration.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, Sensor.RecordingLevel.DEBUG.name); - streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 * 1024L); + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 10 * 1024 * 1024L); streamsConfiguration.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, NUM_THREADS); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); } @@ -527,6 +529,8 @@ private void checkTaskLevelMetrics() { checkMetricByName(listMetricTask, PUNCTUATE_TOTAL, 4); checkMetricByName(listMetricTask, PROCESS_RATE, 4); checkMetricByName(listMetricTask, PROCESS_TOTAL, 4); + checkMetricByName(listMetricTask, INPUT_BUFFER_BYTES_TOTAL, 4); + checkMetricByName(listMetricTask, CACHE_SIZE_BYTES_TOTAL, 3); } private void checkProcessorNodeLevelMetrics() { diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java index 0e367d909e9f..75bf3a1c3b8e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java @@ -813,7 +813,7 @@ public void shouldBackOffTaskAndEmitDataWithinSameTopology() throws Exception { try { final AtomicInteger noOutputExpected = new AtomicInteger(0); final AtomicInteger outputExpected = new AtomicInteger(0); - props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 15000L); props.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath()); props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/OptimizedKTableIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/OptimizedKTableIntegrationTest.java index 44744cd3e2be..f9ab66cbd9c0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/OptimizedKTableIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/OptimizedKTableIntegrationTest.java @@ -198,7 +198,7 @@ private Properties streamsConfiguration() { config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); config.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); - config.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + config.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); config.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 100); config.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 200); config.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java index 15b9ea69241f..52a896668e5c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java @@ -964,7 +964,7 @@ public void shouldBeAbleToQueryMapValuesAfterFilterState() throws Exception { } private void verifyCanQueryState(final int cacheSizeBytes) throws Exception { - streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, cacheSizeBytes); + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, cacheSizeBytes); final StreamsBuilder builder = new StreamsBuilder(); final String[] keys = {"hello", "goodbye", "welcome", "go", "kafka"}; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java index 1bfb1b625ac0..99cb358b6e7f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java @@ -129,7 +129,7 @@ public static void closeCluster() { public void setUp() throws InterruptedException { outputTopic = createTopic(topicSuffixGenerator.incrementAndGet()); final Properties properties = new Properties(); - properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + properties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); properties.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000"); properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java index 2c0e18070a5b..b5aad95a0d2c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java @@ -123,7 +123,7 @@ private Properties props() { final Properties streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, appId); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath()); streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RocksDBMetricsIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RocksDBMetricsIntegrationTest.java index c698d0677223..725b38612a46 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/RocksDBMetricsIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RocksDBMetricsIntegrationTest.java @@ -198,7 +198,7 @@ private Properties streamsConfig() { streamsConfiguration.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, Sensor.RecordingLevel.DEBUG.name); streamsConfiguration.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingGuarantee); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); - streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); return streamsConfiguration; } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java index 4fbe73438b91..c8da3bb973e9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java @@ -400,7 +400,7 @@ private Properties props(final String stateDirPath) { final Properties streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, appId); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, stateDirPath); streamsConfiguration.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); streamsConfiguration.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java index 6c6fc5d24b89..17df2b59a217 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java @@ -94,7 +94,7 @@ private Properties props() { final String safeTestName = safeUniqueTestName(getClass(), testName); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamTableJoinTopologyOptimizationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamTableJoinTopologyOptimizationIntegrationTest.java index 512d1c13bd4c..8d023b8028e8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamTableJoinTopologyOptimizationIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamTableJoinTopologyOptimizationIntegrationTest.java @@ -121,7 +121,7 @@ public void before() throws InterruptedException { streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); - streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java index 0974ed6464b0..49e7913dc0e7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java @@ -51,7 +51,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; -@SuppressWarnings("unchecked") +@SuppressWarnings({"unchecked"}) public class KTableFilterTest { private final Consumed consumed = Consumed.with(Serdes.String(), Serdes.Integer()); private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.Integer()); @@ -59,7 +59,7 @@ public class KTableFilterTest { @Before public void setUp() { // disable caching at the config level - props.setProperty(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, "0"); + props.setProperty(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, "0"); } private final Predicate predicate = (key, value) -> (value % 2) == 0; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java index 8c7d179b84dd..a77dcdb0a261 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java @@ -70,7 +70,7 @@ public void before() { @Test public void shouldCountSessionWindowedWithCachingDisabled() { - props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); shouldCountSessionWindowed(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index f185f1aff800..2b273acf0311 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -941,9 +941,22 @@ public void shouldSetTopologyConfigOnRewriteTopology() { } @Test + @SuppressWarnings("deprecation") + public void shouldUseNonDeprecatedConfigToSetCacheBytesWhenBothDeprecatedAndNonDeprecatedConfigsUsed() { + final Properties globalProps = StreamsTestUtils.getStreamsConfig(); + globalProps.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 200L); + globalProps.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 100L); + final StreamsConfig globalStreamsConfig = new StreamsConfig(globalProps); + final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(globalStreamsConfig); + assertThat(topologyBuilder.topologyConfigs(), equalTo(new TopologyConfig(null, globalStreamsConfig, new Properties()))); + assertThat(topologyBuilder.topologyConfigs().cacheSize, equalTo(200L)); + } + + @Test + @SuppressWarnings("deprecation") public void shouldOverrideGlobalStreamsConfigWhenGivenNamedTopologyProps() { final Properties topologyOverrides = new Properties(); - topologyOverrides.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 12345L); + topologyOverrides.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 12345L); topologyOverrides.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L); topologyOverrides.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1000L); topologyOverrides.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15); @@ -969,9 +982,10 @@ public void shouldOverrideGlobalStreamsConfigWhenGivenNamedTopologyProps() { } @Test + @SuppressWarnings("deprecation") public void shouldNotOverrideGlobalStreamsConfigWhenGivenUnnamedTopologyProps() { final Properties streamsProps = StreamsTestUtils.getStreamsConfig(); - streamsProps.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 12345L); + streamsProps.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 12345L); streamsProps.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L); streamsProps.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1000L); streamsProps.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java index 40602b5edf1c..389d0d58c8d7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java @@ -19,9 +19,12 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Value; +import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.IntegerSerializer; @@ -44,6 +47,8 @@ import java.util.List; import java.util.OptionalLong; import java.util.UUID; +import java.util.Collections; +import java.util.Optional; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -83,6 +88,7 @@ public class PartitionGroupTest { private final Metrics metrics = new Metrics(); private final Sensor enforcedProcessingSensor = metrics.sensor(UUID.randomUUID().toString()); private final MetricName lastLatenessValue = new MetricName("record-lateness-last-value", "", "", mkMap()); + private final MetricName totalBytesValue = new MetricName("total-bytes-last-value", "", "", mkMap()); private static Sensor getValueSensor(final Metrics metrics, final MetricName metricName) { @@ -484,6 +490,7 @@ public void shouldUpdatePartitionQueuesExpand() { mkMap(mkEntry(partition1, queue1)), tp -> OptionalLong.of(0L), getValueSensor(metrics, lastLatenessValue), + getValueSensor(metrics, totalBytesValue), enforcedProcessingSensor, maxTaskIdleMs ); @@ -517,6 +524,7 @@ public void shouldUpdatePartitionQueuesShrinkAndExpand() { mkMap(mkEntry(partition1, queue1)), tp -> OptionalLong.of(0L), getValueSensor(metrics, lastLatenessValue), + getValueSensor(metrics, totalBytesValue), enforcedProcessingSensor, maxTaskIdleMs ); @@ -552,6 +560,7 @@ public void shouldNeverWaitIfIdlingIsDisabled() { ), tp -> OptionalLong.of(0L), getValueSensor(metrics, lastLatenessValue), + getValueSensor(metrics, totalBytesValue), enforcedProcessingSensor, StreamsConfig.MAX_TASK_IDLE_MS_DISABLED ); @@ -590,6 +599,7 @@ public void shouldBeReadyIfAllPartitionsAreBuffered() { ), tp -> OptionalLong.of(0L), getValueSensor(metrics, lastLatenessValue), + getValueSensor(metrics, totalBytesValue), enforcedProcessingSensor, 0L ); @@ -629,6 +639,7 @@ public void shouldWaitForFetchesWhenMetadataIsIncomplete() { ), tp -> lags.getOrDefault(tp, OptionalLong.empty()), getValueSensor(metrics, lastLatenessValue), + getValueSensor(metrics, totalBytesValue), enforcedProcessingSensor, 0L ); @@ -665,6 +676,7 @@ public void shouldWaitForPollWhenLagIsNonzero() { ), tp -> lags.getOrDefault(tp, OptionalLong.empty()), getValueSensor(metrics, lastLatenessValue), + getValueSensor(metrics, totalBytesValue), enforcedProcessingSensor, 0L ); @@ -701,6 +713,7 @@ public void shouldIdleAsSpecifiedWhenLagIsZero() { ), tp -> OptionalLong.of(0L), getValueSensor(metrics, lastLatenessValue), + getValueSensor(metrics, totalBytesValue), enforcedProcessingSensor, 1L ); @@ -763,6 +776,93 @@ public void shouldIdleAsSpecifiedWhenLagIsZero() { } } + @Test + public void shouldUpdateTotalBytesBufferedOnRecordsAdditionAndConsumption() { + final PartitionGroup group = getBasicGroup(); + + assertEquals(0, group.numBuffered()); + assertEquals(0L, group.totalBytesBuffered()); + + // add three 3 records with timestamp 1, 5, 3 to partition-1 + final List> list1 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 1L, new MockTime().milliseconds(), TimestampType.CREATE_TIME, recordKey.length, recordValue.length, recordKey, recordValue, new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 5L, new MockTime().milliseconds(), TimestampType.CREATE_TIME, recordKey.length, recordValue.length, recordKey, recordValue, new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 3L, new MockTime().milliseconds(), TimestampType.CREATE_TIME, recordKey.length, recordValue.length, recordKey, recordValue, new RecordHeaders(), Optional.empty())); + + long partition1TotalBytes = getBytesBufferedForRawRecords(list1); + group.addRawRecords(partition1, list1); + + verifyBuffered(3, 3, 0, group); + assertEquals(group.totalBytesBuffered(), partition1TotalBytes); + assertEquals(-1L, group.streamTime()); + assertEquals(0.0, metrics.metric(lastLatenessValue).metricValue()); + assertThat(metrics.metric(totalBytesValue).metricValue(), is((double) partition1TotalBytes)); + + StampedRecord record; + final PartitionGroup.RecordInfo info = new PartitionGroup.RecordInfo(); + + // get first two records from partition 1 + record = group.nextRecord(info, time.milliseconds()); + assertEquals(record.timestamp, 1L); + record = group.nextRecord(info, time.milliseconds()); + assertEquals(record.timestamp, 5L); + + partition1TotalBytes -= getBytesBufferedForRawRecords(Arrays.asList(list1.get(0), list1.get(0))); + assertEquals(group.totalBytesBuffered(), partition1TotalBytes); + assertThat(metrics.metric(totalBytesValue).metricValue(), is((double) partition1TotalBytes)); + + // add three 3 records with timestamp 2, 4, 6 to partition-2 + final List> list2 = Arrays.asList( + new ConsumerRecord<>("topic", 2, 2L, record.timestamp, TimestampType.CREATE_TIME, recordKey.length, recordValue.length, recordKey, recordValue, new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 2, 4L, record.timestamp, TimestampType.CREATE_TIME, recordKey.length, recordValue.length, recordKey, recordValue, new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 2, 6L, record.timestamp, TimestampType.CREATE_TIME, recordKey.length, recordValue.length, recordKey, recordValue, new RecordHeaders(), Optional.empty())); + + long partition2TotalBytes = getBytesBufferedForRawRecords(list2); + group.addRawRecords(partition2, list2); + // 1:[3] + // 2:[2, 4, 6] + assertEquals(group.totalBytesBuffered(), partition2TotalBytes + partition1TotalBytes); + assertThat(metrics.metric(totalBytesValue).metricValue(), is((double) partition2TotalBytes + partition1TotalBytes)); + + // get one record, next record should be ts=2 from partition 2 + record = group.nextRecord(info, time.milliseconds()); + // 1:[3] + // 2:[4, 6] + partition2TotalBytes -= getBytesBufferedForRawRecords(Collections.singletonList(list2.get(0))); + assertEquals(group.totalBytesBuffered(), partition2TotalBytes + partition1TotalBytes); + assertThat(metrics.metric(totalBytesValue).metricValue(), is((double) partition2TotalBytes + partition1TotalBytes)); + assertEquals(record.timestamp, 2L); + + // get one record, next up should have ts=3 from partition 1 (even though it has seen a larger max timestamp =5) + record = group.nextRecord(info, time.milliseconds()); + // 1:[] + // 2:[4, 6] + partition1TotalBytes -= getBytesBufferedForRawRecords(Collections.singletonList(list2.get(2))); + assertEquals(group.totalBytesBuffered(), partition2TotalBytes + partition1TotalBytes); + assertThat(metrics.metric(totalBytesValue).metricValue(), is((double) partition2TotalBytes + partition1TotalBytes)); + assertEquals(record.timestamp, 3L); + } + + private long getBytesBufferedForRawRecords(final List> rawRecords) { + long rawRecordsSizeInBytes = 0L; + for (final ConsumerRecord rawRecord : rawRecords) { + long headerSizeInBytes = 0L; + + for (final Header header: rawRecord.headers().toArray()) { + headerSizeInBytes += header.key().getBytes().length + header.value().length; + } + + rawRecordsSizeInBytes += rawRecord.serializedKeySize() + + rawRecord.serializedValueSize() + + 8L + // timestamp + 8L + // offset + rawRecord.topic().getBytes().length + + 4L + // partition + headerSizeInBytes; + } + return rawRecordsSizeInBytes; + } + private PartitionGroup getBasicGroup() { return new PartitionGroup( logContext, @@ -773,6 +873,7 @@ private PartitionGroup getBasicGroup() { tp -> OptionalLong.of(0L), getValueSensor(metrics, lastLatenessValue), enforcedProcessingSensor, + getValueSensor(metrics, totalBytesValue), maxTaskIdleMs ); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java index 251a2637c43e..7d0e7165ba7c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java @@ -110,7 +110,7 @@ public class RepartitionOptimizingTest { @Before public void setUp() { streamsConfiguration = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); - streamsConfiguration.setProperty(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, Integer.toString(1024 * 10)); + streamsConfiguration.setProperty(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, Integer.toString(1024 * 10)); streamsConfiguration.setProperty(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(5000)); processorValueCollector.clear(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionWithMergeOptimizingTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionWithMergeOptimizingTest.java index b388a6a67eef..bc04505ebfee 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionWithMergeOptimizingTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionWithMergeOptimizingTest.java @@ -86,7 +86,7 @@ public class RepartitionWithMergeOptimizingTest { @Before public void setUp() { streamsConfiguration = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); - streamsConfiguration.setProperty(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, Integer.toString(1024 * 10)); + streamsConfiguration.setProperty(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, Integer.toString(1024 * 10)); streamsConfiguration.setProperty(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(5000)); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 9a66f27df3a5..b6bb3b5e7a3e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -108,7 +108,6 @@ private StreamsConfig createConfig(final File baseDir) throws IOException { return new StreamsConfig(mkProperties(mkMap( mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, applicationId), mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"), - mkEntry(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"), mkEntry(StreamsConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath()), mkEntry(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()) ))); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 71537ae2802a..be99d732a779 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -230,6 +230,7 @@ private static StreamsConfig createConfig(final String eosConfig, final String e return createConfig(eosConfig, enforcedProcessingValue, LogAndFailExceptionHandler.class.getName()); } + @SuppressWarnings("deprecation") private static StreamsConfig createConfig( final String eosConfig, final String enforcedProcessingValue, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index c0cf9a20bd8a..af02e555114a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -122,6 +122,7 @@ import static org.apache.kafka.streams.processor.internals.ClientUtils.getSharedAdminClientId; import static org.apache.kafka.streams.processor.internals.StateManagerUtil.CHECKPOINT_FILE_NAME; import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.anyInt; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.mock; @@ -161,6 +162,7 @@ public class StreamThreadTest { private final StateDirectory stateDirectory = new StateDirectory(config, mockTime, true, false); private final InternalTopologyBuilder internalTopologyBuilder = new InternalTopologyBuilder(); private final InternalStreamsBuilder internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder); + private final long defaultMaxBufferSizeInBytes = 512 * 1024 * 1024; private StreamsMetadataState streamsMetadataState; private final static BiConsumer HANDLER = (e, b) -> { @@ -200,7 +202,6 @@ private Properties configProps(final boolean enableEoS) { return mkProperties(mkMap( mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, APPLICATION_ID), mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"), - mkEntry(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"), mkEntry(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()), mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath()), mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, enableEoS ? StreamsConfig.EXACTLY_ONCE_V2 : StreamsConfig.AT_LEAST_ONCE), @@ -251,6 +252,7 @@ private StreamThread createStreamThread(@SuppressWarnings("SameParameterValue") mockTime, streamsMetadataState, 0, + defaultMaxBufferSizeInBytes, stateDirectory, new MockStateRestoreListener(), threadIdx, @@ -528,6 +530,7 @@ public void shouldEnforceRebalanceAfterNextScheduledProbingRebalanceTime() throw mockTime, streamsMetadataState, 0, + defaultMaxBufferSizeInBytes, stateDirectory, new MockStateRestoreListener(), threadIdx, @@ -1200,7 +1203,8 @@ public void restore(final Map tasks) { new LinkedList<>(), null, HANDLER, - null + null, + defaultMaxBufferSizeInBytes ).updateThreadMetadata(getSharedAdminClientId(CLIENT_ID)); final StreamsException thrown = assertThrows(StreamsException.class, thread::run); @@ -1587,6 +1591,7 @@ public void shouldReturnActiveTaskMetadataWhileRunningState() { mockTime, streamsMetadataState, 0, + defaultMaxBufferSizeInBytes, stateDirectory, new MockStateRestoreListener(), threadIdx, @@ -2237,7 +2242,8 @@ public void shouldCatchHandleCorruptionOnTaskCorruptedExceptionPath() { new LinkedList<>(), null, HANDLER, - null + null, + defaultMaxBufferSizeInBytes ) { @Override void runOnce() { @@ -2304,7 +2310,8 @@ public void shouldCatchTimeoutExceptionFromHandleCorruptionAndInvokeExceptionHan new LinkedList<>(), null, HANDLER, - null + null, + defaultMaxBufferSizeInBytes ) { @Override void runOnce() { @@ -2379,7 +2386,8 @@ public void shouldCatchTaskMigratedExceptionOnOnTaskCorruptedExceptionPath() { new LinkedList<>(), null, HANDLER, - null + null, + defaultMaxBufferSizeInBytes ) { @Override void runOnce() { @@ -2449,7 +2457,8 @@ public void shouldEnforceRebalanceWhenTaskCorruptedExceptionIsThrownForAnActiveT new LinkedList<>(), null, HANDLER, - null + null, + defaultMaxBufferSizeInBytes ) { @Override void runOnce() { @@ -2517,7 +2526,8 @@ public void shouldNotEnforceRebalanceWhenTaskCorruptedExceptionIsThrownForAnInac new LinkedList<>(), null, HANDLER, - null + null, + defaultMaxBufferSizeInBytes ) { @Override void runOnce() { @@ -2670,6 +2680,205 @@ public void shouldLogAndRecordSkippedRecordsForInvalidTimestamps() { } } + @Test + public void shouldPauseNonEmptyPartitionsWhenTotalBufferSizeExceedsMaxBufferSize() { + final Consumer consumer = EasyMock.createNiceMock(Consumer.class); + final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class); + expect(consumer.groupMetadata()).andStubReturn(consumerGroupMetadata); + expect(consumerGroupMetadata.groupInstanceId()).andReturn(Optional.empty()); + + final Map>> records = new HashMap<>(); + final List assignedPartitions = Collections.singletonList(t1p1); + consumer.assign(assignedPartitions); + records.put(t1p1, Collections.singletonList(new ConsumerRecord<>( + t1p1.topic(), + t1p1.partition(), + 1, + mockTime.milliseconds(), + TimestampType.CREATE_TIME, + 2, + 6, + new byte[2], + new byte[6], + new RecordHeaders(), + Optional.empty()))); + expect(consumer.poll(anyObject())).andReturn(new ConsumerRecords<>(records)).anyTimes(); + EasyMock.replay(consumer, consumerGroupMetadata); + final TaskManager taskManager = EasyMock.createNiceMock(TaskManager.class); + + final MetricName testMetricName = new MetricName("test_metric", "", "", new HashMap<>()); + final Metric testMetric = new KafkaMetric( + new Object(), + testMetricName, + (Measurable) (config, now) -> 0, + null, + new MockTime()); + final Map dummyProducerMetrics = singletonMap(testMetricName, testMetric); + + expect(taskManager.producerMetrics()).andReturn(dummyProducerMetrics); + EasyMock.replay(taskManager); + + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); + + final StreamsMetricsImpl streamsMetrics = + new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST, mockTime); + final StreamThread thread = new StreamThread( + mockTime, + config, + null, + consumer, + consumer, + changelogReader, + null, + taskManager, + streamsMetrics, + topologyMetadata, + CLIENT_ID, + new LogContext(""), + new AtomicInteger(), + new AtomicLong(Long.MAX_VALUE), + new LinkedList<>(), + null, + HANDLER, + null, + 10 + ); + thread.setState(StreamThread.State.STARTING); + thread.setState(StreamThread.State.PARTITIONS_ASSIGNED); + thread.pollPhase(); + thread.setState(StreamThread.State.PARTITIONS_REVOKED); + thread.pollPhase(); + EasyMock.reset(consumer); + consumer.pause(anyObject()); + // Consumer.pause should be called only once, when we added the second record. + EasyMock.expectLastCall().times(1); + } + + @Test + public void shouldResumePartitionsAfterConsumptionWhenTotalBufferSizeIsLTEMaxBufferSize() { + final Consumer consumer = EasyMock.createNiceMock(Consumer.class); + final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class); + final ChangelogReader changelogReader = EasyMock.createNiceMock(ChangelogReader.class); + expect(consumer.groupMetadata()).andStubReturn(consumerGroupMetadata); + expect(consumerGroupMetadata.groupInstanceId()).andReturn(Optional.empty()); + changelogReader.restore(anyObject()); + expectLastCall().andVoid(); + + final Task task1 = mock(Task.class); + final Task task2 = mock(Task.class); + + final TaskId taskId1 = new TaskId(0, 1); + final TaskId taskId2 = new TaskId(0, 2); + + expect(task1.state()).andReturn(Task.State.RUNNING).anyTimes(); + expect(task1.id()).andReturn(taskId1).anyTimes(); + expect(task1.inputPartitions()).andReturn(mkSet(t1p1)).anyTimes(); + expect(task1.committedOffsets()).andReturn(new HashMap<>()).anyTimes(); + expect(task1.highWaterMark()).andReturn(new HashMap<>()).anyTimes(); + expect(task1.timeCurrentIdlingStarted()).andReturn(Optional.empty()).anyTimes(); + + expect(task2.state()).andReturn(Task.State.RUNNING).anyTimes(); + expect(task2.id()).andReturn(taskId2).anyTimes(); + expect(task2.inputPartitions()).andReturn(mkSet(t1p2)).anyTimes(); + expect(task2.committedOffsets()).andReturn(new HashMap<>()).anyTimes(); + expect(task2.highWaterMark()).andReturn(new HashMap<>()).anyTimes(); + expect(task2.timeCurrentIdlingStarted()).andReturn(Optional.empty()).anyTimes(); + EasyMock.replay(task1, task2); + + final Map>> records = new HashMap<>(); + records.put(t1p1, Collections.singletonList(new ConsumerRecord<>( + t1p1.topic(), + t1p1.partition(), + 1, + mockTime.milliseconds(), + TimestampType.CREATE_TIME, + 2, + 6, + new byte[2], + new byte[6], + new RecordHeaders(), + Optional.empty()))); + records.put(t1p2, Collections.singletonList(new ConsumerRecord<>( + t1p2.topic(), + t1p2.partition(), + 1, + mockTime.milliseconds(), + TimestampType.CREATE_TIME, + 2, + 6, + new byte[2], + new byte[6], + new RecordHeaders(), + Optional.empty()))); + + final List assignedPartitions = Arrays.asList(t1p1, t1p2); + consumer.assign(assignedPartitions); + expect(consumer.poll(anyObject())).andReturn(new ConsumerRecords<>(records)); + EasyMock.replay(consumer, consumerGroupMetadata); + + final TaskManager taskManager = EasyMock.createNiceMock(TaskManager.class); + + final MetricName testMetricName = new MetricName("test_metric", "", "", new HashMap<>()); + final Metric testMetric = new KafkaMetric( + new Object(), + testMetricName, + (Measurable) (config, now) -> 0, + null, + new MockTime()); + final Map dummyProducerMetrics = singletonMap(testMetricName, testMetric); + expect(taskManager.producerClientIds()).andStubReturn(Collections.emptySet()); + expect(taskManager.producerMetrics()).andReturn(dummyProducerMetrics); + expect(taskManager.activeTaskMap()).andReturn(mkMap( + mkEntry(taskId1, task1), + mkEntry(taskId2, task2) + )); + expect(taskManager.tasks()).andStubReturn(mkMap( + mkEntry(taskId1, task1), + mkEntry(taskId2, task2) + )); + expect(taskManager.standbyTaskMap()).andReturn(new HashMap<>()); + expect(taskManager.commit(anyObject())).andReturn(0); + expect(taskManager.process(anyInt(), anyObject())).andReturn(1); + expect(taskManager.process(anyInt(), anyObject())).andReturn(1); + expect(taskManager.process(anyInt(), anyObject())).andReturn(0); + + EasyMock.replay(taskManager); + + final StreamsMetricsImpl streamsMetrics = + new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST, mockTime); + final StreamThread thread = new StreamThread( + mockTime, + new StreamsConfig(configProps(true)), + null, + consumer, + consumer, + changelogReader, + null, + taskManager, + streamsMetrics, + new TopologyMetadata(internalTopologyBuilder, config), + CLIENT_ID, + new LogContext(""), + new AtomicInteger(), + new AtomicLong(Long.MAX_VALUE), + new LinkedList<>(), + null, + HANDLER, + null, + 6 + ).updateThreadMetadata(getSharedAdminClientId(CLIENT_ID)); + thread.setState(StreamThread.State.STARTING); + thread.setState(StreamThread.State.PARTITIONS_ASSIGNED); + thread.setState(StreamThread.State.RUNNING); + + thread.runOnce(); + EasyMock.reset(consumer); + consumer.resume(anyObject()); + // Consumer.resume should be called only once, when we added the second record. + EasyMock.expectLastCall().times(1); + } + @Test public void shouldTransmitTaskManagerMetrics() { final Consumer consumer = EasyMock.createNiceMock(Consumer.class); @@ -2738,7 +2947,8 @@ public void shouldConstructAdminMetrics() { new LinkedList<>(), null, HANDLER, - null + null, + defaultMaxBufferSizeInBytes ); final MetricName testMetricName = new MetricName("test_metric", "", "", new HashMap<>()); final Metric testMetric = new KafkaMetric( @@ -2795,7 +3005,8 @@ public void runAndVerifyFailedStreamThreadRecording(final boolean shouldFail) { new LinkedList<>(), null, (e, b) -> { }, - null + null, + defaultMaxBufferSizeInBytes ) { @Override void runOnce() { @@ -2930,7 +3141,8 @@ private StreamThread buildStreamThread(final Consumer consumer, new LinkedList<>(), null, HANDLER, - null + null, + defaultMaxBufferSizeInBytes ); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/TaskMetricsTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/TaskMetricsTest.java index 1d33fea1ecc0..cababb1c3190 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/TaskMetricsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/TaskMetricsTest.java @@ -84,6 +84,47 @@ public void shouldGetActiveBufferCountSensor() { assertThat(sensor, is(expectedSensor)); } + @Test + public void shouldGetTotalBytesSensor() { + final String operation = "input-buffer-bytes-total"; + when(streamsMetrics.taskLevelSensor(THREAD_ID, TASK_ID, operation, RecordingLevel.INFO)) + .thenReturn(expectedSensor); + final String totalBytesDescription = "The total number of bytes accumulated in this task's input buffer"; + when(streamsMetrics.taskLevelTagMap(THREAD_ID, TASK_ID)).thenReturn(tagMap); + StreamsMetricsImpl.addValueMetricToSensor( + expectedSensor, + TASK_LEVEL_GROUP, + tagMap, + operation, + totalBytesDescription + ); + + + final Sensor sensor = TaskMetrics.totalBytesSensor(THREAD_ID, TASK_ID, streamsMetrics); + + assertThat(sensor, is(expectedSensor)); + } + + @Test + public void shouldGetTotalCacheSizeInBytesSensor() { + final String operation = "cache-size-bytes-total"; + when(streamsMetrics.taskLevelSensor(THREAD_ID, TASK_ID, operation, RecordingLevel.INFO)) + .thenReturn(expectedSensor); + final String totalBytesDescription = "The total size in bytes of this task's cache."; + when(streamsMetrics.taskLevelTagMap(THREAD_ID, TASK_ID)).thenReturn(tagMap); + StreamsMetricsImpl.addValueMetricToSensor( + expectedSensor, + TASK_LEVEL_GROUP, + tagMap, + operation, + totalBytesDescription + ); + + final Sensor sensor = TaskMetrics.totalCacheSizeBytesSensor(THREAD_ID, TASK_ID, streamsMetrics); + + assertThat(sensor, is(expectedSensor)); + } + @Test public void shouldGetProcessLatencySensor() { final String operation = "process-latency"; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java index 6d43b4cfc8b9..4d2022ca5af6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeader; @@ -32,6 +33,10 @@ import java.util.Arrays; import java.util.List; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -43,11 +48,12 @@ public class NamedCacheTest { private final Headers headers = new RecordHeaders(new Header[]{new RecordHeader("key", "value".getBytes())}); private NamedCache cache; + private final Metrics innerMetrics = new Metrics(); + private final StreamsMetricsImpl metrics = new MockStreamsMetrics(innerMetrics); + private final MetricName cacheSizeBytesTotal = new MetricName("cache-size-bytes-total", "stream-task-metrics", "", mkMap(mkEntry("thread-id", "Test worker"), mkEntry("task-id", "dummy"))); @Before public void setUp() { - final Metrics innerMetrics = new Metrics(); - final StreamsMetricsImpl metrics = new MockStreamsMetrics(innerMetrics); cache = new NamedCache("dummy-name", metrics); } @@ -82,6 +88,7 @@ public void shouldKeepTrackOfSize() { cache.put(Bytes.wrap(new byte[]{1}), value); cache.put(Bytes.wrap(new byte[]{2}), value); final long size = cache.sizeInBytes(); + assertThat(metrics.metrics().get(cacheSizeBytesTotal).metricValue(), is((double) size)); // 1 byte key + 24 bytes overhead assertEquals((value.size() + 25) * 3, size); } @@ -114,6 +121,7 @@ public void shouldDeleteAndUpdateSize() { final LRUCacheEntry deleted = cache.delete(Bytes.wrap(new byte[]{0})); assertArrayEquals(new byte[] {10}, deleted.value()); assertEquals(0, cache.sizeInBytes()); + assertThat(metrics.metrics().get(cacheSizeBytesTotal).metricValue(), is((double) 0)); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java index 8a402ab9abf5..1e06b4ab3141 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java @@ -72,7 +72,7 @@ public static void main(final String[] args) throws IOException { streamsProperties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); streamsProperties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); streamsProperties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); - streamsProperties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsProperties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsProperties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingMode); final int timeout = 6000; streamsProperties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), timeout); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java index 3b1aa4478f3f..ae9d7527d7ac 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java @@ -106,7 +106,7 @@ private KafkaStreams createKafkaStreams(final Properties props) { props.put(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, Duration.ofMinutes(1).toMillis()); props.put(StreamsConfig.MAX_WARMUP_REPLICAS_CONFIG, Integer.MAX_VALUE); props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3); - props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 5000L); // increase commit interval to make sure a client is killed having an open transaction props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsNamedRepartitionTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsNamedRepartitionTest.java index b98f86141a03..af3614c73269 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsNamedRepartitionTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsNamedRepartitionTest.java @@ -87,7 +87,7 @@ public static void main(final String[] args) throws Exception { final Properties config = new Properties(); config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsNamedRepartitionTest"); - config.setProperty(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, "0"); + config.setProperty(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, "0"); config.setProperty(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); config.setProperty(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsOptimizedTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsOptimizedTest.java index 714aa110ef3e..95945b1b4462 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsOptimizedTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsOptimizedTest.java @@ -110,7 +110,7 @@ public static void main(final String[] args) throws Exception { config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsOptimizedTest"); - config.setProperty(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, "0"); + config.setProperty(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, "0"); config.setProperty(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); config.setProperty(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); config.setProperty(StreamsConfig.adminClientPrefix(AdminClientConfig.RETRIES_CONFIG), "100"); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java index 3c693cc71353..2568b498c970 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java @@ -67,7 +67,7 @@ public static void main(final String[] args) throws IOException { streamsProperties.put(StreamsConfig.APPLICATION_ID_CONFIG, "kafka-streams-standby-tasks"); streamsProperties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); streamsProperties.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); - streamsProperties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsProperties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsProperties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); streamsProperties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); streamsProperties.put(StreamsConfig.producerPrefix(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG), true); diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index dba1cbca1132..d438a44f37b8 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -155,7 +155,7 @@ * *

Note that the {@code TopologyTestDriver} processes input records synchronously. * This implies that {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit.interval.ms} and - * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache.max.bytes.buffering} configuration have no effect. + * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache.max.bytes.buffering} configuration have no effect. * The driver behaves as if both configs would be set to zero, i.e., as if a "commit" (and thus "flush") would happen * after each input record. * @@ -309,6 +309,7 @@ public TopologyTestDriver(final Topology topology, * @param config the configuration for the topology * @param initialWallClockTimeMs the initial value of internally mocked wall-clock time */ + @SuppressWarnings({"unchecked", "deprecation"}) private TopologyTestDriver(final InternalTopologyBuilder builder, final Properties config, final long initialWallClockTimeMs) { @@ -329,7 +330,7 @@ private TopologyTestDriver(final InternalTopologyBuilder builder, final ThreadCache cache = new ThreadCache( logContext, - Math.max(0, streamsConfig.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG)), + Math.max(0, streamsConfig.getTotalCacheSize()), streamsMetrics ); From be4ef3df428b9d783710a8779d5eadc80e10e299 Mon Sep 17 00:00:00 2001 From: Xiaobing Fang Date: Tue, 22 Mar 2022 16:31:46 +0800 Subject: [PATCH 035/447] KAFKA-13752: Uuid compare using equals in java (#11912) This patch fixes a few cases where we use `==` instead of `equals` to compare UUID. The impact of this bug is low because `Uuid.ZERO_UUID` is used by default everywhere. Reviewers: Justine Olshan , dengziming , David Jacot --- .../common/requests/MetadataRequest.java | 2 +- .../common/requests/MetadataResponse.java | 2 +- .../common/requests/MetadataRequestTest.java | 28 ++++++-- .../common/requests/MetadataResponseTest.java | 66 +++++++++++++++++++ .../requests/UpdateMetadataRequestTest.java | 2 +- 5 files changed, 90 insertions(+), 10 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/common/requests/MetadataResponseTest.java diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java index aab5fc684026..48609b1666c6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -112,7 +112,7 @@ public MetadataRequest build(short version) { if (topic.name() == null && version < 12) throw new UnsupportedVersionException("MetadataRequest version " + version + " does not support null topic names."); - if (topic.topicId() != Uuid.ZERO_UUID && version < 12) + if (!Uuid.ZERO_UUID.equals(topic.topicId()) && version < 12) throw new UnsupportedVersionException("MetadataRequest version " + version + " does not support non-zero topic IDs."); }); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index d539fa871982..3696b047abad 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -151,7 +151,7 @@ public Cluster buildCluster() { if (metadata.error == Errors.NONE) { if (metadata.isInternal) internalTopics.add(metadata.topic); - if (metadata.topicId() != null && metadata.topicId() != Uuid.ZERO_UUID) { + if (metadata.topicId() != null && !Uuid.ZERO_UUID.equals(metadata.topicId())) { topicIds.put(metadata.topic, metadata.topicId()); } for (PartitionMetadata partitionMetadata : metadata.partitionMetadata) { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/MetadataRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/MetadataRequestTest.java index 74c217df91f8..84764c29da6e 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/MetadataRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/MetadataRequestTest.java @@ -26,6 +26,7 @@ import java.util.Collections; import java.util.List; +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.assertNull; @@ -82,12 +83,25 @@ public void testTopicIdAndNullTopicNameRequests() { // if version is 10 or 11, the invalid topic metadata should return an error List invalidVersions = Arrays.asList((short) 10, (short) 11); - invalidVersions.forEach(version -> - topics.forEach(topic -> { - MetadataRequestData metadataRequestData = new MetadataRequestData().setTopics(Collections.singletonList(topic)); - MetadataRequest.Builder builder = new MetadataRequest.Builder(metadataRequestData); - assertThrows(UnsupportedVersionException.class, () -> builder.build(version)); - }) - ); + invalidVersions.forEach(version -> topics.forEach(topic -> { + MetadataRequestData metadataRequestData = new MetadataRequestData().setTopics(Collections.singletonList(topic)); + MetadataRequest.Builder builder = new MetadataRequest.Builder(metadataRequestData); + assertThrows(UnsupportedVersionException.class, () -> builder.build(version)); + })); + } + + @Test + public void testTopicIdWithZeroUuid() { + List topics = Arrays.asList( + new MetadataRequestData.MetadataRequestTopic().setName("topic").setTopicId(Uuid.ZERO_UUID), + new MetadataRequestData.MetadataRequestTopic().setName("topic").setTopicId(new Uuid(0L, 0L)), + new MetadataRequestData.MetadataRequestTopic().setName("topic")); + + List invalidVersions = Arrays.asList((short) 10, (short) 11); + invalidVersions.forEach(version -> topics.forEach(topic -> { + MetadataRequestData metadataRequestData = new MetadataRequestData().setTopics(Collections.singletonList(topic)); + MetadataRequest.Builder builder = new MetadataRequest.Builder(metadataRequestData); + assertDoesNotThrow(() -> builder.build(version)); + })); } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/MetadataResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/MetadataResponseTest.java new file mode 100644 index 000000000000..37f7356c6969 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/requests/MetadataResponseTest.java @@ -0,0 +1,66 @@ +/* + * 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.common.requests; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.MetadataResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.junit.jupiter.api.Test; + +import static java.util.Collections.emptyList; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + +public class MetadataResponseTest { + + @Test + void buildClusterTest() { + Uuid zeroUuid = new Uuid(0L, 0L); + Uuid randomUuid = Uuid.randomUuid(); + MetadataResponseData.MetadataResponseTopic topicMetadata1 = new MetadataResponseData.MetadataResponseTopic() + .setName("topic1") + .setErrorCode(Errors.NONE.code()) + .setPartitions(emptyList()) + .setIsInternal(false); + MetadataResponseData.MetadataResponseTopic topicMetadata2 = new MetadataResponseData.MetadataResponseTopic() + .setName("topic2") + .setErrorCode(Errors.NONE.code()) + .setTopicId(zeroUuid) + .setPartitions(emptyList()) + .setIsInternal(false); + MetadataResponseData.MetadataResponseTopic topicMetadata3 = new MetadataResponseData.MetadataResponseTopic() + .setName("topic3") + .setErrorCode(Errors.NONE.code()) + .setTopicId(randomUuid) + .setPartitions(emptyList()) + .setIsInternal(false); + + MetadataResponseData.MetadataResponseTopicCollection topics = + new MetadataResponseData.MetadataResponseTopicCollection(); + topics.add(topicMetadata1); + topics.add(topicMetadata2); + topics.add(topicMetadata3); + MetadataResponse metadataResponse = new MetadataResponse(new MetadataResponseData().setTopics(topics), + ApiKeys.METADATA.latestVersion()); + Cluster cluster = metadataResponse.buildCluster(); + assertNull(cluster.topicName(Uuid.ZERO_UUID)); + assertNull(cluster.topicName(zeroUuid)); + assertEquals("topic3", cluster.topicName(randomUuid)); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/requests/UpdateMetadataRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/UpdateMetadataRequestTest.java index 6f9d5c245460..2dd17f776ec9 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/UpdateMetadataRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/UpdateMetadataRequestTest.java @@ -203,7 +203,7 @@ public void testVersionLogic() { long topicIdCount = deserializedRequest.data().topicStates().stream() .map(UpdateMetadataRequestData.UpdateMetadataTopicState::topicId) - .filter(topicId -> topicId != Uuid.ZERO_UUID).count(); + .filter(topicId -> !Uuid.ZERO_UUID.equals(topicId)).count(); if (version >= 7) assertEquals(2, topicIdCount); else From eddb98df67e792b87719636b33c19b0bc5934ffa Mon Sep 17 00:00:00 2001 From: Idan Kamara Date: Tue, 22 Mar 2022 10:45:04 +0200 Subject: [PATCH 036/447] MINOR: Fix class comparison in `AlterConfigPolicy.RequestMetadata.equals()` (#11900) This patch fixes a bug in the `AlterConfigPolicy.RequestMetadata.equals` method where we were not comparing the class correctly. Co-authored-by: David Jacot Reviewers: David Jacot --- checkstyle/import-control.xml | 3 ++ .../server/policy/AlterConfigPolicy.java | 2 +- .../server/policy/AlterConfigPolicyTest.java | 51 +++++++++++++++++++ 3 files changed, 55 insertions(+), 1 deletion(-) create mode 100644 clients/src/test/java/org/apache/kafka/server/policy/AlterConfigPolicyTest.java diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 3b8a78da205b..ac2da909bab4 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -310,6 +310,9 @@ + + + diff --git a/clients/src/main/java/org/apache/kafka/server/policy/AlterConfigPolicy.java b/clients/src/main/java/org/apache/kafka/server/policy/AlterConfigPolicy.java index 5710a6011aae..7f2c4905c9a7 100644 --- a/clients/src/main/java/org/apache/kafka/server/policy/AlterConfigPolicy.java +++ b/clients/src/main/java/org/apache/kafka/server/policy/AlterConfigPolicy.java @@ -71,7 +71,7 @@ public int hashCode() { @Override public boolean equals(Object o) { - if (o == null || o.getClass() != o.getClass()) return false; + if ((o == null) || (!o.getClass().equals(getClass()))) return false; RequestMetadata other = (RequestMetadata) o; return resource.equals(other.resource) && configs.equals(other.configs); diff --git a/clients/src/test/java/org/apache/kafka/server/policy/AlterConfigPolicyTest.java b/clients/src/test/java/org/apache/kafka/server/policy/AlterConfigPolicyTest.java new file mode 100644 index 000000000000..13971effe9ec --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/server/policy/AlterConfigPolicyTest.java @@ -0,0 +1,51 @@ +/* + * 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.policy; + +import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.config.ConfigResource.Type; +import org.apache.kafka.server.policy.AlterConfigPolicy.RequestMetadata; + +import org.junit.jupiter.api.Test; +import java.util.Collections; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; + +public class AlterConfigPolicyTest { + + @Test + public void testRequestMetadataEquals() { + RequestMetadata requestMetadata = new RequestMetadata( + new ConfigResource(Type.BROKER, "0"), + Collections.singletonMap("foo", "bar") + ); + + assertEquals(requestMetadata, requestMetadata); + + assertNotEquals(requestMetadata, null); + assertNotEquals(requestMetadata, new Object()); + assertNotEquals(requestMetadata, new RequestMetadata( + new ConfigResource(Type.BROKER, "1"), + Collections.singletonMap("foo", "bar") + )); + assertNotEquals(requestMetadata, new RequestMetadata( + new ConfigResource(Type.BROKER, "0"), + Collections.emptyMap() + )); + } +} From a44e1ed449750f8d0d0aadd1fb37856158a88aa8 Mon Sep 17 00:00:00 2001 From: xuexiaoyue <2821566237@qq.com> Date: Tue, 22 Mar 2022 20:55:54 +0800 Subject: [PATCH 037/447] MINOR: Fix typos in `TransactionManager` (#11924) Reviewers: Kvicii , David Jacot --- .../kafka/clients/producer/internals/TransactionManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index be881a972551..4362ad68af3d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -206,8 +206,8 @@ void resetSequenceNumbers(Consumer resetSequence) { private final Map pendingTxnOffsetCommits; - // If a batch bound for a partition expired locally after being sent at least once, the partition has is considered - // to have an unresolved state. We keep track fo such partitions here, and cannot assign any more sequence numbers + // If a batch bound for a partition expired locally after being sent at least once, the partition is considered + // to have an unresolved state. We keep track of such partitions here, and cannot assign any more sequence numbers // for this partition until the unresolved state gets cleared. This may happen if other inflight batches returned // successfully (indicating that the expired batch actually made it to the broker). If we don't get any successful // responses for the partition once the inflight request count falls to zero, we reset the producer id and From b60f4464acaa540de75ef2fc83a911d4ff6a8786 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 22 Mar 2022 17:19:29 -0700 Subject: [PATCH 038/447] Revert "KAFKA-7077: Use default producer settings in Connect Worker (#11475)" (#11932) This reverts commit 76cf7a5793702b55e2cfd98a375f8f1708ff32c3. Connect already allows users to enable idempotent producers for connectors and the Connect workers. Although Kafka producers enabled idempotency by default in 3.0, due to compatibility requirements and the fact that [KIP-318](https://cwiki.apache.org/confluence/display/KAFKA/KIP-318%3A+Make+Kafka+Connect+Source+idempotent) hasn't been explicitly approved, the changes here are reverted. A separate commit will explicitly disable idempotency in producers instantiated by Connect by default until KIP-318 is approved and scheduled for release. --- .../src/main/java/org/apache/kafka/connect/runtime/Worker.java | 2 ++ .../test/java/org/apache/kafka/connect/runtime/WorkerTest.java | 2 ++ 2 files changed, 4 insertions(+) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index 45c7d9f6d13a..582271a19633 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -648,6 +648,8 @@ static Map producerConfigs(ConnectorTaskId id, // These settings will execute infinite retries on retriable exceptions. They *may* be overridden via configs passed to the worker, // but this may compromise the delivery guarantees of Kafka Connect. producerProps.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.toString(Long.MAX_VALUE)); + producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); + producerProps.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1"); producerProps.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, Integer.toString(Integer.MAX_VALUE)); producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, defaultClientId); // User-specified overrides diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index e57e5e33fea8..2b210794207a 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -213,6 +213,8 @@ public void setup() { defaultProducerConfigs.put( ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); defaultProducerConfigs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.toString(Long.MAX_VALUE)); + defaultProducerConfigs.put(ProducerConfig.ACKS_CONFIG, "all"); + defaultProducerConfigs.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1"); defaultProducerConfigs.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, Integer.toString(Integer.MAX_VALUE)); defaultConsumerConfigs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); From a3adf41d8b90d1244232e448b959db3b3f4dc2fe Mon Sep 17 00:00:00 2001 From: Hao Li <1127478+lihaosky@users.noreply.github.com> Date: Tue, 22 Mar 2022 20:53:53 -0700 Subject: [PATCH 039/447] [Emit final][4/N] add time ordered store factory (#11892) Add factory to create time ordered store supplier. Reviewers: Guozhang Wang --- ...edTimeOrderedWindowBytesStoreSupplier.java | 37 +++++++++ ...meOrderedWindowBytesStoreSupplierTest.java | 75 +++++++++++++++++++ 2 files changed, 112 insertions(+) create mode 100644 streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDbIndexedTimeOrderedWindowBytesStoreSupplierTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbIndexedTimeOrderedWindowBytesStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbIndexedTimeOrderedWindowBytesStoreSupplier.java index 84d8a80f6c48..af5417fccf94 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbIndexedTimeOrderedWindowBytesStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbIndexedTimeOrderedWindowBytesStoreSupplier.java @@ -16,6 +16,11 @@ */ package org.apache.kafka.streams.state.internals; +import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; +import static org.apache.kafka.streams.internals.ApiUtils.validateMillisecondDuration; + +import java.time.Duration; +import java.util.Objects; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.state.WindowBytesStoreSupplier; import org.apache.kafka.streams.state.WindowStore; @@ -33,6 +38,38 @@ public enum WindowStoreTypes { private final boolean retainDuplicates; private final WindowStoreTypes windowStoreType; + public static RocksDbIndexedTimeOrderedWindowBytesStoreSupplier create(final String name, + final Duration retentionPeriod, + final Duration windowSize, + final boolean retainDuplicates, + final boolean hasIndex) { + Objects.requireNonNull(name, "name cannot be null"); + final String rpMsgPrefix = prepareMillisCheckFailMsgPrefix(retentionPeriod, "retentionPeriod"); + final long retentionMs = validateMillisecondDuration(retentionPeriod, rpMsgPrefix); + final String wsMsgPrefix = prepareMillisCheckFailMsgPrefix(windowSize, "windowSize"); + final long windowSizeMs = validateMillisecondDuration(windowSize, wsMsgPrefix); + + final long defaultSegmentInterval = Math.max(retentionMs / 2, 60_000L); + + if (retentionMs < 0L) { + throw new IllegalArgumentException("retentionPeriod cannot be negative"); + } + if (windowSizeMs < 0L) { + throw new IllegalArgumentException("windowSize cannot be negative"); + } + if (defaultSegmentInterval < 1L) { + throw new IllegalArgumentException("segmentInterval cannot be zero or negative"); + } + if (windowSizeMs > retentionMs) { + throw new IllegalArgumentException("The retention period of the window store " + + name + " must be no smaller than its window size. Got size=[" + + windowSizeMs + "], retention=[" + retentionMs + "]"); + } + + return new RocksDbIndexedTimeOrderedWindowBytesStoreSupplier(name, retentionMs, + defaultSegmentInterval, windowSizeMs, retainDuplicates, hasIndex); + } + public RocksDbIndexedTimeOrderedWindowBytesStoreSupplier(final String name, final long retentionPeriod, final long segmentInterval, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDbIndexedTimeOrderedWindowBytesStoreSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDbIndexedTimeOrderedWindowBytesStoreSupplierTest.java new file mode 100644 index 000000000000..ed1bbb8fd4a3 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDbIndexedTimeOrderedWindowBytesStoreSupplierTest.java @@ -0,0 +1,75 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.WindowStore; +import org.junit.Test; + +import static java.time.Duration.ZERO; +import static java.time.Duration.ofMillis; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +public class RocksDbIndexedTimeOrderedWindowBytesStoreSupplierTest { + + @Test + public void shouldThrowIfStoreNameIsNull() { + final Exception e = assertThrows(NullPointerException.class, () -> RocksDbIndexedTimeOrderedWindowBytesStoreSupplier.create(null, ZERO, ZERO, false, false)); + assertEquals("name cannot be null", e.getMessage()); + } + + @Test + public void shouldThrowIfRetentionPeriodIsNegative() { + final Exception e = assertThrows(IllegalArgumentException.class, () -> RocksDbIndexedTimeOrderedWindowBytesStoreSupplier.create("anyName", ofMillis(-1L), ZERO, false, false)); + assertEquals("retentionPeriod cannot be negative", e.getMessage()); + } + + @Test + public void shouldThrowIfWindowSizeIsNegative() { + final Exception e = assertThrows(IllegalArgumentException.class, () -> RocksDbIndexedTimeOrderedWindowBytesStoreSupplier.create("anyName", ofMillis(0L), ofMillis(-1L), false, false)); + assertEquals("windowSize cannot be negative", e.getMessage()); + } + + @Test + public void shouldThrowIfWindowSizeIsLargerThanRetention() { + final Exception e = assertThrows(IllegalArgumentException.class, () -> RocksDbIndexedTimeOrderedWindowBytesStoreSupplier.create("anyName", ofMillis(1L), ofMillis(2L), false, false)); + assertEquals("The retention period of the window store anyName must be no smaller than its window size. Got size=[2], retention=[1]", e.getMessage()); + } + + @Test + public void shouldCreateRocksDbTimeOrderedWindowStoreWithIndex() { + final WindowStore store = RocksDbIndexedTimeOrderedWindowBytesStoreSupplier.create("store", ofMillis(1L), ofMillis(1L), false, true).get(); + final StateStore wrapped = ((WrappedStateStore) store).wrapped(); + assertThat(store, instanceOf(RocksDBTimeOrderedWindowStore.class)); + assertThat(wrapped, instanceOf(RocksDBTimeOrderedSegmentedBytesStore.class)); + assertTrue(((RocksDBTimeOrderedSegmentedBytesStore) wrapped).hasIndex()); + } + + @Test + public void shouldCreateRocksDbTimeOrderedWindowStoreWithoutIndex() { + final WindowStore store = RocksDbIndexedTimeOrderedWindowBytesStoreSupplier.create("store", ofMillis(1L), ofMillis(1L), false, false).get(); + final StateStore wrapped = ((WrappedStateStore) store).wrapped(); + assertThat(store, instanceOf(RocksDBTimeOrderedWindowStore.class)); + assertThat(wrapped, instanceOf(RocksDBTimeOrderedSegmentedBytesStore.class)); + assertFalse(((RocksDBTimeOrderedSegmentedBytesStore) wrapped).hasIndex()); + } +} From dcd09de1ed84b43f269eb32fc2baf589a791d468 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Wed, 23 Mar 2022 13:07:02 -0500 Subject: [PATCH 040/447] MINOR: Clarify how to publish specific projects to the local repo (#11938) The current README instruction for local publishing boils the ocean by building and installing every jar in the project with both 2.12 and 2.13. While that is some times what people want to do, they are also often trying to just build a specific jar. Reviewers: Bill Bejeck --- README.md | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 5e409f8dada6..34b50f535ff6 100644 --- a/README.md +++ b/README.md @@ -180,8 +180,8 @@ Please note for this to work you should create/update user maven settings (typic ... -### Installing the jars to the local Maven repository ### -The recommended command is: +### Installing ALL the jars to the local Maven repository ### +The recommended command to build for both Scala 2.12 and 2.13 is: ./gradlewAll publishToMavenLocal @@ -189,6 +189,12 @@ For backwards compatibility, the following also works: ./gradlewAll install +### Installing specific projects to the local Maven repository ### + + ./gradlew -PskipSigning :streams:publishToMavenLocal + +If needed, you can specify the Scala version with `-PscalaVersion=2.13`. + ### Building the test jar ### ./gradlew testJar From 6ce69021fd1daa7bf39a9796152aff260b633b3a Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Wed, 23 Mar 2022 15:03:52 -0700 Subject: [PATCH 041/447] KAFKA-13759: Disable idempotence by default in producers instantiated by Connect (#11933) With AK 3.0, idempotence was enabled by default in Kafka producers. However, if idempotence is enabled, Connect won't be able to communicate via its producers with Kafka brokers older than version 0.11. Perhaps more importantly, for brokers older than version 2.8 the IDEMPOTENT_WRITE ACL is required to be granted to the principal of the Connect worker. Therefore this commit disables producer idempotence by default to all the producers instantiated by Connect. Users can still choose to enable producer idempotence by explicitly setting the right worker and/or connector properties. The changes were tested via existing unit, integration and system tests. Reviewers: Randall Hauch --- .../java/org/apache/kafka/connect/runtime/Worker.java | 6 ++++++ .../kafka/connect/storage/KafkaConfigBackingStore.java | 6 ++++++ .../kafka/connect/storage/KafkaOffsetBackingStore.java | 6 ++++++ .../kafka/connect/storage/KafkaStatusBackingStore.java | 7 ++++++- .../org/apache/kafka/connect/runtime/WorkerTest.java | 6 ++++++ docs/upgrade.html | 10 ++++++++++ 6 files changed, 40 insertions(+), 1 deletion(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index 582271a19633..4adf6ff5e041 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -648,6 +648,12 @@ static Map producerConfigs(ConnectorTaskId id, // These settings will execute infinite retries on retriable exceptions. They *may* be overridden via configs passed to the worker, // but this may compromise the delivery guarantees of Kafka Connect. producerProps.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.toString(Long.MAX_VALUE)); + // By default, Connect disables idempotent behavior for all producers, even though idempotence became + // default for Kafka producers. This is to ensure Connect continues to work with many Kafka broker versions, including older brokers that do not support + // idempotent producers or require explicit steps to enable them (e.g. adding the IDEMPOTENT_WRITE ACL to brokers older than 2.8). + // These settings might change when https://cwiki.apache.org/confluence/display/KAFKA/KIP-318%3A+Make+Kafka+Connect+Source+idempotent + // gets approved and scheduled for release. + producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false"); producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); producerProps.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1"); producerProps.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, Integer.toString(Integer.MAX_VALUE)); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java index 669c72b224df..94b98cb9eb38 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java @@ -511,6 +511,12 @@ KafkaBasedLog setupAndCreateKafkaBasedLog(String topic, final Wo producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); producerProps.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, Integer.MAX_VALUE); + // By default, Connect disables idempotent behavior for all producers, even though idempotence became + // default for Kafka producers. This is to ensure Connect continues to work with many Kafka broker versions, including older brokers that do not support + // idempotent producers or require explicit steps to enable them (e.g. adding the IDEMPOTENT_WRITE ACL to brokers older than 2.8). + // These settings might change when https://cwiki.apache.org/confluence/display/KAFKA/KIP-318%3A+Make+Kafka+Connect+Source+idempotent + // gets approved and scheduled for release. + producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false"); ConnectUtils.addMetricsContextProperties(producerProps, config, clusterId); Map consumerProps = new HashMap<>(originals); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java index 313baf72c58c..f3cbb686fff5 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java @@ -91,6 +91,12 @@ public void configure(final WorkerConfig config) { producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); producerProps.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, Integer.MAX_VALUE); + // By default, Connect disables idempotent behavior for all producers, even though idempotence became + // default for Kafka producers. This is to ensure Connect continues to work with many Kafka broker versions, including older brokers that do not support + // idempotent producers or require explicit steps to enable them (e.g. adding the IDEMPOTENT_WRITE ACL to brokers older than 2.8). + // These settings might change when https://cwiki.apache.org/confluence/display/KAFKA/KIP-318%3A+Make+Kafka+Connect+Source+idempotent + // gets approved and scheduled for release. + producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false"); ConnectUtils.addMetricsContextProperties(producerProps, config, clusterId); Map consumerProps = new HashMap<>(originals); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java index c2aeba808012..3ba6996da8ab 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java @@ -170,7 +170,12 @@ public void configure(final WorkerConfig config) { producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); producerProps.put(ProducerConfig.RETRIES_CONFIG, 0); // we handle retries in this class - producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, false); // disable idempotence since retries is force to 0 + // By default, Connect disables idempotent behavior for all producers, even though idempotence became + // default for Kafka producers. This is to ensure Connect continues to work with many Kafka broker versions, including older brokers that do not support + // idempotent producers or require explicit steps to enable them (e.g. adding the IDEMPOTENT_WRITE ACL to brokers older than 2.8). + // These settings might change when https://cwiki.apache.org/confluence/display/KAFKA/KIP-318%3A+Make+Kafka+Connect+Source+idempotent + // gets approved and scheduled for release. + producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false"); // disable idempotence since retries is force to 0 ConnectUtils.addMetricsContextProperties(producerProps, config, clusterId); Map consumerProps = new HashMap<>(originals); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index 2b210794207a..dcd9286480e8 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -213,6 +213,12 @@ public void setup() { defaultProducerConfigs.put( ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); defaultProducerConfigs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.toString(Long.MAX_VALUE)); + // By default, producers that are instantiated and used by Connect have idempotency disabled even after idempotency became + // default for Kafka producers. This is chosen to avoid breaking changes when Connect contacts Kafka brokers that do not support + // idempotent producers or require explicit steps to enable them (e.g. adding the IDEMPOTENT_WRITE ACL to brokers older than 2.8). + // These settings might change when https://cwiki.apache.org/confluence/display/KAFKA/KIP-318%3A+Make+Kafka+Connect+Source+idempotent + // gets approved and scheduled for release. + defaultProducerConfigs.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false"); defaultProducerConfigs.put(ProducerConfig.ACKS_CONFIG, "all"); defaultProducerConfigs.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1"); defaultProducerConfigs.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, Integer.toString(Integer.MAX_VALUE)); diff --git a/docs/upgrade.html b/docs/upgrade.html index ddcee32053eb..e7700272b39c 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -25,6 +25,11 @@

Notable changes in 3 which meant that idempotence remained disabled unless the user had explicitly set enable.idempotence to true (See KAFKA-13598for more details). This issue was fixed and the default is properly applied in 3.0.1, 3.1.1, and 3.2.0. +
  • A notable exception is Connect that by default disables idempotent behavior for all of its + producers in order to uniformly support using a wide range of Kafka broker versions. + Users can change this behavior to enable idempotence for some or all producers + via Connect worker and/or connector configuration. Connect may enable idempotent producers + by default in a future major release.
  • Upgrading to 3.1.0 from any version 0.8.x through 3.0.x

    @@ -75,6 +80,11 @@
    Notable changes in 3 A bug prevented the producer idempotence default from being applied which meant that it remained disabled unless the user had explicitly set enable.idempotence to true. See KAFKA-13598for more details. This issue was fixed and the default is properly applied. +
  • A notable exception is Connect that by default disables idempotent behavior for all of its + producers in order to uniformly support using a wide range of Kafka broker versions. + Users can change this behavior to enable idempotence for some or all producers + via Connect worker and/or connector configuration. Connect may enable idempotent producers + by default in a future major release.
  • Notable changes in 3.1.0
    From 322a065b9055649c713baf43f154052d45cd1588 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Wed, 23 Mar 2022 22:09:05 -0500 Subject: [PATCH 042/447] KAFKA-13714: Fix cache flush position (#11926) The caching store layers were passing down writes into lower store layers upon eviction, but not setting the context to the evicted records' context. Instead, the context was from whatever unrelated record was being processed at the time. Reviewers: Matthias J. Sax --- README.md | 9 ++- build.gradle | 12 ++- .../state/internals/CachingKeyValueStore.java | 13 +++- .../state/internals/CachingSessionStore.java | 12 ++- .../state/internals/CachingWindowStore.java | 12 ++- .../streams/state/internals/RocksDBStore.java | 2 +- .../state/internals/StoreQueryUtils.java | 4 +- .../integration/IQv2StoreIntegrationTest.java | 73 ++++++++++--------- .../CachingInMemoryKeyValueStoreTest.java | 39 ++++++++-- .../CachingInMemorySessionStoreTest.java | 43 +++++++++++ .../CachingPersistentSessionStoreTest.java | 45 +++++++++++- .../CachingPersistentWindowStoreTest.java | 43 +++++++++++ 12 files changed, 250 insertions(+), 57 deletions(-) diff --git a/README.md b/README.md index 34b50f535ff6..2aa509e08101 100644 --- a/README.md +++ b/README.md @@ -37,13 +37,16 @@ Follow instructions in https://kafka.apache.org/quickstart ./gradlew integrationTest ### Force re-running tests without code change ### - ./gradlew cleanTest test - ./gradlew cleanTest unitTest - ./gradlew cleanTest integrationTest + ./gradlew -Prerun-tests test + ./gradlew -Prerun-tests unitTest + ./gradlew -Prerun-tests integrationTest ### Running a particular unit/integration test ### ./gradlew clients:test --tests RequestResponseTest +### Repeatedly running a particular unit/integration test ### + I=0; while ./gradlew clients:test -Prerun-tests --tests RequestResponseTest --fail-fast; do (( I=$I+1 )); echo "Completed run: $I"; sleep 1; done + ### Running a particular test method within a unit/integration test ### ./gradlew core:test --tests kafka.api.ProducerFailureHandlingTest.testCannotSendToInternalTopic ./gradlew clients:test --tests org.apache.kafka.clients.MetadataTest.testTimeToNextUpdate diff --git a/build.gradle b/build.gradle index ff4bab922c5e..3363a43ac70d 100644 --- a/build.gradle +++ b/build.gradle @@ -207,7 +207,7 @@ if (file('.git').exists()) { } else { rat.enabled = false } -println("Starting build with version $version (commit id ${commitId.take(8)}) using Gradle $gradleVersion, Java ${JavaVersion.current()} and Scala ${versions.scala}") +println("Starting build with version $version (commit id ${commitId == null ? "null" : commitId.take(8)}) using Gradle $gradleVersion, Java ${JavaVersion.current()} and Scala ${versions.scala}") println("Build properties: maxParallelForks=$maxTestForks, maxScalacThreads=$maxScalacThreads, maxTestRetries=$userMaxTestRetries") subprojects { @@ -435,6 +435,11 @@ subprojects { maxRetries = userMaxTestRetries maxFailures = userMaxTestRetryFailures } + + // Allows devs to run tests in a loop to debug flaky tests. See README. + if (project.hasProperty("rerun-tests")) { + outputs.upToDateWhen { false } + } } task integrationTest(type: Test, dependsOn: compileJava) { @@ -468,6 +473,11 @@ subprojects { maxRetries = userMaxTestRetries maxFailures = userMaxTestRetryFailures } + + // Allows devs to run tests in a loop to debug flaky tests. See README. + if (project.hasProperty("rerun-tests")) { + outputs.upToDateWhen { false } + } } task unitTest(type: Test, dependsOn: compileJava) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java index 1d08d20ec238..04f2a0c6230f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java @@ -226,10 +226,9 @@ private void putAndMaybeForward(final ThreadCache.DirtyEntry entry, if (rawNewValue != null || rawOldValue != null) { // we need to get the old values if needed, and then put to store, and then flush final ProcessorRecordContext current = context.recordContext(); - context.setRecordContext(entry.entry().context()); - wrapped().put(entry.key(), entry.newValue()); - try { + context.setRecordContext(entry.entry().context()); + wrapped().put(entry.key(), entry.newValue()); flushListener.apply( new Record<>( entry.key().get(), @@ -241,7 +240,13 @@ private void putAndMaybeForward(final ThreadCache.DirtyEntry entry, } } } else { - wrapped().put(entry.key(), entry.newValue()); + final ProcessorRecordContext current = context.recordContext(); + try { + context.setRecordContext(entry.entry().context()); + wrapped().put(entry.key(), entry.newValue()); + } finally { + context.setRecordContext(current); + } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java index 59d2a0e72943..cff10da5f873 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java @@ -105,11 +105,11 @@ private void putAndMaybeForward(final ThreadCache.DirtyEntry entry, final Intern // we can skip flushing to downstream as well as writing to underlying store if (newValueBytes != null || oldValueBytes != null) { // we need to get the old values if needed, and then put to store, and then flush - wrapped().put(bytesKey, entry.newValue()); final ProcessorRecordContext current = context.recordContext(); - context.setRecordContext(entry.entry().context()); try { + context.setRecordContext(entry.entry().context()); + wrapped().put(bytesKey, entry.newValue()); flushListener.apply( new Record<>( binaryKey.get(), @@ -121,7 +121,13 @@ private void putAndMaybeForward(final ThreadCache.DirtyEntry entry, final Intern } } } else { - wrapped().put(bytesKey, entry.newValue()); + final ProcessorRecordContext current = context.recordContext(); + try { + context.setRecordContext(entry.entry().context()); + wrapped().put(bytesKey, entry.newValue()); + } finally { + context.setRecordContext(current); + } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index 8a1f8865fb69..50ede9c5262a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -122,11 +122,11 @@ private void putAndMaybeForward(final ThreadCache.DirtyEntry entry, // we can skip flushing to downstream as well as writing to underlying store if (rawNewValue != null || rawOldValue != null) { // we need to get the old values if needed, and then put to store, and then flush - wrapped().put(binaryKey, entry.newValue(), windowStartTimestamp); final ProcessorRecordContext current = context.recordContext(); - context.setRecordContext(entry.entry().context()); try { + context.setRecordContext(entry.entry().context()); + wrapped().put(binaryKey, entry.newValue(), windowStartTimestamp); flushListener.apply( new Record<>( binaryWindowKey, @@ -138,7 +138,13 @@ private void putAndMaybeForward(final ThreadCache.DirtyEntry entry, } } } else { - wrapped().put(binaryKey, entry.newValue(), windowStartTimestamp); + final ProcessorRecordContext current = context.recordContext(); + try { + context.setRecordContext(entry.entry().context()); + wrapped().put(binaryKey, entry.newValue(), windowStartTimestamp); + } finally { + context.setRecordContext(current); + } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 919c4402801a..1eb9a70e8c3d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -326,8 +326,8 @@ public synchronized byte[] putIfAbsent(final Bytes key, public void putAll(final List> entries) { try (final WriteBatch batch = new WriteBatch()) { dbAccessor.prepareBatch(entries, batch); - StoreQueryUtils.updatePosition(position, context); write(batch); + StoreQueryUtils.updatePosition(position, context); } catch (final RocksDBException e) { throw new ProcessorStateException("Error while batch writing to store " + name, e); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreQueryUtils.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreQueryUtils.java index 06b37139f07d..4630195c8ae7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreQueryUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreQueryUtils.java @@ -142,7 +142,9 @@ public static void updatePosition( if (stateStoreContext != null && stateStoreContext.recordMetadata().isPresent()) { final RecordMetadata meta = stateStoreContext.recordMetadata().get(); - position.withComponent(meta.topic(), meta.partition(), meta.offset()); + if (meta.topic() != null) { + position.withComponent(meta.topic(), meta.partition(), meta.offset()); + } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java index f534d6d171e1..1c828c745303 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java @@ -760,45 +760,50 @@ public static void after() { @Test public void verifyStore() { - if (storeToTest.global()) { - // See KAFKA-13523 - globalShouldRejectAllQueries(); - } else { - shouldRejectUnknownQuery(); - shouldCollectExecutionInfo(); - shouldCollectExecutionInfoUnderFailure(); - - if (storeToTest.keyValue()) { - if (storeToTest.timestamped()) { - final Function, Integer> valueExtractor = - ValueAndTimestamp::value; - shouldHandleKeyQuery(2, valueExtractor, 2); - shouldHandleRangeQueries(valueExtractor); - } else { - final Function valueExtractor = Function.identity(); - shouldHandleKeyQuery(2, valueExtractor, 2); - shouldHandleRangeQueries(valueExtractor); + try { + if (storeToTest.global()) { + // See KAFKA-13523 + globalShouldRejectAllQueries(); + } else { + shouldRejectUnknownQuery(); + shouldCollectExecutionInfo(); + shouldCollectExecutionInfoUnderFailure(); + + if (storeToTest.keyValue()) { + if (storeToTest.timestamped()) { + final Function, Integer> valueExtractor = + ValueAndTimestamp::value; + shouldHandleKeyQuery(2, valueExtractor, 2); + shouldHandleRangeQueries(valueExtractor); + } else { + final Function valueExtractor = Function.identity(); + shouldHandleKeyQuery(2, valueExtractor, 2); + shouldHandleRangeQueries(valueExtractor); + } } - } - if (storeToTest.isWindowed()) { - if (storeToTest.timestamped()) { - final Function, Integer> valueExtractor = + if (storeToTest.isWindowed()) { + if (storeToTest.timestamped()) { + final Function, Integer> valueExtractor = ValueAndTimestamp::value; - shouldHandleWindowKeyQueries(valueExtractor); - shouldHandleWindowRangeQueries(valueExtractor); - } else { - final Function valueExtractor = Function.identity(); - shouldHandleWindowKeyQueries(valueExtractor); - shouldHandleWindowRangeQueries(valueExtractor); + shouldHandleWindowKeyQueries(valueExtractor); + shouldHandleWindowRangeQueries(valueExtractor); + } else { + final Function valueExtractor = Function.identity(); + shouldHandleWindowKeyQueries(valueExtractor); + shouldHandleWindowRangeQueries(valueExtractor); + } } - } - if (storeToTest.isSession()) { - // Note there's no "timestamped" differentiation here. - // Idiosyncratically, SessionStores are _never_ timestamped. - shouldHandleSessionKeyQueries(); + if (storeToTest.isSession()) { + // Note there's no "timestamped" differentiation here. + // Idiosyncratically, SessionStores are _never_ timestamped. + shouldHandleSessionKeyQueries(); + } } + } catch (final AssertionError e) { + LOG.error("Failed assertion", e); + throw e; } } @@ -1350,7 +1355,7 @@ private static Properties streamsConfiguration(final boolean cache, final boolea final String supplier, final String kind) { final String safeTestName = IQv2StoreIntegrationTest.class.getName() + "-" + cache + "-" + log + "-" + supplier - + "-" + kind; + + "-" + kind + "-" + RANDOM.nextInt(); final Properties config = new Properties(); config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE); config.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemoryKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemoryKeyValueStoreTest.java index 13d78ec49b01..f11f85477c10 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemoryKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemoryKeyValueStoreTest.java @@ -221,17 +221,44 @@ public void shouldPutGetToFromCache() { } @Test - public void shouldMatchPositionAfterPut() { + public void shouldMatchPositionAfterPutWithFlushListener() { + store.setFlushListener(record -> { }, false); + shouldMatchPositionAfterPut(); + } + + @Test + public void shouldMatchPositionAfterPutWithoutFlushListener() { + store.setFlushListener(null, false); + shouldMatchPositionAfterPut(); + } + + private void shouldMatchPositionAfterPut() { context.setRecordContext(new ProcessorRecordContext(0, 1, 0, "", new RecordHeaders())); store.put(bytesKey("key1"), bytesValue("value1")); context.setRecordContext(new ProcessorRecordContext(0, 2, 0, "", new RecordHeaders())); store.put(bytesKey("key2"), bytesValue("value2")); - context.setRecordContext(new ProcessorRecordContext(0, 3, 0, "", new RecordHeaders())); - store.put(bytesKey("key3"), bytesValue("value3")); - final Position expected = Position.fromMap(mkMap(mkEntry("", mkMap(mkEntry(0, 3L))))); - final Position actual = store.getPosition(); - assertEquals(expected, actual); + // Position should correspond to the last record's context, not the current context. + context.setRecordContext( + new ProcessorRecordContext(0, 3, 0, "", new RecordHeaders()) + ); + + assertEquals( + Position.fromMap(mkMap(mkEntry("", mkMap(mkEntry(0, 2L))))), + store.getPosition() + ); + assertEquals(Position.emptyPosition(), underlyingStore.getPosition()); + + store.flush(); + + assertEquals( + Position.fromMap(mkMap(mkEntry("", mkMap(mkEntry(0, 2L))))), + store.getPosition() + ); + assertEquals( + Position.fromMap(mkMap(mkEntry("", mkMap(mkEntry(0, 2L))))), + underlyingStore.getPosition() + ); } private byte[] bytesValue(final String value) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemorySessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemorySessionStoreTest.java index 0de2321e7085..d5aa667c0c56 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemorySessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemorySessionStoreTest.java @@ -36,6 +36,7 @@ import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; +import org.apache.kafka.streams.query.Position; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.SessionStore; import org.apache.kafka.test.InternalMockProcessorContext; @@ -53,6 +54,8 @@ import java.util.Random; import static java.util.Arrays.asList; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList; import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue; @@ -142,6 +145,46 @@ public void shouldPutFetchFromCache() { } } + @Test + public void shouldMatchPositionAfterPutWithFlushListener() { + cachingStore.setFlushListener(record -> { }, false); + shouldMatchPositionAfterPut(); + } + + @Test + public void shouldMatchPositionAfterPutWithoutFlushListener() { + cachingStore.setFlushListener(null, false); + shouldMatchPositionAfterPut(); + } + + private void shouldMatchPositionAfterPut() { + context.setRecordContext(new ProcessorRecordContext(0, 1, 0, "", new RecordHeaders())); + cachingStore.put(new Windowed<>(keyA, new SessionWindow(0, 0)), "1".getBytes()); + context.setRecordContext(new ProcessorRecordContext(0, 2, 0, "", new RecordHeaders())); + cachingStore.put(new Windowed<>(keyA, new SessionWindow(0, 0)), "1".getBytes()); + + // Position should correspond to the last record's context, not the current context. + context.setRecordContext( + new ProcessorRecordContext(0, 3, 0, "", new RecordHeaders()) + ); + + // the caching session store doesn't maintain a separate + // position because it never serves queries from the cache + assertEquals(Position.emptyPosition(), cachingStore.getPosition()); + assertEquals(Position.emptyPosition(), underlyingStore.getPosition()); + + cachingStore.flush(); + + assertEquals( + Position.fromMap(mkMap(mkEntry("", mkMap(mkEntry(0, 2L))))), + cachingStore.getPosition() + ); + assertEquals( + Position.fromMap(mkMap(mkEntry("", mkMap(mkEntry(0, 2L))))), + underlyingStore.getPosition() + ); + } + @Test public void shouldPutFetchAllKeysFromCache() { cachingStore.put(new Windowed<>(keyA, new SessionWindow(0, 0)), "1".getBytes()); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java index 6a622dcf0d4c..50fd88a27695 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java @@ -35,6 +35,7 @@ import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; +import org.apache.kafka.streams.query.Position; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.SessionStore; import org.apache.kafka.test.InternalMockProcessorContext; @@ -52,6 +53,8 @@ import java.util.Random; import static java.util.Arrays.asList; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList; import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue; @@ -80,6 +83,7 @@ public class CachingPersistentSessionStoreTest { private SessionStore underlyingStore; private CachingSessionStore cachingStore; private ThreadCache cache; + private InternalMockProcessorContext context; @Before public void before() { @@ -93,7 +97,7 @@ public void before() { underlyingStore = new RocksDBSessionStore(segmented); cachingStore = new CachingSessionStore(underlyingStore, SEGMENT_INTERVAL); cache = new ThreadCache(new LogContext("testCache "), MAX_CACHE_SIZE_BYTES, new MockStreamsMetrics(new Metrics())); - final InternalMockProcessorContext context = + this.context = new InternalMockProcessorContext<>(TestUtils.tempDirectory(), null, null, null, cache); context.setRecordContext(new ProcessorRecordContext(DEFAULT_TIMESTAMP, 0, 0, TOPIC, new RecordHeaders())); cachingStore.init((StateStoreContext) context, cachingStore); @@ -123,6 +127,45 @@ public void shouldPutFetchFromCache() { assertFalse(b.hasNext()); } } + @Test + public void shouldMatchPositionAfterPutWithFlushListener() { + cachingStore.setFlushListener(record -> { }, false); + shouldMatchPositionAfterPut(); + } + + @Test + public void shouldMatchPositionAfterPutWithoutFlushListener() { + cachingStore.setFlushListener(null, false); + shouldMatchPositionAfterPut(); + } + + private void shouldMatchPositionAfterPut() { + context.setRecordContext(new ProcessorRecordContext(0, 1, 0, "", new RecordHeaders())); + cachingStore.put(new Windowed<>(keyA, new SessionWindow(0, 0)), "1".getBytes()); + context.setRecordContext(new ProcessorRecordContext(0, 2, 0, "", new RecordHeaders())); + cachingStore.put(new Windowed<>(keyA, new SessionWindow(0, 0)), "1".getBytes()); + + // Position should correspond to the last record's context, not the current context. + context.setRecordContext( + new ProcessorRecordContext(0, 3, 0, "", new RecordHeaders()) + ); + + // the caching session store doesn't maintain a separate + // position because it never serves queries from the cache + assertEquals(Position.emptyPosition(), cachingStore.getPosition()); + assertEquals(Position.emptyPosition(), underlyingStore.getPosition()); + + cachingStore.flush(); + + assertEquals( + Position.fromMap(mkMap(mkEntry("", mkMap(mkEntry(0, 2L))))), + cachingStore.getPosition() + ); + assertEquals( + Position.fromMap(mkMap(mkEntry("", mkMap(mkEntry(0, 2L))))), + underlyingStore.getPosition() + ); + } @Test public void shouldPutFetchAllKeysFromCache() { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java index 2d64a44aa063..3426c3ed3075 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java @@ -39,6 +39,7 @@ import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; +import org.apache.kafka.streams.query.Position; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; @@ -63,6 +64,8 @@ import static java.time.Duration.ofMinutes; import static java.time.Instant.ofEpochMilli; import static java.util.Arrays.asList; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize; import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.verifyAllWindowedKeyValues; @@ -260,6 +263,46 @@ public void shouldPutFetchFromCache() { } } + @Test + public void shouldMatchPositionAfterPutWithFlushListener() { + cachingStore.setFlushListener(record -> { }, false); + shouldMatchPositionAfterPut(); + } + + @Test + public void shouldMatchPositionAfterPutWithoutFlushListener() { + cachingStore.setFlushListener(null, false); + shouldMatchPositionAfterPut(); + } + + private void shouldMatchPositionAfterPut() { + context.setRecordContext(new ProcessorRecordContext(0, 1, 0, "", new RecordHeaders())); + cachingStore.put(bytesKey("key1"), bytesValue("value1"), DEFAULT_TIMESTAMP); + context.setRecordContext(new ProcessorRecordContext(0, 2, 0, "", new RecordHeaders())); + cachingStore.put(bytesKey("key2"), bytesValue("value2"), DEFAULT_TIMESTAMP); + + // Position should correspond to the last record's context, not the current context. + context.setRecordContext( + new ProcessorRecordContext(0, 3, 0, "", new RecordHeaders()) + ); + + // the caching window store doesn't maintain a separate + // position because it never serves queries from the cache + assertEquals(Position.emptyPosition(), cachingStore.getPosition()); + assertEquals(Position.emptyPosition(), underlyingStore.getPosition()); + + cachingStore.flush(); + + assertEquals( + Position.fromMap(mkMap(mkEntry("", mkMap(mkEntry(0, 2L))))), + cachingStore.getPosition() + ); + assertEquals( + Position.fromMap(mkMap(mkEntry("", mkMap(mkEntry(0, 2L))))), + underlyingStore.getPosition() + ); + } + private void verifyKeyValue(final KeyValue next, final long expectedKey, final String expectedValue) { From e8f09007e4259d8a6adcba162b0385b66783a7f8 Mon Sep 17 00:00:00 2001 From: Liam Clarke-Hutchinson Date: Thu, 24 Mar 2022 16:54:05 +1300 Subject: [PATCH 043/447] KAFKA-13672: Race condition in DynamicBrokerConfig (#11920) Reviewers: David Jacot , Luke Chen --- .../kafka/server/DynamicBrokerConfig.scala | 22 +++++++++++-------- .../DynamicBrokerReconfigurationTest.scala | 1 - 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index cb6cd84d3b63..2a4fd9501ba0 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -19,6 +19,7 @@ package kafka.server import java.util import java.util.{Collections, Properties} +import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.locks.ReentrantReadWriteLock import kafka.cluster.EndPoint import kafka.log.{LogCleaner, LogConfig, LogManager} @@ -201,8 +202,11 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging private[server] val staticDefaultConfigs = ConfigDef.convertToStringMapWithPasswordValues(KafkaConfig.defaultValues.asJava).asScala private val dynamicBrokerConfigs = mutable.Map[String, String]() private val dynamicDefaultConfigs = mutable.Map[String, String]() - private val reconfigurables = mutable.Buffer[Reconfigurable]() - private val brokerReconfigurables = mutable.Buffer[BrokerReconfigurable]() + + // Use COWArrayList to prevent concurrent modification exception when an item is added by one thread to these + // collections, while another thread is iterating over them. + private val reconfigurables = new CopyOnWriteArrayList[Reconfigurable]() + private val brokerReconfigurables = new CopyOnWriteArrayList[BrokerReconfigurable]() private val lock = new ReentrantReadWriteLock private var currentConfig: KafkaConfig = null private val dynamicConfigPasswordEncoder = maybeCreatePasswordEncoder(kafkaConfig.passwordEncoderSecret) @@ -259,16 +263,16 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging def addReconfigurable(reconfigurable: Reconfigurable): Unit = CoreUtils.inWriteLock(lock) { verifyReconfigurableConfigs(reconfigurable.reconfigurableConfigs.asScala) - reconfigurables += reconfigurable + reconfigurables.add(reconfigurable) } def addBrokerReconfigurable(reconfigurable: BrokerReconfigurable): Unit = CoreUtils.inWriteLock(lock) { verifyReconfigurableConfigs(reconfigurable.reconfigurableConfigs) - brokerReconfigurables += reconfigurable + brokerReconfigurables.add(reconfigurable) } def removeReconfigurable(reconfigurable: Reconfigurable): Unit = CoreUtils.inWriteLock(lock) { - reconfigurables -= reconfigurable + reconfigurables.remove(reconfigurable) } private def verifyReconfigurableConfigs(configNames: Set[String]): Unit = CoreUtils.inWriteLock(lock) { @@ -320,7 +324,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging * changes are processed. At the moment, only listener configs are considered for reloading. */ private[server] def reloadUpdatedFilesWithoutConfigChange(newProps: Properties): Unit = CoreUtils.inWriteLock(lock) { - reconfigurables + reconfigurables.asScala .filter(reconfigurable => ReloadableFileConfigs.exists(reconfigurable.reconfigurableConfigs.contains)) .foreach { case reconfigurable: ListenerReconfigurable => @@ -535,8 +539,8 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging if (changeMap.nonEmpty || deletedKeySet.nonEmpty) { try { val customConfigs = new util.HashMap[String, Object](newConfig.originalsFromThisConfig) // non-Kafka configs - newConfig.valuesFromThisConfig.keySet.forEach(customConfigs.remove(_)) - reconfigurables.foreach { + newConfig.valuesFromThisConfig.keySet.forEach(k => customConfigs.remove(k)) + reconfigurables.forEach { case listenerReconfigurable: ListenerReconfigurable => processListenerReconfigurable(listenerReconfigurable, newConfig, customConfigs, validateOnly, reloadOnly = false) case reconfigurable => @@ -546,7 +550,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging // BrokerReconfigurable updates are processed after config is updated. Only do the validation here. val brokerReconfigurablesToUpdate = mutable.Buffer[BrokerReconfigurable]() - brokerReconfigurables.foreach { reconfigurable => + brokerReconfigurables.forEach { reconfigurable => if (needsReconfiguration(reconfigurable.reconfigurableConfigs.asJava, changeMap.keySet, deletedKeySet)) { reconfigurable.validateReconfiguration(newConfig) if (!validateOnly) diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 1d64106f096f..dee4f01da246 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -728,7 +728,6 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } @Test - @Disabled // TODO: To be re-enabled once we can make it less flaky (KAFKA-13672) def testThreadPoolResize(): Unit = { val requestHandlerPrefix = "data-plane-kafka-request-handler-" val networkThreadPrefix = "data-plane-kafka-network-thread-" From 12bb23157c5dd6e305ce93dd28efcb2ba4782d98 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Thu, 24 Mar 2022 08:49:25 +0100 Subject: [PATCH 044/447] MINOR: A few cleanups in BrokerToControllerChannelManager (#11937) Make the code style more consistent Reviewers: Luke Chen --- .../BrokerToControllerChannelManager.scala | 38 ++++++++++--------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala index b671c700ac66..86395f015c48 100644 --- a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala @@ -78,9 +78,11 @@ class MetadataCacheControllerNodeProvider( } object RaftControllerNodeProvider { - def apply(raftManager: RaftManager[ApiMessageAndVersion], - config: KafkaConfig, - controllerQuorumVoterNodes: Seq[Node]): RaftControllerNodeProvider = { + def apply( + raftManager: RaftManager[ApiMessageAndVersion], + config: KafkaConfig, + controllerQuorumVoterNodes: Seq[Node] + ): RaftControllerNodeProvider = { val controllerListenerName = new ListenerName(config.controllerListenerNames.head) val controllerSecurityProtocol = config.effectiveListenerSecurityProtocolMap.getOrElse(controllerListenerName, SecurityProtocol.forName(controllerListenerName.value())) val controllerSaslMechanism = config.saslMechanismControllerProtocol @@ -98,12 +100,13 @@ object RaftControllerNodeProvider { * Finds the controller node by checking the metadata log manager. * This provider is used when we are using a Raft-based metadata quorum. */ -class RaftControllerNodeProvider(val raftManager: RaftManager[ApiMessageAndVersion], - controllerQuorumVoterNodes: Seq[Node], - val listenerName: ListenerName, - val securityProtocol: SecurityProtocol, - val saslMechanism: String - ) extends ControllerNodeProvider with Logging { +class RaftControllerNodeProvider( + val raftManager: RaftManager[ApiMessageAndVersion], + controllerQuorumVoterNodes: Seq[Node], + val listenerName: ListenerName, + val securityProtocol: SecurityProtocol, + val saslMechanism: String +) extends ControllerNodeProvider with Logging { val idToNode = controllerQuorumVoterNodes.map(node => node.id() -> node).toMap override def get(): Option[Node] = { @@ -133,7 +136,6 @@ object BrokerToControllerChannelManager { } } - trait BrokerToControllerChannelManager { def start(): Unit def shutdown(): Unit @@ -144,7 +146,6 @@ trait BrokerToControllerChannelManager { ): Unit } - /** * This class manages the connection between a broker and the controller. It runs a single * [[BrokerToControllerRequestThread]] which uses the broker's metadata cache as its own metadata to find @@ -250,13 +251,14 @@ class BrokerToControllerChannelManagerImpl( )) } - def controllerApiVersions(): Option[NodeApiVersions] = - requestThread.activeControllerAddress().flatMap( - activeController => if (activeController.id() == config.brokerId) + def controllerApiVersions(): Option[NodeApiVersions] = { + requestThread.activeControllerAddress().flatMap { activeController => + if (activeController.id == config.brokerId) Some(currentNodeApiVersions) else - Option(apiVersions.get(activeController.idString())) - ) + Option(apiVersions.get(activeController.idString)) + } + } } abstract class ControllerRequestCompletionHandler extends RequestCompletionHandler { @@ -351,10 +353,10 @@ class BrokerToControllerRequestThread( requestQueue.putFirst(queueItem) } else if (response.responseBody().errorCounts().containsKey(Errors.NOT_CONTROLLER)) { // just close the controller connection and wait for metadata cache update in doWork - activeControllerAddress().foreach { controllerAddress => { + activeControllerAddress().foreach { controllerAddress => networkClient.disconnect(controllerAddress.idString) updateControllerAddress(null) - }} + } requestQueue.putFirst(queueItem) } else { From db724f23f38cdb6c668a10681ea2a03bb11611ad Mon Sep 17 00:00:00 2001 From: Yang Yu Date: Thu, 24 Mar 2022 08:36:56 -0500 Subject: [PATCH 045/447] KAFKA-13761: KafkaLog4jAppender deadlocks when idempotence is enabled (#11939) When a log entry is appended to a Kafka topic using KafkaLog4jAppender, the producer.send operation may hit a deadlock if the producer network thread also tries to append a log at the same log level. This issue is triggered when idempotence is enabled for the KafkaLog4jAppender and the producer tries to acquire the TransactionManager lock. This is a temporary workaround to avoid deadlocks by disabling idempotence explicitly in KafkaLog4jAppender. Reviewers: Luke Chen , Ismael Juma --- .../org/apache/kafka/log4jappender/KafkaLog4jAppender.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java index 23272a2cb5d3..c561fc23608b 100644 --- a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java +++ b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java @@ -43,6 +43,7 @@ import static org.apache.kafka.clients.producer.ProducerConfig.MAX_BLOCK_MS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.RETRIES_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG; import static org.apache.kafka.common.config.SaslConfigs.SASL_JAAS_CONFIG; import static org.apache.kafka.common.config.SaslConfigs.SASL_KERBEROS_SERVICE_NAME; import static org.apache.kafka.common.config.SaslConfigs.SASL_MECHANISM; @@ -290,6 +291,9 @@ public void activateOptions() { props.put(DELIVERY_TIMEOUT_MS_CONFIG, deliveryTimeoutMs); props.put(LINGER_MS_CONFIG, lingerMs); props.put(BATCH_SIZE_CONFIG, batchSize); + // Disable idempotence to avoid deadlock when the producer network thread writes a log line while interacting + // with the TransactionManager, see KAFKA-13761 for more information. + props.put(ENABLE_IDEMPOTENCE_CONFIG, false); if (securityProtocol != null) { props.put(SECURITY_PROTOCOL_CONFIG, securityProtocol); From c5bc2688da5902015e658667186611d5d060070c Mon Sep 17 00:00:00 2001 From: RivenSun <91005273+RivenSun2@users.noreply.github.com> Date: Fri, 25 Mar 2022 00:29:39 +0800 Subject: [PATCH 046/447] KAFKA-13689: optimize the log output of logUnused method (#11940) Optimize the log output of logUnused method. Reviewers: Guozhang Wang --- .../org/apache/kafka/clients/producer/KafkaProducer.java | 3 +++ .../java/org/apache/kafka/common/config/AbstractConfig.java | 6 ++++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index ef686de7cebe..76c52bed9c81 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -530,6 +530,9 @@ private TransactionManager configureTransactionState(ProducerConfig config, log.info("Instantiated a transactional producer."); else log.info("Instantiated an idempotent producer."); + } else { + // ignore unretrieved configurations related to producer transaction + config.ignore(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG); } return transactionManager; } diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index 7ef46092749e..e3fda4d9f540 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -380,8 +380,10 @@ private void logAll() { * Log warnings for any unused configurations */ public void logUnused() { - for (String key : unused()) - log.warn("The configuration '{}' was supplied but isn't a known config.", key); + Set unusedkeys = unused(); + if (!unusedkeys.isEmpty()) { + log.warn("These configurations '{}' were supplied but are not used yet.", unusedkeys); + } } private T getConfiguredInstance(Object klass, Class t, Map configPairs) { From 01533e3dd7388ca96b9fd3b84260a8df1b2a9d28 Mon Sep 17 00:00:00 2001 From: Rohan Date: Thu, 24 Mar 2022 09:55:26 -0700 Subject: [PATCH 047/447] KAFKA-13692: include metadata wait time in total blocked time (#11805) This patch includes metadata wait time in total blocked time. First, this patch adds a new metric for total producer time spent waiting on metadata, called metadata-wait-time-ms-total. Then, this time is included in the total blocked time computed from StreamsProducer. Reviewers: Guozhang Wang --- .../kafka/clients/producer/KafkaProducer.java | 2 ++ .../internals/KafkaProducerMetrics.java | 11 ++++++++++ .../internals/KafkaProducerMetricsTest.java | 11 ++++++++++ .../processor/internals/StreamsProducer.java | 3 ++- .../internals/StreamsProducerTest.java | 20 +++++++++++++------ 5 files changed, 40 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 76c52bed9c81..7fa640f43966 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -894,6 +894,7 @@ private Future doSend(ProducerRecord record, Callback call throwIfProducerClosed(); // first make sure the metadata for the topic is available long nowMs = time.milliseconds(); + long nowNanos = time.nanoseconds(); ClusterAndWaitTime clusterAndWaitTime; try { clusterAndWaitTime = waitOnMetadata(record.topic(), record.partition(), nowMs, maxBlockTimeMs); @@ -903,6 +904,7 @@ private Future doSend(ProducerRecord record, Callback call throw e; } nowMs += clusterAndWaitTime.waitedOnMetadataMs; + producerMetrics.recordMetadataWait(time.nanoseconds() - nowNanos); long remainingWaitMs = Math.max(0, maxBlockTimeMs - clusterAndWaitTime.waitedOnMetadataMs); Cluster cluster = clusterAndWaitTime.cluster; byte[] serializedKey; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetrics.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetrics.java index 3c6fe2691e2a..7d942d572cfd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetrics.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetrics.java @@ -34,6 +34,7 @@ public class KafkaProducerMetrics implements AutoCloseable { private static final String TXN_COMMIT = "txn-commit"; private static final String TXN_ABORT = "txn-abort"; private static final String TOTAL_TIME_SUFFIX = "-time-ns-total"; + private static final String METADATA_WAIT = "metadata-wait"; private final Map tags; private final Metrics metrics; @@ -43,6 +44,7 @@ public class KafkaProducerMetrics implements AutoCloseable { private final Sensor sendOffsetsSensor; private final Sensor commitTxnSensor; private final Sensor abortTxnSensor; + private final Sensor metadataWaitSensor; public KafkaProducerMetrics(Metrics metrics) { this.metrics = metrics; @@ -71,6 +73,10 @@ public KafkaProducerMetrics(Metrics metrics) { TXN_ABORT, "Total time producer has spent in abortTransaction in nanoseconds." ); + metadataWaitSensor = newLatencySensor( + METADATA_WAIT, + "Total time producer has spent waiting on topic metadata in nanoseconds." + ); } @Override @@ -81,6 +87,7 @@ public void close() { removeMetric(TXN_SEND_OFFSETS); removeMetric(TXN_COMMIT); removeMetric(TXN_ABORT); + removeMetric(METADATA_WAIT); } public void recordFlush(long duration) { @@ -107,6 +114,10 @@ public void recordAbortTxn(long duration) { abortTxnSensor.record(duration); } + public void recordMetadataWait(long duration) { + metadataWaitSensor.record(duration); + } + private Sensor newLatencySensor(String name, String description) { Sensor sensor = metrics.sensor(name + TOTAL_TIME_SUFFIX); sensor.add(metricName(name, description), new CumulativeSum()); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetricsTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetricsTest.java index e0688616b643..46d1ed329eee 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetricsTest.java @@ -32,6 +32,7 @@ class KafkaProducerMetricsTest { private static final String TXN_COMMIT_TIME_TOTAL = "txn-commit-time-ns-total"; private static final String TXN_ABORT_TIME_TOTAL = "txn-abort-time-ns-total"; private static final String TXN_SEND_OFFSETS_TIME_TOTAL = "txn-send-offsets-time-ns-total"; + private static final String METADATA_WAIT_TIME_TOTAL = "metadata-wait-time-ns-total"; private final Metrics metrics = new Metrics(); private final KafkaProducerMetrics producerMetrics = new KafkaProducerMetrics(metrics); @@ -90,6 +91,15 @@ public void shouldRecordSendOffsetsTime() { assertMetricValue(TXN_SEND_OFFSETS_TIME_TOTAL); } + @Test + public void shouldRecordMetadataWaitTime() { + // When: + producerMetrics.recordMetadataWait(METRIC_VALUE); + + // Then: + assertMetricValue(METADATA_WAIT_TIME_TOTAL); + } + @Test public void shouldRemoveMetricsOnClose() { // When: @@ -102,6 +112,7 @@ public void shouldRemoveMetricsOnClose() { assertMetricRemoved(TXN_COMMIT_TIME_TOTAL); assertMetricRemoved(TXN_ABORT_TIME_TOTAL); assertMetricRemoved(TXN_SEND_OFFSETS_TIME_TOTAL); + assertMetricRemoved(METADATA_WAIT_TIME_TOTAL); } private void assertMetricRemoved(final String name) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsProducer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsProducer.java index 7c98189ff1b3..a1b68ff79085 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsProducer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsProducer.java @@ -223,7 +223,8 @@ private double totalBlockedTime(final Producer producer) { + getMetricValue(producer.metrics(), "txn-begin-time-ns-total") + getMetricValue(producer.metrics(), "txn-send-offsets-time-ns-total") + getMetricValue(producer.metrics(), "txn-commit-time-ns-total") - + getMetricValue(producer.metrics(), "txn-abort-time-ns-total"); + + getMetricValue(producer.metrics(), "txn-abort-time-ns-total") + + getMetricValue(producer.metrics(), "metadata-wait-time-ns-total"); } public double totalBlockedTime() { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsProducerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsProducerTest.java index 420d94a94972..9470a7b166e1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsProducerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsProducerTest.java @@ -76,6 +76,7 @@ public class StreamsProducerTest { private static final double TXN_SEND_OFFSETS_TIME = 5; private static final double TXN_COMMIT_TIME = 6; private static final double TXN_ABORT_TIME = 7; + private static final double METADATA_WAIT_TIME = 8; private final LogContext logContext = new LogContext("test "); private final String topic = "topic"; @@ -1167,11 +1168,13 @@ public void shouldComputeTotalBlockedTime() { TXN_BEGIN_TIME, TXN_SEND_OFFSETS_TIME, TXN_COMMIT_TIME, - TXN_ABORT_TIME + TXN_ABORT_TIME, + METADATA_WAIT_TIME ); final double expectedTotalBlocked = BUFFER_POOL_WAIT_TIME + FLUSH_TME + TXN_INIT_TIME + - TXN_BEGIN_TIME + TXN_SEND_OFFSETS_TIME + TXN_COMMIT_TIME + TXN_ABORT_TIME; + TXN_BEGIN_TIME + TXN_SEND_OFFSETS_TIME + TXN_COMMIT_TIME + TXN_ABORT_TIME + + METADATA_WAIT_TIME; assertThat(nonEosStreamsProducer.totalBlockedTime(), closeTo(expectedTotalBlocked, 0.01)); } @@ -1185,10 +1188,12 @@ public void shouldComputeTotalBlockedTimeAfterReset() { TXN_BEGIN_TIME, TXN_SEND_OFFSETS_TIME, TXN_COMMIT_TIME, - TXN_ABORT_TIME + TXN_ABORT_TIME, + METADATA_WAIT_TIME ); final double expectedTotalBlocked = BUFFER_POOL_WAIT_TIME + FLUSH_TME + TXN_INIT_TIME + - TXN_BEGIN_TIME + TXN_SEND_OFFSETS_TIME + TXN_COMMIT_TIME + TXN_ABORT_TIME; + TXN_BEGIN_TIME + TXN_SEND_OFFSETS_TIME + TXN_COMMIT_TIME + TXN_ABORT_TIME + + METADATA_WAIT_TIME; assertThat(eosBetaStreamsProducer.totalBlockedTime(), equalTo(expectedTotalBlocked)); reset(mockTime); final long closeStart = 1L; @@ -1204,7 +1209,8 @@ public void shouldComputeTotalBlockedTimeAfterReset() { TXN_BEGIN_TIME, TXN_SEND_OFFSETS_TIME, TXN_COMMIT_TIME, - TXN_ABORT_TIME + TXN_ABORT_TIME, + METADATA_WAIT_TIME ); assertThat( @@ -1243,7 +1249,8 @@ private void setProducerMetrics( final double txnBeginTime, final double txnSendOffsetsTime, final double txnCommitTime, - final double txnAbortTime) { + final double txnAbortTime, + final double metadataWaitTime) { addMetric(producer, "bufferpool-wait-time-ns-total", bufferPoolWaitTime); addMetric(producer, "flush-time-ns-total", flushTime); addMetric(producer, "txn-init-time-ns-total", txnInitTime); @@ -1251,5 +1258,6 @@ private void setProducerMetrics( addMetric(producer, "txn-send-offsets-time-ns-total", txnSendOffsetsTime); addMetric(producer, "txn-commit-time-ns-total", txnCommitTime); addMetric(producer, "txn-abort-time-ns-total", txnAbortTime); + addMetric(producer, "metadata-wait-time-ns-total", metadataWaitTime); } } From 46df7ee97ce696e8868a654cfdff483e6f180902 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Thu, 24 Mar 2022 14:04:40 -0500 Subject: [PATCH 048/447] MINOR: Add extra notice about IQv2 compatibility (#11944) Added an extra notice about IQv2's API compatibility, as discussed in the KIP-796 vote thread. Reviewers: Bill Bejeck , @Kvicii --- .../src/main/java/org/apache/kafka/streams/KafkaStreams.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 369bc560f80e..7798e58204c9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -1782,6 +1782,9 @@ protected Map> allLocalStorePartitionLags(final Li * This method allows callers outside of the Streams runtime to access the internal state of * stateful processors. See https://kafka.apache.org/documentation/streams/developer-guide/interactive-queries.html * for more information. + *

    + * NOTICE: This functionality is {@link Evolving} and subject to change in minor versions. + * Once it is stabilized, this notice and the evolving annotation will be removed. * * @param The result type specified by the query. * @throws StreamsNotStartedException If Streams has not yet been started. Just call {@link From ce883892270a02a72e91afbdb1fabdd50d7da474 Mon Sep 17 00:00:00 2001 From: Randall Hauch Date: Thu, 24 Mar 2022 21:40:10 -0500 Subject: [PATCH 049/447] KAFKA-13770: Restore compatibility with KafkaBasedLog using older Kafka brokers (#11946) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The `retryEndOffsets(…)` method in `TopicAdmin` recently added (KAFKA-12879, #11797) to allow the `KafkaBasedLog.start()` method to retry any failures reading the last offsets for a topic. However, this introduce a regression when talking to older brokers (0.10.x or earlier). The `KafkaBasedLog` already had logic that expected an `UnsupportedVersionException` thrown by the admin client when a Kafka API is not available on an older broker, but the new retry logic in `TopicAdmin` did not account for this and wrapped the exception, thereby breaking the `KafkaBasedLog` logic and preventing startup. The fix is to propagate this `UnsupportedVersionException` from the `TopicAdmin.retryEndOffsets(…)` method. Added a new unit test that first replicated the problem before the fix, and verified the fix corrects the problem. --- .../apache/kafka/connect/util/TopicAdmin.java | 4 +++ .../kafka/connect/util/TopicAdminTest.java | 25 ++++++++++++++++++- 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java index faf7b372bedc..d97533a87e52 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java @@ -714,6 +714,7 @@ public Map endOffsets(Set partitions) { * must be 0 or more * @return the map of offset for each topic partition, or an empty map if the supplied partitions * are null or empty + * @throws UnsupportedVersionException if the broker is too old to support the admin client API to read end offsets * @throws ConnectException if {@code timeoutDuration} is exhausted * @see TopicAdmin#endOffsets(Set) */ @@ -725,6 +726,9 @@ public Map retryEndOffsets(Set partitions, () -> "list offsets for topic partitions", timeoutDuration, retryBackoffMs); + } catch (UnsupportedVersionException e) { + // Older brokers don't support this admin method, so rethrow it without wrapping it + throw e; } catch (Exception e) { throw new ConnectException("Failed to list offsets for topic partitions.", e); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java index deea050d78bd..cf611db9c0ff 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java @@ -467,8 +467,31 @@ public void verifyingGettingTopicCleanupPolicies() { } } + /** + * TopicAdmin can be used to read the end offsets, but the admin client API used to do this was + * added to the broker in 0.11.0.0. This means that if Connect talks to older brokers, + * the admin client cannot be used to read end offsets, and will throw an UnsupportedVersionException. + */ + @Test + public void retryEndOffsetsShouldRethrowUnknownVersionException() { + String topicName = "myTopic"; + TopicPartition tp1 = new TopicPartition(topicName, 0); + Set tps = Collections.singleton(tp1); + Long offset = null; // response should use error + Cluster cluster = createCluster(1, topicName, 1); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(prepareMetadataResponse(cluster, Errors.NONE)); + // Expect the admin client list offsets will throw unsupported version, simulating older brokers + env.kafkaClient().prepareResponse(listOffsetsResultWithUnsupportedVersion(tp1, offset)); + TopicAdmin admin = new TopicAdmin(null, env.adminClient()); + // The retryEndOffsets should catch and rethrow an unsupported version exception + assertThrows(UnsupportedVersionException.class, () -> admin.retryEndOffsets(tps, Duration.ofMillis(100), 1)); + } + } + @Test - public void retryEndOffsetsShouldThrowConnectException() { + public void retryEndOffsetsShouldWrapNonRetriableExceptionsWithConnectException() { String topicName = "myTopic"; TopicPartition tp1 = new TopicPartition(topicName, 0); Set tps = Collections.singleton(tp1); From 0586f544ef5a3e2ffa263babc811568ce672dfa1 Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Sat, 26 Mar 2022 00:30:02 +0800 Subject: [PATCH 050/447] KAFKA-10405: Set purge interval explicitly in PurgeRepartitionTopicIntegrationTest (#11948) In KIP-811, we added a new config repartition.purge.interval.ms to set repartition purge interval. In this flaky test, we expected the purge interval is the same as commit interval, which is not correct anymore (default is 30 sec). Set the purge interval explicitly to fix this issue. Reviewers: Bruno Cadonna , Guozhang Wang --- .../integration/PurgeRepartitionTopicIntegrationTest.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index ffb35312ea41..37d87435218f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -161,6 +161,7 @@ public void setup() { final Properties streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, APPLICATION_ID); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, PURGE_INTERVAL_MS); + streamsConfiguration.put(StreamsConfig.REPARTITION_PURGE_INTERVAL_MS_CONFIG, PURGE_INTERVAL_MS); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); @@ -203,10 +204,11 @@ public void shouldRestoreState() throws Exception { TestUtils.waitForCondition(new RepartitionTopicCreatedWithExpectedConfigs(), 60000, "Repartition topic " + REPARTITION_TOPIC + " not created with the expected configs after 60000 ms."); + // wait until we received more than 1 segment of data, so that we can confirm the purge succeeds in next verification TestUtils.waitForCondition( - new RepartitionTopicVerified(currentSize -> currentSize > 0), + new RepartitionTopicVerified(currentSize -> currentSize > PURGE_SEGMENT_BYTES), 60000, - "Repartition topic " + REPARTITION_TOPIC + " not received data after 60000 ms." + "Repartition topic " + REPARTITION_TOPIC + " not received more than " + PURGE_SEGMENT_BYTES + "B of data after 60000 ms." ); // we need long enough timeout to by-pass the log manager's InitialTaskDelayMs, which is hard-coded on server side From 0461382adbf8b8dfddc547f28c8d7cfa801e585b Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Sat, 26 Mar 2022 01:16:50 +0800 Subject: [PATCH 051/447] KAFKA-4801: don't verify assignment during broker up and down in testConsumptionWithBrokerFailures (#11949) In this test, we have another thread to let broker down and up, to test if consumer can still work as expected. During the broker down and up, we tried to verify the assignment is as what we expected. But the rebalance will keep triggering while broker down and up. It doesn't make sense to verify the assignment here. Remove it to make the test reliable. Reviewers: Guozhang Wang --- .../scala/integration/kafka/api/ConsumerBounceTest.scala | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index 510a98a90089..2ee6d43f0fc5 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -16,7 +16,6 @@ package kafka.api import java.time import java.util.concurrent._ import java.util.{Collection, Collections, Properties} - import kafka.server.KafkaConfig import kafka.utils.{Logging, ShutdownableThread, TestUtils} import org.apache.kafka.clients.consumer._ @@ -29,6 +28,7 @@ import org.apache.kafka.common.requests.{FindCoordinatorRequest, FindCoordinator import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Disabled, Test} +import java.time.Duration import scala.annotation.nowarn import scala.jdk.CollectionConverters._ import scala.collection.{Seq, mutable} @@ -77,14 +77,12 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { } @Test - @Disabled // To be re-enabled once we can make it less flaky (KAFKA-4801) def testConsumptionWithBrokerFailures(): Unit = consumeWithBrokerFailures(10) /* * 1. Produce a bunch of messages * 2. Then consume the messages while killing and restarting brokers at random */ - @nowarn("cat=deprecation") def consumeWithBrokerFailures(numIters: Int): Unit = { val numRecords = 1000 val producer = createProducer() @@ -99,8 +97,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { scheduler.start() while (scheduler.isRunning) { - val records = consumer.poll(100).asScala - assertEquals(Set(tp), consumer.assignment.asScala) + val records = consumer.poll(Duration.ofMillis(100)).asScala for (record <- records) { assertEquals(consumed, record.offset()) From d706d6cac4622153973d131417e809ee57c60de0 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Fri, 25 Mar 2022 18:18:15 +0100 Subject: [PATCH 052/447] MINOR: Renable SocketServerTest.closingChannelWithBufferedReceives and SocketServerTest.remoteCloseWithoutBufferedReceives (#11927) Reviewers: Guozhang Wang --- core/src/test/scala/unit/kafka/network/SocketServerTest.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 33d15ad10b5a..849646cb644c 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -1389,7 +1389,6 @@ class SocketServerTest { * buffered receive. */ @Test - @Disabled // TODO: re-enabled until KAFKA-13735 is fixed def remoteCloseWithoutBufferedReceives(): Unit = { verifyRemoteCloseWithBufferedReceives(numComplete = 0, hasIncomplete = false) } @@ -1427,7 +1426,6 @@ class SocketServerTest { * The channel must be closed after pending receives are processed. */ @Test - @Disabled // TODO: re-enable after KAFKA-13736 is fixed def closingChannelWithBufferedReceives(): Unit = { verifyRemoteCloseWithBufferedReceives(numComplete = 3, hasIncomplete = false, makeClosing = true) } From ca3271e0f3e3c31cce413668c6a1ed2dacab91fe Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Sun, 27 Mar 2022 15:47:24 +0800 Subject: [PATCH 053/447] MINOR: replace kafka-preferred-replica-election.sh with kafka-leader-election.sh (#11954) Reviewers: Luke Chen --- docs/ops.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ops.html b/docs/ops.html index e5b2fb8e9467..183d8acb49a0 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -78,7 +78,7 @@

    auto.leader.rebalance.enable=true You can also set this to false, but you will then need to manually restore leadership to the restored replicas by running the command: -
      > bin/kafka-preferred-replica-election.sh --bootstrap-server broker_host:port
    +
      > bin/kafka-leader-election.sh --bootstrap-server broker_host:port --election-type preferred --all-topic-partitions

    Balancing Replicas Across Racks

    The rack awareness feature spreads replicas of the same partition across different racks. This extends the guarantees Kafka provides for broker-failure to cover rack-failure, limiting the risk of data loss should all the brokers on a rack fail at once. The feature can also be applied to other broker groupings such as availability zones in EC2. From 110bccac4af74e04368e13c4e638b0b5bcf4100f Mon Sep 17 00:00:00 2001 From: Tim Patterson Date: Tue, 29 Mar 2022 03:48:39 +1300 Subject: [PATCH 054/447] KAFKA-13600: Kafka Streams - Fall back to most caught up client if no caught up clients exist (#11760) The task assignor is modified to consider the Streams client with the most caught up states if no Streams client exists that is caught up, i.e., the lag of the states on that client is less than the acceptable recovery lag. Unit test for case task assignment where no caught up nodes exist. Existing unit and integration tests to verify no other behaviour has been changed Co-authored-by: Bruno Cadonna Reviewer: Bruno Cadonna --- .../HighAvailabilityTaskAssignor.java | 17 ++ .../internals/assignment/TaskMovement.java | 178 +++++++++++++----- .../HighAvailabilityTaskAssignorTest.java | 31 +++ .../assignment/TaskMovementTest.java | 144 ++++++++++++-- 4 files changed, 303 insertions(+), 67 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java index 7111ae28e5fd..c54199ad1773 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java @@ -22,6 +22,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -68,6 +69,8 @@ public boolean assign(final Map clients, configs.acceptableRecoveryLag ); + final Map> tasksToClientByLag = tasksToClientByLag(statefulTasks, clientStates); + // We temporarily need to know which standby tasks were intended as warmups // for active tasks, so that we don't move them (again) when we plan standby // task movements. We can then immediately treat warmups exactly the same as @@ -77,6 +80,7 @@ public boolean assign(final Map clients, final int neededActiveTaskMovements = assignActiveTaskMovements( tasksToCaughtUpClients, + tasksToClientByLag, clientStates, warmups, remainingWarmupReplicas @@ -84,6 +88,7 @@ public boolean assign(final Map clients, final int neededStandbyTaskMovements = assignStandbyTaskMovements( tasksToCaughtUpClients, + tasksToClientByLag, clientStates, remainingWarmupReplicas, warmups @@ -238,6 +243,18 @@ private static Map> tasksToCaughtUpClients(final Set> tasksToClientByLag(final Set statefulTasks, + final Map clientStates) { + final Map> tasksToClientByLag = new HashMap<>(); + for (final TaskId task : statefulTasks) { + final SortedSet clientLag = new TreeSet<>(Comparator.comparingLong(a -> + clientStates.get(a).lagFor(task)).thenComparing(a -> a)); + clientLag.addAll(clientStates.keySet()); + tasksToClientByLag.put(task, clientLag); + } + return tasksToClientByLag; + } + private static boolean unbounded(final long acceptableRecoveryLag) { return acceptableRecoveryLag == Long.MAX_VALUE; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovement.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovement.java index cbfa3daaedf9..38e64276ba0a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovement.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovement.java @@ -29,6 +29,7 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; +import java.util.function.Function; import static java.util.Arrays.asList; import static java.util.Objects.requireNonNull; @@ -42,10 +43,6 @@ private TaskMovement(final TaskId task, final UUID destination, final SortedSet< this.task = task; this.destination = destination; this.caughtUpClients = caughtUpClients; - - if (caughtUpClients == null || caughtUpClients.isEmpty()) { - throw new IllegalStateException("Should not attempt to move a task if no caught up clients exist"); - } } private TaskId task() { @@ -56,25 +53,34 @@ private int numCaughtUpClients() { return caughtUpClients.size(); } - private static boolean taskIsNotCaughtUpOnClientAndOtherCaughtUpClientsExist(final TaskId task, - final UUID client, - final Map> tasksToCaughtUpClients) { - return !taskIsCaughtUpOnClientOrNoCaughtUpClientsExist(task, client, tasksToCaughtUpClients); + private static boolean taskIsNotCaughtUpOnClientAndOtherMoreCaughtUpClientsExist(final TaskId task, + final UUID client, + final Map clientStates, + final Map> tasksToCaughtUpClients, + final Map> tasksToClientByLag) { + final SortedSet taskClients = requireNonNull(tasksToClientByLag.get(task), "uninitialized set"); + if (taskIsCaughtUpOnClient(task, client, tasksToCaughtUpClients)) { + return false; + } + final long mostCaughtUpLag = clientStates.get(taskClients.first()).lagFor(task); + final long clientLag = clientStates.get(client).lagFor(task); + return mostCaughtUpLag < clientLag; } - private static boolean taskIsCaughtUpOnClientOrNoCaughtUpClientsExist(final TaskId task, - final UUID client, - final Map> tasksToCaughtUpClients) { + private static boolean taskIsCaughtUpOnClient(final TaskId task, + final UUID client, + final Map> tasksToCaughtUpClients) { final Set caughtUpClients = requireNonNull(tasksToCaughtUpClients.get(task), "uninitialized set"); - return caughtUpClients.isEmpty() || caughtUpClients.contains(client); + return caughtUpClients.contains(client); } static int assignActiveTaskMovements(final Map> tasksToCaughtUpClients, + final Map> tasksToClientByLag, final Map clientStates, final Map> warmups, final AtomicInteger remainingWarmupReplicas) { final BiFunction caughtUpPredicate = - (client, task) -> taskIsCaughtUpOnClientOrNoCaughtUpClientsExist(task, client, tasksToCaughtUpClients); + (client, task) -> taskIsCaughtUpOnClient(task, client, tasksToCaughtUpClients); final ConstrainedPrioritySet caughtUpClientsByTaskLoad = new ConstrainedPrioritySet( caughtUpPredicate, @@ -89,10 +95,10 @@ static int assignActiveTaskMovements(final Map> tasksToC final UUID client = clientStateEntry.getKey(); final ClientState state = clientStateEntry.getValue(); for (final TaskId task : state.activeTasks()) { - // if the desired client is not caught up, and there is another client that _is_ caught up, then - // we schedule a movement, so we can move the active task to the caught-up client. We'll try to + // if the desired client is not caught up, and there is another client that _is_ more caught up, then + // we schedule a movement, so we can move the active task to a more caught-up client. We'll try to // assign a warm-up to the desired client so that we can move it later on. - if (taskIsNotCaughtUpOnClientAndOtherCaughtUpClientsExist(task, client, tasksToCaughtUpClients)) { + if (taskIsNotCaughtUpOnClientAndOtherMoreCaughtUpClientsExist(task, client, clientStates, tasksToCaughtUpClients, tasksToClientByLag)) { taskMovements.add(new TaskMovement(task, client, tasksToCaughtUpClients.get(task))); } } @@ -102,33 +108,14 @@ static int assignActiveTaskMovements(final Map> tasksToC final int movementsNeeded = taskMovements.size(); for (final TaskMovement movement : taskMovements) { - final UUID standbySourceClient = caughtUpClientsByTaskLoad.poll( - movement.task, - c -> clientStates.get(c).hasStandbyTask(movement.task) - ); - if (standbySourceClient == null) { - // there's not a caught-up standby available to take over the task, so we'll schedule a warmup instead - final UUID sourceClient = requireNonNull( - caughtUpClientsByTaskLoad.poll(movement.task), - "Tried to move task to caught-up client but none exist" - ); - - moveActiveAndTryToWarmUp( - remainingWarmupReplicas, - movement.task, - clientStates.get(sourceClient), - clientStates.get(movement.destination), - warmups.computeIfAbsent(movement.destination, x -> new TreeSet<>()) - ); - caughtUpClientsByTaskLoad.offerAll(asList(sourceClient, movement.destination)); - } else { - // we found a candidate to trade standby/active state with our destination, so we don't need a warmup - swapStandbyAndActive( - movement.task, - clientStates.get(standbySourceClient), - clientStates.get(movement.destination) - ); - caughtUpClientsByTaskLoad.offerAll(asList(standbySourceClient, movement.destination)); + // Attempt to find a caught up standby, otherwise find any caught up client, failing that use the most + // caught up client. + final boolean moved = tryToSwapStandbyAndActiveOnCaughtUpClient(clientStates, caughtUpClientsByTaskLoad, movement) || + tryToMoveActiveToCaughtUpClientAndTryToWarmUp(clientStates, warmups, remainingWarmupReplicas, caughtUpClientsByTaskLoad, movement) || + tryToMoveActiveToMostCaughtUpClient(tasksToClientByLag, clientStates, warmups, remainingWarmupReplicas, caughtUpClientsByTaskLoad, movement); + + if (!moved) { + throw new IllegalStateException("Tried to move task to more caught-up client as scheduled before but none exist"); } } @@ -136,11 +123,12 @@ static int assignActiveTaskMovements(final Map> tasksToC } static int assignStandbyTaskMovements(final Map> tasksToCaughtUpClients, + final Map> tasksToClientByLag, final Map clientStates, final AtomicInteger remainingWarmupReplicas, final Map> warmups) { final BiFunction caughtUpPredicate = - (client, task) -> taskIsCaughtUpOnClientOrNoCaughtUpClientsExist(task, client, tasksToCaughtUpClients); + (client, task) -> taskIsCaughtUpOnClient(task, client, tasksToCaughtUpClients); final ConstrainedPrioritySet caughtUpClientsByTaskLoad = new ConstrainedPrioritySet( caughtUpPredicate, @@ -157,8 +145,8 @@ static int assignStandbyTaskMovements(final Map> tasksTo for (final TaskId task : state.standbyTasks()) { if (warmups.getOrDefault(destination, Collections.emptySet()).contains(task)) { // this is a warmup, so we won't move it. - } else if (taskIsNotCaughtUpOnClientAndOtherCaughtUpClientsExist(task, destination, tasksToCaughtUpClients)) { - // if the desired client is not caught up, and there is another client that _is_ caught up, then + } else if (taskIsNotCaughtUpOnClientAndOtherMoreCaughtUpClientsExist(task, destination, clientStates, tasksToCaughtUpClients, tasksToClientByLag)) { + // if the desired client is not caught up, and there is another client that _is_ more caught up, then // we schedule a movement, so we can move the active task to the caught-up client. We'll try to // assign a warm-up to the desired client so that we can move it later on. taskMovements.add(new TaskMovement(task, destination, tasksToCaughtUpClients.get(task))); @@ -170,11 +158,17 @@ static int assignStandbyTaskMovements(final Map> tasksTo int movementsNeeded = 0; for (final TaskMovement movement : taskMovements) { - final UUID sourceClient = caughtUpClientsByTaskLoad.poll( + final Function eligibleClientPredicate = + clientId -> !clientStates.get(clientId).hasAssignedTask(movement.task); + UUID sourceClient = caughtUpClientsByTaskLoad.poll( movement.task, - clientId -> !clientStates.get(clientId).hasAssignedTask(movement.task) + eligibleClientPredicate ); + if (sourceClient == null) { + sourceClient = mostCaughtUpEligibleClient(tasksToClientByLag, eligibleClientPredicate, movement.task, movement.destination); + } + if (sourceClient == null) { // then there's no caught-up client that doesn't already have a copy of this task, so there's // nowhere to move it. @@ -193,6 +187,74 @@ static int assignStandbyTaskMovements(final Map> tasksTo return movementsNeeded; } + private static boolean tryToSwapStandbyAndActiveOnCaughtUpClient(final Map clientStates, + final ConstrainedPrioritySet caughtUpClientsByTaskLoad, + final TaskMovement movement) { + final UUID caughtUpStandbySourceClient = caughtUpClientsByTaskLoad.poll( + movement.task, + c -> clientStates.get(c).hasStandbyTask(movement.task) + ); + if (caughtUpStandbySourceClient != null) { + swapStandbyAndActive( + movement.task, + clientStates.get(caughtUpStandbySourceClient), + clientStates.get(movement.destination) + ); + caughtUpClientsByTaskLoad.offerAll(asList(caughtUpStandbySourceClient, movement.destination)); + return true; + } + return false; + } + + private static boolean tryToMoveActiveToCaughtUpClientAndTryToWarmUp(final Map clientStates, + final Map> warmups, + final AtomicInteger remainingWarmupReplicas, + final ConstrainedPrioritySet caughtUpClientsByTaskLoad, + final TaskMovement movement) { + final UUID caughtUpSourceClient = caughtUpClientsByTaskLoad.poll(movement.task); + if (caughtUpSourceClient != null) { + moveActiveAndTryToWarmUp( + remainingWarmupReplicas, + movement.task, + clientStates.get(caughtUpSourceClient), + clientStates.get(movement.destination), + warmups.computeIfAbsent(movement.destination, x -> new TreeSet<>()) + ); + caughtUpClientsByTaskLoad.offerAll(asList(caughtUpSourceClient, movement.destination)); + return true; + } + return false; + } + + private static boolean tryToMoveActiveToMostCaughtUpClient(final Map> tasksToClientByLag, + final Map clientStates, + final Map> warmups, + final AtomicInteger remainingWarmupReplicas, + final ConstrainedPrioritySet caughtUpClientsByTaskLoad, + final TaskMovement movement) { + final UUID mostCaughtUpSourceClient = mostCaughtUpEligibleClient(tasksToClientByLag, movement.task, movement.destination); + if (mostCaughtUpSourceClient != null) { + if (clientStates.get(mostCaughtUpSourceClient).hasStandbyTask(movement.task)) { + swapStandbyAndActive( + movement.task, + clientStates.get(mostCaughtUpSourceClient), + clientStates.get(movement.destination) + ); + } else { + moveActiveAndTryToWarmUp( + remainingWarmupReplicas, + movement.task, + clientStates.get(mostCaughtUpSourceClient), + clientStates.get(movement.destination), + warmups.computeIfAbsent(movement.destination, x -> new TreeSet<>()) + ); + } + caughtUpClientsByTaskLoad.offerAll(asList(mostCaughtUpSourceClient, movement.destination)); + return true; + } + return false; + } + private static void moveActiveAndTryToWarmUp(final AtomicInteger remainingWarmupReplicas, final TaskId task, final ClientState sourceClientState, @@ -235,4 +297,24 @@ private static void swapStandbyAndActive(final TaskId task, destinationClientState.assignStandby(task); } + private static UUID mostCaughtUpEligibleClient(final Map> tasksToClientByLag, + final TaskId task, + final UUID destinationClient) { + return mostCaughtUpEligibleClient(tasksToClientByLag, client -> true, task, destinationClient); + } + + private static UUID mostCaughtUpEligibleClient(final Map> tasksToClientByLag, + final Function constraint, + final TaskId task, + final UUID destinationClient) { + for (final UUID client : tasksToClientByLag.get(task)) { + if (destinationClient.equals(client)) { + break; + } else if (constraint.apply(client)) { + return client; + } + } + return null; + } + } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignorTest.java index 36ae42fded37..90e0fed51f38 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignorTest.java @@ -419,6 +419,37 @@ public void shouldComputeNewAssignmentIfActiveTasksWasNotOnCaughtUpClient() { assertBalancedTasks(clientStates); } + @Test + public void shouldAssignToMostCaughtUpIfActiveTasksWasNotOnCaughtUpClient() { + final Set allTasks = mkSet(TASK_0_0); + final Set statefulTasks = mkSet(TASK_0_0); + final ClientState client1 = new ClientState(emptySet(), emptySet(), singletonMap(TASK_0_0, Long.MAX_VALUE), EMPTY_CLIENT_TAGS, 1); + final ClientState client2 = new ClientState(emptySet(), emptySet(), singletonMap(TASK_0_0, 1000L), EMPTY_CLIENT_TAGS, 1); + final ClientState client3 = new ClientState(emptySet(), emptySet(), singletonMap(TASK_0_0, 500L), EMPTY_CLIENT_TAGS, 1); + final Map clientStates = mkMap( + mkEntry(UUID_1, client1), + mkEntry(UUID_2, client2), + mkEntry(UUID_3, client3) + ); + + final boolean probingRebalanceNeeded = + new HighAvailabilityTaskAssignor().assign(clientStates, allTasks, statefulTasks, configWithStandbys); + + assertThat(clientStates.get(UUID_1).activeTasks(), is(emptySet())); + assertThat(clientStates.get(UUID_2).activeTasks(), is(emptySet())); + assertThat(clientStates.get(UUID_3).activeTasks(), is(singleton(TASK_0_0))); + + assertThat(clientStates.get(UUID_1).standbyTasks(), is(singleton(TASK_0_0))); // warm up + assertThat(clientStates.get(UUID_2).standbyTasks(), is(singleton(TASK_0_0))); // standby + assertThat(clientStates.get(UUID_3).standbyTasks(), is(emptySet())); + + assertThat(probingRebalanceNeeded, is(true)); + assertValidAssignment(1, 1, allTasks, emptySet(), clientStates, new StringBuilder()); + assertBalancedActiveAssignment(clientStates, new StringBuilder()); + assertBalancedStatefulAssignment(allTasks, clientStates, new StringBuilder()); + assertBalancedTasks(clientStates); + } + @Test public void shouldAssignStandbysForStatefulTasks() { final Set allTasks = mkSet(TASK_0_0, TASK_0_1); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovementTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovementTest.java index 9b58d18c88f6..baf6d18496c1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovementTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovementTest.java @@ -19,19 +19,21 @@ import org.apache.kafka.streams.processor.TaskId; import org.junit.Test; -import java.util.Collection; +import java.util.Comparator; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.SortedSet; import java.util.TreeMap; +import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import static java.util.Arrays.asList; -import static java.util.Collections.emptyList; +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; import static java.util.Collections.emptySortedSet; -import static java.util.Collections.singletonList; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkSet; @@ -58,17 +60,20 @@ public void shouldAssignTasksToClientsAndReturnFalseWhenAllClientsCaughtUp() { final Set allTasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2, TASK_1_0, TASK_1_1, TASK_1_2); final Map> tasksToCaughtUpClients = new HashMap<>(); + final Map> tasksToClientByLag = new HashMap<>(); for (final TaskId task : allTasks) { tasksToCaughtUpClients.put(task, mkSortedSet(UUID_1, UUID_2, UUID_3)); + tasksToClientByLag.put(task, mkOrderedSet(UUID_1, UUID_2, UUID_3)); } - final ClientState client1 = getClientStateWithActiveAssignment(asList(TASK_0_0, TASK_1_0)); - final ClientState client2 = getClientStateWithActiveAssignment(asList(TASK_0_1, TASK_1_1)); - final ClientState client3 = getClientStateWithActiveAssignment(asList(TASK_0_2, TASK_1_2)); + final ClientState client1 = getClientStateWithActiveAssignment(mkSet(TASK_0_0, TASK_1_0), allTasks, allTasks); + final ClientState client2 = getClientStateWithActiveAssignment(mkSet(TASK_0_1, TASK_1_1), allTasks, allTasks); + final ClientState client3 = getClientStateWithActiveAssignment(mkSet(TASK_0_2, TASK_1_2), allTasks, allTasks); assertThat( assignActiveTaskMovements( tasksToCaughtUpClients, + tasksToClientByLag, getClientStatesMap(client1, client2, client3), new TreeMap<>(), new AtomicInteger(maxWarmupReplicas) @@ -80,10 +85,11 @@ public void shouldAssignTasksToClientsAndReturnFalseWhenAllClientsCaughtUp() { @Test public void shouldAssignAllTasksToClientsAndReturnFalseIfNoClientsAreCaughtUp() { final int maxWarmupReplicas = Integer.MAX_VALUE; + final Set allTasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2, TASK_1_0, TASK_1_1, TASK_1_2); - final ClientState client1 = getClientStateWithActiveAssignment(asList(TASK_0_0, TASK_1_0)); - final ClientState client2 = getClientStateWithActiveAssignment(asList(TASK_0_1, TASK_1_1)); - final ClientState client3 = getClientStateWithActiveAssignment(asList(TASK_0_2, TASK_1_2)); + final ClientState client1 = getClientStateWithActiveAssignment(mkSet(TASK_0_0, TASK_1_0), mkSet(), allTasks); + final ClientState client2 = getClientStateWithActiveAssignment(mkSet(TASK_0_1, TASK_1_1), mkSet(), allTasks); + final ClientState client3 = getClientStateWithActiveAssignment(mkSet(TASK_0_2, TASK_1_2), mkSet(), allTasks); final Map> tasksToCaughtUpClients = mkMap( mkEntry(TASK_0_0, emptySortedSet()), @@ -93,9 +99,18 @@ public void shouldAssignAllTasksToClientsAndReturnFalseIfNoClientsAreCaughtUp() mkEntry(TASK_1_1, emptySortedSet()), mkEntry(TASK_1_2, emptySortedSet()) ); + final Map> tasksToClientByLag = mkMap( + mkEntry(TASK_0_0, mkOrderedSet(UUID_1, UUID_2, UUID_3)), + mkEntry(TASK_0_1, mkOrderedSet(UUID_1, UUID_2, UUID_3)), + mkEntry(TASK_0_2, mkOrderedSet(UUID_1, UUID_2, UUID_3)), + mkEntry(TASK_1_0, mkOrderedSet(UUID_1, UUID_2, UUID_3)), + mkEntry(TASK_1_1, mkOrderedSet(UUID_1, UUID_2, UUID_3)), + mkEntry(TASK_1_2, mkOrderedSet(UUID_1, UUID_2, UUID_3)) + ); assertThat( assignActiveTaskMovements( tasksToCaughtUpClients, + tasksToClientByLag, getClientStatesMap(client1, client2, client3), new TreeMap<>(), new AtomicInteger(maxWarmupReplicas) @@ -107,9 +122,10 @@ public void shouldAssignAllTasksToClientsAndReturnFalseIfNoClientsAreCaughtUp() @Test public void shouldMoveTasksToCaughtUpClientsAndAssignWarmupReplicasInTheirPlace() { final int maxWarmupReplicas = Integer.MAX_VALUE; - final ClientState client1 = getClientStateWithActiveAssignment(singletonList(TASK_0_0)); - final ClientState client2 = getClientStateWithActiveAssignment(singletonList(TASK_0_1)); - final ClientState client3 = getClientStateWithActiveAssignment(singletonList(TASK_0_2)); + final Set allTasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2); + final ClientState client1 = getClientStateWithActiveAssignment(mkSet(TASK_0_0), mkSet(TASK_0_0), allTasks); + final ClientState client2 = getClientStateWithActiveAssignment(mkSet(TASK_0_1), mkSet(TASK_0_2), allTasks); + final ClientState client3 = getClientStateWithActiveAssignment(mkSet(TASK_0_2), mkSet(TASK_0_1), allTasks); final Map clientStates = getClientStatesMap(client1, client2, client3); final Map> tasksToCaughtUpClients = mkMap( @@ -117,11 +133,17 @@ public void shouldMoveTasksToCaughtUpClientsAndAssignWarmupReplicasInTheirPlace( mkEntry(TASK_0_1, mkSortedSet(UUID_3)), mkEntry(TASK_0_2, mkSortedSet(UUID_2)) ); + final Map> tasksToClientByLag = mkMap( + mkEntry(TASK_0_0, mkOrderedSet(UUID_1, UUID_2, UUID_3)), + mkEntry(TASK_0_1, mkOrderedSet(UUID_3, UUID_1, UUID_2)), + mkEntry(TASK_0_2, mkOrderedSet(UUID_2, UUID_1, UUID_3)) + ); assertThat( "should have assigned movements", assignActiveTaskMovements( tasksToCaughtUpClients, + tasksToClientByLag, clientStates, new TreeMap<>(), new AtomicInteger(maxWarmupReplicas) @@ -139,12 +161,60 @@ public void shouldMoveTasksToCaughtUpClientsAndAssignWarmupReplicasInTheirPlace( assertThat(client3, hasProperty("standbyTasks", ClientState::standbyTasks, mkSet(TASK_0_2))); } + @Test + public void shouldMoveTasksToMostCaughtUpClientsAndAssignWarmupReplicasInTheirPlace() { + final int maxWarmupReplicas = Integer.MAX_VALUE; + final Map client1Lags = mkMap(mkEntry(TASK_0_0, 10000L), mkEntry(TASK_0_1, 20000L), mkEntry(TASK_0_2, 30000L)); + final Map client2Lags = mkMap(mkEntry(TASK_0_2, 10000L), mkEntry(TASK_0_0, 20000L), mkEntry(TASK_0_1, 30000L)); + final Map client3Lags = mkMap(mkEntry(TASK_0_1, 10000L), mkEntry(TASK_0_2, 20000L), mkEntry(TASK_0_0, 30000L)); + + final ClientState client1 = getClientStateWithLags(mkSet(TASK_0_0), client1Lags); + final ClientState client2 = getClientStateWithLags(mkSet(TASK_0_1), client2Lags); + final ClientState client3 = getClientStateWithLags(mkSet(TASK_0_2), client3Lags); + // To test when the task is already a standby on the most caught up node + client3.assignStandby(TASK_0_1); + final Map clientStates = getClientStatesMap(client1, client2, client3); + + final Map> tasksToCaughtUpClients = mkMap( + mkEntry(TASK_0_0, mkSortedSet()), + mkEntry(TASK_0_1, mkSortedSet()), + mkEntry(TASK_0_2, mkSortedSet()) + ); + final Map> tasksToClientByLag = mkMap( + mkEntry(TASK_0_0, mkOrderedSet(UUID_1, UUID_2, UUID_3)), + mkEntry(TASK_0_1, mkOrderedSet(UUID_3, UUID_1, UUID_2)), + mkEntry(TASK_0_2, mkOrderedSet(UUID_2, UUID_3, UUID_1)) + ); + + assertThat( + "should have assigned movements", + assignActiveTaskMovements( + tasksToCaughtUpClients, + tasksToClientByLag, + clientStates, + new TreeMap<>(), + new AtomicInteger(maxWarmupReplicas) + ), + is(2) + ); + // The active tasks have changed to the ones that each client is most caught up on + assertThat(client1, hasProperty("activeTasks", ClientState::activeTasks, mkSet(TASK_0_0))); + assertThat(client2, hasProperty("activeTasks", ClientState::activeTasks, mkSet(TASK_0_2))); + assertThat(client3, hasProperty("activeTasks", ClientState::activeTasks, mkSet(TASK_0_1))); + + // we assigned warmups to migrate to the input active assignment + assertThat(client1, hasProperty("standbyTasks", ClientState::standbyTasks, mkSet())); + assertThat(client2, hasProperty("standbyTasks", ClientState::standbyTasks, mkSet(TASK_0_1))); + assertThat(client3, hasProperty("standbyTasks", ClientState::standbyTasks, mkSet(TASK_0_2))); + } + @Test public void shouldOnlyGetUpToMaxWarmupReplicasAndReturnTrue() { final int maxWarmupReplicas = 1; - final ClientState client1 = getClientStateWithActiveAssignment(singletonList(TASK_0_0)); - final ClientState client2 = getClientStateWithActiveAssignment(singletonList(TASK_0_1)); - final ClientState client3 = getClientStateWithActiveAssignment(singletonList(TASK_0_2)); + final Set allTasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2); + final ClientState client1 = getClientStateWithActiveAssignment(mkSet(TASK_0_0), mkSet(TASK_0_0), allTasks); + final ClientState client2 = getClientStateWithActiveAssignment(mkSet(TASK_0_1), mkSet(TASK_0_2), allTasks); + final ClientState client3 = getClientStateWithActiveAssignment(mkSet(TASK_0_2), mkSet(TASK_0_1), allTasks); final Map clientStates = getClientStatesMap(client1, client2, client3); final Map> tasksToCaughtUpClients = mkMap( @@ -152,11 +222,17 @@ public void shouldOnlyGetUpToMaxWarmupReplicasAndReturnTrue() { mkEntry(TASK_0_1, mkSortedSet(UUID_3)), mkEntry(TASK_0_2, mkSortedSet(UUID_2)) ); + final Map> tasksToClientByLag = mkMap( + mkEntry(TASK_0_0, mkOrderedSet(UUID_1, UUID_2, UUID_3)), + mkEntry(TASK_0_1, mkOrderedSet(UUID_3, UUID_1, UUID_2)), + mkEntry(TASK_0_2, mkOrderedSet(UUID_2, UUID_1, UUID_3)) + ); assertThat( "should have assigned movements", assignActiveTaskMovements( tasksToCaughtUpClients, + tasksToClientByLag, clientStates, new TreeMap<>(), new AtomicInteger(maxWarmupReplicas) @@ -182,19 +258,24 @@ public void shouldOnlyGetUpToMaxWarmupReplicasAndReturnTrue() { @Test public void shouldNotCountPreviousStandbyTasksTowardsMaxWarmupReplicas() { final int maxWarmupReplicas = 0; - final ClientState client1 = getClientStateWithActiveAssignment(emptyList()); + final Set allTasks = mkSet(TASK_0_0); + final ClientState client1 = getClientStateWithActiveAssignment(mkSet(), mkSet(TASK_0_0), allTasks); client1.assignStandby(TASK_0_0); - final ClientState client2 = getClientStateWithActiveAssignment(singletonList(TASK_0_0)); + final ClientState client2 = getClientStateWithActiveAssignment(mkSet(TASK_0_0), mkSet(), allTasks); final Map clientStates = getClientStatesMap(client1, client2); final Map> tasksToCaughtUpClients = mkMap( mkEntry(TASK_0_0, mkSortedSet(UUID_1)) ); + final Map> tasksToClientByLag = mkMap( + mkEntry(TASK_0_0, mkOrderedSet(UUID_1, UUID_2)) + ); assertThat( "should have assigned movements", assignActiveTaskMovements( tasksToCaughtUpClients, + tasksToClientByLag, clientStates, new TreeMap<>(), new AtomicInteger(maxWarmupReplicas) @@ -215,10 +296,35 @@ public void shouldNotCountPreviousStandbyTasksTowardsMaxWarmupReplicas() { } - private static ClientState getClientStateWithActiveAssignment(final Collection activeTasks) { - final ClientState client1 = new ClientState(1); + private static ClientState getClientStateWithActiveAssignment(final Set activeTasks, + final Set caughtUpTasks, + final Set allTasks) { + final Map lags = new HashMap<>(); + for (final TaskId task : allTasks) { + if (caughtUpTasks.contains(task)) { + lags.put(task, 0L); + } else { + lags.put(task, 10000L); + } + } + return getClientStateWithLags(activeTasks, lags); + } + + private static ClientState getClientStateWithLags(final Set activeTasks, + final Map taskLags) { + final ClientState client1 = new ClientState(activeTasks, emptySet(), taskLags, emptyMap(), 1); client1.assignActiveTasks(activeTasks); return client1; } + /** + * Creates a SortedSet with the sort order being the order of elements in the parameter list + */ + private static SortedSet mkOrderedSet(final UUID... clients) { + final List clientList = asList(clients); + final SortedSet set = new TreeSet<>(Comparator.comparing(clientList::indexOf)); + set.addAll(clientList); + return set; + } + } From 7243facb8d69a7252e6b9556b5eaee13e41bab7f Mon Sep 17 00:00:00 2001 From: John Roesler Date: Mon, 28 Mar 2022 11:35:43 -0500 Subject: [PATCH 055/447] MINOR: Fix stream-join metadata (#11952) #11356 inadvertently changed the (undefined) header forwarding behavior of stream-stream joins. This change does not define the behavior, but just restores the prior undefined behavior for continuity's sake. Defining the header-forwarding behavior is future work. Reviewers: Matthias J. Sax , Jorge Esteban Quilcate Otoya --- .../kstream/internals/KStreamKStreamJoin.java | 18 ++-- .../KStreamKStreamOuterJoinTest.java | 97 +++++++++++++++++++ .../apache/kafka/test/MockApiProcessor.java | 24 ++++- 3 files changed, 129 insertions(+), 10 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index 4d9ac2e1672d..ecf988d6140f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -155,7 +155,7 @@ public void process(final Record record) { // Emit all non-joined records which window has closed if (inputRecordTimestamp == sharedTimeTracker.streamTime) { - outerJoinStore.ifPresent(this::emitNonJoinedOuterRecords); + outerJoinStore.ifPresent(store -> emitNonJoinedOuterRecords(store, record)); } try (final WindowStoreIterator iter = otherWindowStore.fetch(record.key(), timeFrom, timeTo)) { @@ -209,7 +209,10 @@ public void process(final Record record) { } @SuppressWarnings("unchecked") - private void emitNonJoinedOuterRecords(final KeyValueStore, LeftOrRightValue> store) { + private void emitNonJoinedOuterRecords( + final KeyValueStore, LeftOrRightValue> store, + final Record record) { + // calling `store.all()` creates an iterator what is an expensive operation on RocksDB; // to reduce runtime cost, we try to avoid paying those cost @@ -235,10 +238,9 @@ private void emitNonJoinedOuterRecords(final KeyValueStore prevKey = null; while (it.hasNext()) { - final KeyValue, LeftOrRightValue> record = it.next(); - - final TimestampedKeyAndJoinSide timestampedKeyAndJoinSide = record.key; - final LeftOrRightValue value = record.value; + final KeyValue, LeftOrRightValue> next = it.next(); + final TimestampedKeyAndJoinSide timestampedKeyAndJoinSide = next.key; + final LeftOrRightValue value = next.value; final K key = timestampedKeyAndJoinSide.getKey(); final long timestamp = timestampedKeyAndJoinSide.getTimestamp(); sharedTimeTracker.minTime = timestamp; @@ -259,7 +261,9 @@ private void emitNonJoinedOuterRecords(final KeyValueStore(key, nullJoinedValue, timestamp)); + context().forward( + record.withKey(key).withValue(nullJoinedValue).withTimestamp(timestamp) + ); if (prevKey != null && !prevKey.equals(timestampedKeyAndJoinSide)) { // blind-delete the previous key from the outer window store now it is emitted; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java index acb4be46bbe9..8133e25ec4be 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java @@ -16,6 +16,9 @@ */ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.internals.RecordHeader; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; @@ -29,8 +32,10 @@ import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.StreamJoined; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.WindowBytesStoreSupplier; +import org.apache.kafka.streams.test.TestRecord; import org.apache.kafka.test.MockApiProcessor; import org.apache.kafka.test.MockApiProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; @@ -887,6 +892,98 @@ public void shouldNotEmitLeftJoinResultForAsymmetricAfterWindow() { } } + /** + * NOTE: Header forwarding is undefined behavior, but we still want to understand the + * behavior so that we can make decisions about defining it in the future. + */ + @Test + public void shouldForwardCurrentHeaders() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockApiProcessorSupplier supplier = new MockApiProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.ofTimeDifferenceAndGrace(ofMillis(100L), ofMillis(10L)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()) + ); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockApiProcessor processor = supplier.theCapturedProcessor(); + + inputTopic1.pipeInput(new TestRecord<>( + 0, + "A0", + new RecordHeaders(new Header[]{new RecordHeader("h", new byte[]{0x1})}), + 0L + )); + inputTopic2.pipeInput(new TestRecord<>( + 1, + "a0", + new RecordHeaders(new Header[]{new RecordHeader("h", new byte[]{0x2})}), + 0L + )); + // bump stream-time to trigger outer-join results + inputTopic2.pipeInput(new TestRecord<>( + 3, + "dummy", + new RecordHeaders(new Header[]{new RecordHeader("h", new byte[]{0x3})}), + (long) 211 + )); + + // Again, header forwarding is undefined, but the current observed behavior is that + // the headers pass through the forwarding record. + processor.checkAndClearProcessedRecords( + new Record<>( + 1, + "null+a0", + 0L, + new RecordHeaders(new Header[]{new RecordHeader("h", new byte[]{0x3})}) + ), + new Record<>( + 0, + "A0+null", + 0L, + new RecordHeaders(new Header[]{new RecordHeader("h", new byte[]{0x3})}) + ) + ); + + // verifies joined duplicates are emitted + inputTopic1.pipeInput(new TestRecord<>( + 2, + "A2", + new RecordHeaders(new Header[]{new RecordHeader("h", new byte[]{0x4})}), + 200L + )); + inputTopic2.pipeInput(new TestRecord<>( + 2, + "a2", + new RecordHeaders(new Header[]{new RecordHeader("h", new byte[]{0x5})}), + 200L + )); + + processor.checkAndClearProcessedRecords( + new Record<>( + 2, + "A2+a2", + 200L, + new RecordHeaders(new Header[]{new RecordHeader("h", new byte[]{0x5})}) + ) + ); + } + } + private void testUpperWindowBound(final int[] expectedKeys, final TopologyTestDriver driver, final MockApiProcessor processor) { diff --git a/streams/src/test/java/org/apache/kafka/test/MockApiProcessor.java b/streams/src/test/java/org/apache/kafka/test/MockApiProcessor.java index dd56bad58469..fc62c73f9935 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockApiProcessor.java +++ b/streams/src/test/java/org/apache/kafka/test/MockApiProcessor.java @@ -29,13 +29,14 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; public class MockApiProcessor implements Processor { - private final ArrayList> processed = new ArrayList<>(); + private final ArrayList> processed = new ArrayList<>(); private final Map> lastValueAndTimestampPerKey = new HashMap<>(); private final ArrayList punctuatedStreamTime = new ArrayList<>(); @@ -83,7 +84,7 @@ public void process(final Record record) { lastValueAndTimestampPerKey.remove(key); } - processed.add(keyValueTimestamp); + processed.add(record); if (commitRequested) { context.commit(); @@ -92,6 +93,20 @@ public void process(final Record record) { } public void checkAndClearProcessResult(final KeyValueTimestamp... expected) { + assertThat("the number of outputs:" + processed, processed.size(), is(expected.length)); + for (int i = 0; i < expected.length; i++) { + final Record record = processed.get(i); + assertThat( + "output[" + i + "]:", + new KeyValueTimestamp<>(record.key(), record.value(), record.timestamp()), + is(expected[i]) + ); + } + + processed.clear(); + } + + public void checkAndClearProcessedRecords(final Record... expected) { assertThat("the number of outputs:" + processed, processed.size(), is(expected.length)); for (int i = 0; i < expected.length; i++) { assertThat("output[" + i + "]:", processed.get(i), is(expected[i])); @@ -121,7 +136,10 @@ public void checkAndClearPunctuateResult(final PunctuationType type, final long. } public ArrayList> processed() { - return processed; + return processed + .stream() + .map(r -> new KeyValueTimestamp<>(r.key(), r.value(), r.timestamp())) + .collect(Collectors.toCollection(ArrayList::new)); } public Map> lastValueAndTimestampPerKey() { From 2a270591f868c59bae191ec059ef982dea22db96 Mon Sep 17 00:00:00 2001 From: Sanjana Kaundinya Date: Mon, 28 Mar 2022 16:20:21 -0700 Subject: [PATCH 056/447] MINOR: Improved display names for parameterized KRaft and ZK tests (#11957) This patch adds display names for KRaft and ZK tests. Without this, it becomes hard to understand in Jenkins test reports which test failed. With this addition, it becomes more clear which method in the test suite fails. Reviewers: Colin P. McCabe --- .../kafka/api/AuthorizerIntegrationTest.scala | 194 +++++++++--------- .../api/TransactionsExpirationTest.scala | 4 +- .../scala/kafka/utils/TestInfoUtils.scala | 1 + .../kafka/integration/MinIsrConfigTest.scala | 4 +- .../unit/kafka/metrics/MetricsTest.scala | 26 +-- .../server/CreateTopicsRequestTest.scala | 12 +- .../CreateTopicsRequestWithPolicyTest.scala | 5 +- .../server/DynamicConfigChangeTest.scala | 32 +-- .../kafka/server/MetadataRequestTest.scala | 35 ++-- .../kafka/server/ServerShutdownTest.scala | 14 +- 10 files changed, 164 insertions(+), 163 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index a4323bed2c0b..ef1b8d96fb4d 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -23,7 +23,7 @@ import kafka.log.LogConfig import kafka.security.authorizer.{AclAuthorizer, AclEntry} import kafka.security.authorizer.AclEntry.WildcardHost import kafka.server.{BaseRequestTest, KafkaConfig} -import kafka.utils.TestUtils +import kafka.utils.{TestInfoUtils, TestUtils} import kafka.utils.TestUtils.waitUntilTrue import org.apache.kafka.clients.admin.{Admin, AlterConfigOp} import org.apache.kafka.clients.consumer._ @@ -752,7 +752,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testAuthorizationWithTopicExisting(quorum: String): Unit = { //First create the topic so we have a valid topic ID @@ -806,7 +806,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { /* * even if the topic doesn't exist, request APIs should not leak the topic name */ - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testAuthorizationWithTopicNotExisting(quorum: String): Unit = { val id = Uuid.randomUuid() @@ -831,7 +831,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { sendRequests(requestKeyToRequest, false, topicNames) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @CsvSource(value = Array("zk,false", "zk,true", "kraft,false", "kraft,true")) def testTopicIdAuthorization(quorum: String, withTopicExisting: Boolean): Unit = { val topicId = if (withTopicExisting) { @@ -884,7 +884,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { /* * even if the topic doesn't exist, request APIs should not leak the topic name */ - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testAuthorizationFetchV12WithTopicNotExisting(quorum: String): Unit = { val id = Uuid.ZERO_UUID @@ -896,7 +896,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { sendRequests(requestKeyToRequest, false, topicNames) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCreateTopicAuthorizationWithClusterCreate(quorum: String): Unit = { removeAllClientAcls() @@ -909,7 +909,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { sendRequestAndVerifyResponseError(createTopicsRequest, resources, isAuthorized = true) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testFetchFollowerRequest(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -929,7 +929,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { sendRequestAndVerifyResponseError(request, resources, isAuthorized = true) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testIncrementalAlterConfigsRequestRequiresClusterPermissionForBrokerLogger(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -953,7 +953,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { sendRequestAndVerifyResponseError(request, resources, isAuthorized = true) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testOffsetsForLeaderEpochClusterPermission(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -972,7 +972,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { sendRequestAndVerifyResponseError(request, resources, isAuthorized = true) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testProduceWithNoTopicAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -980,7 +980,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords, tp)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testProduceWithTopicDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -989,7 +989,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords, tp)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testProduceWithTopicRead(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -998,7 +998,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords, tp)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testProduceWithTopicWrite(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1007,13 +1007,13 @@ class AuthorizerIntegrationTest extends BaseRequestTest { sendRecords(producer, numRecords, tp) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCreatePermissionOnTopicToWriteToNonExistentTopic(quorum: String): Unit = { testCreatePermissionNeededToWriteToNonExistentTopic(TOPIC) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCreatePermissionOnClusterToWriteToNonExistentTopic(quorum: String): Unit = { testCreatePermissionNeededToWriteToNonExistentTopic(CLUSTER) @@ -1032,7 +1032,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { sendRecords(producer, numRecords, tp) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testConsumeUsingAssignWithNoAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1047,7 +1047,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertThrows(classOf[TopicAuthorizationException], () => consumeRecords(consumer)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testSimpleConsumeWithOffsetLookupAndNoGroupAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1067,7 +1067,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertEquals(group, e.groupId()) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testSimpleConsumeWithExplicitSeekAndNoGroupAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1087,7 +1087,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { consumeRecords(consumer) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testConsumeWithoutTopicDescribeAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1105,7 +1105,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertEquals(Collections.singleton(topic), e.unauthorizedTopics()) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testConsumeWithTopicDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1124,7 +1124,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertEquals(Collections.singleton(topic), e.unauthorizedTopics()) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testConsumeWithTopicWrite(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1143,7 +1143,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertEquals(Collections.singleton(topic), e.unauthorizedTopics()) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testConsumeWithTopicAndGroupRead(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1162,7 +1162,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } @nowarn("cat=deprecation") - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testPatternSubscriptionWithNoTopicAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1180,7 +1180,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertTrue(consumer.subscription.isEmpty) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testPatternSubscriptionWithTopicDescribeOnlyAndGroupRead(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1199,7 +1199,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } @nowarn("cat=deprecation") - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testPatternSubscriptionWithTopicAndGroupRead(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1232,7 +1232,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } @nowarn("cat=deprecation") - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testPatternSubscriptionMatchingInternalTopic(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1262,7 +1262,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testPatternSubscriptionMatchingInternalTopicWithDescribeOnlyPermission(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1288,7 +1288,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertEquals(Collections.singleton(GROUP_METADATA_TOPIC_NAME), e.unauthorizedTopics()) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testPatternSubscriptionNotMatchingInternalTopic(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1307,7 +1307,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { consumeRecords(consumer) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCreatePermissionOnTopicToReadFromNonExistentTopic(quorum: String): Unit = { testCreatePermissionNeededToReadFromNonExistentTopic("newTopic", @@ -1315,7 +1315,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { TOPIC) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCreatePermissionOnClusterToReadFromNonExistentTopic(quorum: String): Unit = { testCreatePermissionNeededToReadFromNonExistentTopic("newTopic", @@ -1348,7 +1348,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { }, "Partition metadata not propagated.") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCreatePermissionMetadataRequestAutoCreate(quorum: String): Unit = { val readAcls = topicReadAcl(topicResource) @@ -1370,14 +1370,14 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCommitWithNoAccess(quorum: String): Unit = { val consumer = createConsumer() assertThrows(classOf[GroupAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCommitWithNoTopicAccess(quorum: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupResource) @@ -1385,7 +1385,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertThrows(classOf[TopicAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCommitWithTopicWrite(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1396,7 +1396,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertThrows(classOf[TopicAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCommitWithTopicDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1407,7 +1407,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertThrows(classOf[TopicAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCommitWithNoGroupAccess(quorum: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicResource) @@ -1415,7 +1415,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertThrows(classOf[GroupAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCommitWithTopicAndGroupRead(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1425,7 +1425,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testOffsetFetchWithNoAccess(quorum: String): Unit = { val consumer = createConsumer() @@ -1433,7 +1433,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertThrows(classOf[TopicAuthorizationException], () => consumer.position(tp)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testOffsetFetchWithNoGroupAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1443,7 +1443,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertThrows(classOf[GroupAuthorizationException], () => consumer.position(tp)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testOffsetFetchWithNoTopicAccess(quorum: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupResource) @@ -1452,7 +1452,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertThrows(classOf[TopicAuthorizationException], () => consumer.position(tp)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testOffsetFetchAllTopicPartitionsAuthorization(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1484,7 +1484,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertEquals(offset, offsetFetchResponse.partitionDataMap(group).get(tp).offset) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testOffsetFetchMultipleGroupsAuthorization(quorum: String): Unit = { val groups: Seq[String] = (1 to 5).map(i => s"group$i") @@ -1640,7 +1640,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testOffsetFetchTopicDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1651,7 +1651,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { consumer.position(tp) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testOffsetFetchWithTopicAndGroupRead(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1662,14 +1662,14 @@ class AuthorizerIntegrationTest extends BaseRequestTest { consumer.position(tp) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testMetadataWithNoTopicAccess(quorum: String): Unit = { val consumer = createConsumer() assertThrows(classOf[TopicAuthorizationException], () => consumer.partitionsFor(topic)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testMetadataWithTopicDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1678,14 +1678,14 @@ class AuthorizerIntegrationTest extends BaseRequestTest { consumer.partitionsFor(topic) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testListOffsetsWithNoTopicAccess(quorum: String): Unit = { val consumer = createConsumer() assertThrows(classOf[TopicAuthorizationException], () => consumer.endOffsets(Set(tp).asJava)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testListOffsetsWithTopicDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1694,7 +1694,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { consumer.endOffsets(Set(tp).asJava) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDescribeGroupApiWithNoGroupAcl(quorum: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DESCRIBE, ALLOW)), topicResource) @@ -1702,7 +1702,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { TestUtils.assertFutureExceptionTypeEquals(result.describedGroups().get(group), classOf[GroupAuthorizationException]) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDescribeGroupApiWithGroupDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1711,7 +1711,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { createAdminClient().describeConsumerGroups(Seq(group).asJava).describedGroups().get(group).get() } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDescribeGroupCliWithGroupDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1725,7 +1725,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { consumerGroupService.close() } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testListGroupApiWithAndWithoutListGroupAcls(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1774,7 +1774,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { otherConsumer.close() } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDeleteGroupApiWithDeleteGroupAcl(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1788,7 +1788,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { createAdminClient().deleteConsumerGroups(Seq(group).asJava).deletedGroups().get(group).get() } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDeleteGroupApiWithNoDeleteGroupAcl(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1802,14 +1802,14 @@ class AuthorizerIntegrationTest extends BaseRequestTest { TestUtils.assertFutureExceptionTypeEquals(result.deletedGroups().get(group), classOf[GroupAuthorizationException]) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDeleteGroupApiWithNoDeleteGroupAcl2(quorum: String): Unit = { val result = createAdminClient().deleteConsumerGroups(Seq(group).asJava) TestUtils.assertFutureExceptionTypeEquals(result.deletedGroups().get(group), classOf[GroupAuthorizationException]) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDeleteGroupOffsetsWithAcl(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1825,7 +1825,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertNull(result.partitionResult(tp).get()) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDeleteGroupOffsetsWithoutDeleteAcl(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1840,7 +1840,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { TestUtils.assertFutureExceptionTypeEquals(result.all(), classOf[GroupAuthorizationException]) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDeleteGroupOffsetsWithDeleteAclWithoutTopicAcl(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1861,21 +1861,21 @@ class AuthorizerIntegrationTest extends BaseRequestTest { TestUtils.assertFutureExceptionTypeEquals(result.partitionResult(tp), classOf[TopicAuthorizationException]) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDeleteGroupOffsetsWithNoAcl(quorum: String): Unit = { val result = createAdminClient().deleteConsumerGroupOffsets(group, Set(tp).asJava) TestUtils.assertFutureExceptionTypeEquals(result.all(), classOf[GroupAuthorizationException]) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testUnauthorizedDeleteTopicsWithoutDescribe(quorum: String): Unit = { val deleteResponse = connectAndReceive[DeleteTopicsResponse](deleteTopicsRequest) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, deleteResponse.data.responses.find(topic).errorCode) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testUnauthorizedDeleteTopicsWithDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1884,7 +1884,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, deleteResponse.data.responses.find(topic).errorCode) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDeleteTopicsWithWildCardAuth(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1893,7 +1893,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertEquals(Errors.NONE.code, deleteResponse.data.responses.find(topic).errorCode) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testUnauthorizedDeleteRecordsWithoutDescribe(quorum: String): Unit = { val deleteRecordsResponse = connectAndReceive[DeleteRecordsResponse](deleteRecordsRequest) @@ -1901,7 +1901,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { partitions.asScala.head.errorCode) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testUnauthorizedDeleteRecordsWithDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1911,7 +1911,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { partitions.asScala.head.errorCode) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDeleteRecordsWithWildCardAuth(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1921,14 +1921,14 @@ class AuthorizerIntegrationTest extends BaseRequestTest { partitions.asScala.head.errorCode) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testUnauthorizedCreatePartitions(quorum: String): Unit = { val createPartitionsResponse = connectAndReceive[CreatePartitionsResponse](createPartitionsRequest) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, createPartitionsResponse.data.results.asScala.head.errorCode) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCreatePartitionsWithWildCardAuth(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1937,7 +1937,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertEquals(Errors.NONE.code, createPartitionsResponse.data.results.asScala.head.errorCode) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testTransactionalProducerInitTransactionsNoWriteTransactionalIdAcl(quorum: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DESCRIBE, ALLOW)), transactionalIdResource) @@ -1945,14 +1945,14 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertThrows(classOf[TransactionalIdAuthorizationException], () => producer.initTransactions()) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testTransactionalProducerInitTransactionsNoDescribeTransactionalIdAcl(quorum: String): Unit = { val producer = buildTransactionalProducer() assertThrows(classOf[TransactionalIdAuthorizationException], () => producer.initTransactions()) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testSendOffsetsWithNoConsumerGroupDescribeAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1968,7 +1968,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { () => producer.sendOffsetsToTransaction(Map(tp -> new OffsetAndMetadata(0L)).asJava, new ConsumerGroupMetadata(group))) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testSendOffsetsWithNoConsumerGroupWriteAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -1983,7 +1983,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { () => producer.sendOffsetsToTransaction(Map(tp -> new OffsetAndMetadata(0L)).asJava, new ConsumerGroupMetadata(group))) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testIdempotentProducerNoIdempotentWriteAclInInitProducerId(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2022,7 +2022,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertClusterAuthFailure() } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testIdempotentProducerNoIdempotentWriteAclInProduce(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2051,7 +2051,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertTrue(e.getCause.isInstanceOf[TopicAuthorizationException]) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def shouldInitTransactionsWhenAclSet(quorum: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, WRITE, ALLOW)), transactionalIdResource) @@ -2059,7 +2059,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { producer.initTransactions() } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testTransactionalProducerTopicAuthorizationExceptionInSendCallback(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2076,7 +2076,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertEquals(Set(topic), e.unauthorizedTopics.asScala) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testTransactionalProducerTopicAuthorizationExceptionInCommit(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2094,7 +2094,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { }) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessDuringSend(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2109,7 +2109,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { JTestUtils.assertFutureThrows(future, classOf[TransactionalIdAuthorizationException]) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnEndTransaction(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2124,7 +2124,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertThrows(classOf[TransactionalIdAuthorizationException], () => producer.commitTransaction()) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testListTransactionsAuthorization(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2158,7 +2158,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertListTransactionResult(expectedTransactionalIds = Set(transactionalId)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def shouldNotIncludeUnauthorizedTopicsInDescribeTransactionsResponse(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2181,7 +2181,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertEquals(List.empty, transactionStateData.topics.asScala.toList) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def shouldSuccessfullyAbortTransactionAfterTopicAuthorizationException(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2201,7 +2201,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { producer.abortTransaction() } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnSendOffsetsToTxn(quorum: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, WRITE, ALLOW)), transactionalIdResource) @@ -2217,7 +2217,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { }) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def shouldSendSuccessfullyWhenIdempotentAndHasCorrectACL(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2228,7 +2228,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } // Verify that metadata request without topics works without any ACLs and returns cluster id - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testClusterId(quorum: String): Unit = { val request = new requests.MetadataRequest.Builder(List.empty.asJava, false).build() @@ -2237,7 +2237,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertFalse(response.clusterId.isEmpty, "Cluster id not returned") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testAuthorizeByResourceTypeMultipleAddAndRemove(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2255,7 +2255,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testAuthorizeByResourceTypeIsolationUnrelatedDenyWontDominateAllow(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2278,7 +2278,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertIdempotentSendSuccess() } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testAuthorizeByResourceTypeDenyTakesPrecedence(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2291,7 +2291,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertIdempotentSendAuthorizationFailure() } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testAuthorizeByResourceTypeWildcardResourceDenyDominate(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2309,7 +2309,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertIdempotentSendAuthorizationFailure() } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testAuthorizeByResourceTypePrefixedResourceDenyDominate(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2323,7 +2323,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertIdempotentSendAuthorizationFailure() } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testMetadataClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String): Unit = { removeAllClientAcls() @@ -2334,7 +2334,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testMetadataClusterAuthorizedOperationsWithDescribeAndAlterCluster(quorum: String): Unit = { removeAllClientAcls() @@ -2355,7 +2355,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDescribeTopicAclWithOperationAll(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2380,7 +2380,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { assertEquals(Errors.NONE, topicResponse.error) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDescribeTopicConfigsAclWithOperationAll(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2420,7 +2420,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDescribeClusterClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String): Unit = { removeAllClientAcls() @@ -2430,7 +2430,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDescribeClusterClusterAuthorizedOperationsWithDescribeAndAlterCluster(quorum: String): Unit = { removeAllClientAcls() @@ -2450,7 +2450,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testHostAddressBasedAcls(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) @@ -2484,7 +2484,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCreateAndCloseConsumerWithNoAccess(quorum: String): Unit = { val consumer = createConsumer() diff --git a/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala index 91e7ae2cc188..ddf3a9746028 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala @@ -21,7 +21,7 @@ import java.util.Properties import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig -import kafka.utils.TestUtils +import kafka.utils.{TestInfoUtils, TestUtils} import kafka.utils.TestUtils.consumeRecords import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer.KafkaProducer @@ -68,7 +68,7 @@ class TransactionsExpirationTest extends KafkaServerTestHarness { super.tearDown() } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testBumpTransactionalEpochAfterInvalidProducerIdMapping(quorum: String): Unit = { producer.initTransactions() diff --git a/core/src/test/scala/kafka/utils/TestInfoUtils.scala b/core/src/test/scala/kafka/utils/TestInfoUtils.scala index ecd656e0fb65..fa48024f313d 100644 --- a/core/src/test/scala/kafka/utils/TestInfoUtils.scala +++ b/core/src/test/scala/kafka/utils/TestInfoUtils.scala @@ -43,4 +43,5 @@ object TestInfoUtils { false } } + final val TestWithParameterizedQuorumName = "{displayName}.quorum={0}" } diff --git a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala index 35f7cc4a6bbf..3ed7e8dc57cb 100644 --- a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala +++ b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala @@ -21,7 +21,7 @@ import java.util.Properties import scala.collection.Seq import kafka.server.KafkaConfig -import kafka.utils.TestUtils +import kafka.utils.{TestInfoUtils, TestUtils} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -30,7 +30,7 @@ class MinIsrConfigTest extends KafkaServerTestHarness { overridingProps.put(KafkaConfig.MinInSyncReplicasProp, "5") def generateConfigs: Seq[KafkaConfig] = TestUtils.createBrokerConfigs(1, zkConnectOrNull).map(KafkaConfig.fromProps(_, overridingProps)) - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDeaultKafkaConfig(quorum: String): Unit = { assert(brokers.head.logManager.initialDefaultConfig.minInSyncReplicas == 5) diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala index f4e69f9f2b4f..3889ff350296 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -53,7 +53,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { val nMessages = 2 - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testMetricsReporterAfterDeletingTopic(quorum: String): Unit = { val topic = "test-topic-metric" @@ -63,7 +63,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertEquals(Set.empty, topicMetricGroups(topic), "Topic metrics exists after deleteTopic") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testBrokerTopicMetricsUnregisteredAfterDeletingTopic(quorum: String): Unit = { val topic = "test-broker-topic-metric" @@ -78,7 +78,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertEquals(Set.empty, topicMetricGroups(topic), "Topic metrics exists after deleteTopic") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testClusterIdMetric(quorum: String): Unit = { // Check if clusterId metric exists. @@ -86,7 +86,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=ClusterId"), 1) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testBrokerStateMetric(quorum: String): Unit = { // Check if BrokerState metric exists. @@ -94,7 +94,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=BrokerState"), 1) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testYammerMetricsCountMetric(quorum: String): Unit = { // Check if yammer-metrics-count metric exists. @@ -102,7 +102,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=yammer-metrics-count"), 1) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testLinuxIoMetrics(quorum: String): Unit = { // Check if linux-disk-{read,write}-bytes metrics either do or do not exist depending on whether we are or are not @@ -114,7 +114,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=$name"), expectedCount)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testJMXFilter(quorum: String): Unit = { // Check if cluster id metrics is not exposed in JMX @@ -124,7 +124,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { .isRegistered(new ObjectName(s"$requiredKafkaServerPrefix=ClusterId"))) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testUpdateJMXFilter(quorum: String): Unit = { // verify previously exposed metrics are removed and existing matching metrics are added @@ -137,7 +137,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { .isRegistered(new ObjectName(s"$requiredKafkaServerPrefix=ClusterId"))) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testGeneralBrokerTopicMetricsAreGreedilyRegistered(quorum: String): Unit = { val topic = "test-broker-topic-metric" @@ -152,7 +152,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertTrue(topicMetricGroups(topic).nonEmpty, "Topic metrics aren't registered") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testWindowsStyleTagNames(quorum: String): Unit = { val path = "C:\\windows-path\\kafka-logs" @@ -162,7 +162,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assert(metric.getMBeanName.endsWith(expectedMBeanName)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testBrokerTopicMetricsBytesInOut(quorum: String): Unit = { val topic = "test-bytes-in-out" @@ -210,7 +210,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertTrue(TestUtils.meterCount(bytesOut) > initialBytesOut) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk")) def testZkControllerMetrics(quorum: String): Unit = { val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics @@ -232,7 +232,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { * Test that the metrics are created with the right name, testZooKeeperStateChangeRateMetrics * and testZooKeeperSessionStateMetric in ZooKeeperClientTest test the metrics behaviour. */ - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testSessionExpireListenerMetrics(quorum: String): Unit = { val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala index 94eb213446cd..57834234cc10 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala @@ -31,7 +31,7 @@ import org.junit.jupiter.params.provider.ValueSource import scala.jdk.CollectionConverters._ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testValidCreateTopicsRequests(quorum: String): Unit = { // Generated assignments @@ -61,7 +61,7 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { topicReq("topic14", replicationFactor = -1, numPartitions = 2)))) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testErrorCreateTopicsRequests(quorum: String): Unit = { val existingTopic = "existing-topic" @@ -102,7 +102,7 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { validateTopicExists("partial-none") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk")) def testCreateTopicsWithVeryShortTimeouts(quorum: String): Unit = { // When using ZooKeeper, we don't expect a request to ever complete within 1ms. @@ -132,7 +132,7 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testInvalidCreateTopicsRequests(quorum: String): Unit = { // Partitions/ReplicationFactor and ReplicaAssignment @@ -147,7 +147,7 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { Map("bad-args-topic" -> error(Errors.INVALID_REQUEST)), checkErrorMessage = false) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk")) def testNotController(quorum: String): Unit = { // Note: we don't run this test when in KRaft mode, because KRaft doesn't have this @@ -157,7 +157,7 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { assertEquals(1, response.errorCounts().get(Errors.NOT_CONTROLLER)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk")) def testCreateTopicsRequestVersions(quorum: String): Unit = { // Note: we don't run this test when in KRaft mode, because kraft does not yet support returning topic diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala index d480c7b472f8..fc46640c1b65 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala @@ -20,6 +20,7 @@ package kafka.server import java.util import java.util.Properties import kafka.log.LogConfig +import kafka.utils.TestInfoUtils import org.apache.kafka.common.errors.PolicyViolationException import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.protocol.Errors @@ -44,7 +45,7 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest Seq(properties) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testValidCreateTopicsRequests(quorum: String): Unit = { validateValidCreateTopicsRequests(topicsReq(Seq(topicReq("topic1", @@ -63,7 +64,7 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest assignment = Map(0 -> List(1, 0), 1 -> List(0, 1)))))) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testErrorCreateTopicsRequests(quorum: String): Unit = { val existingTopic = "existing-topic" diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 3a3725976b2d..09491bf1e1f9 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -56,7 +56,7 @@ import scala.jdk.CollectionConverters._ class DynamicConfigChangeTest extends KafkaServerTestHarness { def generateConfigs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnectOrNull))) - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testConfigChange(quorum: String): Unit = { if (!isKRaftTest()) { @@ -94,7 +94,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDynamicTopicConfigChange(quorum: String): Unit = { val tp = new TopicPartition("test", 0) @@ -136,7 +136,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } @nowarn("cat=deprecation") - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk")) def testMessageFormatVersionChange(quorum: String): Unit = { val tp = new TopicPartition("test", 0) @@ -206,7 +206,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testClientIdQuotaConfigChange(quorum: String): Unit = { val m = new util.HashMap[String, String] @@ -214,7 +214,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testUserQuotaConfigChange(quorum: String): Unit = { val m = new util.HashMap[String, String] @@ -222,7 +222,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testUserClientIdQuotaChange(quorum: String): Unit = { val m = new util.HashMap[String, String] @@ -231,7 +231,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDefaultClientIdQuotaConfigChange(quorum: String): Unit = { val m = new util.HashMap[String, String] @@ -239,7 +239,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDefaultUserQuotaConfigChange(quorum: String): Unit = { val m = new util.HashMap[String, String] @@ -247,7 +247,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDefaultUserClientIdQuotaConfigChange(quorum: String): Unit = { val m = new util.HashMap[String, String] @@ -256,7 +256,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk")) def testQuotaInitialization(quorum: String): Unit = { val server = servers.head @@ -288,7 +288,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { assertEquals(Quota.upperBound(200000), quotaManagers.fetch.quota("ANONYMOUS", "overriddenUserClientId")) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testIpQuotaInitialization(quorum: String): Unit = { val broker = brokers.head @@ -328,7 +328,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testIpQuotaConfigChange(quorum: String): Unit = { val admin = createAdminClient() @@ -370,7 +370,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk")) def testConfigChangeOnNonExistingTopic(quorum: String): Unit = { val topic = TestUtils.tempTopic() @@ -379,7 +379,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { assertThrows(classOf[UnknownTopicOrPartitionException], () => adminZkClient.changeTopicConfig(topic, logProps)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testConfigChangeOnNonExistingTopicWithAdminClient(quorum: String): Unit = { val topic = TestUtils.tempTopic() @@ -397,7 +397,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk")) def testProcessNotification(quorum: String): Unit = { val props = new Properties() @@ -430,7 +430,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { verify(handler).processConfigChanges(anyString, any[Properties]) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testConfigureDefaultTopic(quorum: String): Unit = { val admin = createAdminClient() diff --git a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala index 27a9f6b05954..7b9576026c18 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala @@ -18,8 +18,7 @@ package kafka.server import java.util.Optional - -import kafka.utils.TestUtils +import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.common.Uuid import org.apache.kafka.common.errors.UnsupportedVersionException import org.apache.kafka.common.internals.Topic @@ -42,7 +41,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { doSetup(testInfo, createOffsetsTopic = false) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testClusterIdWithRequestVersion1(quorum: String): Unit = { val v1MetadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) @@ -50,7 +49,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { assertNull(v1ClusterId, s"v1 clusterId should be null") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testClusterIdIsValid(quorum: String): Unit = { val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(2.toShort)) @@ -61,9 +60,9 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { * This test only runs in ZK mode because in KRaft mode, the controller ID visible to * the client is randomized. */ - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk")) - def testControllerId(): Unit = { + def testControllerId(quorum: String): Unit = { val controllerServer = servers.find(_.kafkaController.isActive).get val controllerId = controllerServer.config.brokerId val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) @@ -84,7 +83,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { }, "Controller id should match the active controller after failover", 5000) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testRack(quorum: String): Unit = { val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) @@ -94,7 +93,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testIsInternal(quorum: String): Unit = { val internalTopic = Topic.GROUP_METADATA_TOPIC_NAME @@ -116,7 +115,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { assertEquals(Set(internalTopic).asJava, metadataResponse.buildCluster().internalTopics) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testNoTopicsRequest(quorum: String): Unit = { // create some topics @@ -130,7 +129,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { assertTrue(metadataResponse.topicMetadata.isEmpty, "Response should have no topics") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testAutoTopicCreation(quorum: String): Unit = { val topic1 = "t1" @@ -161,7 +160,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testAutoCreateTopicWithInvalidReplicationFactor(quorum: String): Unit = { // Shutdown all but one broker so that the number of brokers is less than the default replication factor @@ -181,7 +180,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { assertEquals(0, topicMetadata.partitionMetadata.size) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk")) def testAutoCreateOfCollidingTopics(quorum: String): Unit = { val topic1 = "testAutoCreate.Topic" @@ -212,7 +211,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { assertTrue(partitionMetadata.leaderId.get >= 0) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testAllTopicsRequest(quorum: String): Unit = { // create some topics @@ -230,7 +229,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { assertEquals(2, metadataResponseV1.topicMetadata.size(), "V1 Response should have 2 (all) topics") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testTopicIdsInResponse(quorum: String): Unit = { val replicaAssignment = Map(0 -> Seq(1, 2, 0), 1 -> Seq(2, 0, 1)) @@ -260,7 +259,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { /** * Preferred replica should be the first item in the replicas list */ - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testPreferredReplica(quorum: String): Unit = { val replicaAssignment = Map(0 -> Seq(1, 2, 0), 1 -> Seq(2, 0, 1)) @@ -284,7 +283,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testReplicaDownResponse(quorum: String): Unit = { val replicaDownTopic = "replicaDown" @@ -330,7 +329,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { assertEquals(replicaCount, v1PartitionMetadata.replicaIds.size, s"Response should have $replicaCount replicas") } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testIsrAfterBrokerShutDownAndJoinsBack(quorum: String): Unit = { def checkIsr[B <: KafkaBroker]( @@ -368,7 +367,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { checkIsr(brokers, topic) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testAliveBrokersWithNoTopics(quorum: String): Unit = { def checkMetadata[B <: KafkaBroker]( diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 16d17d2fd219..0464a340744a 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -16,7 +16,7 @@ */ package kafka.server -import kafka.utils.{CoreUtils, TestUtils} +import kafka.utils.{CoreUtils, TestInfoUtils, TestUtils} import java.io.{DataInputStream, File} import java.net.ServerSocket @@ -82,7 +82,7 @@ class ServerShutdownTest extends KafkaServerTestHarness { super.setUp(testInfo) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCleanShutdown(quorum: String): Unit = { @@ -141,7 +141,7 @@ class ServerShutdownTest extends KafkaServerTestHarness { producer.close() } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCleanShutdownAfterFailedStartup(quorum: String): Unit = { if (quorum == "zk") { @@ -156,7 +156,7 @@ class ServerShutdownTest extends KafkaServerTestHarness { } } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testCleanShutdownAfterFailedStartupDueToCorruptLogs(quorum: String): Unit = { createTopic(topic) @@ -168,7 +168,7 @@ class ServerShutdownTest extends KafkaServerTestHarness { verifyCleanShutdownAfterFailedStartup[KafkaStorageException](quorum) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk")) def testCleanShutdownWithZkUnavailable(quorum: String): Unit = { shutdownZooKeeper() @@ -177,7 +177,7 @@ class ServerShutdownTest extends KafkaServerTestHarness { verifyNonDaemonThreadsStatus() } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("kraft")) def testCleanShutdownWithKRaftControllerUnavailable(quorum: String): Unit = { shutdownKRaftController() @@ -212,7 +212,7 @@ class ServerShutdownTest extends KafkaServerTestHarness { .count(isNonDaemonKafkaThread)) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testConsecutiveShutdown(quorum: String): Unit = { shutdownBroker() From 19a62697803a91fc24c1a704881e531f848a759c Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 28 Mar 2022 21:19:22 -0700 Subject: [PATCH 057/447] MINOR: Fix log4j entry in RepartitionTopics (#11958) I noticed two issues in the log4j entry: 1. It's formatted as "{}...{}" + param1, param2; effectively it is one param only, and the printed line is effectively mis-aligned: we always print Subtopology [sourceTopics set] was missing source topics {} 2. Even fix 1) is not enough, since topologyName may be null. On the other hand I think the original goal is not to print the topology name but the sub-topology id since it's within the per-sub-topology loop. Reviewers: Guozhang Wang --- .../streams/processor/internals/RepartitionTopics.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RepartitionTopics.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RepartitionTopics.java index 44be19ef8548..ad94bc88cf8d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RepartitionTopics.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RepartitionTopics.java @@ -154,15 +154,17 @@ private Map computeRepartitionTopicConfig(final Clu final Set missingSourceTopicsForSubtopology = computeMissingExternalSourceTopics(topicsInfo, clusterMetadata); missingSourceTopicsForTopology.addAll(missingSourceTopicsForSubtopology); if (!missingSourceTopicsForSubtopology.isEmpty()) { - missingInputTopicsBySubtopology.put(subtopologyEntry.getKey(), missingSourceTopicsForSubtopology); + final Subtopology subtopology = subtopologyEntry.getKey(); + missingInputTopicsBySubtopology.put(subtopology, missingSourceTopicsForSubtopology); log.error("Subtopology {} was missing source topics {} and will be excluded from the current assignment, " + "this can be due to the consumer client's metadata being stale or because they have " + "not been created yet. Please verify that you have created all input topics; if they " + "do exist, you just need to wait for the metadata to be updated, at which time a new " - + "rebalance will be kicked off automatically and the topology will be retried at that time." - + topologyName, missingSourceTopicsForTopology); + + "rebalance will be kicked off automatically and the topology will be retried at that time.", + subtopology.nodeGroupId, missingSourceTopicsForSubtopology); } } + if (missingSourceTopicsForTopology.isEmpty()) { allRepartitionTopicConfigs.putAll(repartitionTopicConfigsForTopology); allTopicsInfo.addAll(topicsInfoForTopology); From db2485cb597c6f9cc0787a24b49273e992e8e310 Mon Sep 17 00:00:00 2001 From: bozhao12 <102274736+bozhao12@users.noreply.github.com> Date: Tue, 29 Mar 2022 16:13:05 +0800 Subject: [PATCH 058/447] KAFKA-13767; Fetch from consumers should return immediately when preferred read replica is defined by the leader (#11942) When a replica selector is configured, the partition leader computes a preferred read replica for any fetch from the consumers. When the preferred read replica is not the leader, the leader returns the preferred read replica with `FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY)` to the `ReplicaManager`. This causes the fetch to go into in the fetch purgatory because the exit conditions are not met. In turns out that the delayed fetch is not completed until the timeout is reached because the delayed fetch ignores partition with an unknown offset (-1). If the fetch contains only one partition, the fetch is unnecessarily delayed by the timeout time (500ms by default) to only inform the consumer that it has to read from a follower. This patch fixes the issue by completing the fetch request immediately when a preferred read replica is defined. Reviewers: David Jacot --- .../scala/kafka/server/ReplicaManager.scala | 8 +- .../kafka/server/ReplicaManagerTest.scala | 76 +++++++++++++++++-- 2 files changed, 75 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 42124aaa3db9..4b77a4a56c5d 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -1028,15 +1028,17 @@ class ReplicaManager(val config: KafkaConfig, var bytesReadable: Long = 0 var errorReadingData = false var hasDivergingEpoch = false + var hasPreferredReadReplica = false val logReadResultMap = new mutable.HashMap[TopicIdPartition, LogReadResult] logReadResults.foreach { case (topicIdPartition, logReadResult) => brokerTopicStats.topicStats(topicIdPartition.topicPartition.topic).totalFetchRequestRate.mark() brokerTopicStats.allTopicsStats.totalFetchRequestRate.mark() - if (logReadResult.error != Errors.NONE) errorReadingData = true if (logReadResult.divergingEpoch.nonEmpty) hasDivergingEpoch = true + if (logReadResult.preferredReadReplica.nonEmpty) + hasPreferredReadReplica = true bytesReadable = bytesReadable + logReadResult.info.records.sizeInBytes logReadResultMap.put(topicIdPartition, logReadResult) } @@ -1046,7 +1048,9 @@ class ReplicaManager(val config: KafkaConfig, // 3) has enough data to respond // 4) some error happens while reading data // 5) we found a diverging epoch - if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData || hasDivergingEpoch) { + // 6) has a preferred read replica + if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData || + hasDivergingEpoch || hasPreferredReadReplica) { val fetchPartitionData = logReadResults.map { case (tp, result) => val isReassignmentFetch = isFromFollower && isAddingReplica(tp.topicPartition, replicaId) tp -> result.toFetchPartitionData(isReassignmentFetch) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index dd644c8edf2d..a17c70b72173 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -1265,7 +1265,7 @@ class ReplicaManagerTest { initializeLogAndTopicId(replicaManager, tp0, topicId) - // Make this replica the follower + // Make this replica the leader val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, Seq(new LeaderAndIsrPartitionState() .setTopicName(topic) @@ -1281,14 +1281,14 @@ class ReplicaManagerTest { Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() replicaManager.becomeLeaderOrFollower(1, leaderAndIsrRequest2, (_, _) => ()) - val metadata: ClientMetadata = new DefaultClientMetadata("rack-a", "client-id", + val metadata = new DefaultClientMetadata("rack-a", "client-id", InetAddress.getByName("localhost"), KafkaPrincipal.ANONYMOUS, "default") val consumerResult = fetchAsConsumer(replicaManager, tidp0, new PartitionData(Uuid.ZERO_UUID, 0, 0, 100000, Optional.empty()), clientMetadata = Some(metadata)) - // Fetch from follower succeeds + // Fetch from leader succeeds assertTrue(consumerResult.isFired) // Returns a preferred replica (should just be the leader, which is None) @@ -1300,6 +1300,66 @@ class ReplicaManagerTest { TestUtils.assertNoNonDaemonThreads(this.getClass.getName) } + @Test + def testFetchShouldReturnImmediatelyWhenPreferredReadReplicaIsDefined(): Unit = { + val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), + propsModifier = props => props.put(KafkaConfig.ReplicaSelectorClassProp, "org.apache.kafka.common.replica.RackAwareReplicaSelector")) + + try { + val leaderBrokerId = 0 + val followerBrokerId = 1 + val brokerList = Seq[Integer](leaderBrokerId, followerBrokerId).asJava + val topicId = Uuid.randomUuid() + val tp0 = new TopicPartition(topic, 0) + val tidp0 = new TopicIdPartition(topicId, tp0) + + initializeLogAndTopicId(replicaManager, tp0, topicId) + + when(replicaManager.metadataCache.getPartitionReplicaEndpoints( + tp0, + new ListenerName("default") + )).thenReturn(Map( + leaderBrokerId -> new Node(leaderBrokerId, "host1", 9092, "rack-a"), + followerBrokerId -> new Node(followerBrokerId, "host2", 9092, "rack-b") + ).toMap) + + // Make this replica the leader + val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(1) + .setIsr(brokerList) + .setZkVersion(0) + .setReplicas(brokerList) + .setIsNew(false)).asJava, + Collections.singletonMap(topic, topicId), + Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() + replicaManager.becomeLeaderOrFollower(1, leaderAndIsrRequest, (_, _) => ()) + // Avoid the replica selector ignore the follower replica if it not have the data that need to fetch + replicaManager.getPartitionOrException(tp0).updateFollowerFetchState(followerBrokerId, new LogOffsetMetadata(0), 0, 0, 0) + + val metadata = new DefaultClientMetadata("rack-b", "client-id", + InetAddress.getLocalHost, KafkaPrincipal.ANONYMOUS, "default") + + // If a preferred read replica is selected, the fetch response returns immediately, even if min bytes and timeout conditions are not met. + val consumerResult = fetchAsConsumer(replicaManager, tidp0, + new PartitionData(Uuid.ZERO_UUID, 0, 0, 100000, Optional.empty()), + minBytes = 1, clientMetadata = Some(metadata), timeout = 5000) + + // Fetch from leader succeeds + assertTrue(consumerResult.isFired) + + // No delayed fetch was inserted + assertEquals(0, replicaManager.delayedFetchPurgatory.watched) + + // Returns a preferred replica + assertTrue(consumerResult.assertFired.preferredReadReplica.isDefined) + } finally replicaManager.shutdown(checkpointHW = false) + } + @Test def testFollowerFetchWithDefaultSelectorNoForcedHwPropagation(): Unit = { val topicPartition = 0 @@ -2023,8 +2083,9 @@ class ReplicaManagerTest { partitionData: PartitionData, minBytes: Int = 0, isolationLevel: IsolationLevel = IsolationLevel.READ_UNCOMMITTED, - clientMetadata: Option[ClientMetadata] = None): CallbackResult[FetchPartitionData] = { - fetchMessages(replicaManager, replicaId = -1, partition, partitionData, minBytes, isolationLevel, clientMetadata) + clientMetadata: Option[ClientMetadata] = None, + timeout: Long = 1000): CallbackResult[FetchPartitionData] = { + fetchMessages(replicaManager, replicaId = -1, partition, partitionData, minBytes, isolationLevel, clientMetadata, timeout) } private def fetchAsFollower(replicaManager: ReplicaManager, @@ -2042,7 +2103,8 @@ class ReplicaManagerTest { partitionData: PartitionData, minBytes: Int, isolationLevel: IsolationLevel, - clientMetadata: Option[ClientMetadata]): CallbackResult[FetchPartitionData] = { + clientMetadata: Option[ClientMetadata], + timeout: Long = 1000): CallbackResult[FetchPartitionData] = { val result = new CallbackResult[FetchPartitionData]() def fetchCallback(responseStatus: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = { assertEquals(1, responseStatus.size) @@ -2052,7 +2114,7 @@ class ReplicaManagerTest { } replicaManager.fetchMessages( - timeout = 1000, + timeout = timeout, replicaId = replicaId, fetchMinBytes = minBytes, fetchMaxBytes = Int.MaxValue, From 35ae4f248b34cc60a522bc856a42158e503d20b7 Mon Sep 17 00:00:00 2001 From: Levani Kokhreidze Date: Tue, 29 Mar 2022 15:08:51 +0300 Subject: [PATCH 059/447] KAFKA-6718: Add documentation for KIP-708 (#11923) Adds documentation for KIP-708: Rack awareness for Kafka Streams Co-authored-by: Bruno Cadonna Reviewers: Luke Chen , Bruno Cadonna --- .../apache/kafka/common/config/ConfigDef.java | 9 +++- .../kafka/common/config/ConfigDefTest.java | 5 +++ docs/streams/architecture.html | 6 +++ .../developer-guide/config-streams.html | 42 +++++++++++++++++++ 4 files changed, 60 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index 8c91a2523e5d..9331f992268a 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.common.config; -import java.util.function.Function; -import java.util.stream.Collectors; import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.utils.Utils; @@ -33,8 +31,10 @@ import java.util.Map; import java.util.Set; import java.util.function.BiConsumer; +import java.util.function.Function; import java.util.function.Supplier; import java.util.regex.Pattern; +import java.util.stream.Collectors; /** * This class is used for specifying the set of expected configurations. For each configuration, you can specify @@ -1140,6 +1140,11 @@ public void ensureValid(final String name, final Object value) { throw new ConfigException(name, value, "exceeds maximum list size of [" + maxSize + "]."); } } + + @Override + public String toString() { + return "List containing maximum of " + maxSize + " elements"; + } } public static class ConfigKey { diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index 0e5af1f5cb38..76c20df4edff 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -758,4 +758,9 @@ public void testNoExceptionIsThrownWhenListSizeIsBelowTheLimit() { "lst doc")); } + @Test + public void testListSizeValidatorToString() { + assertEquals("List containing maximum of 5 elements", ListSize.atMostOfSize(5).toString()); + } + } diff --git a/docs/streams/architecture.html b/docs/streams/architecture.html index a1773c5fa125..e561231c311e 100644 --- a/docs/streams/architecture.html +++ b/docs/streams/architecture.html @@ -161,6 +161,12 @@

    rack.aware.assignment.tags + in the Kafka Streams Developer Guide section. +

    +
    +

    rack.aware.assignment.tags +

    +
    +
    +

    + This configuration sets a list of tag keys used to distribute standby replicas across Kafka Streams + clients. When configured, Kafka Streams will make a best-effort to distribute the standby tasks over + clients with different tag values. +

    +

    + Tags for the Kafka Streams clients can be set via client.tag. + prefix. Example: +

    +
    
    +Client-1                                   | Client-2
    +_______________________________________________________________________
    +client.tag.zone: eu-central-1a             | client.tag.zone: eu-central-1b
    +client.tag.cluster: k8s-cluster1           | client.tag.cluster: k8s-cluster1
    +rack.aware.assignment.tags: zone,cluster   | rack.aware.assignment.tags: zone,cluster
    +
    +
    +Client-3                                   | Client-4
    +_______________________________________________________________________
    +client.tag.zone: eu-central-1a             | client.tag.zone: eu-central-1b
    +client.tag.cluster: k8s-cluster2           | client.tag.cluster: k8s-cluster2
    +rack.aware.assignment.tags: zone,cluster   | rack.aware.assignment.tags: zone,cluster
    +

    + In the above example, we have four Kafka Streams clients across two zones (eu-central-1a, eu-central-1b) and across two clusters (k8s-cluster1, k8s-cluster2). + For an active task located on Client-1, Kafka Streams will allocate a standby task on Client-4, since Client-4 has a different zone and a different cluster than Client-1. +

    +
    +
    +

    max.task.idle.ms

    From f2aa0c439cb0045e6505d1bbdf8bbbb731a253ca Mon Sep 17 00:00:00 2001 From: David Jacot Date: Tue, 29 Mar 2022 14:31:12 +0200 Subject: [PATCH 060/447] MINOR: Disable SocketServerTest.closingChannelWithBufferedReceives and SocketServerTest.remoteCloseWithoutBufferedReceives (#11960) This reverts commit d706d6cac4622153973d131417e809ee57c60de0. Reviewers: Bruno Cadonna --- core/src/test/scala/unit/kafka/network/SocketServerTest.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 849646cb644c..33d15ad10b5a 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -1389,6 +1389,7 @@ class SocketServerTest { * buffered receive. */ @Test + @Disabled // TODO: re-enabled until KAFKA-13735 is fixed def remoteCloseWithoutBufferedReceives(): Unit = { verifyRemoteCloseWithBufferedReceives(numComplete = 0, hasIncomplete = false) } @@ -1426,6 +1427,7 @@ class SocketServerTest { * The channel must be closed after pending receives are processed. */ @Test + @Disabled // TODO: re-enable after KAFKA-13736 is fixed def closingChannelWithBufferedReceives(): Unit = { verifyRemoteCloseWithBufferedReceives(numComplete = 3, hasIncomplete = false, makeClosing = true) } From 5aed178048dd5a79112274ee5e8a72611d9f0ec2 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 29 Mar 2022 14:59:38 -0700 Subject: [PATCH 061/447] KAFKA-13418: Support key updates with TLS 1.3 (#11966) Key updates with TLS 1.3 trigger code paths similar to renegotiation with TLS 1.2. Update the read/write paths not to throw an exception in this case (kept the exception in the `handshake` method). With the default configuration, key updates happen after 2^37 bytes are encrypted. There is a security property to adjust this configuration, but the change has to be done before it is used for the first time and it cannot be changed after that. As such, it is best done via a system test (filed KAFKA-13779). To validate the change, I wrote a unit test that forces key updates and manually ran a producer workload that produced more than 2^37 bytes. Both cases failed without these changes and pass with them. Note that Shylaja Kokoori attached a patch with the SslTransportLayer fix and hence included them as a co-author of this change. Reviewers: Rajini Sivaram Co-authored-by: Shylaja Kokoori --- .../common/network/SslTransportLayer.java | 16 +++- .../kafka/common/network/SelectorTest.java | 5 - .../kafka/common/network/SslSelectorTest.java | 44 +-------- .../common/network/Tls12SelectorTest.java | 72 +++++++++++++++ .../common/network/Tls13SelectorTest.java | 92 +++++++++++++++++++ 5 files changed, 180 insertions(+), 49 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/common/network/Tls12SelectorTest.java create mode 100644 clients/src/test/java/org/apache/kafka/common/network/Tls13SelectorTest.java diff --git a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java index 893fd6a4eced..844c2bd2c17d 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java @@ -71,6 +71,8 @@ private enum State { CLOSING } + private static final String TLS13 = "TLSv1.3"; + private final String channelId; private final SSLEngine sslEngine; private final SelectionKey key; @@ -449,7 +451,7 @@ private void handshakeFinished() throws IOException { if (netWriteBuffer.hasRemaining()) key.interestOps(key.interestOps() | SelectionKey.OP_WRITE); else { - state = sslEngine.getSession().getProtocol().equals("TLSv1.3") ? State.POST_HANDSHAKE : State.READY; + state = sslEngine.getSession().getProtocol().equals(TLS13) ? State.POST_HANDSHAKE : State.READY; key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE); SSLSession session = sslEngine.getSession(); log.debug("SSL handshake completed successfully with peerHost '{}' peerPort {} peerPrincipal '{}' cipherSuite '{}'", @@ -585,10 +587,11 @@ public int read(ByteBuffer dst) throws IOException { throw e; } netReadBuffer.compact(); - // handle ssl renegotiation. + // reject renegotiation if TLS < 1.3, key updates for TLS 1.3 are allowed if (unwrapResult.getHandshakeStatus() != HandshakeStatus.NOT_HANDSHAKING && unwrapResult.getHandshakeStatus() != HandshakeStatus.FINISHED && - unwrapResult.getStatus() == Status.OK) { + unwrapResult.getStatus() == Status.OK && + !sslEngine.getSession().getProtocol().equals(TLS13)) { log.error("Renegotiation requested, but it is not supported, channelId {}, " + "appReadBuffer pos {}, netReadBuffer pos {}, netWriteBuffer pos {} handshakeStatus {}", channelId, appReadBuffer.position(), netReadBuffer.position(), netWriteBuffer.position(), unwrapResult.getHandshakeStatus()); @@ -706,9 +709,12 @@ public int write(ByteBuffer src) throws IOException { SSLEngineResult wrapResult = sslEngine.wrap(src, netWriteBuffer); netWriteBuffer.flip(); - //handle ssl renegotiation - if (wrapResult.getHandshakeStatus() != HandshakeStatus.NOT_HANDSHAKING && wrapResult.getStatus() == Status.OK) + // reject renegotiation if TLS < 1.3, key updates for TLS 1.3 are allowed + if (wrapResult.getHandshakeStatus() != HandshakeStatus.NOT_HANDSHAKING && + wrapResult.getStatus() == Status.OK && + !sslEngine.getSession().getProtocol().equals(TLS13)) { throw renegotiationException(); + } if (wrapResult.getStatus() == Status.OK) { written += wrapResult.bytesConsumed(); diff --git a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java index f276cd4211a3..43b095656e1c 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java @@ -110,10 +110,6 @@ public void tearDown() throws Exception { } } - public SecurityProtocol securityProtocol() { - return SecurityProtocol.PLAINTEXT; - } - protected Map clientConfigs() { return new HashMap<>(); } @@ -1015,7 +1011,6 @@ public void testChannelCloseWhileProcessingReceives() throws Exception { private String blockingRequest(String node, String s) throws IOException { selector.send(createSend(node, s)); - selector.poll(1000L); while (true) { selector.poll(1000L); for (NetworkReceive receive : selector.completedReceives()) diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslSelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SslSelectorTest.java index 7f95566c9f98..0ddfce652285 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslSelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslSelectorTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.common.network; import java.nio.channels.SelectionKey; +import java.security.GeneralSecurityException; import javax.net.ssl.SSLEngine; import org.apache.kafka.common.config.SecurityConfig; @@ -43,11 +44,9 @@ import java.nio.channels.ServerSocketChannel; import java.nio.channels.SocketChannel; import java.security.Security; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; @@ -61,7 +60,7 @@ /** * A set of tests for the selector. These use a test harness that runs a simple socket server that echos back responses. */ -public class SslSelectorTest extends SelectorTest { +public abstract class SslSelectorTest extends SelectorTest { private Map sslClientConfigs; @@ -73,7 +72,7 @@ public void setUp() throws Exception { this.server = new EchoServer(SecurityProtocol.SSL, sslServerConfigs); this.server.start(); this.time = new MockTime(); - sslClientConfigs = TestSslUtils.createSslConfig(false, false, Mode.CLIENT, trustStoreFile, "client"); + sslClientConfigs = createSslClientConfigs(trustStoreFile); LogContext logContext = new LogContext(); this.channelBuilder = new SslChannelBuilder(Mode.CLIENT, null, false, logContext); this.channelBuilder.configure(sslClientConfigs); @@ -81,6 +80,8 @@ public void setUp() throws Exception { this.selector = new Selector(5000, metrics, time, "MetricGroup", channelBuilder, logContext); } + protected abstract Map createSslClientConfigs(File trustStoreFile) throws GeneralSecurityException, IOException; + @AfterEach public void tearDown() throws Exception { this.selector.close(); @@ -88,11 +89,6 @@ public void tearDown() throws Exception { this.metrics.close(); } - @Override - public SecurityProtocol securityProtocol() { - return SecurityProtocol.PLAINTEXT; - } - @Override protected Map clientConfigs() { return sslClientConfigs; @@ -100,7 +96,6 @@ protected Map clientConfigs() { @Test public void testConnectionWithCustomKeyManager() throws Exception { - TestProviderCreator testProviderCreator = new TestProviderCreator(); int requestSize = 100 * 1024; @@ -249,35 +244,6 @@ void pollSelectionKeys(Set selectionKeys, boolean isImmediatelyCon verifySelectorEmpty(); } - /** - * Renegotiation is not supported since it is potentially unsafe and it has been removed in TLS 1.3 - */ - @Test - public void testRenegotiationFails() throws Exception { - String node = "0"; - // create connections - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); - selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - - // send echo requests and receive responses - while (!selector.isChannelReady(node)) { - selector.poll(1000L); - } - selector.send(createSend(node, node + "-" + 0)); - selector.poll(0L); - server.renegotiate(); - selector.send(createSend(node, node + "-" + 1)); - long expiryTime = System.currentTimeMillis() + 2000; - - List disconnected = new ArrayList<>(); - while (!disconnected.contains(node) && System.currentTimeMillis() < expiryTime) { - selector.poll(10); - disconnected.addAll(selector.disconnected().keySet()); - } - assertTrue(disconnected.contains(node), "Renegotiation should cause disconnection"); - - } - @Override @Test public void testMuteOnOOM() throws Exception { diff --git a/clients/src/test/java/org/apache/kafka/common/network/Tls12SelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/Tls12SelectorTest.java new file mode 100644 index 000000000000..59903b5173c9 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/network/Tls12SelectorTest.java @@ -0,0 +1,72 @@ +/* + * 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.common.network; + +import static java.util.Arrays.asList; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.security.GeneralSecurityException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.test.TestSslUtils; +import org.junit.jupiter.api.Test; + +public class Tls12SelectorTest extends SslSelectorTest { + + @Override + protected Map createSslClientConfigs(File trustStoreFile) + throws GeneralSecurityException, IOException { + Map configs = TestSslUtils.createSslConfig(false, false, Mode.CLIENT, + trustStoreFile, "client"); + configs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, asList("TLSv1.2")); + return configs; + } + + /** + * Renegotiation is not supported when TLS 1.2 is used (renegotiation was removed from TLS 1.3) + */ + @Test + public void testRenegotiationFails() throws Exception { + String node = "0"; + // create connections + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + + // send echo requests and receive responses + while (!selector.isChannelReady(node)) { + selector.poll(1000L); + } + selector.send(createSend(node, node + "-" + 0)); + selector.poll(0L); + server.renegotiate(); + selector.send(createSend(node, node + "-" + 1)); + long expiryTime = System.currentTimeMillis() + 2000; + + List disconnected = new ArrayList<>(); + while (!disconnected.contains(node) && System.currentTimeMillis() < expiryTime) { + selector.poll(10); + disconnected.addAll(selector.disconnected().keySet()); + } + assertTrue(disconnected.contains(node), "Renegotiation should cause disconnection"); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/network/Tls13SelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/Tls13SelectorTest.java new file mode 100644 index 000000000000..afae3e20c7a4 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/network/Tls13SelectorTest.java @@ -0,0 +1,92 @@ +/* + * 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.common.network; + +import static java.util.Arrays.asList; +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.security.GeneralSecurityException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.test.TestSslUtils; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.EnabledForJreRange; +import org.junit.jupiter.api.condition.JRE; + +@EnabledForJreRange(min = JRE.JAVA_11) // TLS 1.3 is only supported with Java 11 and newer +public class Tls13SelectorTest extends SslSelectorTest { + + @Override + protected Map createSslClientConfigs(File trustStoreFile) throws GeneralSecurityException, IOException { + Map configs = TestSslUtils.createSslConfig(false, false, Mode.CLIENT, + trustStoreFile, "client"); + configs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, asList("TLSv1.3")); + return configs; + } + + /** + * TLS 1.3 has a post-handshake key and IV update, which will update the sending and receiving keys + * for one side of the connection. + * + * Key Usage Limits will trigger an update when the algorithm limits are reached, but the default + * value is too large (2^37 bytes of plaintext data) for a unit test. This value can be overridden + * via the security property `jdk.tls.keyLimits`, but that's also difficult to achieve in a unit + * test. + * + * Applications can also trigger an update by calling `SSLSocket.startHandshake()` or + * `SSLEngine.beginHandshake()` (this would trigger `renegotiation` with TLS 1.2) and that's the + * approach we take here. + */ + @Test + public void testKeyUpdate() throws Exception { + String node = "0"; + // create connections + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + // send echo requests and receive responses + while (!selector.isChannelReady(node)) { + selector.poll(1000L); + } + selector.send(createSend(node, node + "-" + 0)); + selector.poll(0L); + server.renegotiate(); + selector.send(createSend(node, node + "-" + 1)); + List received = new ArrayList<>(); + TestUtils.waitForCondition(() -> { + try { + selector.poll(1000L); + } catch (IOException e) { + throw new RuntimeException(e); + } + for (NetworkReceive receive : selector.completedReceives()) { + if (receive.source().equals(node)) + received.add(receive); + } + return received.size() == 2; + }, "Expected two receives, got " + received.size()); + + assertEquals(asList("0-0", "0-1"), received.stream().map(this::asString).collect(Collectors.toList())); + } +} From 3904652b3e2dd2908ee30cff228041ce80508548 Mon Sep 17 00:00:00 2001 From: Bruno Cadonna Date: Wed, 30 Mar 2022 14:04:47 +0200 Subject: [PATCH 062/447] Upgrade RocksDB from 6.27.3 to 6.29.4.1 (#11967) RocksDB 6.27.3 does not run on arm64 M1 Macs which would prevent people on this platform to run Kafka Streams. Thus, this PR upgrades RocksDB to 6.29.4.1 which contains the following fix to allow to run RocksDB on arm64 M1 Macs: facebook/rocksdb#7720 The source compatibility report between 6.27.3 and 6.29.4.1 (attached to the ticket) reports a couple of incompatibilities. However, the incompatibilities do not seem to affect Kafka Streams' backwards compatibility. The changes to class RocksDB only apply when inheriting from RocksDB. RocksDB is not exposed to users in Streams. The changes to class WriteBatch and class WriteBatchInterface also only apply with inheritance. Both classes are not exposed to users in Streams. -The change to enum SanityLevel seem also not to apply to Streams since SanityLevel is only used in ConfigOptions which is only used to load options from files and properties objects. Loading options from files or properties is not exposed to users in Streams. Reviewers: Bill Bejeck , Matthias J. Sax , A. Sophie Blee-Goldman --- gradle/dependencies.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 3c317b53ba29..cd7051135bbf 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -107,7 +107,7 @@ versions += [ netty: "4.1.73.Final", powermock: "2.0.9", reflections: "0.9.12", - rocksDB: "6.27.3", + rocksDB: "6.29.4.1", scalaCollectionCompat: "2.6.0", scalafmt: "2.7.5", scalaJava8Compat : "1.0.2", From 8e205b503a39ba8c798e3ce14cd887b66a88551c Mon Sep 17 00:00:00 2001 From: sunshujie1990 Date: Wed, 30 Mar 2022 20:58:58 +0800 Subject: [PATCH 063/447] KAFKA-13719: Fix connector restart cause duplicate tasks (#11869) Reviewers: Mickael Maison , Luke Chen , Chris Egerton Co-authored-by: Chris Egerton --- .../runtime/distributed/DistributedHerder.java | 6 +++--- .../distributed/DistributedHerderTest.java | 15 ++++++++------- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index 357796c9d1af..65a8e7e15b81 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -1180,8 +1180,8 @@ protected synchronized void doRestartConnectorAndTasks(RestartRequest request) { } } if (restartTasks) { - log.debug("Restarting {} of {} tasks for {}", plan.restartTaskCount(), plan.totalTaskCount(), request); - plan.taskIdsToRestart().forEach(taskId -> { + log.debug("Restarting {} of {} tasks for {}", assignedIdsToRestart.size(), plan.totalTaskCount(), request); + assignedIdsToRestart.forEach(taskId -> { try { if (startTask(taskId)) { log.info("Task '{}' restart successful", taskId); @@ -1192,7 +1192,7 @@ protected synchronized void doRestartConnectorAndTasks(RestartRequest request) { log.error("Task '{}' restart failed", taskId, t); } }); - log.debug("Restarted {} of {} tasks for {} as requested", plan.restartTaskCount(), plan.totalTaskCount(), request); + log.debug("Restarted {} of {} tasks for {} as requested", assignedIdsToRestart.size(), plan.totalTaskCount(), request); } log.info("Completed {}", plan); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java index 6ddf04776bfd..996c8407c6ec 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java @@ -1289,24 +1289,25 @@ public void testDoRestartConnectorAndTasksOnlyConnector() { @Test public void testDoRestartConnectorAndTasksOnlyTasks() { - ConnectorTaskId taskId = new ConnectorTaskId(CONN1, 0); RestartRequest restartRequest = new RestartRequest(CONN1, false, true); RestartPlan restartPlan = PowerMock.createMock(RestartPlan.class); EasyMock.expect(restartPlan.shouldRestartConnector()).andReturn(true).anyTimes(); EasyMock.expect(restartPlan.shouldRestartTasks()).andReturn(true).anyTimes(); - EasyMock.expect(restartPlan.taskIdsToRestart()).andReturn(Collections.singletonList(taskId)).anyTimes(); - EasyMock.expect(restartPlan.restartTaskCount()).andReturn(1).anyTimes(); - EasyMock.expect(restartPlan.totalTaskCount()).andReturn(1).anyTimes(); + // The connector has three tasks + EasyMock.expect(restartPlan.taskIdsToRestart()).andReturn(Arrays.asList(TASK0, TASK1, TASK2)).anyTimes(); + EasyMock.expect(restartPlan.restartTaskCount()).andReturn(3).anyTimes(); + EasyMock.expect(restartPlan.totalTaskCount()).andReturn(3).anyTimes(); EasyMock.expect(herder.buildRestartPlan(restartRequest)).andReturn(Optional.of(restartPlan)).anyTimes(); herder.assignment = PowerMock.createMock(ExtendedAssignment.class); EasyMock.expect(herder.assignment.connectors()).andReturn(Collections.emptyList()).anyTimes(); - EasyMock.expect(herder.assignment.tasks()).andReturn(Collections.singletonList(taskId)).anyTimes(); + // But only one task is assigned to this worker + EasyMock.expect(herder.assignment.tasks()).andReturn(Collections.singletonList(TASK0)).anyTimes(); - worker.stopAndAwaitTasks(Collections.singletonList(taskId)); + worker.stopAndAwaitTasks(Collections.singletonList(TASK0)); PowerMock.expectLastCall(); - herder.onRestart(taskId); + herder.onRestart(TASK0); EasyMock.expectLastCall(); worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), From bb60eb86e1a0028274f02482eb70d9e018b7014e Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Wed, 30 Mar 2022 17:09:19 +0100 Subject: [PATCH 064/447] MINOR: Increase wait in ZooKeeperClientTest (#11973) Increase wait in ZooKeeperClientTest.testReinitializeAfterAuthFailure so that the testcase of https://github.com/apache/kafka/pull/11563 actually fails without the corresponding source code fix. Followup of https://issues.apache.org/jira/browse/KAFKA-13461. Co-Authored-By: Gantigmaa Selenge Reviewers: Rajini Sivaram --- .../test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala index 5af2ba8db044..5a8e4c382a8d 100644 --- a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala +++ b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala @@ -658,7 +658,7 @@ class ZooKeeperClientTest extends QuorumTestHarness { connectionStateOverride = Some(States.CONNECTED) zooKeeperClient.ZooKeeperClientWatcher.process(new WatchedEvent(EventType.None, KeeperState.AuthFailed, null)) - assertFalse(sessionInitializedCountDownLatch.await(10, TimeUnit.MILLISECONDS), "Unexpected session initialization when connection is alive") + assertFalse(sessionInitializedCountDownLatch.await(1200, TimeUnit.MILLISECONDS), "Unexpected session initialization when connection is alive") connectionStateOverride = Some(States.AUTH_FAILED) zooKeeperClient.ZooKeeperClientWatcher.process(new WatchedEvent(EventType.None, KeeperState.AuthFailed, null)) From 76ca62a39673f90c7cb65d90f27f68ba035a164f Mon Sep 17 00:00:00 2001 From: Edwin Date: Wed, 30 Mar 2022 21:36:34 +0300 Subject: [PATCH 065/447] KAFKA-13775: CVE-2020-36518 - Upgrade jackson-databind to 2.12.6.1 (#11962) CVE-2020-36518 vulnerability affects jackson-databind (see GHSA-57j2-w4cx-62h2). Upgrading to jackson-databind version 2.12.6.1 addresses this CVE. Reviewers: Luke Chen , Bruno Cadonna --- LICENSE-binary | 2 +- gradle/dependencies.gradle | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index 42a8d79f86c9..72fa1a0a7263 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -210,7 +210,7 @@ commons-cli-1.4 commons-lang3-3.8.1 jackson-annotations-2.12.3 jackson-core-2.12.3 -jackson-databind-2.12.3 +jackson-databind-2.12.6.1 jackson-dataformat-csv-2.12.3 jackson-datatype-jdk8-2.12.3 jackson-jaxrs-base-2.12.3 diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index cd7051135bbf..6f3cb8fdb053 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -67,6 +67,7 @@ versions += [ httpclient: "4.5.13", easymock: "4.3", jackson: "2.12.6", + jacksonDatabind: "2.12.6.1", jacoco: "0.8.7", javassist: "3.27.0-GA", jetty: "9.4.44.v20210927", @@ -135,7 +136,7 @@ libs += [ commonsCli: "commons-cli:commons-cli:$versions.commonsCli", easymock: "org.easymock:easymock:$versions.easymock", jacksonAnnotations: "com.fasterxml.jackson.core:jackson-annotations:$versions.jackson", - jacksonDatabind: "com.fasterxml.jackson.core:jackson-databind:$versions.jackson", + jacksonDatabind: "com.fasterxml.jackson.core:jackson-databind:$versions.jacksonDatabind", jacksonDataformatCsv: "com.fasterxml.jackson.dataformat:jackson-dataformat-csv:$versions.jackson", jacksonModuleScala: "com.fasterxml.jackson.module:jackson-module-scala_$versions.baseScala:$versions.jackson", jacksonJDK8Datatypes: "com.fasterxml.jackson.datatype:jackson-datatype-jdk8:$versions.jackson", From ecb0e8eece0b47a1376643a11056c77d0aa5ad6e Mon Sep 17 00:00:00 2001 From: Mike Lothian Date: Wed, 30 Mar 2022 19:54:01 +0100 Subject: [PATCH 066/447] KAFKA-13660: Switch log4j12 to reload4j (#11743) This bumps the slf4j version to 1.7.36 and swaps out log4j 1.2.17 with reload4j 1.2.19 Signed-off-by: Mike Lothian Reviewers: Luke Chen , Ismael Juma , Bruno Cadonna --- LICENSE-binary | 6 +++--- build.gradle | 1 + docs/upgrade.html | 2 ++ gradle/dependencies.gradle | 6 +++--- 4 files changed, 9 insertions(+), 6 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index 72fa1a0a7263..dfe7d44f0815 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -233,7 +233,6 @@ jetty-util-ajax-9.4.43.v20210629 jersey-common-2.34 jersey-server-2.34 jose4j-0.7.8 -log4j-1.2.17 lz4-java-1.8.0 maven-artifact-3.8.1 metrics-core-4.1.12.1 @@ -246,6 +245,7 @@ netty-transport-4.1.68.Final netty-transport-native-epoll-4.1.68.Final netty-transport-native-unix-common-4.1.68.Final plexus-utils-3.2.1 +reload4j-1.2.19 rocksdbjni-6.22.1.1 scala-collection-compat_2.13-2.4.4 scala-library-2.13.6 @@ -300,8 +300,8 @@ MIT License argparse4j-0.7.0, see: licenses/argparse-MIT jopt-simple-5.0.4, see: licenses/jopt-simple-MIT -slf4j-api-1.7.30, see: licenses/slf4j-MIT -slf4j-log4j12-1.7.30, see: licenses/slf4j-MIT +slf4j-api-1.7.36, see: licenses/slf4j-MIT +slf4j-reload4j-1.7.36, see: licenses/slf4j-MIT --------------------------------------- BSD 2-Clause diff --git a/build.gradle b/build.gradle index 3363a43ac70d..3efb88d4b019 100644 --- a/build.gradle +++ b/build.gradle @@ -149,6 +149,7 @@ allprojects { } } } + task printAllDependencies(type: DependencyReportTask) {} } def determineCommitId() { diff --git a/docs/upgrade.html b/docs/upgrade.html index e7700272b39c..313057d86672 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -30,6 +30,8 @@
    Notable changes in 3 Users can change this behavior to enable idempotence for some or all producers via Connect worker and/or connector configuration. Connect may enable idempotent producers by default in a future major release. +
  • Kafka has replaced log4j and slf4j-log4j12 with reload4j and slf4j-reload4j due to security concerns. + More information can be found at reload4j.
  • Upgrading to 3.1.0 from any version 0.8.x through 3.0.x

    diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 6f3cb8fdb053..6910f640728c 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -75,7 +75,6 @@ versions += [ jline: "3.21.0", jmh: "1.34", hamcrest: "2.2", - log4j: "1.2.17", scalaLogging: "3.9.4", jaxb: "2.3.0", jaxrs: "2.1.1", @@ -108,12 +107,13 @@ versions += [ netty: "4.1.73.Final", powermock: "2.0.9", reflections: "0.9.12", + reload4j: "1.2.19", rocksDB: "6.29.4.1", scalaCollectionCompat: "2.6.0", scalafmt: "2.7.5", scalaJava8Compat : "1.0.2", scoverage: "1.4.11", - slf4j: "1.7.32", + slf4j: "1.7.36", snappy: "1.1.8.4", spotbugs: "4.2.2", zinc: "1.3.5", @@ -178,7 +178,7 @@ libs += [ kafkaStreams_28: "org.apache.kafka:kafka-streams:$versions.kafka_28", kafkaStreams_30: "org.apache.kafka:kafka-streams:$versions.kafka_30", kafkaStreams_31: "org.apache.kafka:kafka-streams:$versions.kafka_31", - log4j: "log4j:log4j:$versions.log4j", + log4j: "ch.qos.reload4j:reload4j:$versions.reload4j", lz4: "org.lz4:lz4-java:$versions.lz4", metrics: "com.yammer.metrics:metrics-core:$versions.metrics", dropwizardMetrics: "io.dropwizard.metrics:metrics-core:$versions.dropwizardMetrics", From dd62ef2eda571576a222d757beabcd7690dd8c16 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Wed, 30 Mar 2022 13:15:42 -0700 Subject: [PATCH 067/447] KAFKA-13748: Do not include file stream connectors in Connect's CLASSPATH and plugin.path by default (#11908) With this change we stop including the non-production grade connectors that are meant to be used for demos and quick starts by default in the CLASSPATH and plugin.path of Connect deployments. The package of these connector will still be shipped with the Apache Kafka distribution and will be available for explicit inclusion. The changes have been tested through the system tests and the existing unit and integration tests. Reviewers: Mickael Maison , Randall Hauch --- bin/kafka-run-class.sh | 4 +- docs/connect.html | 1 + docs/quickstart.html | 19 +++++++-- tests/kafkatest/services/connect.py | 42 ++++++++++++++++--- .../tests/connect/connect_distributed_test.py | 12 +++--- .../tests/connect/connect_rest_test.py | 3 +- tests/kafkatest/tests/connect/connect_test.py | 12 ++++-- 7 files changed, 73 insertions(+), 20 deletions(-) diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 6167583780bd..490f930b8cc4 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -32,7 +32,7 @@ if [ -z "$INCLUDE_TEST_JARS" ]; then fi # Exclude jars not necessary for running commands. -regex="(-(test|test-sources|src|scaladoc|javadoc)\.jar|jar.asc)$" +regex="(-(test|test-sources|src|scaladoc|javadoc)\.jar|jar.asc|connect-file.*\.jar)$" should_include_file() { if [ "$INCLUDE_TEST_JARS" = true ]; then return 0 @@ -171,7 +171,7 @@ do CLASSPATH="$CLASSPATH:$dir/*" done -for cc_pkg in "api" "transforms" "runtime" "file" "mirror" "mirror-client" "json" "tools" "basic-auth-extension" +for cc_pkg in "api" "transforms" "runtime" "mirror" "mirror-client" "json" "tools" "basic-auth-extension" do for file in "$base_dir"/connect/${cc_pkg}/build/libs/connect-${cc_pkg}*.jar; do diff --git a/docs/connect.html b/docs/connect.html index 66d621248dec..1251c3ce683e 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -48,6 +48,7 @@

    Running Kafka Connectbootstrap.servers - List of Kafka servers used to bootstrap connections to Kafka
  • key.converter - Converter class used to convert between Kafka Connect format and the serialized form that is written to Kafka. This controls the format of the keys in messages written to or read from Kafka, and since this is independent of connectors it allows any connector to work with any serialization format. Examples of common formats include JSON and Avro.
  • value.converter - Converter class used to convert between Kafka Connect format and the serialized form that is written to Kafka. This controls the format of the values in messages written to or read from Kafka, and since this is independent of connectors it allows any connector to work with any serialization format. Examples of common formats include JSON and Avro.
  • +
  • plugin.path (default empty) - a list of paths that contain Connect plugins (connectors, converters, transformations). Before running quick starts, users must add the absolute path that contains the example FileStreamSourceConnector and FileStreamSinkConnector packaged in connect-file-"version".jar, because these connectors are not included by default to the CLASSPATH or the plugin.path of the Connect worker (see plugin.path property for examples).
  • The important configuration options specific to standalone mode are:

    diff --git a/docs/quickstart.html b/docs/quickstart.html index 2ef56c850574..70b13146e650 100644 --- a/docs/quickstart.html +++ b/docs/quickstart.html @@ -32,8 +32,8 @@

    the latest Kafka release and extract it:

    -
    $ tar -xzf kafka_2.13-3.1.0.tgz
    -$ cd kafka_2.13-3.1.0
    +
    $ tar -xzf kafka_{{scalaVersion}}-{{fullDotVersion}}.tgz
    +$ cd kafka_{{scalaVersion}}-{{fullDotVersion}}

    @@ -173,7 +173,20 @@

    - First, we'll start by creating some seed data to test with: + First, make sure to add connect-file-{{fullDotVersion}}.jar to the plugin.path property in the Connect worker's configuration. + For the purpose of this quickstart we'll use a relative path and consider the connectors' package as an uber jar, which works when the quickstart commands are run from the installation directory. + However, it's worth noting that for production deployments using absolute paths is always preferable. See plugin.path for a detailed description of how to set this config. +

    + +

    + Edit the config/connect-standalone.properties file, add or change the plugin.path configuration property match the following, and save the file: +

    + +
    +> echo "plugin.path=lib/connect-file-{{fullDotVersion}}.jar"
    + +

    + Then, start by creating some seed data to test with:

    diff --git a/tests/kafkatest/services/connect.py b/tests/kafkatest/services/connect.py
    index 26c0d927dccd..41c33ccb9e10 100644
    --- a/tests/kafkatest/services/connect.py
    +++ b/tests/kafkatest/services/connect.py
    @@ -69,7 +69,8 @@ class ConnectServiceBase(KafkaPathResolverMixin, Service):
                 "collect_default": True}
         }
     
    -    def __init__(self, context, num_nodes, kafka, files, startup_timeout_sec = 60):
    +    def __init__(self, context, num_nodes, kafka, files, startup_timeout_sec=60,
    +                 include_filestream_connectors=False):
             super(ConnectServiceBase, self).__init__(context, num_nodes)
             self.kafka = kafka
             self.security_config = kafka.security_config.client_config()
    @@ -78,6 +79,8 @@ def __init__(self, context, num_nodes, kafka, files, startup_timeout_sec = 60):
             self.startup_timeout_sec = startup_timeout_sec
             self.environment = {}
             self.external_config_template_func = None
    +        self.include_filestream_connectors = include_filestream_connectors
    +        self.logger.debug("include_filestream_connectors % s", include_filestream_connectors)
     
         def pids(self, node):
             """Return process ids for Kafka Connect processes."""
    @@ -279,12 +282,34 @@ def append_to_environment_variable(self, envvar, value):
                 env_opts = "\"%s %s\"" % (env_opts.strip('\"'), value)
             self.environment[envvar] = env_opts
     
    +    def append_filestream_connectors_to_classpath(self):
    +        if self.include_filestream_connectors:
    +            cwd = os.getcwd()
    +            self.logger.info("Including filestream connectors when starting Connect. "
    +                             "Looking for jar locally in: %s" % cwd)
    +            relative_path = "/connect/file/build/libs/"
    +            local_dir = cwd + relative_path
    +            lib_dir = self.path.home() + relative_path
    +            for pwd, dirs, files in os.walk(local_dir):
    +                for file in files:
    +                    if file.startswith("connect-file") and file.endswith(".jar"):
    +                        # Use the expected directory on the node instead of the path in the driver node
    +                        file_path = lib_dir + file
    +                        self.logger.debug("Appending %s to Connect worker's CLASSPATH" % file_path)
    +                        return "export CLASSPATH=${CLASSPATH}:%s; " % file_path
    +            self.logger.info("Jar with filestream connectors was not found under %s" % lib_dir)
    +        else:
    +            self.logger.info("Starting Connect without filestream connectors in the CLASSPATH")
    +
    +        return None
    +
     
     class ConnectStandaloneService(ConnectServiceBase):
         """Runs Kafka Connect in standalone mode."""
     
    -    def __init__(self, context, kafka, files, startup_timeout_sec = 60):
    -        super(ConnectStandaloneService, self).__init__(context, 1, kafka, files, startup_timeout_sec)
    +    def __init__(self, context, kafka, files, startup_timeout_sec=60, include_filestream_connectors=False):
    +        super(ConnectStandaloneService, self).__init__(context, 1, kafka, files, startup_timeout_sec,
    +                                                       include_filestream_connectors)
     
         # For convenience since this service only makes sense with a single node
         @property
    @@ -299,6 +324,9 @@ def start_cmd(self, node, connector_configs):
     
             cmd += fix_opts_for_new_jvm(node)
             cmd += "export KAFKA_OPTS=\"%s %s\"; " % (heap_kafka_opts, other_kafka_opts)
    +        classpath = self.append_filestream_connectors_to_classpath()
    +        cmd += classpath if classpath else ""
    +
             for envvar in self.environment:
                 cmd += "export %s=%s; " % (envvar, str(self.environment[envvar]))
             cmd += "%s %s " % (self.path.script("connect-standalone.sh", node), self.CONFIG_FILE)
    @@ -339,8 +367,9 @@ class ConnectDistributedService(ConnectServiceBase):
         """Runs Kafka Connect in distributed mode."""
     
         def __init__(self, context, num_nodes, kafka, files, offsets_topic="connect-offsets",
    -                 configs_topic="connect-configs", status_topic="connect-status", startup_timeout_sec = 60):
    -        super(ConnectDistributedService, self).__init__(context, num_nodes, kafka, files, startup_timeout_sec)
    +                 configs_topic="connect-configs", status_topic="connect-status", startup_timeout_sec=60,
    +                 include_filestream_connectors=False):
    +        super(ConnectDistributedService, self).__init__(context, num_nodes, kafka, files, startup_timeout_sec, include_filestream_connectors)
             self.startup_mode = self.STARTUP_MODE_JOIN
             self.offsets_topic = offsets_topic
             self.configs_topic = configs_topic
    @@ -355,6 +384,9 @@ def start_cmd(self, node, connector_configs):
             cmd += "export KAFKA_OPTS=\"%s %s\"; " % (heap_kafka_opts, other_kafka_opts)
             for envvar in self.environment:
                 cmd += "export %s=%s; " % (envvar, str(self.environment[envvar]))
    +
    +        classpath = self.append_filestream_connectors_to_classpath()
    +        cmd += classpath if classpath else ""
             cmd += "%s %s " % (self.path.script("connect-distributed.sh", node), self.CONFIG_FILE)
             cmd += " & echo $! >&3 ) 1>> %s 2>> %s 3> %s" % (self.STDOUT_FILE, self.STDERR_FILE, self.PID_FILE)
             return cmd
    diff --git a/tests/kafkatest/tests/connect/connect_distributed_test.py b/tests/kafkatest/tests/connect/connect_distributed_test.py
    index 6bc52b0d35f4..970779f723fb 100644
    --- a/tests/kafkatest/tests/connect/connect_distributed_test.py
    +++ b/tests/kafkatest/tests/connect/connect_distributed_test.py
    @@ -80,7 +80,7 @@ def __init__(self, test_context):
             self.value_converter = "org.apache.kafka.connect.json.JsonConverter"
             self.schemas = True
     
    -    def setup_services(self, security_protocol=SecurityConfig.PLAINTEXT, timestamp_type=None, broker_version=DEV_BRANCH, auto_create_topics=False):
    +    def setup_services(self, security_protocol=SecurityConfig.PLAINTEXT, timestamp_type=None, broker_version=DEV_BRANCH, auto_create_topics=False, include_filestream_connectors=False):
             self.kafka = KafkaService(self.test_context, self.num_brokers, self.zk,
                                       security_protocol=security_protocol, interbroker_security_protocol=security_protocol,
                                       topics=self.topics, version=broker_version,
    @@ -89,7 +89,8 @@ def setup_services(self, security_protocol=SecurityConfig.PLAINTEXT, timestamp_t
                 for node in self.kafka.nodes:
                     node.config[config_property.MESSAGE_TIMESTAMP_TYPE] = timestamp_type
     
    -        self.cc = ConnectDistributedService(self.test_context, 3, self.kafka, [self.INPUT_FILE, self.OUTPUT_FILE])
    +        self.cc = ConnectDistributedService(self.test_context, 3, self.kafka, [self.INPUT_FILE, self.OUTPUT_FILE],
    +                                            include_filestream_connectors=include_filestream_connectors)
             self.cc.log_level = "DEBUG"
     
             self.zk.start()
    @@ -370,7 +371,7 @@ def test_file_source_and_sink(self, security_protocol, connect_protocol):
             """
     
             self.CONNECT_PROTOCOL = connect_protocol
    -        self.setup_services(security_protocol=security_protocol)
    +        self.setup_services(security_protocol=security_protocol, include_filestream_connectors=True)
             self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node))
     
             self.cc.start()
    @@ -522,7 +523,7 @@ def test_bounce(self, clean, connect_protocol):
         @matrix(connect_protocol=['sessioned', 'compatible', 'eager'])
         def test_transformations(self, connect_protocol):
             self.CONNECT_PROTOCOL = connect_protocol
    -        self.setup_services(timestamp_type='CreateTime')
    +        self.setup_services(timestamp_type='CreateTime', include_filestream_connectors=True)
             self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node))
             self.cc.start()
     
    @@ -610,7 +611,8 @@ def test_broker_compatibility(self, broker_version, auto_create_topics, security
             or relies upon the broker to auto-create the topics (v0.10.0.x and before).
             """
             self.CONNECT_PROTOCOL = connect_protocol
    -        self.setup_services(broker_version=KafkaVersion(broker_version), auto_create_topics=auto_create_topics, security_protocol=security_protocol)
    +        self.setup_services(broker_version=KafkaVersion(broker_version), auto_create_topics=auto_create_topics,
    +                            security_protocol=security_protocol, include_filestream_connectors=True)
             self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node))
     
             self.cc.start()
    diff --git a/tests/kafkatest/tests/connect/connect_rest_test.py b/tests/kafkatest/tests/connect/connect_rest_test.py
    index 4d978a232d20..ff44d9412f1a 100644
    --- a/tests/kafkatest/tests/connect/connect_rest_test.py
    +++ b/tests/kafkatest/tests/connect/connect_rest_test.py
    @@ -73,7 +73,8 @@ def __init__(self, test_context):
                 'test': {'partitions': 1, 'replication-factor': 1}
             })
     
    -        self.cc = ConnectDistributedService(test_context, 2, self.kafka, [self.INPUT_FILE, self.INPUT_FILE2, self.OUTPUT_FILE])
    +        self.cc = ConnectDistributedService(test_context, 2, self.kafka, [self.INPUT_FILE, self.INPUT_FILE2, self.OUTPUT_FILE],
    +                                            include_filestream_connectors=True)
     
         @cluster(num_nodes=4)
         @matrix(connect_protocol=['compatible', 'eager'])
    diff --git a/tests/kafkatest/tests/connect/connect_test.py b/tests/kafkatest/tests/connect/connect_test.py
    index 1a7f6abfeb8b..4c2a91a6036b 100644
    --- a/tests/kafkatest/tests/connect/connect_test.py
    +++ b/tests/kafkatest/tests/connect/connect_test.py
    @@ -91,8 +91,10 @@ def test_file_source_and_sink(self, converter="org.apache.kafka.connect.json.Jso
                                       security_protocol=security_protocol, interbroker_security_protocol=security_protocol,
                                       topics=self.topics, controller_num_nodes_override=self.num_zk)
     
    -        self.source = ConnectStandaloneService(self.test_context, self.kafka, [self.INPUT_FILE, self.OFFSETS_FILE])
    -        self.sink = ConnectStandaloneService(self.test_context, self.kafka, [self.OUTPUT_FILE, self.OFFSETS_FILE])
    +        self.source = ConnectStandaloneService(self.test_context, self.kafka, [self.INPUT_FILE, self.OFFSETS_FILE],
    +                                               include_filestream_connectors=True)
    +        self.sink = ConnectStandaloneService(self.test_context, self.kafka, [self.OUTPUT_FILE, self.OFFSETS_FILE],
    +                                             include_filestream_connectors=True)
             self.consumer_validator = ConsoleConsumer(self.test_context, 1, self.kafka, self.TOPIC_TEST,
                                                       consumer_timeout_ms=10000)
     
    @@ -164,8 +166,10 @@ def test_skip_and_log_to_dlq(self, error_tolerance):
             else:
                 faulty_records = faulty_records[0]
     
    -        self.source = ConnectStandaloneService(self.test_context, self.kafka, [self.INPUT_FILE, self.OFFSETS_FILE])
    -        self.sink = ConnectStandaloneService(self.test_context, self.kafka, [self.OUTPUT_FILE, self.OFFSETS_FILE])
    +        self.source = ConnectStandaloneService(self.test_context, self.kafka, [self.INPUT_FILE, self.OFFSETS_FILE],
    +                                               include_filestream_connectors=True)
    +        self.sink = ConnectStandaloneService(self.test_context, self.kafka, [self.OUTPUT_FILE, self.OFFSETS_FILE],
    +                                             include_filestream_connectors=True)
     
             self.zk.start()
             self.kafka.start()
    
    From b2cb6caa1e9267c720c00fa367a277ee8509baea Mon Sep 17 00:00:00 2001
    From: Jason Gustafson 
    Date: Wed, 30 Mar 2022 13:59:22 -0700
    Subject: [PATCH 068/447] MINOR: Move `KafkaYammerMetrics` to server-common
     (#11970)
    
    With major server components like the new quorum controller being moved outside of the `core` module, it is useful to have shared dependencies moved into `server-common`. An example of this is Yammer metrics which server components still rely heavily upon. All server components should have access to the default registry used by the broker so that new metrics can be registered and metric naming conventions should be standardized. This is particularly important in KRaft where we are attempting to recreate identically named metrics in the controller context. This patch takes a step in this direction. It moves `KafkaYammerMetrics` into `server-common` and it implements
    standard metric naming utilities there.
    
    Reviewers: Manikumar Reddy 
    ---
     build.gradle                                  |   1 +
     checkstyle/import-control.xml                 |   5 +
     .../kafka/metrics/KafkaYammerMetrics.java     |  76 --------
     .../metrics/KafkaCSVMetricsReporter.scala     |   1 +
     .../kafka/metrics/KafkaMetricsGroup.scala     |   3 +-
     .../scala/kafka/server/BrokerServer.scala     |   5 +-
     .../scala/kafka/server/ControllerServer.scala |   3 +-
     .../main/scala/kafka/server/KafkaBroker.scala |   3 +-
     .../scala/kafka/server/KafkaRaftServer.scala  |   3 +-
     .../main/scala/kafka/server/KafkaServer.scala |   3 +-
     .../integration/kafka/api/BaseQuotaTest.scala |   3 +-
     .../kafka/api/EndToEndAuthorizationTest.scala |   4 +-
     .../integration/kafka/api/MetricsTest.scala   |   3 +-
     .../kafka/api/SslAdminIntegrationTest.scala   |   3 +-
     .../DynamicBrokerReconfigurationTest.scala    |   2 +-
     .../unit/kafka/cluster/PartitionTest.scala    |   4 +-
     .../ControllerEventManagerTest.scala          |   2 +-
     .../ControllerIntegrationTest.scala           |   3 +-
     .../group/GroupMetadataManagerTest.scala      |   4 +-
     .../TransactionMarkerChannelManagerTest.scala |   2 +-
     ...tricsDuringTopicCreationDeletionTest.scala |   2 +-
     .../kafka/log/LogCleanerIntegrationTest.scala |   3 +-
     .../scala/unit/kafka/log/LogManagerTest.scala |   5 +-
     .../unit/kafka/log/LogValidatorTest.scala     |   3 +-
     .../scala/unit/kafka/log/UnifiedLogTest.scala |   3 +-
     .../kafka/metrics/KafkaMetricsGroupTest.scala |  79 ++++++++
     .../unit/kafka/metrics/MetricsTest.scala      |   3 +-
     .../unit/kafka/network/SocketServerTest.scala |   7 +-
     .../server/AbstractFetcherManagerTest.scala   |   2 +-
     .../server/AbstractFetcherThreadTest.scala    |   2 +-
     .../kafka/server/BrokerMetricNamesTest.scala  |   2 +-
     .../kafka/server/ProduceRequestTest.scala     |   2 +-
     .../scala/unit/kafka/utils/TestUtils.scala    |   2 +-
     .../kafka/zookeeper/ZooKeeperClientTest.scala |   2 +-
     .../controller/QuorumControllerMetrics.java   |   6 +-
     .../server}/metrics/FilteringJmxReporter.java |   2 +-
     .../server/metrics/KafkaYammerMetrics.java    | 181 ++++++++++++++++++
     .../metrics/KafkaYammerMetricsTest.java       |  98 ++++++++++
     38 files changed, 420 insertions(+), 117 deletions(-)
     delete mode 100644 core/src/main/java/kafka/metrics/KafkaYammerMetrics.java
     create mode 100644 core/src/test/scala/unit/kafka/metrics/KafkaMetricsGroupTest.scala
     rename {core/src/main/java/kafka => server-common/src/main/java/org/apache/kafka/server}/metrics/FilteringJmxReporter.java (97%)
     create mode 100644 server-common/src/main/java/org/apache/kafka/server/metrics/KafkaYammerMetrics.java
     create mode 100644 server-common/src/test/java/org/apache/kafka/server/metrics/KafkaYammerMetricsTest.java
    
    diff --git a/build.gradle b/build.gradle
    index 3efb88d4b019..5b28b9389b07 100644
    --- a/build.gradle
    +++ b/build.gradle
    @@ -1415,6 +1415,7 @@ project(':server-common') {
       dependencies {
         api project(':clients')
         implementation libs.slf4jApi
    +    implementation libs.metrics
     
         testImplementation project(':clients')
         testImplementation project(':clients').sourceSets.test.output
    diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
    index ac2da909bab4..3b744979433f 100644
    --- a/checkstyle/import-control.xml
    +++ b/checkstyle/import-control.xml
    @@ -232,6 +232,7 @@
         
         
         
    +    
         
         
         
    @@ -317,6 +318,10 @@
           
         
     
    +    
    +      
    +    
    +
         
           
           
    diff --git a/core/src/main/java/kafka/metrics/KafkaYammerMetrics.java b/core/src/main/java/kafka/metrics/KafkaYammerMetrics.java
    deleted file mode 100644
    index dd650fdd0f79..000000000000
    --- a/core/src/main/java/kafka/metrics/KafkaYammerMetrics.java
    +++ /dev/null
    @@ -1,76 +0,0 @@
    -/*
    - * 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 kafka.metrics;
    -
    -import com.yammer.metrics.core.MetricsRegistry;
    -
    -import org.apache.kafka.common.Reconfigurable;
    -import org.apache.kafka.common.config.ConfigException;
    -import org.apache.kafka.common.metrics.JmxReporter;
    -
    -import java.util.Map;
    -import java.util.Set;
    -import java.util.function.Predicate;
    -
    -/**
    - * This class encapsulates the default yammer metrics registry for Kafka server,
    - * and configures the set of exported JMX metrics for Yammer metrics.
    - *
    - * KafkaYammerMetrics.defaultRegistry() should always be used instead of Metrics.defaultRegistry()
    - */
    -public class KafkaYammerMetrics implements Reconfigurable {
    -
    -    public static final KafkaYammerMetrics INSTANCE = new KafkaYammerMetrics();
    -
    -    /**
    -     * convenience method to replace {@link com.yammer.metrics.Metrics#defaultRegistry()}
    -     */
    -    public static MetricsRegistry defaultRegistry() {
    -        return INSTANCE.metricsRegistry;
    -    }
    -
    -    private final MetricsRegistry metricsRegistry = new MetricsRegistry();
    -    private final FilteringJmxReporter jmxReporter = new FilteringJmxReporter(metricsRegistry,
    -        metricName -> true);
    -
    -    private KafkaYammerMetrics() {
    -        jmxReporter.start();
    -        Runtime.getRuntime().addShutdownHook(new Thread(jmxReporter::shutdown));
    -    }
    -
    -    @Override
    -    public void configure(Map configs) {
    -        reconfigure(configs);
    -    }
    -
    -    @Override
    -    public Set reconfigurableConfigs() {
    -        return JmxReporter.RECONFIGURABLE_CONFIGS;
    -    }
    -
    -    @Override
    -    public void validateReconfiguration(Map configs) throws ConfigException {
    -        JmxReporter.compilePredicate(configs);
    -    }
    -
    -    @Override
    -    public void reconfigure(Map configs) {
    -        Predicate mBeanPredicate = JmxReporter.compilePredicate(configs);
    -        jmxReporter.updatePredicate(metricName -> mBeanPredicate.test(metricName.getMBeanName()));
    -    }
    -}
    diff --git a/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala b/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala
    index 0d8354728ac8..fa3c45a49459 100755
    --- a/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala
    +++ b/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala
    @@ -28,6 +28,7 @@ import java.util.concurrent.TimeUnit
     
     import kafka.utils.{Logging, VerifiableProperties}
     import org.apache.kafka.common.utils.Utils
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     
     private trait KafkaCSVMetricsReporterMBean extends KafkaMetricsReporterMBean
     
    diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala
    index 177edea52738..d174f3ea839e 100644
    --- a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala
    +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala
    @@ -19,9 +19,10 @@ package kafka.metrics
     
     import java.util.concurrent.TimeUnit
     
    -import com.yammer.metrics.core.{Gauge, MetricName, Meter, Histogram, Timer}
    +import com.yammer.metrics.core.{Gauge, Histogram, Meter, MetricName, Timer}
     import kafka.utils.Logging
     import org.apache.kafka.common.utils.Sanitizer
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     
     trait KafkaMetricsGroup extends Logging {
     
    diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala
    index 56cd2412fe59..05aa0f954c04 100644
    --- a/core/src/main/scala/kafka/server/BrokerServer.scala
    +++ b/core/src/main/scala/kafka/server/BrokerServer.scala
    @@ -22,11 +22,11 @@ import java.util
     import java.util.concurrent.atomic.AtomicBoolean
     import java.util.concurrent.locks.ReentrantLock
     import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit, TimeoutException}
    +
     import kafka.cluster.Broker.ServerInfo
     import kafka.coordinator.group.GroupCoordinator
     import kafka.coordinator.transaction.{ProducerIdManager, TransactionCoordinator}
     import kafka.log.LogManager
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.network.{DataPlaneAcceptor, SocketServer}
     import kafka.raft.RaftManager
     import kafka.security.CredentialProvider
    @@ -47,6 +47,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.metrics.KafkaYammerMetrics
     import org.apache.kafka.snapshot.SnapshotWriter
     
     import scala.collection.{Map, Seq}
    @@ -152,7 +153,7 @@ class BrokerServer(
     
       var metadataPublisher: BrokerMetadataPublisher = null
     
    -  def kafkaYammerMetrics: kafka.metrics.KafkaYammerMetrics = KafkaYammerMetrics.INSTANCE
    +  def kafkaYammerMetrics: KafkaYammerMetrics = KafkaYammerMetrics.INSTANCE
     
       private def maybeChangeStatus(from: ProcessStatus, to: ProcessStatus): Boolean = {
         lock.lock()
    diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala
    index 8c3502984787..1644c9258d58 100644
    --- a/core/src/main/scala/kafka/server/ControllerServer.scala
    +++ b/core/src/main/scala/kafka/server/ControllerServer.scala
    @@ -24,7 +24,7 @@ import java.util.concurrent.{CompletableFuture, TimeUnit}
     
     import kafka.api.KAFKA_3_2_IV0
     import kafka.cluster.Broker.ServerInfo
    -import kafka.metrics.{KafkaMetricsGroup, KafkaYammerMetrics, LinuxIoMetricsCollector}
    +import kafka.metrics.{KafkaMetricsGroup, LinuxIoMetricsCollector}
     import kafka.network.{DataPlaneAcceptor, SocketServer}
     import kafka.raft.RaftManager
     import kafka.security.CredentialProvider
    @@ -45,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.metrics.KafkaYammerMetrics
     import org.apache.kafka.server.policy.{AlterConfigPolicy, CreateTopicPolicy}
     
     import scala.jdk.CollectionConverters._
    diff --git a/core/src/main/scala/kafka/server/KafkaBroker.scala b/core/src/main/scala/kafka/server/KafkaBroker.scala
    index f4c6abc30653..46f2e7e8b19c 100644
    --- a/core/src/main/scala/kafka/server/KafkaBroker.scala
    +++ b/core/src/main/scala/kafka/server/KafkaBroker.scala
    @@ -20,7 +20,7 @@ package kafka.server
     import com.yammer.metrics.core.MetricName
     import kafka.coordinator.group.GroupCoordinator
     import kafka.log.LogManager
    -import kafka.metrics.{KafkaMetricsGroup, KafkaYammerMetrics, LinuxIoMetricsCollector}
    +import kafka.metrics.{KafkaMetricsGroup, LinuxIoMetricsCollector}
     import kafka.network.SocketServer
     import kafka.security.CredentialProvider
     import kafka.utils.KafkaScheduler
    @@ -31,6 +31,7 @@ import org.apache.kafka.common.network.ListenerName
     import org.apache.kafka.common.utils.Time
     import org.apache.kafka.metadata.BrokerState
     import org.apache.kafka.server.authorizer.Authorizer
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     
     import scala.collection.Seq
     import scala.jdk.CollectionConverters._
    diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala
    index 20c87ed9e51a..5ec8d3a38aa6 100644
    --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala
    +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala
    @@ -21,7 +21,7 @@ import java.util.concurrent.CompletableFuture
     
     import kafka.common.InconsistentNodeIdException
     import kafka.log.{LogConfig, UnifiedLog}
    -import kafka.metrics.{KafkaMetricsReporter, KafkaYammerMetrics}
    +import kafka.metrics.KafkaMetricsReporter
     import kafka.raft.KafkaRaftManager
     import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole}
     import kafka.utils.{CoreUtils, Logging, Mx4jLoader, VerifiableProperties}
    @@ -31,6 +31,7 @@ import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid}
     import org.apache.kafka.metadata.{KafkaConfigSchema, MetadataRecordSerde}
     import org.apache.kafka.raft.RaftConfig
     import org.apache.kafka.server.common.ApiMessageAndVersion
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     
     import scala.collection.Seq
     import scala.jdk.CollectionConverters._
    diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
    index 67013d339193..28cb1cbc6f8e 100755
    --- a/core/src/main/scala/kafka/server/KafkaServer.scala
    +++ b/core/src/main/scala/kafka/server/KafkaServer.scala
    @@ -29,7 +29,7 @@ import kafka.controller.KafkaController
     import kafka.coordinator.group.GroupCoordinator
     import kafka.coordinator.transaction.{ProducerIdManager, TransactionCoordinator}
     import kafka.log.LogManager
    -import kafka.metrics.{KafkaMetricsReporter, KafkaYammerMetrics}
    +import kafka.metrics.KafkaMetricsReporter
     import kafka.network.{ControlPlaneAcceptor, DataPlaneAcceptor, RequestChannel, SocketServer}
     import kafka.security.CredentialProvider
     import kafka.server.metadata.{ZkConfigRepository, ZkMetadataCache}
    @@ -50,6 +50,7 @@ import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time, Utils}
     import org.apache.kafka.common.{Endpoint, Node}
     import org.apache.kafka.metadata.BrokerState
     import org.apache.kafka.server.authorizer.Authorizer
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.apache.zookeeper.client.ZKClientConfig
     
     import scala.collection.{Map, Seq}
    diff --git a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala
    index 9f73236d0ba0..f8eedf172e97 100644
    --- a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala
    +++ b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala
    @@ -17,9 +17,9 @@ package kafka.api
     import java.time.Duration
     import java.util.concurrent.TimeUnit
     import java.util.{Collections, HashMap, Properties}
    +
     import com.yammer.metrics.core.{Histogram, Meter}
     import kafka.api.QuotaTestClients._
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.server.{ClientQuotaManager, ClientQuotaManagerConfig, KafkaConfig, KafkaServer, QuotaType}
     import kafka.utils.TestUtils
     import org.apache.kafka.clients.admin.Admin
    @@ -33,6 +33,7 @@ import org.apache.kafka.common.protocol.ApiKeys
     import org.apache.kafka.common.quota.ClientQuotaAlteration
     import org.apache.kafka.common.quota.ClientQuotaEntity
     import org.apache.kafka.common.security.auth.KafkaPrincipal
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.junit.jupiter.api.Assertions._
     import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
     
    diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala
    index 7c8ac014ad11..eb9522a50003 100644
    --- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala
    +++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala
    @@ -18,12 +18,11 @@
     package kafka.api
     
     import com.yammer.metrics.core.Gauge
    -
     import java.io.File
     import java.util.{Collections, Properties}
     import java.util.concurrent.ExecutionException
    +
     import kafka.admin.AclCommand
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.security.authorizer.AclAuthorizer
     import kafka.security.authorizer.AclEntry.WildcardHost
     import kafka.server._
    @@ -40,6 +39,7 @@ import org.apache.kafka.common.resource._
     import org.apache.kafka.common.resource.ResourceType._
     import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED}
     import org.apache.kafka.common.security.auth.KafkaPrincipal
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.junit.jupiter.api.Assertions._
     import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
     import org.junit.jupiter.params.ParameterizedTest
    diff --git a/core/src/test/scala/integration/kafka/api/MetricsTest.scala b/core/src/test/scala/integration/kafka/api/MetricsTest.scala
    index f5a3ae439f12..612092f41eb4 100644
    --- a/core/src/test/scala/integration/kafka/api/MetricsTest.scala
    +++ b/core/src/test/scala/integration/kafka/api/MetricsTest.scala
    @@ -13,11 +13,11 @@
     package kafka.api
     
     import java.util.{Locale, Properties}
    +
     import kafka.log.LogConfig
     import kafka.server.{KafkaConfig, KafkaServer}
     import kafka.utils.{JaasTestUtils, TestUtils}
     import com.yammer.metrics.core.{Gauge, Histogram, Meter}
    -import kafka.metrics.KafkaYammerMetrics
     import org.apache.kafka.clients.consumer.KafkaConsumer
     import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
     import org.apache.kafka.common.{Metric, MetricName, TopicPartition}
    @@ -26,6 +26,7 @@ import org.apache.kafka.common.errors.{InvalidTopicException, UnknownTopicOrPart
     import org.apache.kafka.common.network.ListenerName
     import org.apache.kafka.common.security.auth.SecurityProtocol
     import org.apache.kafka.common.security.authenticator.TestJaasConfig
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
     import org.junit.jupiter.api.Assertions._
     
    diff --git a/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala
    index b9180815c6f0..d04a09023cc3 100644
    --- a/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala
    +++ b/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala
    @@ -15,8 +15,8 @@ package kafka.api
     import java.io.File
     import java.util
     import java.util.concurrent._
    +
     import com.yammer.metrics.core.Gauge
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.security.authorizer.AclAuthorizer
     import kafka.server.KafkaConfig
     import kafka.utils.TestUtils
    @@ -26,6 +26,7 @@ import org.apache.kafka.common.protocol.ApiKeys
     import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType}
     import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
     import org.apache.kafka.server.authorizer._
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotNull, assertTrue}
     import org.junit.jupiter.api.{AfterEach, Test}
     
    diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
    index dee4f01da246..24d1eb969e7b 100644
    --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
    +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
    @@ -34,7 +34,6 @@ import kafka.api.{KafkaSasl, SaslSetup}
     import kafka.controller.{ControllerBrokerStateInfo, ControllerChannelManager}
     import kafka.log.{CleanerConfig, LogConfig}
     import kafka.message.ProducerCompressionCodec
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.network.{Processor, RequestChannel}
     import kafka.server.QuorumTestHarness
     import kafka.utils._
    @@ -61,6 +60,7 @@ import org.apache.kafka.common.record.TimestampType
     import org.apache.kafka.common.security.auth.SecurityProtocol
     import org.apache.kafka.common.security.scram.ScramCredential
     import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer}
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils}
     import org.junit.jupiter.api.Assertions._
     import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Test, TestInfo}
    diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
    index a5e791800695..deccad259a5f 100644
    --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
    +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
    @@ -20,7 +20,6 @@ import com.yammer.metrics.core.Metric
     import kafka.api.{ApiVersion, KAFKA_2_6_IV0}
     import kafka.common.UnexpectedAppendOffsetException
     import kafka.log.{Defaults => _, _}
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.server._
     import kafka.server.checkpoints.OffsetCheckpoints
     import kafka.utils._
    @@ -41,11 +40,12 @@ import org.mockito.ArgumentMatchers
     import org.mockito.ArgumentMatchers.{any, anyString}
     import org.mockito.Mockito._
     import org.mockito.invocation.InvocationOnMock
    -
     import java.nio.ByteBuffer
     import java.util.Optional
     import java.util.concurrent.{CountDownLatch, Semaphore}
    +
     import kafka.server.epoch.LeaderEpochFileCache
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     
     import scala.jdk.CollectionConverters._
     
    diff --git a/core/src/test/scala/unit/kafka/controller/ControllerEventManagerTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerEventManagerTest.scala
    index 26bbf94e9014..97d046f5ea7a 100644
    --- a/core/src/test/scala/unit/kafka/controller/ControllerEventManagerTest.scala
    +++ b/core/src/test/scala/unit/kafka/controller/ControllerEventManagerTest.scala
    @@ -22,12 +22,12 @@ import java.util.concurrent.atomic.AtomicInteger
     
     import com.yammer.metrics.core.{Histogram, MetricName, Timer}
     import kafka.controller
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.utils.TestUtils
     import org.apache.kafka.common.message.UpdateMetadataResponseData
     import org.apache.kafka.common.protocol.Errors
     import org.apache.kafka.common.requests.UpdateMetadataResponse
     import org.apache.kafka.common.utils.MockTime
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue, fail}
     import org.junit.jupiter.api.{AfterEach, Test}
     
    diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
    index 9d539fa7f608..1aa5ec25d325 100644
    --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
    +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
    @@ -19,10 +19,10 @@ package kafka.controller
     
     import java.util.Properties
     import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingQueue, TimeUnit}
    +
     import com.yammer.metrics.core.Timer
     import kafka.api.{ApiVersion, KAFKA_2_6_IV0, KAFKA_2_7_IV0, LeaderAndIsr}
     import kafka.controller.KafkaController.AlterPartitionCallback
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.server.{KafkaConfig, KafkaServer, QuorumTestHarness}
     import kafka.utils.{LogCaptureAppender, TestUtils}
     import kafka.zk.{FeatureZNodeStatus, _}
    @@ -32,6 +32,7 @@ import org.apache.kafka.common.metrics.KafkaMetric
     import org.apache.kafka.common.protocol.Errors
     import org.apache.kafka.common.{ElectionType, TopicPartition, Uuid}
     import org.apache.kafka.metadata.LeaderRecoveryState
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.apache.log4j.Level
     import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertTrue}
     import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
    diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
    index 7132ca79d372..2104fd3e589a 100644
    --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
    +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
    @@ -21,14 +21,13 @@ import java.lang.management.ManagementFactory
     import java.nio.ByteBuffer
     import java.util.concurrent.locks.ReentrantLock
     import java.util.{Collections, Optional}
    -import com.yammer.metrics.core.Gauge
     
    +import com.yammer.metrics.core.Gauge
     import javax.management.ObjectName
     import kafka.api._
     import kafka.cluster.Partition
     import kafka.common.OffsetAndMetadata
     import kafka.log.{AppendOrigin, LogAppendInfo, UnifiedLog}
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.server.{FetchDataInfo, FetchLogEnd, HostedPartition, KafkaConfig, LogOffsetMetadata, ReplicaManager, RequestLocal}
     import kafka.utils.{KafkaScheduler, MockTime, TestUtils}
     import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor
    @@ -42,6 +41,7 @@ import org.apache.kafka.common.record._
     import org.apache.kafka.common.requests.OffsetFetchResponse
     import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
     import org.apache.kafka.common.utils.Utils
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.junit.jupiter.api.Assertions._
     import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
     import org.mockito.{ArgumentCaptor, ArgumentMatchers}
    diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala
    index 20dbddc9546e..a15980990555 100644
    --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala
    +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala
    @@ -22,7 +22,6 @@ import java.util.Collections
     import java.util.concurrent.{Callable, Executors, Future}
     
     import kafka.common.RequestAndCompletionHandler
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.server.{KafkaConfig, MetadataCache}
     import kafka.utils.TestUtils
     import org.apache.kafka.clients.{ClientResponse, NetworkClient}
    @@ -31,6 +30,7 @@ import org.apache.kafka.common.record.RecordBatch
     import org.apache.kafka.common.requests.{RequestHeader, TransactionResult, WriteTxnMarkersRequest, WriteTxnMarkersResponse}
     import org.apache.kafka.common.utils.MockTime
     import org.apache.kafka.common.{Node, TopicPartition}
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.junit.jupiter.api.Assertions._
     import org.junit.jupiter.api.Test
     import org.mockito.ArgumentMatchers.any
    diff --git a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala
    index e045ea9b7792..7d363d13b3cc 100644
    --- a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala
    +++ b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala
    @@ -25,7 +25,7 @@ import kafka.utils.{Logging, TestUtils}
     import scala.jdk.CollectionConverters._
     import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
     import com.yammer.metrics.core.Gauge
    -import kafka.metrics.KafkaYammerMetrics
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     
     class MetricsDuringTopicCreationDeletionTest extends KafkaServerTestHarness with Logging {
     
    diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
    index c9797439a4b1..7cfa764d379f 100644
    --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
    +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
    @@ -20,10 +20,11 @@ package kafka.log
     import java.io.PrintWriter
     
     import com.yammer.metrics.core.{Gauge, MetricName}
    -import kafka.metrics.{KafkaMetricsGroup, KafkaYammerMetrics}
    +import kafka.metrics.KafkaMetricsGroup
     import kafka.utils.{MockTime, TestUtils}
     import org.apache.kafka.common.TopicPartition
     import org.apache.kafka.common.record.{CompressionType, RecordBatch}
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.junit.jupiter.api.Assertions._
     import org.junit.jupiter.api.{AfterEach, Test}
     
    diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
    index 11b511e3da6e..5353df6db385 100755
    --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
    +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
    @@ -18,7 +18,6 @@
     package kafka.log
     
     import com.yammer.metrics.core.MetricName
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.server.checkpoints.OffsetCheckpointFile
     import kafka.server.metadata.{ConfigRepository, MockConfigRepository}
     import kafka.server.{FetchDataInfo, FetchLogEnd}
    @@ -32,11 +31,13 @@ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
     import org.mockito.ArgumentMatchers.any
     import org.mockito.{ArgumentMatchers, Mockito}
     import org.mockito.Mockito.{doAnswer, mock, never, spy, times, verify}
    -
     import java.io._
     import java.nio.file.Files
     import java.util.concurrent.Future
     import java.util.{Collections, Properties}
    +
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
    +
     import scala.collection.mutable
     import scala.jdk.CollectionConverters._
     import scala.util.{Failure, Try}
    diff --git a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
    index 427568423073..57a7073ff1c0 100644
    --- a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
    +++ b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
    @@ -18,17 +18,18 @@ package kafka.log
     
     import java.nio.ByteBuffer
     import java.util.concurrent.TimeUnit
    +
     import kafka.api.{ApiVersion, KAFKA_2_0_IV1, KAFKA_2_3_IV1}
     import kafka.common.{LongRef, RecordValidationException}
     import kafka.log.LogValidator.ValidationAndOffsetAssignResult
     import kafka.message._
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.server.{BrokerTopicStats, RequestLocal}
     import kafka.utils.TestUtils.meterCount
     import org.apache.kafka.common.errors.{InvalidTimestampException, UnsupportedCompressionTypeException, UnsupportedForMessageFormatException}
     import org.apache.kafka.common.record._
     import org.apache.kafka.common.utils.Time
     import org.apache.kafka.common.{InvalidRecordException, TopicPartition}
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.apache.kafka.test.TestUtils
     import org.junit.jupiter.api.Assertions._
     import org.junit.jupiter.api.Test
    diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala
    index 79aa743eb6e3..e8c2c28e762d 100755
    --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala
    +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala
    @@ -22,8 +22,8 @@ import java.nio.ByteBuffer
     import java.nio.file.Files
     import java.util.concurrent.{Callable, Executors}
     import java.util.{Optional, Properties}
    +
     import kafka.common.{OffsetsOutOfOrderException, RecordValidationException, UnexpectedAppendOffsetException}
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.server.checkpoints.LeaderEpochCheckpointFile
     import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache}
     import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchIsolation, FetchLogEnd, FetchTxnCommitted, KafkaConfig, LogOffsetMetadata, PartitionMetadataFile}
    @@ -36,6 +36,7 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter
     import org.apache.kafka.common.record._
     import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse}
     import org.apache.kafka.common.utils.{BufferSupplier, Time, Utils}
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.junit.jupiter.api.Assertions._
     import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
     
    diff --git a/core/src/test/scala/unit/kafka/metrics/KafkaMetricsGroupTest.scala b/core/src/test/scala/unit/kafka/metrics/KafkaMetricsGroupTest.scala
    new file mode 100644
    index 000000000000..918553589d6c
    --- /dev/null
    +++ b/core/src/test/scala/unit/kafka/metrics/KafkaMetricsGroupTest.scala
    @@ -0,0 +1,79 @@
    +/*
    + * 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 kafka.metrics
    +
    +import org.junit.jupiter.api.Assertions.{assertEquals, assertNull}
    +import org.junit.jupiter.api.Test
    +
    +class KafkaMetricsGroupTest {
    +
    +  @Test
    +  def testUntaggedMetricName(): Unit = {
    +    val metricName = KafkaMetricsGroup.explicitMetricName(
    +      group = "kafka.metrics",
    +      typeName = "TestMetrics",
    +      name = "TaggedMetric",
    +      Map.empty
    +    )
    +
    +    assertEquals("kafka.metrics", metricName.getGroup)
    +    assertEquals("TestMetrics", metricName.getType)
    +    assertEquals("TaggedMetric", metricName.getName)
    +    assertEquals("kafka.metrics:type=TestMetrics,name=TaggedMetric",
    +      metricName.getMBeanName)
    +    assertNull(metricName.getScope)
    +  }
    +
    +  @Test
    +  def testTaggedMetricName(): Unit = {
    +    val tags = Map("foo" -> "bar", "bar" -> "baz", "baz" -> "raz.taz")
    +    val metricName = KafkaMetricsGroup.explicitMetricName(
    +      group = "kafka.metrics",
    +      typeName = "TestMetrics",
    +      name = "TaggedMetric",
    +      tags
    +    )
    +
    +    assertEquals("kafka.metrics", metricName.getGroup)
    +    assertEquals("TestMetrics", metricName.getType)
    +    assertEquals("TaggedMetric", metricName.getName)
    +    assertEquals("kafka.metrics:type=TestMetrics,name=TaggedMetric,foo=bar,bar=baz,baz=raz.taz",
    +      metricName.getMBeanName)
    +    assertEquals("bar.baz.baz.raz_taz.foo.bar", metricName.getScope)
    +  }
    +
    +  @Test
    +  def testTaggedMetricNameWithEmptyValue(): Unit = {
    +    val tags = Map("foo" -> "bar", "bar" -> "", "baz" -> "raz.taz")
    +    val metricName = KafkaMetricsGroup.explicitMetricName(
    +      group = "kafka.metrics",
    +      typeName = "TestMetrics",
    +      name = "TaggedMetric",
    +      tags
    +    )
    +
    +    assertEquals("kafka.metrics", metricName.getGroup)
    +    assertEquals("TestMetrics", metricName.getType)
    +    assertEquals("TaggedMetric", metricName.getName)
    +    assertEquals("kafka.metrics:type=TestMetrics,name=TaggedMetric,foo=bar,baz=raz.taz",
    +      metricName.getMBeanName)
    +    assertEquals("baz.raz_taz.foo.bar", metricName.getScope)
    +  }
    +
    +
    +}
    diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala
    index 3889ff350296..fb2ed99ef157 100644
    --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala
    +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala
    @@ -33,7 +33,8 @@ import kafka.log.LogConfig
     import org.apache.kafka.common.TopicPartition
     import org.apache.kafka.common.metrics.JmxReporter
     import org.apache.kafka.common.utils.Time
    -import org.junit.jupiter.api.Timeout;
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
    +import org.junit.jupiter.api.Timeout
     import org.junit.jupiter.params.ParameterizedTest
     import org.junit.jupiter.params.provider.ValueSource
     
    diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
    index 33d15ad10b5a..443edb1a79c3 100644
    --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
    +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
    @@ -25,12 +25,11 @@ import java.nio.charset.StandardCharsets
     import java.util
     import java.util.concurrent.{CompletableFuture, ConcurrentLinkedQueue, Executors, TimeUnit}
     import java.util.{Properties, Random}
    +
     import com.fasterxml.jackson.databind.node.{JsonNodeFactory, ObjectNode, TextNode}
     import com.yammer.metrics.core.{Gauge, Meter}
    -
     import javax.net.ssl._
     import kafka.cluster.EndPoint
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.security.CredentialProvider
     import kafka.server.{ApiVersionManager, KafkaConfig, SimpleApiVersionManager, ThrottleCallback, ThrottledChannel}
     import kafka.utils.Implicits._
    @@ -51,8 +50,10 @@ import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils}
     import org.apache.log4j.Level
     import org.junit.jupiter.api.Assertions._
     import org.junit.jupiter.api._
    -
     import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
    +
     import scala.collection.mutable
     import scala.collection.mutable.ArrayBuffer
     import scala.jdk.CollectionConverters._
    diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala
    index 647f8ae61a6e..3c02613e8501 100644
    --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala
    +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala
    @@ -18,9 +18,9 @@ package kafka.server
     
     import com.yammer.metrics.core.Gauge
     import kafka.cluster.BrokerEndPoint
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.utils.TestUtils
     import org.apache.kafka.common.{TopicPartition, Uuid}
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.junit.jupiter.api.{BeforeEach, Test}
     import org.junit.jupiter.api.Assertions._
     import org.mockito.Mockito.{mock, verify, when}
    diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
    index 148a903187b1..e34069b6f0bf 100644
    --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
    +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
    @@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicInteger
     import kafka.cluster.BrokerEndPoint
     import kafka.log.LogAppendInfo
     import kafka.message.NoCompressionCodec
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.server.AbstractFetcherThread.ReplicaFetch
     import kafka.server.AbstractFetcherThread.ResultWithPartitions
     import kafka.utils.Implicits.MapExtensionMethods
    @@ -38,6 +37,7 @@ import org.apache.kafka.common.record._
     import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
     import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
     import org.apache.kafka.common.utils.Time
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.junit.jupiter.api.Assertions._
     import org.junit.jupiter.api.Assumptions.assumeTrue
     import org.junit.jupiter.api.{BeforeEach, Test}
    diff --git a/core/src/test/scala/unit/kafka/server/BrokerMetricNamesTest.scala b/core/src/test/scala/unit/kafka/server/BrokerMetricNamesTest.scala
    index 3bd9c6da6102..c1322fe6fe2e 100644
    --- a/core/src/test/scala/unit/kafka/server/BrokerMetricNamesTest.scala
    +++ b/core/src/test/scala/unit/kafka/server/BrokerMetricNamesTest.scala
    @@ -17,11 +17,11 @@
     
     package kafka.server
     
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.test.ClusterInstance
     import kafka.test.annotation.{ClusterTest, ClusterTestDefaults, Type}
     import kafka.test.junit.ClusterTestExtensions
     import kafka.utils.TestUtils
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.junit.jupiter.api.AfterEach
     import org.junit.jupiter.api.Assertions.assertEquals
     import org.junit.jupiter.api.extension.ExtendWith
    diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala
    index 7d3ded57c489..97be68c68105 100644
    --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala
    +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala
    @@ -22,13 +22,13 @@ import java.util.{Collections, Properties}
     
     import kafka.log.LogConfig
     import kafka.message.ZStdCompressionCodec
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.utils.TestUtils
     import org.apache.kafka.common.TopicPartition
     import org.apache.kafka.common.message.ProduceRequestData
     import org.apache.kafka.common.protocol.Errors
     import org.apache.kafka.common.record._
     import org.apache.kafka.common.requests.{ProduceRequest, ProduceResponse}
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.junit.jupiter.api.Assertions._
     import org.junit.jupiter.api.Test
     
    diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
    index 37819d296185..deb6a115d22c 100755
    --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
    +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
    @@ -35,7 +35,6 @@ import kafka.api._
     import kafka.cluster.{Broker, EndPoint, IsrChangeListener}
     import kafka.controller.{ControllerEventManager, LeaderIsrAndControllerEpoch}
     import kafka.log._
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.network.RequestChannel
     import kafka.server._
     import kafka.server.checkpoints.OffsetCheckpointFile
    @@ -70,6 +69,7 @@ import org.apache.kafka.common.utils.{Time, Utils}
     import org.apache.kafka.common.{KafkaFuture, TopicPartition}
     import org.apache.kafka.controller.QuorumController
     import org.apache.kafka.server.authorizer.{Authorizer => JAuthorizer}
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils}
     import org.apache.zookeeper.KeeperException.SessionExpiredException
     import org.apache.zookeeper.ZooDefs._
    diff --git a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala
    index 5a8e4c382a8d..40a8f1e60ff0 100644
    --- a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala
    +++ b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala
    @@ -24,11 +24,11 @@ import java.util.concurrent.{ArrayBlockingQueue, ConcurrentLinkedQueue, CountDow
     import scala.collection.Seq
     import com.yammer.metrics.core.{Gauge, Meter, MetricName}
     import kafka.server.KafkaConfig
    -import kafka.metrics.KafkaYammerMetrics
     import kafka.utils.TestUtils
     import kafka.server.QuorumTestHarness
     import org.apache.kafka.common.security.JaasUtils
     import org.apache.kafka.common.utils.Time
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics
     import org.apache.zookeeper.KeeperException.{Code, NoNodeException}
     import org.apache.zookeeper.Watcher.Event.{EventType, KeeperState}
     import org.apache.zookeeper.ZooKeeper.States
    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 03cd47c8272c..d0280651aa60 100644
    --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java
    +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java
    @@ -21,6 +21,7 @@
     import com.yammer.metrics.core.Histogram;
     import com.yammer.metrics.core.MetricName;
     import com.yammer.metrics.core.MetricsRegistry;
    +import org.apache.kafka.server.metrics.KafkaYammerMetrics;
     
     import java.util.Arrays;
     import java.util.Objects;
    @@ -210,9 +211,6 @@ public void close() {
         }
     
         private static MetricName getMetricName(String type, String name) {
    -        final String group = "kafka.controller";
    -        final StringBuilder mbeanNameBuilder = new StringBuilder();
    -        mbeanNameBuilder.append(group).append(":type=").append(type).append(",name=").append(name);
    -        return new MetricName(group, type, name, null, mbeanNameBuilder.toString());
    +        return KafkaYammerMetrics.getMetricName("kafka.controller", type, name);
         }
     }
    diff --git a/core/src/main/java/kafka/metrics/FilteringJmxReporter.java b/server-common/src/main/java/org/apache/kafka/server/metrics/FilteringJmxReporter.java
    similarity index 97%
    rename from core/src/main/java/kafka/metrics/FilteringJmxReporter.java
    rename to server-common/src/main/java/org/apache/kafka/server/metrics/FilteringJmxReporter.java
    index 3794448a78ed..93a27c5d0f75 100644
    --- a/core/src/main/java/kafka/metrics/FilteringJmxReporter.java
    +++ b/server-common/src/main/java/org/apache/kafka/server/metrics/FilteringJmxReporter.java
    @@ -15,7 +15,7 @@
      * limitations under the License.
      */
     
    -package kafka.metrics;
    +package org.apache.kafka.server.metrics;
     
     import com.yammer.metrics.core.Metric;
     import com.yammer.metrics.core.MetricName;
    diff --git a/server-common/src/main/java/org/apache/kafka/server/metrics/KafkaYammerMetrics.java b/server-common/src/main/java/org/apache/kafka/server/metrics/KafkaYammerMetrics.java
    new file mode 100644
    index 000000000000..e4b7afc3eac9
    --- /dev/null
    +++ b/server-common/src/main/java/org/apache/kafka/server/metrics/KafkaYammerMetrics.java
    @@ -0,0 +1,181 @@
    +/*
    + * 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.metrics;
    +
    +import com.yammer.metrics.core.MetricName;
    +import com.yammer.metrics.core.MetricsRegistry;
    +import org.apache.kafka.common.Reconfigurable;
    +import org.apache.kafka.common.config.ConfigException;
    +import org.apache.kafka.common.metrics.JmxReporter;
    +import org.apache.kafka.common.utils.Sanitizer;
    +
    +import java.util.Iterator;
    +import java.util.LinkedHashMap;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.function.Predicate;
    +import java.util.function.Supplier;
    +
    +/**
    + * This class encapsulates the default yammer metrics registry for Kafka server,
    + * and configures the set of exported JMX metrics for Yammer metrics.
    + *
    + * KafkaYammerMetrics.defaultRegistry() should always be used instead of Metrics.defaultRegistry()
    + */
    +public class KafkaYammerMetrics implements Reconfigurable {
    +
    +    public static final KafkaYammerMetrics INSTANCE = new KafkaYammerMetrics();
    +
    +    /**
    +     * convenience method to replace {@link com.yammer.metrics.Metrics#defaultRegistry()}
    +     */
    +    public static MetricsRegistry defaultRegistry() {
    +        return INSTANCE.metricsRegistry;
    +    }
    +
    +    private final MetricsRegistry metricsRegistry = new MetricsRegistry();
    +    private final FilteringJmxReporter jmxReporter = new FilteringJmxReporter(metricsRegistry,
    +        metricName -> true);
    +
    +    private KafkaYammerMetrics() {
    +        jmxReporter.start();
    +        Runtime.getRuntime().addShutdownHook(new Thread(jmxReporter::shutdown));
    +    }
    +
    +    @Override
    +    public void configure(Map configs) {
    +        reconfigure(configs);
    +    }
    +
    +    @Override
    +    public Set reconfigurableConfigs() {
    +        return JmxReporter.RECONFIGURABLE_CONFIGS;
    +    }
    +
    +    @Override
    +    public void validateReconfiguration(Map configs) throws ConfigException {
    +        JmxReporter.compilePredicate(configs);
    +    }
    +
    +    @Override
    +    public void reconfigure(Map configs) {
    +        Predicate mBeanPredicate = JmxReporter.compilePredicate(configs);
    +        jmxReporter.updatePredicate(metricName -> mBeanPredicate.test(metricName.getMBeanName()));
    +    }
    +
    +    public static MetricName getMetricName(
    +        String group,
    +        String typeName,
    +        String name
    +    ) {
    +        return getMetricName(
    +            group,
    +            typeName,
    +            name,
    +            null
    +        );
    +    }
    +
    +    public static MetricName getMetricName(
    +        String group,
    +        String typeName,
    +        String name,
    +        LinkedHashMap tags
    +    ) {
    +        StringBuilder nameBuilder = new StringBuilder();
    +        nameBuilder.append(group);
    +        nameBuilder.append(":type=");
    +        nameBuilder.append(typeName);
    +
    +        if (name.length() > 0) {
    +            nameBuilder.append(",name=");
    +            nameBuilder.append(name);
    +        }
    +
    +        String scope = toScope(tags).orElse(null);
    +        Optional tagsName = toMBeanName(tags);
    +        tagsName.ifPresent(nameBuilder::append);
    +
    +        return new MetricName(group, typeName, name, scope, nameBuilder.toString());
    +    }
    +
    +    private static Optional toMBeanName(LinkedHashMap tags) {
    +        if (tags == null) {
    +            return Optional.empty();
    +        }
    +
    +        LinkedHashMap nonEmptyTags = collectNonEmptyTags(tags, LinkedHashMap::new);
    +        if (nonEmptyTags.isEmpty()) {
    +            return Optional.empty();
    +        } else {
    +            StringBuilder tagsString = new StringBuilder();
    +            for (Map.Entry tagEntry : nonEmptyTags.entrySet()) {
    +                String sanitizedValue = Sanitizer.jmxSanitize(tagEntry.getValue());
    +                tagsString.append(",");
    +                tagsString.append(tagEntry.getKey());
    +                tagsString.append("=");
    +                tagsString.append(sanitizedValue);
    +            }
    +            return Optional.of(tagsString.toString());
    +        }
    +    }
    +
    +    private static > T collectNonEmptyTags(
    +        Map tags,
    +        Supplier mapSupplier
    +    ) {
    +        T result = mapSupplier.get();
    +        for (Map.Entry tagEntry : tags.entrySet()) {
    +            String tagValue = tagEntry.getValue();
    +            if (!"".equals(tagValue)) {
    +                result.put(tagEntry.getKey(), tagValue);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    private static Optional toScope(Map tags) {
    +        if (tags == null) {
    +            return Optional.empty();
    +        }
    +
    +        SortedMap nonEmptyTags = collectNonEmptyTags(tags, TreeMap::new);
    +        if (nonEmptyTags.isEmpty()) {
    +            return Optional.empty();
    +        } else {
    +            StringBuilder tagsString = new StringBuilder();
    +
    +            for (Iterator> iterator = nonEmptyTags.entrySet().iterator(); iterator.hasNext();) {
    +                // convert dot to _ since reporters like Graphite typically use dot to represent hierarchy
    +                Map.Entry tagEntry = iterator.next();
    +                String convertedValue = tagEntry.getValue().replaceAll("\\.", "_");
    +                tagsString.append(tagEntry.getKey());
    +                tagsString.append(".");
    +                tagsString.append(convertedValue);
    +
    +                if (iterator.hasNext()) {
    +                    tagsString.append(".");
    +                }
    +            }
    +            return Optional.of(tagsString.toString());
    +        }
    +    }
    +}
    diff --git a/server-common/src/test/java/org/apache/kafka/server/metrics/KafkaYammerMetricsTest.java b/server-common/src/test/java/org/apache/kafka/server/metrics/KafkaYammerMetricsTest.java
    new file mode 100644
    index 000000000000..dfe0aaf416e5
    --- /dev/null
    +++ b/server-common/src/test/java/org/apache/kafka/server/metrics/KafkaYammerMetricsTest.java
    @@ -0,0 +1,98 @@
    +/*
    + * 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.metrics;
    +
    +import com.yammer.metrics.core.MetricName;
    +import org.junit.jupiter.api.Test;
    +
    +import java.util.LinkedHashMap;
    +
    +import static org.junit.jupiter.api.Assertions.assertEquals;
    +import static org.junit.jupiter.api.Assertions.assertNull;
    +
    +class KafkaYammerMetricsTest {
    +
    +    @Test
    +    public void testUntaggedMetric() {
    +        MetricName metricName = KafkaYammerMetrics.getMetricName(
    +            "kafka.metrics",
    +            "TestMetrics",
    +            "UntaggedMetric"
    +        );
    +
    +        assertEquals("kafka.metrics", metricName.getGroup());
    +        assertEquals("TestMetrics", metricName.getType());
    +        assertEquals("UntaggedMetric", metricName.getName());
    +        assertEquals("kafka.metrics:type=TestMetrics,name=UntaggedMetric",
    +            metricName.getMBeanName());
    +        assertNull(metricName.getScope());
    +    }
    +
    +    @Test
    +    public void testTaggedMetricName() {
    +        LinkedHashMap tags = new LinkedHashMap<>();
    +        tags.put("foo", "bar");
    +        tags.put("bar", "baz");
    +        tags.put("baz", "raz.taz");
    +
    +        MetricName metricName = KafkaYammerMetrics.getMetricName(
    +            "kafka.metrics",
    +            "TestMetrics",
    +            "TaggedMetric",
    +            tags
    +        );
    +
    +        assertEquals("kafka.metrics", metricName.getGroup());
    +        assertEquals("TestMetrics", metricName.getType());
    +        assertEquals("TaggedMetric", metricName.getName());
    +
    +        // MBean name should preserve initial ordering
    +        assertEquals("kafka.metrics:type=TestMetrics,name=TaggedMetric,foo=bar,bar=baz,baz=raz.taz",
    +            metricName.getMBeanName());
    +
    +        // Scope should be sorted by key
    +        assertEquals("bar.baz.baz.raz_taz.foo.bar", metricName.getScope());
    +    }
    +
    +    @Test
    +    public void testTaggedMetricNameWithEmptyValue() {
    +        LinkedHashMap tags = new LinkedHashMap<>();
    +        tags.put("foo", "bar");
    +        tags.put("bar", "");
    +        tags.put("baz", "raz.taz");
    +
    +        MetricName metricName = KafkaYammerMetrics.getMetricName(
    +            "kafka.metrics",
    +            "TestMetrics",
    +            "TaggedMetric",
    +            tags
    +        );
    +
    +        assertEquals("kafka.metrics", metricName.getGroup());
    +        assertEquals("TestMetrics", metricName.getType());
    +        assertEquals("TaggedMetric", metricName.getName());
    +
    +        // MBean name should preserve initial ordering (with empty key value removed)
    +        assertEquals("kafka.metrics:type=TestMetrics,name=TaggedMetric,foo=bar,baz=raz.taz",
    +            metricName.getMBeanName());
    +
    +        // Scope should be sorted by key (with empty key value removed)
    +        assertEquals("baz.raz_taz.foo.bar", metricName.getScope());
    +    }
    +
    +}
    \ No newline at end of file
    
    From 1317f3f77a9e1e432e7a81de2dcb88365feeac43 Mon Sep 17 00:00:00 2001
    From: "A. Sophie Blee-Goldman" 
    Date: Wed, 30 Mar 2022 16:24:01 -0700
    Subject: [PATCH 069/447] MINOR: log warning when topology override for cache
     size is non-zero (#11959)
    
    Since the topology-level cache size config only controls whether we disable the caching layer entirely for that topology, setting it to anything other than 0 has no effect. The actual cache memory is still just split evenly between the threads, and shared by all topologies.
    
    It's possible we'll want to change this in the future, but for now we should make sure to log a warning so that users who do try to set this override to some nonzero value are made aware that it doesn't work like this.
    
    Also includes some minor refactoring plus a fix for an off-by-one error in #11796
    
    Reviewers: Luke Chen , Walker Carlson , Sagar Rao 
    ---
     checkstyle/suppressions.xml                   |  2 +-
     .../apache/kafka/streams/KafkaStreams.java    |  7 +-
     .../apache/kafka/streams/StreamsConfig.java   | 23 -------
     .../apache/kafka/streams/TopologyConfig.java  | 62 +++++++++++-------
     .../streams/internals/StreamsConfigUtils.java | 31 +++++++++
     .../kafka/streams/KafkaStreamsTest.java       | 25 +++++++
     .../kafka/streams/StreamsConfigTest.java      |  9 +--
     .../AdjustStreamThreadCountTest.java          | 65 ++-----------------
     .../kafka/streams/TopologyTestDriver.java     |  3 +-
     9 files changed, 114 insertions(+), 113 deletions(-)
    
    diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
    index cd82efe1421c..0824e0b60b73 100644
    --- a/checkstyle/suppressions.xml
    +++ b/checkstyle/suppressions.xml
    @@ -188,7 +188,7 @@
                   files="StreamsMetricsImpl.java"/>
     
         
    +              files="(GlobalStateManagerImpl|KafkaStreams|KStreamImplJoin|StreamsPartitionAssignor|StreamThread|TaskManager|TopologyConfig).java"/>
     
         
    diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
    index 7798e58204c9..9d9d57f8ab99 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
    @@ -109,6 +109,7 @@
     import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
     import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix;
     import static org.apache.kafka.streams.internals.ApiUtils.validateMillisecondDuration;
    +import static org.apache.kafka.streams.internals.StreamsConfigUtils.getTotalCacheSize;
     import static org.apache.kafka.streams.processor.internals.ClientUtils.fetchEndOffsets;
     
     /**
    @@ -939,7 +940,7 @@ private KafkaStreams(final TopologyMetadata topologyMetadata,
             streamsUncaughtExceptionHandler = this::defaultStreamsUncaughtExceptionHandler;
             delegatingStateRestoreListener = new DelegatingStateRestoreListener();
     
    -        totalCacheSize = applicationConfigs.getTotalCacheSize();
    +        totalCacheSize = getTotalCacheSize(applicationConfigs);
             inputBufferMaxBytes = applicationConfigs.getLong(StreamsConfig.INPUT_BUFFER_MAX_BYTES_CONFIG);
             final int numStreamThreads = topologyMetadata.getNumStreamThreads(applicationConfigs);
     
    @@ -1047,9 +1048,9 @@ public Optional addStreamThread() {
                     // and then resize them later
                     streamThread = createAndAddStreamThread(0L, 0L, threadIdx);
                     final int numLiveThreads = getNumLiveStreamThreads();
    -                resizeThreadCacheAndBufferMemory(numLiveThreads + 1);
    +                resizeThreadCacheAndBufferMemory(numLiveThreads);
                     log.info("Adding StreamThread-{}, there are now {} threads with cache size/max buffer size values as {} per thread.",
    -                        threadIdx, numLiveThreads + 1, getThreadCacheAndBufferMemoryString());
    +                        threadIdx, numLiveThreads, getThreadCacheAndBufferMemoryString());
                 }
     
                 synchronized (stateLock) {
    diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
    index e9e0cca05a87..740f7af5da2c 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
    @@ -1536,29 +1536,6 @@ public Map getProducerConfigs(final String clientId) {
             return props;
         }
     
    -    public long getTotalCacheSize() {
    -        // both deprecated and new config set. Warn and use the new one.
    -        if (originals().containsKey(CACHE_MAX_BYTES_BUFFERING_CONFIG) && originals().containsKey(STATESTORE_CACHE_MAX_BYTES_CONFIG)) {
    -            if (!getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG).equals(getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG))) {
    -                log.warn("Both deprecated config {} and the new config {} are set, hence {} is ignored and {} is used instead.",
    -                        CACHE_MAX_BYTES_BUFFERING_CONFIG,
    -                        STATESTORE_CACHE_MAX_BYTES_CONFIG,
    -                        CACHE_MAX_BYTES_BUFFERING_CONFIG,
    -                        STATESTORE_CACHE_MAX_BYTES_CONFIG);
    -            }
    -            return getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG);
    -        } else if (originals().containsKey(CACHE_MAX_BYTES_BUFFERING_CONFIG)) {
    -            // only deprecated config set.
    -            log.warn("Deprecated config {} is set, and will be used; we suggest setting the new config {} instead as deprecated {} would be removed in the future.",
    -                    CACHE_MAX_BYTES_BUFFERING_CONFIG,
    -                    STATESTORE_CACHE_MAX_BYTES_CONFIG,
    -                    CACHE_MAX_BYTES_BUFFERING_CONFIG);
    -            return getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG);
    -        }
    -        // only new or no config set. Use default or user specified value.
    -        return getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG);
    -    }
    -
         /**
          * Get the configs for the {@link Admin admin client}.
          * @param clientId clientId
    diff --git a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java
    index de8aec94ce4a..da9a8c89bfa5 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java
    @@ -49,6 +49,7 @@
     import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_DOC;
     import static org.apache.kafka.streams.StreamsConfig.ROCKS_DB;
     import static org.apache.kafka.streams.StreamsConfig.IN_MEMORY;
    +import static org.apache.kafka.streams.internals.StreamsConfigUtils.getTotalCacheSize;
     
     /**
      * Streams configs that apply at the topology level. The values in the {@link StreamsConfig} parameter of the
    @@ -136,37 +137,54 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo
                 maxBufferedSize = getInt(BUFFERED_RECORDS_PER_PARTITION_CONFIG);
                 log.info("Topology {} is overriding {} to {}", topologyName, BUFFERED_RECORDS_PER_PARTITION_CONFIG, maxBufferedSize);
             } else {
    +            // If the user hasn't explicitly set the buffered.records.per.partition config, then leave it unbounded
    +            // and rely on the input.buffer.max.bytes instead to keep the memory usage under control
                 maxBufferedSize = globalAppConfigs.originals().containsKey(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG)
                         ? globalAppConfigs.getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG) : -1;
             }
     
    -        if (isTopologyOverride(STATESTORE_CACHE_MAX_BYTES_CONFIG, topologyOverrides) && isTopologyOverride(CACHE_MAX_BYTES_BUFFERING_CONFIG, topologyOverrides)) {
    -            cacheSize = getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG);
    -            log.info("Topology {} is using both deprecated config {} and new config {}, hence {} is ignored and the new config {} (value {}) is used",
    -                    topologyName,
    -                    CACHE_MAX_BYTES_BUFFERING_CONFIG,
    -                    STATESTORE_CACHE_MAX_BYTES_CONFIG,
    -                    CACHE_MAX_BYTES_BUFFERING_CONFIG,
    -                    STATESTORE_CACHE_MAX_BYTES_CONFIG,
    -                    cacheSize);
    -        } else if (isTopologyOverride(CACHE_MAX_BYTES_BUFFERING_CONFIG, topologyOverrides)) {
    -            cacheSize = getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG);
    -            log.info("Topology {} is using only deprecated config {}, and will be used to set cache size to {}; " +
    -                            "we suggest setting the new config {} instead as deprecated {} would be removed in the future.",
    -                    topologyName,
    -                    CACHE_MAX_BYTES_BUFFERING_CONFIG,
    -                    cacheSize,
    -                    STATESTORE_CACHE_MAX_BYTES_CONFIG,
    -                    CACHE_MAX_BYTES_BUFFERING_CONFIG);
    -        } else if (isTopologyOverride(STATESTORE_CACHE_MAX_BYTES_CONFIG, topologyOverrides)) {
    -            cacheSize = getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG);
    +        final boolean stateStoreCacheMaxBytesOverridden = isTopologyOverride(STATESTORE_CACHE_MAX_BYTES_CONFIG, topologyOverrides);
    +        final boolean cacheMaxBytesBufferingOverridden = isTopologyOverride(CACHE_MAX_BYTES_BUFFERING_CONFIG, topologyOverrides);
    +
    +        if (!stateStoreCacheMaxBytesOverridden && !cacheMaxBytesBufferingOverridden) {
    +            cacheSize = getTotalCacheSize(globalAppConfigs);
             } else {
    -            cacheSize = globalAppConfigs.getTotalCacheSize();
    +            if (stateStoreCacheMaxBytesOverridden && cacheMaxBytesBufferingOverridden) {
    +                cacheSize = getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG);
    +                log.info("Topology {} is using both deprecated config {} and new config {}, hence {} is ignored and the new config {} (value {}) is used",
    +                         topologyName,
    +                         CACHE_MAX_BYTES_BUFFERING_CONFIG,
    +                         STATESTORE_CACHE_MAX_BYTES_CONFIG,
    +                         CACHE_MAX_BYTES_BUFFERING_CONFIG,
    +                         STATESTORE_CACHE_MAX_BYTES_CONFIG,
    +                         cacheSize);
    +            } else if (cacheMaxBytesBufferingOverridden) {
    +                cacheSize = getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG);
    +                log.info("Topology {} is using only deprecated config {}, and will be used to set cache size to {}; " +
    +                             "we suggest setting the new config {} instead as deprecated {} would be removed in the future.",
    +                         topologyName,
    +                         CACHE_MAX_BYTES_BUFFERING_CONFIG,
    +                         cacheSize,
    +                         STATESTORE_CACHE_MAX_BYTES_CONFIG,
    +                         CACHE_MAX_BYTES_BUFFERING_CONFIG);
    +            } else {
    +                cacheSize = getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG);
    +            }
    +
    +            if (cacheSize != 0) {
    +                log.warn("Topology {} is overriding cache size to {} but this will not have any effect as the "
    +                             + "topology-level cache size config only controls whether record buffering is enabled "
    +                             + "or disabled, thus the only valid override value is 0",
    +                         topologyName, cacheSize);
    +            } else {
    +                log.info("Topology {} is overriding cache size to {}, record buffering will be disabled",
    +                         topologyName, cacheSize);
    +            }
             }
     
             if (isTopologyOverride(MAX_TASK_IDLE_MS_CONFIG, topologyOverrides)) {
                 maxTaskIdleMs = getLong(MAX_TASK_IDLE_MS_CONFIG);
    -            log.info("Topology {} is overridding {} to {}", topologyName, MAX_TASK_IDLE_MS_CONFIG, maxTaskIdleMs);
    +            log.info("Topology {} is overriding {} to {}", topologyName, MAX_TASK_IDLE_MS_CONFIG, maxTaskIdleMs);
             } else {
                 maxTaskIdleMs = globalAppConfigs.getLong(MAX_TASK_IDLE_MS_CONFIG);
             }
    diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/StreamsConfigUtils.java b/streams/src/main/java/org/apache/kafka/streams/internals/StreamsConfigUtils.java
    index e271a42ab891..6f169826f069 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/internals/StreamsConfigUtils.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/internals/StreamsConfigUtils.java
    @@ -17,9 +17,16 @@
     package org.apache.kafka.streams.internals;
     
     import org.apache.kafka.streams.StreamsConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.apache.kafka.streams.StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG;
    +import static org.apache.kafka.streams.StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG;
     
     public class StreamsConfigUtils {
     
    +    private static final Logger LOG = LoggerFactory.getLogger(StreamsConfigUtils.class);
    +
         public enum ProcessingMode {
             AT_LEAST_ONCE("AT_LEAST_ONCE"),
     
    @@ -66,4 +73,28 @@ public static boolean eosEnabled(final ProcessingMode processingMode) {
             return processingMode == ProcessingMode.EXACTLY_ONCE_ALPHA ||
                 processingMode == ProcessingMode.EXACTLY_ONCE_V2;
         }
    +
    +    @SuppressWarnings("deprecation")
    +    public static long getTotalCacheSize(final StreamsConfig config) {
    +        // both deprecated and new config set. Warn and use the new one.
    +        if (config.originals().containsKey(CACHE_MAX_BYTES_BUFFERING_CONFIG) && config.originals().containsKey(STATESTORE_CACHE_MAX_BYTES_CONFIG)) {
    +            if (!config.getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG).equals(config.getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG))) {
    +                LOG.warn("Both deprecated config {} and the new config {} are set, hence {} is ignored and {} is used instead.",
    +                         CACHE_MAX_BYTES_BUFFERING_CONFIG,
    +                         STATESTORE_CACHE_MAX_BYTES_CONFIG,
    +                         CACHE_MAX_BYTES_BUFFERING_CONFIG,
    +                         STATESTORE_CACHE_MAX_BYTES_CONFIG);
    +            }
    +            return config.getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG);
    +        } else if (config.originals().containsKey(CACHE_MAX_BYTES_BUFFERING_CONFIG)) {
    +            // only deprecated config set.
    +            LOG.warn("Deprecated config {} is set, and will be used; we suggest setting the new config {} instead as deprecated {} would be removed in the future.",
    +                     CACHE_MAX_BYTES_BUFFERING_CONFIG,
    +                     STATESTORE_CACHE_MAX_BYTES_CONFIG,
    +                     CACHE_MAX_BYTES_BUFFERING_CONFIG);
    +            return config.getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG);
    +        }
    +        // only new or no config set. Use default or user specified value.
    +        return config.getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG);
    +    }
     }
    diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
    index b5620d5c13ed..eabb8edc1147 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
    @@ -143,6 +143,8 @@ public class KafkaStreamsTest {
         private GlobalStreamThread globalStreamThread;
         @Mock
         private Metrics metrics;
    +    @Mock
    +    private State state;
     
         private StateListenerStub streamsStateListener;
         private Capture> metricsReportersCapture;
    @@ -216,6 +218,7 @@ private void prepareStreams() throws Exception {
             ClientMetrics.addStateMetric(anyObject(StreamsMetricsImpl.class), anyObject());
             ClientMetrics.addNumAliveStreamThreadMetric(anyObject(StreamsMetricsImpl.class), anyObject());
     
    +
             // setup stream threads
             PowerMock.mockStatic(StreamThread.class);
             EasyMock.expect(StreamThread.create(
    @@ -240,6 +243,7 @@ private void prepareStreams() throws Exception {
             PowerMock.mockStatic(StreamsConfigUtils.class);
             EasyMock.expect(StreamsConfigUtils.processingMode(anyObject(StreamsConfig.class))).andReturn(StreamsConfigUtils.ProcessingMode.AT_LEAST_ONCE).anyTimes();
             EasyMock.expect(StreamsConfigUtils.eosEnabled(anyObject(StreamsConfig.class))).andReturn(false).anyTimes();
    +        EasyMock.expect(StreamsConfigUtils.getTotalCacheSize(anyObject(StreamsConfig.class))).andReturn(10 * 1024 * 1024L).anyTimes();
             EasyMock.expect(streamThreadOne.getId()).andReturn(1L).anyTimes();
             EasyMock.expect(streamThreadTwo.getId()).andReturn(2L).anyTimes();
             EasyMock.expect(streamThreadOne.getCacheSize()).andReturn(10485760L).anyTimes();
    @@ -598,6 +602,27 @@ public void shouldAddThreadWhenRunning() throws InterruptedException {
                 streams.start();
                 final int oldSize = streams.threads.size();
                 waitForCondition(() -> streams.state() == KafkaStreams.State.RUNNING, 15L, "wait until running");
    +            EasyMock.reset(streamThreadOne, streamThreadTwo);
    +            EasyMock.expect(streamThreadOne.isRunning()).andStubReturn(true);
    +            EasyMock.expect(streamThreadTwo.isRunning()).andStubReturn(true);
    +            EasyMock.expect(streamThreadOne.state()).andStubReturn(StreamThread.State.RUNNING);
    +            EasyMock.expect(streamThreadTwo.state()).andStubReturn(StreamThread.State.RUNNING);
    +            EasyMock.expect(streamThreadOne.getName()).andStubReturn("processId-StreamThread-1");
    +            EasyMock.expect(streamThreadTwo.getName()).andStubReturn("processId-StreamThread-2");
    +            EasyMock.expect(streamThreadTwo.getId()).andStubReturn(2L);
    +            EasyMock.expect(streamThreadOne.getCacheSize()).andReturn(10485760L).anyTimes();
    +            EasyMock.expect(streamThreadOne.getMaxBufferSize()).andReturn(536870912L).anyTimes();
    +            EasyMock.expect(streamThreadTwo.getCacheSize()).andReturn(10485760L).anyTimes();
    +            EasyMock.expect(streamThreadTwo.getMaxBufferSize()).andReturn(536870912L).anyTimes();
    +            streamThreadTwo.setStateListener(EasyMock.anyObject());
    +            streamThreadTwo.start();
    +
    +            streamThreadOne.resizeCacheAndBufferMemory(5 * 1024 * 1024L, 256 * 1024 * 1024L);
    +            streamThreadTwo.resizeCacheAndBufferMemory(5 * 1024 * 1024L, 256 * 1024 * 1024L);
    +            streamThreadOne.shutdown();
    +            streamThreadTwo.shutdown();
    +            EasyMock.expect(state.isRunningOrRebalancing()).andStubReturn(true);
    +            EasyMock.replay(streamThreadOne, streamThreadTwo, state);
                 assertThat(streams.addStreamThread(), equalTo(Optional.of("processId-StreamThread-" + 2)));
                 assertThat(streams.threads.size(), equalTo(oldSize + 1));
             }
    diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
    index 243a4741677b..4deed5ec9032 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
    @@ -63,6 +63,7 @@
     import static org.apache.kafka.streams.StreamsConfig.adminClientPrefix;
     import static org.apache.kafka.streams.StreamsConfig.consumerPrefix;
     import static org.apache.kafka.streams.StreamsConfig.producerPrefix;
    +import static org.apache.kafka.streams.internals.StreamsConfigUtils.getTotalCacheSize;
     import static org.apache.kafka.test.StreamsTestUtils.getStreamsConfig;
     import static org.hamcrest.CoreMatchers.containsString;
     import static org.hamcrest.CoreMatchers.hasItem;
    @@ -1258,7 +1259,7 @@ public void shouldUseStateStoreCacheMaxBytesWhenBothOldAndNewConfigsAreSet() {
             props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 100);
             props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10);
             final StreamsConfig config = new StreamsConfig(props);
    -        assertEquals(config.getTotalCacheSize(), 100);
    +        assertEquals(getTotalCacheSize(config), 100);
         }
     
         @Test
    @@ -1266,20 +1267,20 @@ public void shouldUseStateStoreCacheMaxBytesWhenBothOldAndNewConfigsAreSet() {
         public void shouldUseCacheMaxBytesBufferingConfigWhenOnlyDeprecatedConfigIsSet() {
             props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10);
             final StreamsConfig config = new StreamsConfig(props);
    -        assertEquals(config.getTotalCacheSize(), 10);
    +        assertEquals(getTotalCacheSize(config), 10);
         }
     
         @Test
         public void shouldUseStateStoreCacheMaxBytesWhenNewConfigIsSet() {
             props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 10);
             final StreamsConfig config = new StreamsConfig(props);
    -        assertEquals(config.getTotalCacheSize(), 10);
    +        assertEquals(getTotalCacheSize(config), 10);
         }
     
         @Test
         public void shouldUseDefaultStateStoreCacheMaxBytesConfigWhenNoConfigIsSet() {
             final StreamsConfig config = new StreamsConfig(props);
    -        assertEquals(config.getTotalCacheSize(), 10 * 1024 * 1024);
    +        assertEquals(getTotalCacheSize(config), 10 * 1024 * 1024);
         }
     
         static class MisconfiguredSerde implements Serde {
    diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java
    index 1119818b9272..e5d0b9348a5c 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java
    @@ -421,12 +421,14 @@ public void shouldResizeMaxBufferAfterThreadRemovalTimesOut() throws Interrupted
         }
     
         @Test
    -    public void shouldResizeCacheAfterThreadReplacement() throws InterruptedException {
    +    public void shouldResizeCacheAndInputBufferAfterThreadReplacement() throws InterruptedException {
             final long totalCacheBytes = 10L;
    +        final long maxBufferBytes = 100L;
             final Properties props = new Properties();
             props.putAll(properties);
             props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2);
             props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, totalCacheBytes);
    +        props.put(StreamsConfig.INPUT_BUFFER_MAX_BYTES_CONFIG, maxBufferBytes);
     
             final AtomicBoolean injectError = new AtomicBoolean(false);
     
    @@ -466,64 +468,9 @@ public void close() {
                     waitForTransitionFromRebalancingToRunning();
     
                     for (final String log : appender.getMessages()) {
    -                    // after we replace the thread there should be two remaining threads with 5 bytes each
    -                    if (log.endsWith("Adding StreamThread-3, there are now 3 threads with cache size/max buffer size values as 3/178956970 per thread.")) {
    -                        return;
    -                    }
    -                }
    -            }
    -        }
    -        fail();
    -    }
    -
    -    @Test
    -    public void shouldResizeMaxBufferAfterThreadReplacement() throws InterruptedException {
    -        final long totalCacheBytes = 10L;
    -        final Properties props = new Properties();
    -        props.putAll(properties);
    -        props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2);
    -        props.put(StreamsConfig.INPUT_BUFFER_MAX_BYTES_CONFIG, totalCacheBytes);
    -
    -        final AtomicBoolean injectError = new AtomicBoolean(false);
    -
    -        final StreamsBuilder builder  = new StreamsBuilder();
    -        final KStream stream = builder.stream(inputTopic);
    -        stream.transform(() -> new Transformer>() {
    -            @Override
    -            public void init(final ProcessorContext context) {
    -                context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, timestamp -> {
    -                    if (Thread.currentThread().getName().endsWith("StreamThread-1") && injectError.get()) {
    -                        injectError.set(false);
    -                        throw new RuntimeException("BOOM");
    -                    }
    -                });
    -            }
    -
    -            @Override
    -            public KeyValue transform(final String key, final String value) {
    -                return new KeyValue<>(key, value);
    -            }
    -
    -            @Override
    -            public void close() {
    -            }
    -        });
    -
    -        try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), props)) {
    -            addStreamStateChangeListener(kafkaStreams);
    -            kafkaStreams.setUncaughtExceptionHandler(e -> StreamThreadExceptionResponse.REPLACE_THREAD);
    -            startStreamsAndWaitForRunning(kafkaStreams);
    -
    -            stateTransitionHistory.clear();
    -            try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister()) {
    -                injectError.set(true);
    -                waitForCondition(() -> !injectError.get(), "StreamThread did not hit and reset the injected error");
    -
    -                waitForTransitionFromRebalancingToRunning();
    -
    -                for (final String log : appender.getMessages()) {
    -                    // after we replace the thread there should be two remaining threads with 5 bytes each
    -                    if (log.endsWith("Adding StreamThread-3, there are now 3 threads with cache size/max buffer size values as 3495253/3 per thread.")) {
    +                    // after we replace the thread there should be two remaining threads with 5 bytes each for
    +                    // the cache and 50 for the input buffer
    +                    if (log.endsWith("Adding StreamThread-3, there are now 2 threads with cache size/max buffer size values as 5/50 per thread.")) {
                             return;
                         }
                     }
    diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java
    index d438a44f37b8..49ebf732e961 100644
    --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java
    +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java
    @@ -117,6 +117,7 @@
     import static org.apache.kafka.streams.internals.StreamsConfigUtils.ProcessingMode.AT_LEAST_ONCE;
     import static org.apache.kafka.streams.internals.StreamsConfigUtils.ProcessingMode.EXACTLY_ONCE_ALPHA;
     import static org.apache.kafka.streams.internals.StreamsConfigUtils.ProcessingMode.EXACTLY_ONCE_V2;
    +import static org.apache.kafka.streams.internals.StreamsConfigUtils.getTotalCacheSize;
     import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
     
     /**
    @@ -330,7 +331,7 @@ private TopologyTestDriver(final InternalTopologyBuilder builder,
     
             final ThreadCache cache = new ThreadCache(
                 logContext,
    -            Math.max(0, streamsConfig.getTotalCacheSize()),
    +            Math.max(0, getTotalCacheSize(streamsConfig)),
                 streamsMetrics
             );
     
    
    From 366b998a229f26aa4601e6b114c2198de0697562 Mon Sep 17 00:00:00 2001
    From: yun-yun <704863678@qq.com>
    Date: Thu, 31 Mar 2022 09:45:33 +0800
    Subject: [PATCH 070/447] KAFKA-13777: Fix potential FetchResponse#responseData
     race condition issue (#11963)
    
    In Fix FetchResponse#responseData, we did a double-checked lock for the responseData, but the assignment of lazy-initialized object(responseData) didn't assign in the last step, which would let other threads get the partial object.
    
    Reviewers: David Jacot , Luke Chen 
    ---
     .../org/apache/kafka/common/requests/FetchResponse.java   | 8 ++++++--
     1 file changed, 6 insertions(+), 2 deletions(-)
    
    diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
    index 2e0a02ec1685..0d7049d75514 100644
    --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
    +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
    @@ -100,7 +100,10 @@ public LinkedHashMap responseDa
             if (responseData == null) {
                 synchronized (this) {
                     if (responseData == null) {
    -                    responseData = new LinkedHashMap<>();
    +                    // Assigning the lazy-initialized responseData in the last step
    +                    // to avoid other threads accessing a half-initialized object.
    +                    final LinkedHashMap responseDataTmp =
    +                            new LinkedHashMap<>();
                         data.responses().forEach(topicResponse -> {
                             String name;
                             if (version < 13) {
    @@ -110,9 +113,10 @@ public LinkedHashMap responseDa
                             }
                             if (name != null) {
                                 topicResponse.partitions().forEach(partition ->
    -                                responseData.put(new TopicPartition(name, partition.partitionIndex()), partition));
    +                                responseDataTmp.put(new TopicPartition(name, partition.partitionIndex()), partition));
                             }
                         });
    +                    responseData = responseDataTmp;
                     }
                 }
             }
    
    From 8965240da3ba704e1606f1a00024f365aa0b3d2a Mon Sep 17 00:00:00 2001
    From: Xiaobing Fang 
    Date: Thu, 31 Mar 2022 09:54:42 +0800
    Subject: [PATCH 071/447] MINOR: Fix doc variable typos in `TopicConfig`
     (#11972)
    
    Reviewers: Luke Chen 
    ---
     .../main/java/org/apache/kafka/common/config/TopicConfig.java   | 2 +-
     core/src/main/scala/kafka/log/LogConfig.scala                   | 2 +-
     2 files changed, 2 insertions(+), 2 deletions(-)
    
    diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
    index 73439c5f29af..dc80b7c139f7 100755
    --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
    +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
    @@ -99,7 +99,7 @@ public class TopicConfig {
             "limit only applies to a single record in that case.";
     
         public static final String INDEX_INTERVAL_BYTES_CONFIG = "index.interval.bytes";
    -    public static final String INDEX_INTERVAL_BYTES_DOCS = "This setting controls how frequently " +
    +    public static final String INDEX_INTERVAL_BYTES_DOC = "This setting controls how frequently " +
             "Kafka adds an index entry to its offset index. The default setting ensures that we index a " +
             "message roughly every 4096 bytes. More indexing allows reads to jump closer to the exact " +
             "position in the log but makes the index larger. You probably don't need to change this.";
    diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala
    index 845f80a3a133..7ad85799c55d 100755
    --- a/core/src/main/scala/kafka/log/LogConfig.scala
    +++ b/core/src/main/scala/kafka/log/LogConfig.scala
    @@ -234,7 +234,7 @@ object LogConfig {
       val LocalLogRetentionMsDoc = TopicConfig.LOCAL_LOG_RETENTION_MS_DOC
       val LocalLogRetentionBytesDoc = TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC
       val MaxMessageSizeDoc = TopicConfig.MAX_MESSAGE_BYTES_DOC
    -  val IndexIntervalDoc = TopicConfig.INDEX_INTERVAL_BYTES_DOCS
    +  val IndexIntervalDoc = TopicConfig.INDEX_INTERVAL_BYTES_DOC
       val FileDeleteDelayMsDoc = TopicConfig.FILE_DELETE_DELAY_MS_DOC
       val DeleteRetentionMsDoc = TopicConfig.DELETE_RETENTION_MS_DOC
       val MinCompactionLagMsDoc = TopicConfig.MIN_COMPACTION_LAG_MS_DOC
    
    From 669a49063db814b7d7181f0798bd9822bdf5a3d4 Mon Sep 17 00:00:00 2001
    From: dengziming 
    Date: Thu, 31 Mar 2022 10:34:39 +0800
    Subject: [PATCH 072/447] MINOR: Fix an uncompatible bug in GetOffsetShell
     (#11936)
    
    In KIP-815 we replaced KafkaConsumer with AdminClient in GetOffsetShell. In the previous implementation, partitions were just ignored if there is no offset for them, however, we will print -1 instead now, This PR fix this inconsistency.
    
    Reviewers: David Jacot , Luke Chen 
    ---
     core/src/main/scala/kafka/tools/GetOffsetShell.scala     | 8 ++++++--
     core/src/test/scala/kafka/tools/GetOffsetShellTest.scala | 7 +++++++
     2 files changed, 13 insertions(+), 2 deletions(-)
    
    diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala
    index d4e81bcaf5c4..03f9c819260b 100644
    --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala
    +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala
    @@ -22,7 +22,7 @@ import joptsimple._
     import kafka.utils.{CommandLineUtils, Exit, IncludeList, ToolsUtils}
     import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, ListTopicsOptions, OffsetSpec}
     import org.apache.kafka.common.{KafkaException, TopicPartition}
    -import org.apache.kafka.common.requests.ListOffsetsRequest
    +import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse}
     import org.apache.kafka.common.utils.Utils
     
     import java.util.Properties
    @@ -135,7 +135,11 @@ object GetOffsetShell {
           val partitionOffsets = partitionInfos.flatMap { tp =>
             try {
               val partitionInfo = listOffsetsResult.partitionResult(tp).get
    -          Some((tp, partitionInfo.offset))
    +          if (partitionInfo.offset != ListOffsetsResponse.UNKNOWN_OFFSET) {
    +            Some((tp, partitionInfo.offset))
    +          } else {
    +            None
    +          }
             } catch {
               case e: ExecutionException =>
                 e.getCause match {
    diff --git a/core/src/test/scala/kafka/tools/GetOffsetShellTest.scala b/core/src/test/scala/kafka/tools/GetOffsetShellTest.scala
    index 021642663793..cbce57319274 100644
    --- a/core/src/test/scala/kafka/tools/GetOffsetShellTest.scala
    +++ b/core/src/test/scala/kafka/tools/GetOffsetShellTest.scala
    @@ -166,6 +166,13 @@ class GetOffsetShellTest extends KafkaServerTestHarness with Logging {
         )
       }
     
    +  @Test
    +  def testNoOffsetIfTimestampGreaterThanLatestRecord(): Unit = {
    +    val time = (System.currentTimeMillis() * 2).toString
    +    val offsets = executeAndParse(Array("--topic-partitions", "topic.*", "--time", time))
    +    assertEquals(List.empty, offsets)
    +  }
    +
       @Test
       def testTopicPartitionsArgWithInternalExcluded(): Unit = {
         val offsets = executeAndParse(Array("--topic-partitions",
    
    From ce7788aadaf989214c70e4df7ec286ed080e8cd1 Mon Sep 17 00:00:00 2001
    From: David Jacot 
    Date: Thu, 31 Mar 2022 14:31:31 +0200
    Subject: [PATCH 073/447] KAFKA-13783; Remove reason prefixing in
     JoinGroupRequest and LeaveGroupRequest (#11971)
    
    KIP-800 introduced a mechanism to pass a reason in the join group request and in the leave group request. A default reason is used unless one is provided by the user. In this case, the custom reason is prefixed by the default one.
    
    When we tried to used this in Kafka Streams, we noted a significant degradation of the performances, see https://github.com/apache/kafka/pull/11873. It is not clear wether the prefixing is the root cause of the issue or not. To be on the safe side, I think that we should remove the prefixing. It does not bring much anyway as we are still able to distinguish a custom reason from the default one on the broker side.
    
    This patch removes prefixing the user provided reasons. So if a the user provides a reason, the reason is used directly. If the reason is empty or null, the default reason is used.
    
    Reviewers: Luke Chen , , Hao Li 
    ---
     .../kafka/clients/admin/KafkaAdminClient.java | 26 +++++---
     .../kafka/clients/admin/MemberToRemove.java   |  4 +-
     .../kafka/clients/consumer/KafkaConsumer.java |  4 +-
     .../clients/admin/KafkaAdminClientTest.java   | 31 ++++++----
     .../clients/consumer/KafkaConsumerTest.java   | 61 +++++++++++++++++++
     5 files changed, 101 insertions(+), 25 deletions(-)
    
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
    index 03322fdcf1dc..cf99556b0f2c 100644
    --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
    @@ -306,9 +306,9 @@ public class KafkaAdminClient extends AdminClient {
         private static final long INVALID_SHUTDOWN_TIME = -1;
     
         /**
    -     * The base reason for a LeaveGroupRequest
    +     * The default reason for a LeaveGroupRequest.
          */
    -    static final String LEAVE_GROUP_REASON = "member was removed by an admin";
    +    static final String DEFAULT_LEAVE_GROUP_REASON = "member was removed by an admin";
     
         /**
          * Thread name prefix for admin client network thread
    @@ -3713,7 +3713,7 @@ private Integer nodeFor(ConfigResource resource) {
             }
         }
     
    -    private List getMembersFromGroup(String groupId) {
    +    private List getMembersFromGroup(String groupId, String reason) {
             Collection members;
             try {
                 members = describeConsumerGroups(Collections.singleton(groupId)).describedGroups().get(groupId).get().members();
    @@ -3723,11 +3723,15 @@ private List getMembersFromGroup(String groupId) {
     
             List membersToRemove = new ArrayList<>();
             for (final MemberDescription member : members) {
    +            MemberIdentity memberIdentity = new MemberIdentity().setReason(reason);
    +
                 if (member.groupInstanceId().isPresent()) {
    -                membersToRemove.add(new MemberIdentity().setGroupInstanceId(member.groupInstanceId().get()));
    +                memberIdentity.setGroupInstanceId(member.groupInstanceId().get());
                 } else {
    -                membersToRemove.add(new MemberIdentity().setMemberId(member.consumerId()));
    +                memberIdentity.setMemberId(member.consumerId());
                 }
    +
    +            membersToRemove.add(memberIdentity);
             }
             return membersToRemove;
         }
    @@ -3735,15 +3739,17 @@ private List getMembersFromGroup(String groupId) {
         @Override
         public RemoveMembersFromConsumerGroupResult removeMembersFromConsumerGroup(String groupId,
                                                                                    RemoveMembersFromConsumerGroupOptions options) {
    +        String reason = options.reason() == null || options.reason().isEmpty() ?
    +            DEFAULT_LEAVE_GROUP_REASON : options.reason();
    +
             List members;
             if (options.removeAll()) {
    -            members = getMembersFromGroup(groupId);
    +            members = getMembersFromGroup(groupId, reason);
             } else {
    -            members = options.members().stream().map(MemberToRemove::toMemberIdentity).collect(Collectors.toList());
    +            members = options.members().stream()
    +                .map(m -> m.toMemberIdentity().setReason(reason))
    +                .collect(Collectors.toList());
             }
    -        
    -        String reason = options.reason() == null ? LEAVE_GROUP_REASON : LEAVE_GROUP_REASON + ": " + options.reason();
    -        members.forEach(member -> member.setReason(reason));
     
             SimpleAdminApiFuture> future =
                     RemoveMembersFromConsumerGroupHandler.newFuture(groupId);
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/MemberToRemove.java b/clients/src/main/java/org/apache/kafka/clients/admin/MemberToRemove.java
    index 4c7b16b1da65..5ca5463d3f28 100644
    --- a/clients/src/main/java/org/apache/kafka/clients/admin/MemberToRemove.java
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/MemberToRemove.java
    @@ -48,8 +48,8 @@ public int hashCode() {
     
         MemberIdentity toMemberIdentity() {
             return new MemberIdentity()
    -                   .setGroupInstanceId(groupInstanceId)
    -                   .setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID);
    +            .setGroupInstanceId(groupInstanceId)
    +            .setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID);
         }
     
         public String groupInstanceId() {
    diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
    index 0fd4ea9cf421..d0ef1a0cbebc 100644
    --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
    +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
    @@ -563,7 +563,7 @@ public class KafkaConsumer implements Consumer {
         private static final long NO_CURRENT_THREAD = -1L;
         private static final String JMX_PREFIX = "kafka.consumer";
         static final long DEFAULT_CLOSE_TIMEOUT_MS = 30 * 1000;
    -    private static final String DEFAULT_REASON = "rebalance enforced by user";
    +    static final String DEFAULT_REASON = "rebalance enforced by user";
     
         // Visible for testing
         final Metrics metrics;
    @@ -2322,7 +2322,7 @@ public void enforceRebalance(final String reason) {
                 if (coordinator == null) {
                     throw new IllegalStateException("Tried to force a rebalance but consumer does not have a group.");
                 }
    -            coordinator.requestRejoin(reason == null ? DEFAULT_REASON : DEFAULT_REASON + ": " + reason);
    +            coordinator.requestRejoin(reason == null || reason.isEmpty() ? DEFAULT_REASON : reason);
             } finally {
                 release();
             }
    diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
    index 3d7bb94fe706..c337831b0722 100644
    --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
    +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
    @@ -236,7 +236,7 @@
     import static java.util.Collections.emptySet;
     import static java.util.Collections.singleton;
     import static java.util.Collections.singletonList;
    -import static org.apache.kafka.clients.admin.KafkaAdminClient.LEAVE_GROUP_REASON;
    +import static org.apache.kafka.clients.admin.KafkaAdminClient.DEFAULT_LEAVE_GROUP_REASON;
     import static org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignablePartitionResponse;
     import static org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignableTopicResponse;
     import static org.apache.kafka.common.message.ListPartitionReassignmentsResponseData.OngoingPartitionReassignment;
    @@ -3947,35 +3947,44 @@ private void testRemoveMembersFromGroup(String reason, String expectedReason) th
                     LeaveGroupRequestData leaveGroupRequest = ((LeaveGroupRequest) body).data();
     
                     return leaveGroupRequest.members().stream().allMatch(
    -                        member -> member.reason().equals(expectedReason)
    +                    member -> member.reason().equals(expectedReason)
                     );
                 }, new LeaveGroupResponse(new LeaveGroupResponseData().setErrorCode(Errors.NONE.code()).setMembers(
    -                    Arrays.asList(
    -                            new MemberResponse().setGroupInstanceId("instance-1"),
    -                            new MemberResponse().setGroupInstanceId("instance-2")
    -                    ))
    +                Arrays.asList(
    +                    new MemberResponse().setGroupInstanceId("instance-1"),
    +                    new MemberResponse().setGroupInstanceId("instance-2")
    +                ))
                 ));
     
    -            Collection membersToRemove = Arrays.asList(new MemberToRemove("instance-1"), new MemberToRemove("instance-2"));
    +            MemberToRemove memberToRemove1 = new MemberToRemove("instance-1");
    +            MemberToRemove memberToRemove2 = new MemberToRemove("instance-2");
     
    -            RemoveMembersFromConsumerGroupOptions options = new RemoveMembersFromConsumerGroupOptions(membersToRemove);
    +            RemoveMembersFromConsumerGroupOptions options = new RemoveMembersFromConsumerGroupOptions(Arrays.asList(
    +                memberToRemove1,
    +                memberToRemove2
    +            ));
                 options.reason(reason);
     
                 final RemoveMembersFromConsumerGroupResult result = env.adminClient().removeMembersFromConsumerGroup(
    -                    GROUP_ID, options);
    +                GROUP_ID,
    +                options
    +            );
     
                 assertNull(result.all().get());
    +            assertNull(result.memberResult(memberToRemove1).get());
    +            assertNull(result.memberResult(memberToRemove2).get());
             }
         }
     
         @Test
         public void testRemoveMembersFromGroupReason() throws Exception {
    -        testRemoveMembersFromGroup("testing remove members reason", LEAVE_GROUP_REASON + ": testing remove members reason");
    +        testRemoveMembersFromGroup("testing remove members reason", "testing remove members reason");
         }
     
         @Test
         public void testRemoveMembersFromGroupDefaultReason() throws Exception {
    -        testRemoveMembersFromGroup(null, LEAVE_GROUP_REASON);
    +        testRemoveMembersFromGroup(null, DEFAULT_LEAVE_GROUP_REASON);
    +        testRemoveMembersFromGroup("", DEFAULT_LEAVE_GROUP_REASON);
         }
     
         @Test
    diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
    index 27c108bcdaca..ae54efc2a12e 100644
    --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
    +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
    @@ -138,6 +138,7 @@
     import static java.util.Collections.singleton;
     import static java.util.Collections.singletonList;
     import static java.util.Collections.singletonMap;
    +import static org.apache.kafka.clients.consumer.KafkaConsumer.DEFAULT_REASON;
     import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID;
     import static org.junit.jupiter.api.Assertions.assertEquals;
     import static org.junit.jupiter.api.Assertions.assertFalse;
    @@ -2819,6 +2820,66 @@ public void testEnforceRebalanceTriggersRebalanceOnNextPoll() {
             assertEquals(countingRebalanceListener.revokedCount, 1);
         }
     
    +    @Test
    +    public void testEnforceRebalanceReason() {
    +        Time time = new MockTime(1L);
    +
    +        ConsumerMetadata metadata = createMetadata(subscription);
    +        MockClient client = new MockClient(time, metadata);
    +        initMetadata(client, Utils.mkMap(Utils.mkEntry(topic, 1)));
    +        Node node = metadata.fetch().nodes().get(0);
    +
    +        KafkaConsumer consumer = newConsumer(
    +            time,
    +            client,
    +            subscription,
    +            metadata,
    +            assignor,
    +            true,
    +            groupInstanceId
    +        );
    +        consumer.subscribe(Collections.singletonList(topic));
    +
    +        // Lookup coordinator.
    +        client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node);
    +        consumer.poll(Duration.ZERO);
    +
    +        // Initial join sends an empty reason.
    +        prepareJoinGroupAndVerifyReason(client, node, "");
    +        consumer.poll(Duration.ZERO);
    +
    +        // A null reason should be replaced by the default reason.
    +        consumer.enforceRebalance(null);
    +        prepareJoinGroupAndVerifyReason(client, node, DEFAULT_REASON);
    +        consumer.poll(Duration.ZERO);
    +
    +        // An empty reason should be replaced by the default reason.
    +        consumer.enforceRebalance("");
    +        prepareJoinGroupAndVerifyReason(client, node, DEFAULT_REASON);
    +        consumer.poll(Duration.ZERO);
    +
    +        // A non-null and non-empty reason is sent as-is.
    +        String customReason = "user provided reason";
    +        consumer.enforceRebalance(customReason);
    +        prepareJoinGroupAndVerifyReason(client, node, customReason);
    +        consumer.poll(Duration.ZERO);
    +    }
    +
    +    private void prepareJoinGroupAndVerifyReason(
    +        MockClient client,
    +        Node node,
    +        String expectedReason
    +    ) {
    +        client.prepareResponseFrom(
    +            body -> {
    +                JoinGroupRequest joinGroupRequest = (JoinGroupRequest) body;
    +                return expectedReason.equals(joinGroupRequest.data().reason());
    +            },
    +            joinGroupFollowerResponse(assignor, 1, memberId, leaderId, Errors.NONE),
    +            node
    +        );
    +    }
    +
         @Test
         public void configurableObjectsShouldSeeGeneratedClientId() {
             Properties props = new Properties();
    
    From 430f9c99012d1585aa544d4dadf449963296c1fd Mon Sep 17 00:00:00 2001
    From: Yu 
    Date: Thu, 31 Mar 2022 20:45:59 +0800
    Subject: [PATCH 074/447] KAFKA-13772: Partitions are not correctly
     re-partitioned when the fetcher thread pool is resized (#11953)
    
    Partitions are assigned to fetcher threads based on their hash modulo the number of fetcher threads. When we resize the fetcher thread pool, we basically re-distribute all the partitions based on the new fetcher thread pool size. The issue is that the logic that resizes the fetcher thread pool updates the `fetcherThreadMap` while iterating over it. The `Map` does not give any guarantee in this case - especially when the underlying map is re-hashed - and that led to not iterating over all the fetcher threads during the process and thus in leaving some partitions in the wrong fetcher threads.
    
    Reviewers: Luke Chen , David Jacot 
    ---
     .../kafka/server/AbstractFetcherManager.scala |  23 ++--
     .../kafka/server/AbstractFetcherThread.scala  |  12 ++
     .../server/AbstractFetcherManagerTest.scala   | 119 +++++++++++++++++-
     3 files changed, 145 insertions(+), 9 deletions(-)
    
    diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala
    index 778053548e65..0843fe8164d8 100755
    --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala
    +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala
    @@ -62,19 +62,22 @@ abstract class AbstractFetcherManager[T <: AbstractFetcherThread](val name: Stri
     
       def resizeThreadPool(newSize: Int): Unit = {
         def migratePartitions(newSize: Int): Unit = {
    +      val allRemovedPartitionsMap = mutable.Map[TopicPartition, InitialFetchState]()
           fetcherThreadMap.forKeyValue { (id, thread) =>
    -        val partitionStates = removeFetcherForPartitions(thread.partitions)
    +        val partitionStates = thread.removeAllPartitions()
             if (id.fetcherId >= newSize)
               thread.shutdown()
    -        val fetchStates = partitionStates.map { case (topicPartition, currentFetchState) =>
    -          val initialFetchState = InitialFetchState(currentFetchState.topicId, thread.sourceBroker,
    -            currentLeaderEpoch = currentFetchState.currentLeaderEpoch,
    -            initOffset = currentFetchState.fetchOffset)
    -          topicPartition -> initialFetchState
    +        partitionStates.forKeyValue { (topicPartition, currentFetchState) =>
    +            val initialFetchState = InitialFetchState(currentFetchState.topicId, thread.sourceBroker,
    +              currentLeaderEpoch = currentFetchState.currentLeaderEpoch,
    +              initOffset = currentFetchState.fetchOffset)
    +            allRemovedPartitionsMap += topicPartition -> initialFetchState
             }
    -        addFetcherForPartitions(fetchStates)
           }
    +      // failed partitions are removed when adding partitions to fetcher
    +      addFetcherForPartitions(allRemovedPartitionsMap)
         }
    +
         lock synchronized {
           val currentSize = numFetchersPerBroker
           info(s"Resizing fetcher thread pool size from $currentSize to $newSize")
    @@ -145,7 +148,7 @@ abstract class AbstractFetcherManager[T <: AbstractFetcherThread](val name: Stri
               case None =>
                 addAndStartFetcherThread(brokerAndFetcherId, brokerIdAndFetcherId)
             }
    -
    +        // failed partitions are removed when added partitions to thread
             addPartitionsToFetcherThread(fetcherThread, initialFetchOffsets)
           }
         }
    @@ -251,6 +254,10 @@ class FailedPartitions {
       def contains(topicPartition: TopicPartition): Boolean = synchronized {
         failedPartitionsSet.contains(topicPartition)
       }
    +
    +  def partitions(): Set[TopicPartition] = synchronized {
    +    failedPartitionsSet.toSet
    +  }
     }
     
     case class BrokerAndFetcherId(broker: BrokerEndPoint, fetcherId: Int)
    diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
    index 492cec425e34..507d6bb9597a 100755
    --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
    +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
    @@ -743,6 +743,18 @@ abstract class AbstractFetcherThread(name: String,
         } finally partitionMapLock.unlock()
       }
     
    +  def removeAllPartitions(): Map[TopicPartition, PartitionFetchState] = {
    +    partitionMapLock.lockInterruptibly()
    +    try {
    +      val allPartitionState = partitionStates.partitionStateMap.asScala.toMap
    +      allPartitionState.keys.foreach { tp =>
    +        partitionStates.remove(tp)
    +        fetcherLagStats.unregister(tp)
    +      }
    +      allPartitionState
    +    } finally partitionMapLock.unlock()
    +  }
    +
       def partitionCount: Int = {
         partitionMapLock.lockInterruptibly()
         try partitionStates.size
    diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala
    index 3c02613e8501..5a7a07730f8c 100644
    --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala
    +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala
    @@ -18,13 +18,20 @@ package kafka.server
     
     import com.yammer.metrics.core.Gauge
     import kafka.cluster.BrokerEndPoint
    +import kafka.log.LogAppendInfo
    +import kafka.server.AbstractFetcherThread.{ReplicaFetch, ResultWithPartitions}
    +import kafka.utils.Implicits.MapExtensionMethods
     import kafka.utils.TestUtils
    +import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
    +import org.apache.kafka.common.requests.FetchRequest
    +import org.apache.kafka.common.utils.Utils
     import org.apache.kafka.common.{TopicPartition, Uuid}
     import org.apache.kafka.server.metrics.KafkaYammerMetrics
    -import org.junit.jupiter.api.{BeforeEach, Test}
     import org.junit.jupiter.api.Assertions._
    +import org.junit.jupiter.api.{BeforeEach, Test}
     import org.mockito.Mockito.{mock, verify, when}
     
    +import scala.collection.{Map, Set, mutable}
     import scala.jdk.CollectionConverters._
     
     class AbstractFetcherManagerTest {
    @@ -100,6 +107,7 @@ class AbstractFetcherManagerTest {
         fetcherManager.removeFetcherForPartitions(Set(tp))
         assertEquals(0, getMetricValue(metricName))
       }
    +
       @Test
       def testDeadThreadCountMetric(): Unit = {
         val fetcher: AbstractFetcherThread = mock(classOf[AbstractFetcherThread])
    @@ -210,4 +218,113 @@ class AbstractFetcherManagerTest {
         verify(fetcher).maybeUpdateTopicIds(Set(tp1), topicIds)
         verify(fetcher).maybeUpdateTopicIds(Set(tp2), topicIds)
       }
    +
    +  @Test
    +  def testExpandThreadPool(): Unit = {
    +    testResizeThreadPool(10, 50)
    +  }
    +
    +  @Test
    +  def testShrinkThreadPool(): Unit = {
    +    testResizeThreadPool(50, 10)
    +  }
    +
    +  private def testResizeThreadPool(currentFetcherSize: Int, newFetcherSize: Int, brokerNum: Int = 6): Unit = {
    +    val fetchingTopicPartitions = makeTopicPartition(10, 100)
    +    val failedTopicPartitions = makeTopicPartition(2, 5, "topic_failed")
    +    val fetcherManager = new AbstractFetcherManager[AbstractFetcherThread]("fetcher-manager", "fetcher-manager", currentFetcherSize) {
    +      override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = {
    +        new TestResizeFetcherThread(sourceBroker, failedPartitions)
    +      }
    +    }
    +    try {
    +      fetcherManager.addFetcherForPartitions(fetchingTopicPartitions.map { tp =>
    +        val brokerId = getBrokerId(tp, brokerNum)
    +        val brokerEndPoint = new BrokerEndPoint(brokerId, s"kafka-host-$brokerId", 9092)
    +        tp -> InitialFetchState(None, brokerEndPoint, 0, 0)
    +      }.toMap)
    +
    +      // Mark some of these partitions failed within resizing scope
    +      fetchingTopicPartitions.take(20).foreach(fetcherManager.addFailedPartition)
    +      // Mark failed partitions out of resizing scope
    +      failedTopicPartitions.foreach(fetcherManager.addFailedPartition)
    +
    +      fetcherManager.resizeThreadPool(newFetcherSize)
    +
    +      val ownedPartitions = mutable.Set.empty[TopicPartition]
    +      fetcherManager.fetcherThreadMap.forKeyValue { (brokerIdAndFetcherId, fetcherThread) =>
    +        val fetcherId = brokerIdAndFetcherId.fetcherId
    +        val brokerId = brokerIdAndFetcherId.brokerId
    +
    +        fetcherThread.partitions.foreach { tp =>
    +          ownedPartitions += tp
    +          assertEquals(fetcherManager.getFetcherId(tp), fetcherId)
    +          assertEquals(getBrokerId(tp, brokerNum), brokerId)
    +        }
    +      }
    +      // Verify that all partitions are owned by the fetcher threads.
    +      assertEquals(fetchingTopicPartitions, ownedPartitions)
    +
    +      // Only failed partitions should still be kept after resizing
    +      assertEquals(failedTopicPartitions, fetcherManager.failedPartitions.partitions())
    +    } finally {
    +      fetcherManager.closeAllFetchers()
    +    }
    +  }
    +
    +
    +  private def makeTopicPartition(topicNum: Int, partitionNum: Int, topicPrefix: String = "topic_"): Set[TopicPartition] = {
    +    val res = mutable.Set[TopicPartition]()
    +    for (i <- 0 to topicNum - 1) {
    +      val topic = topicPrefix + i
    +      for (j <- 0 to partitionNum - 1) {
    +        res += new TopicPartition(topic, j)
    +      }
    +    }
    +    res.toSet
    +  }
    +
    +  private def getBrokerId(tp: TopicPartition, brokerNum: Int): Int = {
    +    Utils.abs(tp.hashCode) % brokerNum
    +  }
    +
    +  private class TestResizeFetcherThread(sourceBroker: BrokerEndPoint, failedPartitions: FailedPartitions)
    +    extends AbstractFetcherThread(
    +      name = "test-resize-fetcher",
    +      clientId = "mock-fetcher",
    +      sourceBroker,
    +      failedPartitions,
    +      fetchBackOffMs = 0,
    +      brokerTopicStats = new BrokerTopicStats) {
    +
    +    override protected def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, partitionData: FetchData): Option[LogAppendInfo] = {
    +      None
    +    }
    +
    +    override protected def truncate(topicPartition: TopicPartition, truncationState: OffsetTruncationState): Unit = {}
    +
    +    override protected def truncateFullyAndStartAt(topicPartition: TopicPartition, offset: Long): Unit = {}
    +
    +    override protected def buildFetch(partitionMap: Map[TopicPartition, PartitionFetchState]): ResultWithPartitions[Option[ReplicaFetch]] = ResultWithPartitions(None, Set.empty)
    +
    +    override protected def latestEpoch(topicPartition: TopicPartition): Option[Int] = Some(0)
    +
    +    override protected def logStartOffset(topicPartition: TopicPartition): Long = 1
    +
    +    override protected def logEndOffset(topicPartition: TopicPartition): Long = 1
    +
    +    override protected def endOffsetForEpoch(topicPartition: TopicPartition, epoch: Int): Option[OffsetAndEpoch] = Some(OffsetAndEpoch(1, 0))
    +
    +    override protected def fetchEpochEndOffsets(partitions: Map[TopicPartition, EpochData]): Map[TopicPartition, EpochEndOffset] = Map.empty
    +
    +    override protected def fetchFromLeader(fetchRequest: FetchRequest.Builder): Map[TopicPartition, FetchData] = Map.empty
    +
    +    override protected def fetchEarliestOffsetFromLeader(topicPartition: TopicPartition, currentLeaderEpoch: Int): Long = 1
    +
    +    override protected def fetchLatestOffsetFromLeader(topicPartition: TopicPartition, currentLeaderEpoch: Int): Long = 1
    +
    +    override protected val isOffsetForLeaderEpochSupported: Boolean = false
    +    override protected val isTruncationOnFetchSupported: Boolean = false
    +  }
    +
     }
    
    From 3c279b63fa862671330b568f5f58df69f9352c35 Mon Sep 17 00:00:00 2001
    From: Bounkong Khamphousone 
    Date: Thu, 31 Mar 2022 16:05:53 +0200
    Subject: [PATCH 075/447] fix: make sliding window works without grace period
     (#kafka-13739) (#11928)
    
    Fix upperbound for sliding window, making it compatible with no grace period (kafka-13739)
    
    Added unit test for early sliding window and "normal" sliding window for both events within one time difference (small input) and above window time difference (large input).
    
    Fixing this window interval may slightly change stream behavior but probability to happen is extremely slow and may not have a huge impact on the result given.
    
    Reviewers Leah Thomas , Bill Bejeck 
    ---
     .../KStreamSlidingWindowAggregate.java        |   2 +-
     .../KStreamSlidingWindowAggregateTest.java    | 304 ++++++++++++++++++
     2 files changed, 305 insertions(+), 1 deletion(-)
    
    diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
    index fd0198b71c73..73c901cfaf6d 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
    @@ -473,7 +473,7 @@ private void updateWindowAndForward(final Window window,
                                                 final long closeTime) {
                 final long windowStart = window.start();
                 final long windowEnd = window.end();
    -            if (windowEnd > closeTime) {
    +            if (windowEnd >= closeTime) {
                     //get aggregate from existing window
                     final VAgg oldAgg = getValueOrNull(valueAndTime);
                     final VAgg newAgg = aggregator.apply(record.key(), record.value(), oldAgg);
    diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
    index 8e8115f35e47..b227c7183381 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
    @@ -698,6 +698,310 @@ public void testEarlyRecordsLargeInput() {
             );
         }
     
    +    @Test
    +    public void testEarlyNoGracePeriodSmallInput() {
    +        final StreamsBuilder builder = new StreamsBuilder();
    +        final String topic = "topic";
    +
    +        final KTable, String> table2 = builder
    +            .stream(topic, Consumed.with(Serdes.String(), Serdes.String()))
    +            .groupByKey(Grouped.with(Serdes.String(), Serdes.String()))
    +            .windowedBy(SlidingWindows.ofTimeDifferenceWithNoGrace(ofMillis(50)))
    +            .aggregate(
    +                MockInitializer.STRING_INIT,
    +                MockAggregator.TOSTRING_ADDER,
    +                Materialized.>as("topic-Canonized").withValueSerde(Serdes.String())
    +            );
    +        final MockApiProcessorSupplier, String, Void, Void> supplier = new MockApiProcessorSupplier<>();
    +        table2.toStream().process(supplier);
    +
    +        // all events are considered as early events since record timestamp is less than time difference of the window
    +        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
    +            final TestInputTopic inputTopic =
    +                driver.createInputTopic(topic, new StringSerializer(), new StringSerializer());
    +
    +            inputTopic.pipeInput("A", "1", 0L);
    +            inputTopic.pipeInput("A", "2", 5L);
    +            inputTopic.pipeInput("A", "3", 6L);
    +            inputTopic.pipeInput("A", "4", 3L);
    +            inputTopic.pipeInput("A", "5", 13L);
    +            inputTopic.pipeInput("A", "6", 10L);
    +        }
    +
    +        final Map> actual = new HashMap<>();
    +        for (final KeyValueTimestamp, String> entry : supplier.theCapturedProcessor().processed()) {
    +            final Windowed window = entry.key();
    +            final Long start = window.window().start();
    +            final ValueAndTimestamp valueAndTimestamp = ValueAndTimestamp.make(entry.value(), entry.timestamp());
    +            if (actual.putIfAbsent(start, valueAndTimestamp) != null) {
    +                actual.replace(start, valueAndTimestamp);
    +            }
    +        }
    +
    +        final Map> expected = new HashMap<>();
    +        expected.put(0L, ValueAndTimestamp.make("0+1+2+3+4+5+6", 13L));
    +        expected.put(1L, ValueAndTimestamp.make("0+2+3+4+5+6", 13L));
    +        expected.put(4L, ValueAndTimestamp.make("0+2+3+5+6", 13L));
    +        expected.put(6L, ValueAndTimestamp.make("0+3+5+6", 13L));
    +        expected.put(7L, ValueAndTimestamp.make("0+5+6", 13L));
    +        expected.put(11L, ValueAndTimestamp.make("0+5", 13L));
    +
    +        assertEquals(expected, actual);
    +    }
    +
    +    @Test
    +    public void testNoGracePeriodSmallInput() {
    +        final StreamsBuilder builder = new StreamsBuilder();
    +        final String topic = "topic";
    +
    +        final KTable, String> table2 = builder
    +                .stream(topic, Consumed.with(Serdes.String(), Serdes.String()))
    +                .groupByKey(Grouped.with(Serdes.String(), Serdes.String()))
    +                .windowedBy(SlidingWindows.ofTimeDifferenceWithNoGrace(ofMillis(50)))
    +                .aggregate(
    +                        MockInitializer.STRING_INIT,
    +                        MockAggregator.TOSTRING_ADDER,
    +                        Materialized.>as("topic-Canonized").withValueSerde(Serdes.String())
    +                );
    +        final MockApiProcessorSupplier, String, Void, Void> supplier = new MockApiProcessorSupplier<>();
    +        table2.toStream().process(supplier);
    +
    +        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
    +            final TestInputTopic inputTopic =
    +                    driver.createInputTopic(topic, new StringSerializer(), new StringSerializer());
    +
    +            inputTopic.pipeInput("A", "1", 100L);
    +            inputTopic.pipeInput("A", "2", 105L);
    +            inputTopic.pipeInput("A", "3", 106L);
    +            inputTopic.pipeInput("A", "4", 103L);
    +            inputTopic.pipeInput("A", "5", 113L);
    +            inputTopic.pipeInput("A", "6", 110L);
    +        }
    +
    +        final Map> actual = new HashMap<>();
    +        for (final KeyValueTimestamp, String> entry : supplier.theCapturedProcessor().processed()) {
    +            final Windowed window = entry.key();
    +            final Long start = window.window().start();
    +            final ValueAndTimestamp valueAndTimestamp = ValueAndTimestamp.make(entry.value(), entry.timestamp());
    +            if (actual.putIfAbsent(start, valueAndTimestamp) != null) {
    +                actual.replace(start, valueAndTimestamp);
    +            }
    +        }
    +
    +        final Map> expected = new HashMap<>();
    +        expected.put(50L, ValueAndTimestamp.make("0+1", 100L));
    +        expected.put(55L, ValueAndTimestamp.make("0+1+2", 105L));
    +        expected.put(56L, ValueAndTimestamp.make("0+1+2+3+4", 106L));
    +        expected.put(63L, ValueAndTimestamp.make("0+1+2+3+4+5+6", 113L));
    +        expected.put(101L, ValueAndTimestamp.make("0+2+3+4+5+6", 113L));
    +        expected.put(104L, ValueAndTimestamp.make("0+2+3+5+6", 113L));
    +        expected.put(106L, ValueAndTimestamp.make("0+3+5+6", 113L));
    +        expected.put(107L, ValueAndTimestamp.make("0+5+6", 113L));
    +        expected.put(111L, ValueAndTimestamp.make("0+5", 113L));
    +
    +        assertEquals(expected, actual);
    +    }
    +
    +    @Test
    +    public void testEarlyNoGracePeriodLargeInput() {
    +        final StreamsBuilder builder = new StreamsBuilder();
    +        final String topic = "topic";
    +        final WindowBytesStoreSupplier storeSupplier =
    +                inOrderIterator
    +                        ? new InOrderMemoryWindowStoreSupplier("InOrder", 500L, 10L, false)
    +                        : Stores.inMemoryWindowStore("Reverse", Duration.ofMillis(500), Duration.ofMillis(10), false);
    +
    +        final KTable, String> table2 = builder
    +                .stream(topic, Consumed.with(Serdes.String(), Serdes.String()))
    +                .groupByKey(Grouped.with(Serdes.String(), Serdes.String()))
    +                .windowedBy(SlidingWindows.ofTimeDifferenceWithNoGrace(ofMillis(10)))
    +                .aggregate(
    +                        MockInitializer.STRING_INIT,
    +                        MockAggregator.TOSTRING_ADDER,
    +                        Materialized.as(storeSupplier)
    +                );
    +
    +        final MockApiProcessorSupplier, String, Void, Void> supplier = new MockApiProcessorSupplier<>();
    +        table2.toStream().process(supplier);
    +
    +        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
    +            final TestInputTopic inputTopic1 =
    +                    driver.createInputTopic(topic, new StringSerializer(), new StringSerializer());
    +
    +            inputTopic1.pipeInput("E", "1", 0L);
    +            inputTopic1.pipeInput("E", "3", 5L);
    +            inputTopic1.pipeInput("E", "4", 6L);
    +            inputTopic1.pipeInput("E", "2", 3L);
    +            inputTopic1.pipeInput("E", "6", 13L);
    +            inputTopic1.pipeInput("E", "5", 10L);
    +            inputTopic1.pipeInput("E", "7", 4L);
    +            inputTopic1.pipeInput("E", "8", 2L);
    +            inputTopic1.pipeInput("E", "9", 15L);
    +        }
    +        final Comparator, String>> comparator =
    +                Comparator.comparing((KeyValueTimestamp, String> o) -> o.key().key())
    +                        .thenComparing((KeyValueTimestamp, String> o) -> o.key().window().start());
    +
    +        final ArrayList, String>> actual = supplier.theCapturedProcessor().processed();
    +        actual.sort(comparator);
    +        assertEquals(
    +            asList(
    +                // E@0
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(0, 10)), "0+1", 0),
    +                // E@5
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(0, 10)), "0+1+3", 5),
    +                // E@6
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(0, 10)), "0+1+3+4", 6),
    +                // E@3
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(0, 10)), "0+1+3+4+2", 6),
    +                // E@5
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(1, 11)), "0+3", 5),
    +                // E@6
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(1, 11)), "0+3+4", 6),
    +                // E@3
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(1, 11)), "0+3+4+2", 6),
    +                //E@13
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(3, 13)), "0+3+4+2+6", 13),
    +                //E@10
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(3, 13)), "0+3+4+2+6+5", 13),
    +                //E@4
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(3, 13)), "0+3+4+2+6+5+7", 13),
    +                // E@3
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(4, 14)), "0+3+4", 6),
    +                //E@13
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(4, 14)), "0+3+4+6", 13),
    +                //E@10
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(4, 14)), "0+3+4+6+5", 13),
    +                //E@4
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(4, 14)), "0+3+4+6+5+7", 13),
    +                //E@4
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(5, 15)), "0+3+4+6+5", 13),
    +                //E@15
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(5, 15)), "0+3+4+6+5+9", 15),
    +                // E@6
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(6, 16)), "0+4", 6),
    +                //E@13
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(6, 16)), "0+4+6", 13),
    +                //E@10
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(6, 16)), "0+4+6+5", 13),
    +                //E@15
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(6, 16)), "0+4+6+5+9", 15),
    +                //E@13
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(7, 17)), "0+6", 13),
    +                //E@10
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(7, 17)), "0+6+5", 13),
    +                //E@15
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(7, 17)), "0+6+5+9", 15),
    +                //E@10
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(11, 21)), "0+6", 13),
    +                //E@15
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(11, 21)), "0+6+9", 15),
    +                //E@15
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(14, 24)), "0+9", 15)),
    +            actual
    +        );
    +    }
    +
    +    @Test
    +    public void testNoGracePeriodLargeInput() {
    +        final StreamsBuilder builder = new StreamsBuilder();
    +        final String topic = "topic";
    +        final WindowBytesStoreSupplier storeSupplier =
    +                inOrderIterator
    +                        ? new InOrderMemoryWindowStoreSupplier("InOrder", 500L, 10L, false)
    +                        : Stores.inMemoryWindowStore("Reverse", Duration.ofMillis(500), Duration.ofMillis(10), false);
    +
    +        final KTable, String> table2 = builder
    +                .stream(topic, Consumed.with(Serdes.String(), Serdes.String()))
    +                .groupByKey(Grouped.with(Serdes.String(), Serdes.String()))
    +                .windowedBy(SlidingWindows.ofTimeDifferenceWithNoGrace(ofMillis(10)))
    +                .aggregate(
    +                        MockInitializer.STRING_INIT,
    +                        MockAggregator.TOSTRING_ADDER,
    +                        Materialized.as(storeSupplier)
    +                );
    +
    +        final MockApiProcessorSupplier, String, Void, Void> supplier = new MockApiProcessorSupplier<>();
    +        table2.toStream().process(supplier);
    +
    +        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
    +            final TestInputTopic inputTopic1 =
    +                    driver.createInputTopic(topic, new StringSerializer(), new StringSerializer());
    +
    +            inputTopic1.pipeInput("E", "1", 100L);
    +            inputTopic1.pipeInput("E", "3", 105L);
    +            inputTopic1.pipeInput("E", "4", 106L);
    +            inputTopic1.pipeInput("E", "2", 103L);
    +            inputTopic1.pipeInput("E", "6", 113L);
    +            inputTopic1.pipeInput("E", "5", 110L);
    +            inputTopic1.pipeInput("E", "7", 104L);
    +            inputTopic1.pipeInput("E", "8", 102L);
    +            inputTopic1.pipeInput("E", "9", 115L);
    +        }
    +        final Comparator, String>> comparator =
    +                Comparator.comparing((KeyValueTimestamp, String> o) -> o.key().key())
    +                        .thenComparing((KeyValueTimestamp, String> o) -> o.key().window().start());
    +
    +        final ArrayList, String>> actual = supplier.theCapturedProcessor().processed();
    +        actual.sort(comparator);
    +        assertEquals(
    +            asList(
    +                // E@0
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(90, 100)), "0+1", 100),
    +                // E@5
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(95, 105)), "0+1+3", 105),
    +                // E@6
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(96, 106)), "0+1+3+4", 106),
    +                // E@3
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(96, 106)), "0+1+3+4+2", 106),
    +                // E@5
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(101, 111)), "0+3", 105),
    +                // E@6
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(101, 111)), "0+3+4", 106),
    +                // E@3
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(101, 111)), "0+3+4+2", 106),
    +                //E@13
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(103, 113)), "0+3+4+2+6", 113),
    +                //E@10
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(103, 113)), "0+3+4+2+6+5", 113),
    +                //E@4
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(103, 113)), "0+3+4+2+6+5+7", 113),
    +                // E@3
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(104, 114)), "0+3+4", 106),
    +                //E@13
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(104, 114)), "0+3+4+6", 113),
    +                //E@10
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(104, 114)), "0+3+4+6+5", 113),
    +                //E@4
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(104, 114)), "0+3+4+6+5+7", 113),
    +                //E@4
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(105, 115)), "0+3+4+6+5", 113),
    +                //E@15
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(105, 115)), "0+3+4+6+5+9", 115),
    +                // E@6
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(106, 116)), "0+4", 106),
    +                //E@13
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(106, 116)), "0+4+6", 113),
    +                //E@10
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(106, 116)), "0+4+6+5", 113),
    +                //E@15
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(106, 116)), "0+4+6+5+9", 115),
    +                //E@13
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(107, 117)), "0+6", 113),
    +                //E@10
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(107, 117)), "0+6+5", 113),
    +                //E@15
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(107, 117)), "0+6+5+9", 115),
    +                //E@10
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(111, 121)), "0+6", 113),
    +                //E@15
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(111, 121)), "0+6+9", 115),
    +                //E@15
    +                new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(114, 124)), "0+9", 115)),
    +            actual
    +        );
    +    }
    +
         @Test
         public void shouldLogAndMeterWhenSkippingNullKey() {
             final String builtInMetricsVersion = StreamsConfig.METRICS_LATEST;
    
    From 6b2a0bcf8c76e6387f0f5ec13d1a7f00341ba110 Mon Sep 17 00:00:00 2001
    From: Hao Li <1127478+lihaosky@users.noreply.github.com>
    Date: Thu, 31 Mar 2022 09:48:21 -0700
    Subject: [PATCH 076/447] KAFKA-13785: add processor metadata to be committed
     with offset (#11829)
    
    Part of KIP-825
    
    Reviewers: Matthias J. Sax 
    ---
     .../internals/AbstractProcessorContext.java   |  23 +++
     .../internals/InternalProcessorContext.java   |   9 +
     .../internals/ProcessorMetadata.java          | 145 +++++++++++++
     .../processor/internals/StreamTask.java       |  96 ++++-----
     .../internals/TopicPartitionMetadata.java     | 113 ++++++++++
     .../internals/ProcessorContextImplTest.java   |  33 +++
     .../internals/ProcessorMetadataTest.java      | 163 +++++++++++++++
     .../processor/internals/StreamTaskTest.java   | 193 ++++++++++++++----
     .../internals/TopicPartitionMetadataTest.java | 106 ++++++++++
     .../test/MockInternalNewProcessorContext.java |  26 +++
     .../test/MockInternalProcessorContext.java    |  26 +++
     .../org/apache/kafka/test/MockProcessor.java  |   8 +
     12 files changed, 842 insertions(+), 99 deletions(-)
     create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorMetadata.java
     create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/TopicPartitionMetadata.java
     create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorMetadataTest.java
     create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/TopicPartitionMetadataTest.java
    
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java
    index 4140f129be33..af6190e9645d 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java
    @@ -48,6 +48,7 @@ public abstract class AbstractProcessorContext implements InternalPr
         protected ProcessorNode currentNode;
         private long cachedSystemTimeMs;
         protected ThreadCache cache;
    +    private ProcessorMetadata processorMetadata;
     
         public AbstractProcessorContext(final TaskId taskId,
                                         final StreamsConfig config,
    @@ -60,6 +61,7 @@ public AbstractProcessorContext(final TaskId taskId,
             valueSerde = null;
             keySerde = null;
             this.cache = cache;
    +        processorMetadata = new ProcessorMetadata();
         }
     
         protected abstract StateManager stateManager();
    @@ -254,4 +256,25 @@ public TaskType taskType() {
         public String changelogFor(final String storeName) {
             return stateManager().changelogFor(storeName);
         }
    +
    +    @Override
    +    public void addProcessorMetadataKeyValue(final String key, final long value) {
    +        processorMetadata.put(key, value);
    +    }
    +
    +    @Override
    +    public Long processorMetadataForKey(final String key) {
    +        return processorMetadata.get(key);
    +    }
    +
    +    @Override
    +    public void setProcessorMetadata(final ProcessorMetadata metadata) {
    +        Objects.requireNonNull(metadata);
    +        processorMetadata = metadata;
    +    }
    +
    +    @Override
    +    public ProcessorMetadata getProcessorMetadata() {
    +        return processorMetadata;
    +    }
     }
    \ No newline at end of file
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalProcessorContext.java
    index 6c12f2a60c86..a22a68f1732f 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalProcessorContext.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalProcessorContext.java
    @@ -120,4 +120,13 @@ void logChange(final String storeName,
                        final Position position);
     
         String changelogFor(final String storeName);
    +
    +    void addProcessorMetadataKeyValue(final String key, final long value);
    +
    +    Long processorMetadataForKey(final String key);
    +
    +    void setProcessorMetadata(final ProcessorMetadata metadata);
    +
    +    ProcessorMetadata getProcessorMetadata();
    +
     }
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorMetadata.java
    new file mode 100644
    index 000000000000..943beb64e017
    --- /dev/null
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorMetadata.java
    @@ -0,0 +1,145 @@
    +/*
    + * 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.streams.processor.internals;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.nio.ByteBuffer;
    +import java.nio.charset.StandardCharsets;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.Objects;
    +
    +/**
    + * ProcessorMetadata to be access and populated by processor node. This will be committed along with
    + * offset. This metadata is mainly for windowed aggregation processor to store last emitted timestamp
    + * for now. Therefore, the supported metadata value type is only Long which is timestamp type.
    + */
    +public class ProcessorMetadata {
    +
    +    private final Map metadata;
    +
    +    // Whether metadata should be committed. We only need to commit if metadata is updated via
    +    // put() or set explicitly
    +    private boolean needsCommit;
    +
    +    public ProcessorMetadata() {
    +        this(new HashMap<>());
    +    }
    +
    +    public ProcessorMetadata(final Map metadata) {
    +        this.metadata = metadata;
    +        needsCommit = false;
    +    }
    +
    +    public static ProcessorMetadata deserialize(final byte[] metaDataBytes) {
    +        if (metaDataBytes == null || metaDataBytes.length == 0) {
    +            return new ProcessorMetadata();
    +        }
    +
    +        final ByteBuffer buffer = ByteBuffer.wrap(metaDataBytes);
    +        final int entrySize = buffer.getInt();
    +        final Map metadata = new HashMap<>(entrySize);
    +        for (int i = 0; i < entrySize; i++) {
    +            final int keySize = buffer.getInt();
    +            final byte[] keyBytes = new byte[keySize];
    +            buffer.get(keyBytes);
    +            final Long value = buffer.getLong();
    +            metadata.put(new String(keyBytes, StandardCharsets.UTF_8), value);
    +        }
    +        return new ProcessorMetadata(metadata);
    +    }
    +
    +    public byte[] serialize() {
    +        if (metadata.isEmpty()) {
    +            return new byte[0];
    +        }
    +
    +        final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
    +        final byte[] mapSizeBytes = ByteBuffer.allocate(Integer.BYTES).putInt(metadata.size()).array();
    +        outputStream.write(mapSizeBytes, 0, mapSizeBytes.length);
    +
    +        for (final Map.Entry entry : metadata.entrySet()) {
    +            final byte[] keyBytes = entry.getKey().getBytes(StandardCharsets.UTF_8);
    +            final int keyLen = keyBytes.length;
    +            final byte[] buffer = ByteBuffer.allocate(Integer.BYTES + keyBytes.length + Long.BYTES)
    +                .putInt(keyLen)
    +                .put(keyBytes)
    +                .putLong(entry.getValue())
    +                .array();
    +            outputStream.write(buffer, 0, buffer.length);
    +        }
    +        return outputStream.toByteArray();
    +    }
    +
    +    public void put(final String key, final long value) {
    +        metadata.put(key, value);
    +        needsCommit = true;
    +    }
    +
    +    public Long get(final String key) {
    +        return metadata.get(key);
    +    }
    +
    +    /**
    +     * Merge with other metadata. Missing keys will be added. Existing key's value will be updated to
    +     * max
    +     * @param other Other metadata to be merged
    +     */
    +    public void update(final ProcessorMetadata other) {
    +        if (other == null) {
    +            return;
    +        }
    +        for (final Map.Entry kv : other.metadata.entrySet()) {
    +            final Long value = metadata.get(kv.getKey());
    +            if (value == null || value < kv.getValue()) {
    +                metadata.put(kv.getKey(), kv.getValue());
    +            }
    +        }
    +    }
    +
    +    public void setNeedsCommit(final boolean needsCommit) {
    +        this.needsCommit = needsCommit;
    +    }
    +
    +    /**
    +     * Whether metadata needs to be committed. It should be committed only if put is or
    +     * {@link #setNeedsCommit} is called explicitly
    +     *
    +     * @return If metadata needs to be committed.
    +     */
    +    public boolean needsCommit() {
    +        return needsCommit;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        // needsCommit is not considered in hashCode or equals
    +        return Objects.hashCode(metadata);
    +    }
    +
    +    @Override
    +    public boolean equals(final Object obj) {
    +        if (obj == null || obj.getClass() != getClass()) {
    +            return false;
    +        }
    +        if (this == obj) {
    +            return true;
    +        }
    +
    +        return metadata.equals(((ProcessorMetadata) obj).metadata);
    +    }
    +}
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
    index b2df5ce096d5..3e6513cf7be2 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
    @@ -49,8 +49,6 @@
     import java.io.IOException;
     import java.io.PrintWriter;
     import java.io.StringWriter;
    -import java.nio.ByteBuffer;
    -import java.util.Base64;
     import java.util.Collections;
     import java.util.HashMap;
     import java.util.HashSet;
    @@ -69,9 +67,6 @@
      */
     public class StreamTask extends AbstractTask implements ProcessorNodePunctuator, Task {
     
    -    // visible for testing
    -    static final byte LATEST_MAGIC_BYTE = 1;
    -
         private final Time time;
         private final Consumer mainConsumer;
     
    @@ -419,6 +414,25 @@ public Map prepareCommit() {
             }
         }
     
    +    private Long findOffset(final TopicPartition partition) {
    +        Long offset = partitionGroup.headRecordOffset(partition);
    +        if (offset == null) {
    +            try {
    +                offset = mainConsumer.position(partition);
    +            } catch (final TimeoutException error) {
    +                // the `consumer.position()` call should never block, because we know that we did process data
    +                // for the requested partition and thus the consumer should have a valid local position
    +                // that it can return immediately
    +
    +                // hence, a `TimeoutException` indicates a bug and thus we rethrow it as fatal `IllegalStateException`
    +                throw new IllegalStateException(error);
    +            } catch (final KafkaException fatal) {
    +                throw new StreamsException(fatal);
    +            }
    +        }
    +        return offset;
    +    }
    +
         private Map committableOffsetsAndMetadata() {
             final Map committableOffsets;
     
    @@ -433,28 +447,18 @@ private Map committableOffsetsAndMetadata() {
                 case SUSPENDED:
                     final Map partitionTimes = extractPartitionTimes();
     
    -                committableOffsets = new HashMap<>(consumedOffsets.size());
    -                for (final Map.Entry entry : consumedOffsets.entrySet()) {
    -                    final TopicPartition partition = entry.getKey();
    -                    Long offset = partitionGroup.headRecordOffset(partition);
    -                    if (offset == null) {
    -                        try {
    -                            offset = mainConsumer.position(partition);
    -                        } catch (final TimeoutException error) {
    -                            // the `consumer.position()` call should never block, because we know that we did process data
    -                            // for the requested partition and thus the consumer should have a valid local position
    -                            // that it can return immediately
    -
    -                            // hence, a `TimeoutException` indicates a bug and thus we rethrow it as fatal `IllegalStateException`
    -                            throw new IllegalStateException(error);
    -                        } catch (final KafkaException fatal) {
    -                            throw new StreamsException(fatal);
    -                        }
    -                    }
    +                // If there's processor metadata to be committed. We need to commit them to all
    +                // input partitions
    +                final Set partitionsNeedCommit = processorContext.getProcessorMetadata().needsCommit() ?
    +                    inputPartitions() : consumedOffsets.keySet();
    +                committableOffsets = new HashMap<>(partitionsNeedCommit.size());
    +
    +                for (final TopicPartition partition : partitionsNeedCommit) {
    +                    final Long offset = findOffset(partition);
                         final long partitionTime = partitionTimes.get(partition);
    -                    committableOffsets.put(partition, new OffsetAndMetadata(offset, encodeTimestamp(partitionTime)));
    +                    committableOffsets.put(partition, new OffsetAndMetadata(offset,
    +                        new TopicPartitionMetadata(partitionTime, processorContext.getProcessorMetadata()).encode()));
                     }
    -
                     break;
     
                 case CLOSED:
    @@ -506,6 +510,7 @@ private void clearCommitStatuses() {
             commitNeeded = false;
             commitRequested = false;
             hasPendingTxCommit = false;
    +        processorContext.getProcessorMetadata().setNeedsCommit(false);
         }
     
         private Map extractPartitionTimes() {
    @@ -537,6 +542,7 @@ public void closeDirty() {
         public void updateInputPartitions(final Set topicPartitions, final Map> allTopologyNodesToSourceTopics) {
             super.updateInputPartitions(topicPartitions, allTopologyNodesToSourceTopics);
             partitionGroup.updatePartitions(topicPartitions, recordQueueCreator::createQueue);
    +        processorContext.getProcessorMetadata().setNeedsCommit(true);
         }
     
         @Override
    @@ -889,7 +895,7 @@ private void resetOffsetsIfNeededAndInitializeMetadata(final java.util.function.
                 offsetResetter.accept(resetOffsetsForPartitions);
                 resetOffsetsForPartitions.clear();
     
    -            initializeTaskTime(offsetsAndMetadata.entrySet().stream()
    +            initializeTaskTimeAndProcessorMetadata(offsetsAndMetadata.entrySet().stream()
                     .filter(e -> e.getValue() != null)
                     .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))
                 );
    @@ -907,20 +913,26 @@ private void resetOffsetsIfNeededAndInitializeMetadata(final java.util.function.
             }
         }
     
    -    private void initializeTaskTime(final Map offsetsAndMetadata) {
    +    private void initializeTaskTimeAndProcessorMetadata(final Map offsetsAndMetadata) {
    +        final ProcessorMetadata finalProcessMetadata = new ProcessorMetadata();
             for (final Map.Entry entry : offsetsAndMetadata.entrySet()) {
                 final TopicPartition partition = entry.getKey();
                 final OffsetAndMetadata metadata = entry.getValue();
     
                 if (metadata != null) {
    -                final long committedTimestamp = decodeTimestamp(metadata.metadata());
    +                final TopicPartitionMetadata committedTimestampAndMeta = TopicPartitionMetadata.decode(metadata.metadata());
    +                final long committedTimestamp = committedTimestampAndMeta.partitionTime();
                     partitionGroup.setPartitionTime(partition, committedTimestamp);
                     log.debug("A committed timestamp was detected: setting the partition time of partition {}"
                         + " to {} in stream task {}", partition, committedTimestamp, id);
    +
    +                final ProcessorMetadata processorMetadata = committedTimestampAndMeta.processorMetadata();
    +                finalProcessMetadata.update(processorMetadata);
                 } else {
                     log.debug("No committed timestamp was found in metadata for partition {}", partition);
                 }
             }
    +        processorContext.setProcessorMetadata(finalProcessMetadata);
     
             final Set nonCommitted = new HashSet<>(inputPartitions());
             nonCommitted.removeAll(offsetsAndMetadata.keySet());
    @@ -1095,34 +1107,6 @@ public boolean commitRequested() {
             return commitRequested;
         }
     
    -    static String encodeTimestamp(final long partitionTime) {
    -        final ByteBuffer buffer = ByteBuffer.allocate(9);
    -        buffer.put(LATEST_MAGIC_BYTE);
    -        buffer.putLong(partitionTime);
    -        return Base64.getEncoder().encodeToString(buffer.array());
    -    }
    -
    -    long decodeTimestamp(final String encryptedString) {
    -        if (encryptedString.isEmpty()) {
    -            return RecordQueue.UNKNOWN;
    -        }
    -        try {
    -            final ByteBuffer buffer = ByteBuffer.wrap(Base64.getDecoder().decode(encryptedString));
    -            final byte version = buffer.get();
    -            switch (version) {
    -                case LATEST_MAGIC_BYTE:
    -                    return buffer.getLong();
    -                default:
    -                    log.warn("Unsupported offset metadata version found. Supported version {}. Found version {}.",
    -                            LATEST_MAGIC_BYTE, version);
    -                    return RecordQueue.UNKNOWN;
    -            }
    -        } catch (final Exception exception) {
    -            log.warn("Unsupported offset metadata found");
    -            return RecordQueue.UNKNOWN;
    -        }
    -    }
    -
         @SuppressWarnings("rawtypes")
         public InternalProcessorContext processorContext() {
             return processorContext;
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopicPartitionMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopicPartitionMetadata.java
    new file mode 100644
    index 000000000000..dae072d25bba
    --- /dev/null
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopicPartitionMetadata.java
    @@ -0,0 +1,113 @@
    +/*
    + * 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.streams.processor.internals;
    +
    +import java.nio.ByteBuffer;
    +import java.util.Base64;
    +import java.util.Objects;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Metadata to be committed together with TopicPartition offset
    + */
    +public class TopicPartitionMetadata {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(TopicPartitionMetadata.class);
    +
    +    // visible for testing
    +    static final byte LATEST_MAGIC_BYTE = 2;
    +
    +    private final long partitionTime;
    +    private final ProcessorMetadata processorMetadata;
    +
    +    public TopicPartitionMetadata(final long partitionTime, final ProcessorMetadata processorMetadata) {
    +        Objects.requireNonNull(processorMetadata);
    +        this.partitionTime = partitionTime;
    +        this.processorMetadata = processorMetadata;
    +    }
    +
    +    public long partitionTime() {
    +        return partitionTime;
    +    }
    +
    +    public ProcessorMetadata processorMetadata() {
    +        return processorMetadata;
    +    }
    +
    +    public String encode() {
    +        final byte[] serializedMeta = processorMetadata.serialize();
    +        // Format: MAGIC_BYTE(1) + PartitionTime(8) + processMeta
    +        final ByteBuffer buffer = ByteBuffer.allocate(Byte.BYTES + Long.BYTES + serializedMeta.length);
    +        buffer.put(LATEST_MAGIC_BYTE);
    +        buffer.putLong(partitionTime);
    +        buffer.put(serializedMeta);
    +        return Base64.getEncoder().encodeToString(buffer.array());
    +    }
    +
    +    public static TopicPartitionMetadata decode(final String encryptedString) {
    +        long timestamp = RecordQueue.UNKNOWN;
    +        ProcessorMetadata metadata = new ProcessorMetadata();
    +
    +        if (encryptedString.isEmpty()) {
    +            return new TopicPartitionMetadata(timestamp, metadata);
    +        }
    +        try {
    +            final ByteBuffer buffer = ByteBuffer.wrap(Base64.getDecoder().decode(encryptedString));
    +            final byte version = buffer.get();
    +            switch (version) {
    +                case (byte) 1:
    +                    timestamp = buffer.getLong();
    +                    break;
    +                case LATEST_MAGIC_BYTE:
    +                    timestamp = buffer.getLong();
    +                    if (buffer.remaining() > 0) {
    +                        final byte[] metaBytes = new byte[buffer.remaining()];
    +                        buffer.get(metaBytes);
    +                        metadata = ProcessorMetadata.deserialize(metaBytes);
    +                    }
    +                    break;
    +                default:
    +                    LOG.warn(
    +                        "Unsupported offset metadata version found. Supported version <= {}. Found version {}.",
    +                        LATEST_MAGIC_BYTE, version);
    +            }
    +        } catch (final Exception exception) {
    +            LOG.warn("Unsupported offset metadata found");
    +        }
    +        return new TopicPartitionMetadata(timestamp, metadata);
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        return Objects.hash(partitionTime, processorMetadata);
    +    }
    +
    +    @Override
    +    public boolean equals(final Object obj) {
    +        if (obj == null || obj.getClass() != getClass()) {
    +            return false;
    +        }
    +
    +        if (obj == this) {
    +            return true;
    +        }
    +
    +        return partitionTime == ((TopicPartitionMetadata) obj).partitionTime
    +            && Objects.equals(processorMetadata, ((TopicPartitionMetadata) obj).processorMetadata);
    +    }
    +}
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java
    index f5e4f6364aaa..b6cc7a789ebe 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java
    @@ -58,6 +58,8 @@
     import java.util.function.Consumer;
     
     import static java.util.Arrays.asList;
    +import static org.apache.kafka.common.utils.Utils.mkEntry;
    +import static org.apache.kafka.common.utils.Utils.mkMap;
     import static org.apache.kafka.streams.processor.internals.ProcessorContextImpl.BYTEARRAY_VALUE_SERIALIZER;
     import static org.apache.kafka.streams.processor.internals.ProcessorContextImpl.BYTES_KEY_SERIALIZER;
     import static org.easymock.EasyMock.anyLong;
    @@ -69,6 +71,7 @@
     import static org.easymock.EasyMock.replay;
     import static org.easymock.EasyMock.verify;
     import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertNull;
     import static org.junit.Assert.assertThrows;
     import static org.junit.Assert.assertTrue;
     import static org.junit.Assert.fail;
    @@ -568,6 +571,36 @@ public void shouldMatchStreamTime() {
             assertEquals(STREAM_TIME, context.currentStreamTimeMs());
         }
     
    +    @Test
    +    public void shouldAddAndGetProcessorKeyValue() {
    +        context.addProcessorMetadataKeyValue("key1", 100L);
    +        final Long value = context.processorMetadataForKey("key1");
    +        assertEquals(100L, value.longValue());
    +
    +        final Long noValue = context.processorMetadataForKey("nokey");
    +        assertNull(noValue);
    +    }
    +
    +    @Test
    +    public void shouldSetAndGetProcessorMetaData() {
    +        final ProcessorMetadata emptyMetadata = new ProcessorMetadata();
    +        context.setProcessorMetadata(emptyMetadata);
    +        assertEquals(emptyMetadata, context.getProcessorMetadata());
    +
    +        final ProcessorMetadata metadata = new ProcessorMetadata(
    +            mkMap(
    +                mkEntry("key1", 10L),
    +                mkEntry("key2", 100L)
    +            )
    +        );
    +
    +        context.setProcessorMetadata(metadata);
    +        assertEquals(10L, context.processorMetadataForKey("key1").longValue());
    +        assertEquals(100L, context.processorMetadataForKey("key2").longValue());
    +
    +        assertThrows(NullPointerException.class, () -> context.setProcessorMetadata(null));
    +    }
    +
         @SuppressWarnings("unchecked")
         private KeyValueStore keyValueStoreMock() {
             final KeyValueStore keyValueStoreMock = mock(KeyValueStore.class);
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorMetadataTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorMetadataTest.java
    new file mode 100644
    index 000000000000..c7ce0c80602b
    --- /dev/null
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorMetadataTest.java
    @@ -0,0 +1,163 @@
    +/*
    + * 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.streams.processor.internals;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +import org.junit.Test;
    +
    +import static org.hamcrest.CoreMatchers.is;
    +import static org.hamcrest.CoreMatchers.nullValue;
    +import static org.hamcrest.MatcherAssert.assertThat;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.assertTrue;
    +
    +public class ProcessorMetadataTest {
    +
    +    @Test
    +    public void shouldAddandGetKeyValueWithEmptyConstructor() {
    +        final ProcessorMetadata metadata = new ProcessorMetadata();
    +        final String key = "some_key";
    +        final long value = 100L;
    +
    +        metadata.put(key, value);
    +        final Long actualValue =  metadata.get(key);
    +
    +        assertThat(actualValue, is(value));
    +
    +        final Long noValue = metadata.get("no_key");
    +        assertThat(noValue, is(nullValue()));
    +    }
    +
    +    @Test
    +    public void shouldAddandGetKeyValueWithExistingMeta() {
    +        final Map map = new HashMap<>();
    +        map.put("key1", 1L);
    +        map.put("key2", 2L);
    +
    +        final ProcessorMetadata metadata = new ProcessorMetadata(map);
    +
    +        final long value1 = metadata.get("key1");
    +        assertThat(value1, is(1L));
    +
    +        final long value2 = metadata.get("key2");
    +        assertThat(value2, is(2L));
    +
    +        final Long noValue = metadata.get("key3");
    +        assertThat(noValue, is(nullValue()));
    +
    +        metadata.put("key3", 3L);
    +        final long value3 = metadata.get("key3");
    +        assertThat(value3, is(3L));
    +    }
    +
    +    @Test
    +    public void shouldSerializeAndDeserialize() {
    +        final ProcessorMetadata metadata = new ProcessorMetadata();
    +        final String key1 = "key1", key2 = "key2", key3 = "key3";
    +        final long value1 = 1L, value2 = 2L, value3 = 3L;
    +
    +        metadata.put(key1, value1);
    +        metadata.put(key2, value2);
    +        metadata.put(key3, value3);
    +
    +        final byte[] serialized = metadata.serialize();
    +        final ProcessorMetadata deserialized = ProcessorMetadata.deserialize(serialized);
    +
    +        assertThat(deserialized.get(key1), is(value1));
    +        assertThat(deserialized.get(key2), is(value2));
    +        assertThat(deserialized.get(key3), is(value3));
    +    }
    +
    +    @Test
    +    public void shouldDeserializeNull() {
    +        final ProcessorMetadata deserialized = ProcessorMetadata.deserialize(null);
    +        assertThat(deserialized, is(new ProcessorMetadata()));
    +    }
    +
    +    @Test
    +    public void shouldUpdate() {
    +        final ProcessorMetadata emptyMeta = new ProcessorMetadata();
    +        emptyMeta.update(null);
    +
    +        assertThat(emptyMeta, is(new ProcessorMetadata()));
    +
    +        {
    +            final Map map1 = new HashMap<>();
    +            map1.put("key1", 1L);
    +            map1.put("key2", 2L);
    +            final ProcessorMetadata metadata1 = new ProcessorMetadata(map1);
    +            emptyMeta.update(metadata1);
    +            assertThat(emptyMeta.get("key1"), is(1L));
    +            assertThat(emptyMeta.get("key2"), is(2L));
    +        }
    +
    +        {
    +            final Map map1 = new HashMap<>();
    +            map1.put("key1", 0L);
    +            map1.put("key2", 1L);
    +            final ProcessorMetadata metadata1 = new ProcessorMetadata(map1);
    +            emptyMeta.update(metadata1);
    +            assertThat(emptyMeta.get("key1"), is(1L));
    +            assertThat(emptyMeta.get("key2"), is(2L));
    +        }
    +
    +        {
    +            final Map map1 = new HashMap<>();
    +            map1.put("key1", 2L);
    +            map1.put("key2", 3L);
    +            final ProcessorMetadata metadata1 = new ProcessorMetadata(map1);
    +            emptyMeta.update(metadata1);
    +            assertThat(emptyMeta.get("key1"), is(2L));
    +            assertThat(emptyMeta.get("key2"), is(3L));
    +        }
    +    }
    +
    +    @Test
    +    public void shouldUpdateCommitFlag() {
    +        final ProcessorMetadata emptyMeta = new ProcessorMetadata();
    +        assertFalse(emptyMeta.needsCommit());
    +
    +        emptyMeta.setNeedsCommit(true);
    +        assertTrue(emptyMeta.needsCommit());
    +
    +        emptyMeta.setNeedsCommit(false);
    +        assertFalse(emptyMeta.needsCommit());
    +
    +        emptyMeta.put("key1", 1L);
    +        assertTrue(emptyMeta.needsCommit());
    +
    +        final Map map1 = new HashMap<>();
    +        map1.put("key1", 2L);
    +        map1.put("key2", 3L);
    +        final ProcessorMetadata metadata1 = new ProcessorMetadata(map1);
    +        emptyMeta.update(metadata1);
    +        assertTrue(emptyMeta.needsCommit());
    +    }
    +
    +    @Test
    +    public void shouldNotUseCommitFlagForHashcodeAndEquals() {
    +        final ProcessorMetadata metadata1 = new ProcessorMetadata();
    +        metadata1.setNeedsCommit(true);
    +        final ProcessorMetadata metadata2 = new ProcessorMetadata();
    +        metadata2.setNeedsCommit(false);
    +
    +        assertEquals(metadata1, metadata2);
    +        assertEquals(metadata1.hashCode(), metadata2.hashCode());
    +    }
    +}
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
    index be99d732a779..9974fa92b6ec 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
    @@ -80,7 +80,6 @@
     import java.io.IOException;
     import java.nio.ByteBuffer;
     import java.time.Duration;
    -import java.util.Base64;
     import java.util.Collections;
     import java.util.HashSet;
     import java.util.List;
    @@ -103,7 +102,6 @@
     import static org.apache.kafka.common.utils.Utils.mkProperties;
     import static org.apache.kafka.common.utils.Utils.mkSet;
     import static org.apache.kafka.streams.StreamsConfig.AT_LEAST_ONCE;
    -import static org.apache.kafka.streams.processor.internals.StreamTask.encodeTimestamp;
     import static org.apache.kafka.streams.processor.internals.Task.State.CREATED;
     import static org.apache.kafka.streams.processor.internals.Task.State.RESTORING;
     import static org.apache.kafka.streams.processor.internals.Task.State.RUNNING;
    @@ -130,7 +128,6 @@ public class StreamTaskTest {
     
         private static final String APPLICATION_ID = "stream-task-test";
         private static final File BASE_DIR = TestUtils.tempDirectory();
    -    private static final long DEFAULT_TIMESTAMP = 1000;
     
         private final LogContext logContext = new LogContext("[test] ");
         private final String topic1 = "topic1";
    @@ -406,12 +403,17 @@ public void seek(final TopicPartition partition, final long offset) {
         }
     
         @Test
    -    public void shouldReadCommittedStreamTimeOnInitialize() {
    +    public void shouldReadCommittedStreamTimeAndProcessorMetadataOnInitialize() {
             stateDirectory = EasyMock.createNiceMock(StateDirectory.class);
             EasyMock.replay(stateDirectory);
     
    +        final ProcessorMetadata processorMetadata = new ProcessorMetadata(mkMap(
    +            mkEntry("key1", 1L),
    +            mkEntry("key2", 2L)
    +        ));
    +
             consumer.commitSync(partitions.stream()
    -            .collect(Collectors.toMap(Function.identity(), tp -> new OffsetAndMetadata(0L, encodeTimestamp(10L)))));
    +            .collect(Collectors.toMap(Function.identity(), tp -> new OffsetAndMetadata(0L, new TopicPartitionMetadata(10L, processorMetadata).encode()))));
     
             task = createStatelessTask(createConfig("100"));
     
    @@ -421,6 +423,49 @@ public void shouldReadCommittedStreamTimeOnInitialize() {
             task.completeRestoration(noOpResetter -> { });
     
             assertEquals(10L, task.streamTime());
    +        assertEquals(1L, task.processorContext().processorMetadataForKey("key1").longValue());
    +        assertEquals(2L, task.processorContext().processorMetadataForKey("key2").longValue());
    +    }
    +
    +    @Test
    +    public void shouldReadCommittedStreamTimeAndMergeProcessorMetadataOnInitialize() {
    +        stateDirectory = EasyMock.createNiceMock(StateDirectory.class);
    +        EasyMock.replay(stateDirectory);
    +
    +        final ProcessorMetadata processorMetadata1 = new ProcessorMetadata(mkMap(
    +            mkEntry("key1", 1L),
    +            mkEntry("key2", 2L)
    +        ));
    +
    +        final Map meta1 = mkMap(
    +            mkEntry(partition1, new OffsetAndMetadata(0L, new TopicPartitionMetadata(10L, processorMetadata1).encode())
    +            )
    +        );
    +
    +        final ProcessorMetadata processorMetadata2 = new ProcessorMetadata(mkMap(
    +            mkEntry("key1", 10L),
    +            mkEntry("key3", 30L)
    +        ));
    +
    +        final Map meta2 = mkMap(
    +            mkEntry(partition2, new OffsetAndMetadata(0L, new TopicPartitionMetadata(20L, processorMetadata2).encode())
    +            )
    +        );
    +
    +        consumer.commitSync(meta1);
    +        consumer.commitSync(meta2);
    +
    +        task = createStatelessTask(createConfig("100"));
    +
    +        assertEquals(RecordQueue.UNKNOWN, task.streamTime());
    +
    +        task.initializeIfNeeded();
    +        task.completeRestoration(noOpResetter -> { });
    +
    +        assertEquals(20L, task.streamTime());
    +        assertEquals(10L, task.processorContext().processorMetadataForKey("key1").longValue());
    +        assertEquals(2L, task.processorContext().processorMetadataForKey("key2").longValue());
    +        assertEquals(30L, task.processorContext().processorMetadataForKey("key3").longValue());
         }
     
         @Test
    @@ -1096,7 +1141,7 @@ public void shouldRespectCommitNeeded() {
         }
     
         @Test
    -    public void shouldCommitNextOffsetFromQueueIfAvailable() {
    +    public void shouldCommitNextOffsetAndProcessorMetadataFromQueueIfAvailable() {
             task = createSingleSourceStateless(createConfig(AT_LEAST_ONCE, "0"), StreamsConfig.METRICS_LATEST);
             task.initializeIfNeeded();
             task.completeRestoration(noOpResetter -> { });
    @@ -1107,11 +1152,21 @@ public void shouldCommitNextOffsetFromQueueIfAvailable() {
                 getConsumerRecordWithOffsetAsTimestamp(partition1, 5L)));
     
             task.process(0L);
    +        processorStreamTime.mockProcessor.addProcessorMetadata("key1", 100L);
             task.process(0L);
    +        processorSystemTime.mockProcessor.addProcessorMetadata("key2", 200L);
     
             final Map offsetsAndMetadata = task.prepareCommit();
    +        final TopicPartitionMetadata expected = new TopicPartitionMetadata(3L,
    +            new ProcessorMetadata(
    +                mkMap(
    +                    mkEntry("key1", 100L),
    +                    mkEntry("key2", 200L)
    +                )
    +            )
    +        );
     
    -        assertThat(offsetsAndMetadata, equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(5L, encodeTimestamp(3L))))));
    +        assertThat(offsetsAndMetadata, equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(5L, expected.encode())))));
         }
     
         @Test
    @@ -1130,8 +1185,16 @@ public void shouldCommitConsumerPositionIfRecordQueueIsEmpty() {
             task.addRecords(partition2, singletonList(getConsumerRecordWithOffsetAsTimestamp(partition2, 0L)));
             task.process(0L);
     
    +        final TopicPartitionMetadata metadata = new TopicPartitionMetadata(0, new ProcessorMetadata());
    +
             assertTrue(task.commitNeeded());
    -        assertThat(task.prepareCommit(), equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(3L, encodeTimestamp(0L))))));
    +        assertThat(task.prepareCommit(), equalTo(
    +            mkMap(
    +                mkEntry(partition1,
    +                    new OffsetAndMetadata(3L, metadata.encode())
    +                )
    +            )
    +        ));
             task.postCommit(false);
     
             // the task should still be committed since the processed records have not reached the consumer position
    @@ -1141,8 +1204,79 @@ public void shouldCommitConsumerPositionIfRecordQueueIsEmpty() {
             task.process(0L);
     
             assertTrue(task.commitNeeded());
    -        assertThat(task.prepareCommit(), equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(3L, encodeTimestamp(0L))),
    -                                                       mkEntry(partition2, new OffsetAndMetadata(1L, encodeTimestamp(0L))))));
    +        assertThat(task.prepareCommit(), equalTo(
    +            mkMap(
    +                mkEntry(partition1, new OffsetAndMetadata(3L, metadata.encode())),
    +                mkEntry(partition2, new OffsetAndMetadata(1L, metadata.encode()))
    +            )
    +        ));
    +        task.postCommit(false);
    +
    +        assertFalse(task.commitNeeded());
    +    }
    +
    +    @Test
    +    public void shouldCommitOldProcessorMetadataWhenNotDirty() {
    +        task = createStatelessTask(createConfig());
    +        task.initializeIfNeeded();
    +        task.completeRestoration(noOpResetter -> { });
    +
    +        consumer.addRecord(getConsumerRecordWithOffsetAsTimestamp(partition1, 0L));
    +        consumer.addRecord(getConsumerRecordWithOffsetAsTimestamp(partition1, 1L));
    +        consumer.addRecord(getConsumerRecordWithOffsetAsTimestamp(partition2, 0L));
    +        consumer.addRecord(getConsumerRecordWithOffsetAsTimestamp(partition2, 1L));
    +        consumer.poll(Duration.ZERO);
    +
    +        task.addRecords(partition1, singletonList(getConsumerRecordWithOffsetAsTimestamp(partition1, 0L)));
    +        task.addRecords(partition1, singletonList(getConsumerRecordWithOffsetAsTimestamp(partition1, 1L)));
    +
    +        task.process(0L);
    +        processorStreamTime.mockProcessor.addProcessorMetadata("key1", 100L);
    +
    +        final TopicPartitionMetadata expectedMetadata1 = new TopicPartitionMetadata(0L,
    +            new ProcessorMetadata(
    +                mkMap(
    +                    mkEntry("key1", 100L)
    +                )
    +            )
    +        );
    +
    +        final TopicPartitionMetadata expectedMetadata2 = new TopicPartitionMetadata(RecordQueue.UNKNOWN,
    +            new ProcessorMetadata(
    +                mkMap(
    +                    mkEntry("key1", 100L)
    +                )
    +            )
    +        );
    +
    +        assertTrue(task.commitNeeded());
    +
    +        assertThat(task.prepareCommit(), equalTo(
    +            mkMap(
    +                mkEntry(partition1, new OffsetAndMetadata(1L, expectedMetadata1.encode())),
    +                mkEntry(partition2, new OffsetAndMetadata(2L, expectedMetadata2.encode()))
    +            )));
    +        task.postCommit(false);
    +
    +        // the task should still be committed since the processed records have not reached the consumer position
    +        assertTrue(task.commitNeeded());
    +
    +        consumer.poll(Duration.ZERO);
    +        task.process(0L);
    +
    +        final TopicPartitionMetadata expectedMetadata3 = new TopicPartitionMetadata(1L,
    +            new ProcessorMetadata(
    +                mkMap(
    +                    mkEntry("key1", 100L)
    +                )
    +            )
    +        );
    +        assertTrue(task.commitNeeded());
    +
    +        // Processor metadata not updated, we just need to commit to partition1 again with new offset
    +        assertThat(task.prepareCommit(), equalTo(
    +            mkMap(mkEntry(partition1, new OffsetAndMetadata(2L, expectedMetadata3.encode())))
    +        ));
             task.postCommit(false);
     
             assertFalse(task.commitNeeded());
    @@ -1172,35 +1306,6 @@ public void shouldRespectCommitRequested() {
             assertTrue(task.commitRequested());
         }
     
    -    @Test
    -    public void shouldEncodeAndDecodeMetadata() {
    -        task = createStatelessTask(createConfig("100"));
    -        assertEquals(DEFAULT_TIMESTAMP, task.decodeTimestamp(encodeTimestamp(DEFAULT_TIMESTAMP)));
    -    }
    -
    -    @Test
    -    public void shouldReturnUnknownTimestampIfUnknownVersion() {
    -        task = createStatelessTask(createConfig("100"));
    -
    -        final byte[] emptyMessage = {StreamTask.LATEST_MAGIC_BYTE + 1};
    -        final String encodedString = Base64.getEncoder().encodeToString(emptyMessage);
    -        assertEquals(RecordQueue.UNKNOWN, task.decodeTimestamp(encodedString));
    -    }
    -
    -    @Test
    -    public void shouldReturnUnknownTimestampIfEmptyMessage() {
    -        task = createStatelessTask(createConfig("100"));
    -
    -        assertEquals(RecordQueue.UNKNOWN, task.decodeTimestamp(""));
    -    }
    -
    -    @Test
    -    public void shouldReturnUnknownTimestampIfInvalidMetadata() {
    -        task = createStatelessTask(createConfig("100"));
    -        final String invalidBase64String = "{}";
    -        assertEquals(RecordQueue.UNKNOWN, task.decodeTimestamp(invalidBase64String));
    -    }
    -
         @Test
         public void shouldBeProcessableIfAllPartitionsBuffered() {
             task = createStatelessTask(createConfig("100"));
    @@ -2263,7 +2368,9 @@ public void shouldUpdateOffsetIfAllRecordsAreCorrupted() {
             assertTrue(task.commitNeeded());
             assertThat(
                 task.prepareCommit(),
    -            equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(offset + 1, encodeTimestamp(-1)))))
    +            equalTo(mkMap(mkEntry(partition1,
    +                new OffsetAndMetadata(offset + 1,
    +                    new TopicPartitionMetadata(RecordQueue.UNKNOWN, new ProcessorMetadata()).encode()))))
             );
         }
     
    @@ -2291,7 +2398,7 @@ public void shouldUpdateOffsetIfValidRecordFollowsCorrupted() {
             assertTrue(task.commitNeeded());
             assertThat(
                 task.prepareCommit(),
    -            equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(offset + 1, encodeTimestamp(offset)))))
    +            equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(offset + 1, new TopicPartitionMetadata(offset, new ProcessorMetadata()).encode()))))
             );
         }
     
    @@ -2318,14 +2425,14 @@ public void shouldUpdateOffsetIfCorruptedRecordFollowsValid() {
             assertTrue(task.commitNeeded());
             assertThat(
                 task.prepareCommit(),
    -            equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(1, encodeTimestamp(0)))))
    +            equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(1, new TopicPartitionMetadata(0, new ProcessorMetadata()).encode()))))
             );
     
             assertTrue(task.process(offset));
             assertTrue(task.commitNeeded());
             assertThat(
                 task.prepareCommit(),
    -            equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(2, encodeTimestamp(0)))))
    +            equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(2, new TopicPartitionMetadata(0, new ProcessorMetadata()).encode()))))
             );
         }
     
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TopicPartitionMetadataTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TopicPartitionMetadataTest.java
    new file mode 100644
    index 000000000000..e9bef49ff255
    --- /dev/null
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TopicPartitionMetadataTest.java
    @@ -0,0 +1,106 @@
    +/*
    + * 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.streams.processor.internals;
    +
    +import java.nio.ByteBuffer;
    +import java.util.Base64;
    +import org.junit.Test;
    +
    +import static org.hamcrest.CoreMatchers.is;
    +import static org.hamcrest.MatcherAssert.assertThat;
    +
    +public class TopicPartitionMetadataTest {
    +
    +    @Test
    +    public void shouldGetPartitonTimeAndProcessorMeta() {
    +        final ProcessorMetadata metadata = new ProcessorMetadata();
    +        final String key = "some_key";
    +        final long value = 100L;
    +        metadata.put(key, value);
    +
    +        final TopicPartitionMetadata topicMeta = new TopicPartitionMetadata(100L, metadata);
    +
    +        assertThat(topicMeta.partitionTime(), is(100L));
    +        assertThat(topicMeta.processorMetadata(), is(metadata));
    +    }
    +
    +    @Test
    +    public void shouldDecodeVersionOne() {
    +        final byte[] serialized = ByteBuffer.allocate(Byte.BYTES + Long.BYTES)
    +            .put((byte) 1)
    +            .putLong(100L)
    +            .array();
    +        final String serializedString = Base64.getEncoder().encodeToString(serialized);
    +
    +        final TopicPartitionMetadata topicMeta = TopicPartitionMetadata.decode(serializedString);
    +
    +        assertThat(topicMeta.partitionTime(), is(100L));
    +        assertThat(topicMeta.processorMetadata(), is(new ProcessorMetadata()));
    +    }
    +
    +    @Test
    +    public void shouldEncodeDecodeVersionTwo() {
    +        final ProcessorMetadata metadata = new ProcessorMetadata();
    +        final String key = "some_key";
    +        final long value = 100L;
    +        metadata.put(key, value);
    +
    +        final TopicPartitionMetadata expected = new TopicPartitionMetadata(100L, metadata);
    +        final String serializedString = expected.encode();
    +        final TopicPartitionMetadata topicMeta = TopicPartitionMetadata.decode(serializedString);
    +
    +        assertThat(topicMeta, is(expected));
    +    }
    +
    +    @Test
    +    public void shouldEncodeDecodeEmptyMetaVersionTwo() {
    +        final TopicPartitionMetadata expected = new TopicPartitionMetadata(100L, new ProcessorMetadata());
    +        final String serializedString = expected.encode();
    +        final TopicPartitionMetadata topicMeta = TopicPartitionMetadata.decode(serializedString);
    +
    +        assertThat(topicMeta, is(expected));
    +    }
    +
    +    @Test
    +    public void shouldDecodeEmptyStringVersionTwo() {
    +        final TopicPartitionMetadata expected = new TopicPartitionMetadata(RecordQueue.UNKNOWN, new ProcessorMetadata());
    +        final TopicPartitionMetadata topicMeta = TopicPartitionMetadata.decode("");
    +
    +        assertThat(topicMeta, is(expected));
    +    }
    +
    +    @Test
    +    public void shouldReturnUnknownTimestampIfUnknownVersion() {
    +        final byte[] emptyMessage = {TopicPartitionMetadata.LATEST_MAGIC_BYTE + 1};
    +        final String encodedString = Base64.getEncoder().encodeToString(emptyMessage);
    +
    +        final TopicPartitionMetadata decoded = TopicPartitionMetadata.decode(encodedString);
    +
    +        assertThat(decoded.partitionTime(), is(RecordQueue.UNKNOWN));
    +        assertThat(decoded.processorMetadata(), is(new ProcessorMetadata()));
    +    }
    +
    +    @Test
    +    public void shouldReturnUnknownTimestampIfInvalidMetadata() {
    +        final String invalidBase64String = "{}";
    +
    +        final TopicPartitionMetadata decoded = TopicPartitionMetadata.decode(invalidBase64String);
    +
    +        assertThat(decoded.partitionTime(), is(RecordQueue.UNKNOWN));
    +        assertThat(decoded.processorMetadata(), is(new ProcessorMetadata()));
    +    }
    +}
    diff --git a/streams/src/test/java/org/apache/kafka/test/MockInternalNewProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockInternalNewProcessorContext.java
    index 7131e861ba55..4e974e0947b6 100644
    --- a/streams/src/test/java/org/apache/kafka/test/MockInternalNewProcessorContext.java
    +++ b/streams/src/test/java/org/apache/kafka/test/MockInternalNewProcessorContext.java
    @@ -16,6 +16,7 @@
      */
     package org.apache.kafka.test;
     
    +import java.util.Objects;
     import org.apache.kafka.common.header.Headers;
     import org.apache.kafka.common.header.internals.RecordHeaders;
     import org.apache.kafka.common.utils.Bytes;
    @@ -26,6 +27,7 @@
     import org.apache.kafka.streams.processor.To;
     import org.apache.kafka.streams.processor.api.MockProcessorContext;
     import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
    +import org.apache.kafka.streams.processor.internals.ProcessorMetadata;
     import org.apache.kafka.streams.processor.internals.ProcessorNode;
     import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
     import org.apache.kafka.streams.processor.internals.RecordCollector;
    @@ -48,12 +50,15 @@ public class MockInternalNewProcessorContext extends MockProcessorCo
     
         private long timestamp = 0;
         private Headers headers = new RecordHeaders();
    +    private ProcessorMetadata processorMetadata;
     
         public MockInternalNewProcessorContext() {
    +        processorMetadata = new ProcessorMetadata();
         }
     
         public MockInternalNewProcessorContext(final Properties config, final TaskId taskId, final File stateDir) {
             super(config, taskId, stateDir);
    +        processorMetadata = new ProcessorMetadata();
         }
     
         @Override
    @@ -206,4 +211,25 @@ public  T getStateStore(StoreBuilder builder) {
         public String changelogFor(final String storeName) {
             return "mock-changelog";
         }
    +
    +    @Override
    +    public void addProcessorMetadataKeyValue(final String key, final long value) {
    +        processorMetadata.put(key, value);
    +    }
    +
    +    @Override
    +    public Long processorMetadataForKey(final String key) {
    +        return processorMetadata.get(key);
    +    }
    +
    +    @Override
    +    public void setProcessorMetadata(final ProcessorMetadata metadata) {
    +        Objects.requireNonNull(metadata);
    +        processorMetadata = metadata;
    +    }
    +
    +    @Override
    +    public ProcessorMetadata getProcessorMetadata() {
    +        return processorMetadata;
    +    }
     }
    \ No newline at end of file
    diff --git a/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java
    index 6f8bcd8aeb2e..f1cac2760b34 100644
    --- a/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java
    +++ b/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java
    @@ -16,6 +16,7 @@
      */
     package org.apache.kafka.test;
     
    +import java.util.Objects;
     import org.apache.kafka.common.utils.Bytes;
     import org.apache.kafka.streams.processor.CommitCallback;
     import org.apache.kafka.streams.processor.MockProcessorContext;
    @@ -26,6 +27,7 @@
     import org.apache.kafka.streams.processor.api.Record;
     import org.apache.kafka.streams.processor.api.RecordMetadata;
     import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
    +import org.apache.kafka.streams.processor.internals.ProcessorMetadata;
     import org.apache.kafka.streams.processor.internals.ProcessorNode;
     import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
     import org.apache.kafka.streams.processor.internals.RecordCollector;
    @@ -49,12 +51,15 @@ public class MockInternalProcessorContext extends MockProcessorContext implement
         private RecordCollector recordCollector;
         private long currentSystemTimeMs;
         private TaskType taskType = TaskType.ACTIVE;
    +    private ProcessorMetadata processorMetadata;
     
         public MockInternalProcessorContext() {
    +        processorMetadata = new ProcessorMetadata();
         }
     
         public MockInternalProcessorContext(final Properties config, final TaskId taskId, final File stateDir) {
             super(config, taskId, stateDir);
    +        processorMetadata = new ProcessorMetadata();
         }
     
         @Override
    @@ -181,4 +186,25 @@ public void registerCacheFlushListener(final String namespace, final DirtyEntryF
         public String changelogFor(final String storeName) {
             return "mock-changelog";
         }
    +
    +    @Override
    +    public void addProcessorMetadataKeyValue(final String key, final long value) {
    +        processorMetadata.put(key, value);
    +    }
    +
    +    @Override
    +    public Long processorMetadataForKey(final String key) {
    +        return processorMetadata.get(key);
    +    }
    +
    +    @Override
    +    public void setProcessorMetadata(final ProcessorMetadata metadata) {
    +        Objects.requireNonNull(metadata);
    +        processorMetadata = metadata;
    +    }
    +
    +    @Override
    +    public ProcessorMetadata getProcessorMetadata() {
    +        return processorMetadata;
    +    }
     }
    \ No newline at end of file
    diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java
    index a3bb87d3034d..dc2a38938a81 100644
    --- a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java
    +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java
    @@ -21,6 +21,7 @@
     import org.apache.kafka.streams.processor.ProcessorContext;
     import org.apache.kafka.streams.processor.PunctuationType;
     import org.apache.kafka.streams.processor.api.Record;
    +import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
     import org.apache.kafka.streams.state.ValueAndTimestamp;
     
     import java.util.ArrayList;
    @@ -83,4 +84,11 @@ public Cancellable scheduleCancellable() {
         public ArrayList> processed() {
             return delegate.processed();
         }
    +
    +    @SuppressWarnings("unchecked")
    +    public void addProcessorMetadata(final String key, final long value) {
    +        if (context instanceof InternalProcessorContext) {
    +            ((InternalProcessorContext) context).addProcessorMetadataKeyValue(key, value);
    +        }
    +    }
     }
    
    From eefdf9d6a7fd79a21bb9aea2df25ea642062f28c Mon Sep 17 00:00:00 2001
    From: Yang Yu 
    Date: Thu, 31 Mar 2022 12:56:07 -0500
    Subject: [PATCH 077/447] KAFKA-12875: Change Log layer segment map mutations
     to avoid absence of active segment (#11950)
    
    Reviewers: Kowshik Prakasam , Jun Rao 
    ---
     core/src/main/scala/kafka/log/LocalLog.scala  | 64 ++++++++++++++++---
     .../src/main/scala/kafka/log/UnifiedLog.scala | 20 +++---
     .../scala/unit/kafka/log/LocalLogTest.scala   | 29 +++++++++
     3 files changed, 96 insertions(+), 17 deletions(-)
    
    diff --git a/core/src/main/scala/kafka/log/LocalLog.scala b/core/src/main/scala/kafka/log/LocalLog.scala
    index 86ac672448bd..8e83070ff048 100644
    --- a/core/src/main/scala/kafka/log/LocalLog.scala
    +++ b/core/src/main/scala/kafka/log/LocalLog.scala
    @@ -315,6 +315,44 @@ class LocalLog(@volatile private var _dir: File,
         }
       }
     
    +  /**
    +   * This method deletes the given segment and creates a new segment with the given new base offset. It ensures an
    +   * active segment exists in the log at all times during this process.
    +   *
    +   * Asynchronous deletion allows reads to happen concurrently without synchronization and without the possibility of
    +   * physically deleting a file while it is being read.
    +   *
    +   * This method does not convert IOException to KafkaStorageException, the immediate caller
    +   * is expected to catch and handle IOException.
    +   *
    +   * @param newOffset The base offset of the new segment
    +   * @param segmentToDelete The old active segment to schedule for deletion
    +   * @param asyncDelete Whether the segment files should be deleted asynchronously
    +   * @param reason The reason for the segment deletion
    +   */
    +  private[log] def createAndDeleteSegment(newOffset: Long,
    +                                          segmentToDelete: LogSegment,
    +                                          asyncDelete: Boolean,
    +                                          reason: SegmentDeletionReason): LogSegment = {
    +    if (newOffset == segmentToDelete.baseOffset)
    +      segmentToDelete.changeFileSuffixes("", DeletedFileSuffix)
    +
    +    val newSegment = LogSegment.open(dir,
    +      baseOffset = newOffset,
    +      config,
    +      time = time,
    +      initFileSize = config.initFileSize,
    +      preallocate = config.preallocate)
    +    segments.add(newSegment)
    +
    +    reason.logReason(List(segmentToDelete))
    +    if (newOffset != segmentToDelete.baseOffset)
    +      segments.remove(segmentToDelete.baseOffset)
    +    LocalLog.deleteSegmentFiles(List(segmentToDelete), asyncDelete, dir, topicPartition, config, scheduler, logDirFailureChannel, logIdent)
    +
    +    newSegment
    +  }
    +
       /**
        * Given a message offset, find its corresponding offset metadata in the log.
        * If the message offset is out of range, throw an OffsetOutOfRangeException
    @@ -465,7 +503,10 @@ class LocalLog(@volatile private var _dir: File,
                 s"=max(provided offset = $expectedNextOffset, LEO = $logEndOffset) while it already " +
                 s"exists and is active with size 0. Size of time index: ${activeSegment.timeIndex.entries}," +
                 s" size of offset index: ${activeSegment.offsetIndex.entries}.")
    -          removeAndDeleteSegments(Seq(activeSegment), asyncDelete = true, LogRoll(this))
    +          val newSegment = createAndDeleteSegment(newOffset, activeSegment, asyncDelete = true, LogRoll(this))
    +          updateLogEndOffset(nextOffsetMetadata.messageOffset)
    +          info(s"Rolled new log segment at offset $newOffset in ${time.hiResClockMs() - start} ms.")
    +          return newSegment
             } else {
               throw new KafkaException(s"Trying to roll a new log segment for topic partition $topicPartition with start offset $newOffset" +
                 s" =max(provided offset = $expectedNextOffset, LEO = $logEndOffset) while it already exists. Existing " +
    @@ -517,14 +558,16 @@ class LocalLog(@volatile private var _dir: File,
           debug(s"Truncate and start at offset $newOffset")
           checkIfMemoryMappedBufferClosed()
           val segmentsToDelete = List[LogSegment]() ++ segments.values
    -      removeAndDeleteSegments(segmentsToDelete, asyncDelete = true, LogTruncation(this))
    -      segments.add(LogSegment.open(dir,
    -        baseOffset = newOffset,
    -        config = config,
    -        time = time,
    -        initFileSize = config.initFileSize,
    -        preallocate = config.preallocate))
    +
    +      if (segmentsToDelete.nonEmpty) {
    +        removeAndDeleteSegments(segmentsToDelete.dropRight(1), asyncDelete = true, LogTruncation(this))
    +        // Use createAndDeleteSegment() to create new segment first and then delete the old last segment to prevent missing
    +        // active segment during the deletion process
    +        createAndDeleteSegment(newOffset, segmentsToDelete.last, asyncDelete = true, LogTruncation(this))
    +      }
    +
           updateLogEndOffset(newOffset)
    +
           segmentsToDelete
         }
       }
    @@ -941,7 +984,10 @@ object LocalLog extends Logging {
                                           scheduler: Scheduler,
                                           logDirFailureChannel: LogDirFailureChannel,
                                           logPrefix: String): Unit = {
    -    segmentsToDelete.foreach(_.changeFileSuffixes("", DeletedFileSuffix))
    +    segmentsToDelete.foreach { segment =>
    +      if (!segment.hasSuffix(DeletedFileSuffix))
    +        segment.changeFileSuffixes("", DeletedFileSuffix)
    +    }
     
         def deleteSegments(): Unit = {
           info(s"${logPrefix}Deleting segment files ${segmentsToDelete.mkString(",")}")
    diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala
    index 3742d63f108c..7b81523cf4e1 100644
    --- a/core/src/main/scala/kafka/log/UnifiedLog.scala
    +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala
    @@ -1355,15 +1355,19 @@ class UnifiedLog(@volatile var logStartOffset: Long,
           val numToDelete = deletable.size
           if (numToDelete > 0) {
             // we must always have at least one segment, so if we are going to delete all the segments, create a new one first
    -        if (localLog.segments.numberOfSegments == numToDelete)
    -          roll()
    -        lock synchronized {
    -          localLog.checkIfMemoryMappedBufferClosed()
    -          // remove the segments for lookups
    -          localLog.removeAndDeleteSegments(deletable, asyncDelete = true, reason)
    -          deleteProducerSnapshots(deletable, asyncDelete = true)
    -          maybeIncrementLogStartOffset(localLog.segments.firstSegmentBaseOffset.get, SegmentDeletion)
    +        var segmentsToDelete = deletable
    +        if (localLog.segments.numberOfSegments == numToDelete) {
    +          val newSegment = roll()
    +          if (deletable.last.baseOffset == newSegment.baseOffset) {
    +            warn(s"Empty active segment at ${deletable.last.baseOffset} was deleted and recreated due to $reason")
    +            segmentsToDelete = deletable.dropRight(1)
    +          }
             }
    +        localLog.checkIfMemoryMappedBufferClosed()
    +        // remove the segments for lookups
    +        localLog.removeAndDeleteSegments(segmentsToDelete, asyncDelete = true, reason)
    +        deleteProducerSnapshots(deletable, asyncDelete = true)
    +        maybeIncrementLogStartOffset(localLog.segments.firstSegmentBaseOffset.get, SegmentDeletion)
           }
           numToDelete
         }
    diff --git a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala
    index 67e3a79b8557..d92f1576cd3d 100644
    --- a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala
    +++ b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala
    @@ -128,6 +128,16 @@ class LocalLogTest {
         assertTrue(logDir.exists)
       }
     
    +  @Test
    +  def testRollEmptyActiveSegment(): Unit = {
    +    val oldActiveSegment = log.segments.activeSegment
    +    log.roll()
    +    assertEquals(1, log.segments.numberOfSegments)
    +    assertNotEquals(oldActiveSegment, log.segments.activeSegment)
    +    assertFalse(logDir.listFiles.isEmpty)
    +    assertTrue(oldActiveSegment.hasSuffix(LocalLog.DeletedFileSuffix))
    +  }
    +
       @Test
       def testLogDeleteDirSuccessWhenEmptyAndFailureWhenNonEmpty(): Unit ={
         val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
    @@ -384,6 +394,24 @@ class LocalLogTest {
         assertEquals(log.segments.nonActiveLogSegmentsFrom(0L).toSeq, deletableSegments.toSeq)
       }
     
    +  @Test
    +  def testCreateAndDeleteSegment(): Unit = {
    +    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
    +    appendRecords(List(record))
    +    val newOffset = log.segments.activeSegment.baseOffset + 1
    +    val oldActiveSegment = log.segments.activeSegment
    +    val newActiveSegment = log.createAndDeleteSegment(newOffset, log.segments.activeSegment, asyncDelete = true, LogTruncation(log))
    +    assertEquals(1, log.segments.numberOfSegments)
    +    assertEquals(newActiveSegment, log.segments.activeSegment)
    +    assertNotEquals(oldActiveSegment, log.segments.activeSegment)
    +    assertTrue(oldActiveSegment.hasSuffix(LocalLog.DeletedFileSuffix))
    +    assertEquals(newOffset, log.segments.activeSegment.baseOffset)
    +    assertEquals(0L, log.recoveryPoint)
    +    assertEquals(newOffset, log.logEndOffset)
    +    val fetchDataInfo = readRecords(startOffset = newOffset)
    +    assertTrue(fetchDataInfo.records.records.asScala.isEmpty)
    +  }
    +
       @Test
       def testTruncateFullyAndStartAt(): Unit = {
         val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
    @@ -397,6 +425,7 @@ class LocalLogTest {
           appendRecords(List(record), initialOffset = offset)
         }
         assertEquals(5, log.segments.numberOfSegments)
    +    assertNotEquals(10L, log.segments.activeSegment.baseOffset)
         val expected = List[LogSegment]() ++ log.segments.values
         val deleted = log.truncateFullyAndStartAt(10L)
         assertEquals(expected, deleted)
    
    From 502f2caca4fca520a6b22191573112b3784f5e34 Mon Sep 17 00:00:00 2001
    From: dengziming 
    Date: Fri, 1 Apr 2022 11:39:50 +0800
    Subject: [PATCH 078/447] MINOR: Remove some unused codes (#11935)
    
    `validateChars` and `BaseEnum` are used in old version of clients. Remove them.
    
    Reviewers: Luke Chen 
    ---
     .../scala/kafka/admin/ConfigCommand.scala     |  5 +--
     .../main/scala/kafka/common/BaseEnum.scala    | 26 ------------
     core/src/main/scala/kafka/common/Config.scala | 41 -------------------
     3 files changed, 2 insertions(+), 70 deletions(-)
     delete mode 100644 core/src/main/scala/kafka/common/BaseEnum.scala
     delete mode 100644 core/src/main/scala/kafka/common/Config.scala
    
    diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala
    index 5e5ccefa4540..4676bfd1013a 100644
    --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala
    +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala
    @@ -22,11 +22,10 @@ import java.util.concurrent.TimeUnit
     import java.util.{Collections, Properties}
     
     import joptsimple._
    -import kafka.common.Config
     import kafka.log.LogConfig
     import kafka.server.DynamicConfig.QuotaConfigs
     import kafka.server.{ConfigEntityName, ConfigType, Defaults, DynamicBrokerConfig, DynamicConfig, KafkaConfig}
    -import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit, PasswordEncoder}
    +import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit, Logging, PasswordEncoder}
     import kafka.utils.Implicits._
     import kafka.zk.{AdminZkClient, KafkaZkClient}
     import org.apache.kafka.clients.admin.{Admin, AlterClientQuotasOptions, AlterConfigOp, AlterConfigsOptions, ConfigEntry, DescribeClusterOptions, DescribeConfigsOptions, ListTopicsOptions, ScramCredentialInfo, UserScramCredentialDeletion, UserScramCredentialUpsertion, Config => JConfig, ScramMechanism => PublicScramMechanism}
    @@ -74,7 +73,7 @@ import scala.collection._
      * For example, this allows password configs to be stored encrypted in ZK before brokers are started,
      * avoiding cleartext passwords in `server.properties`.
      */
    -object ConfigCommand extends Config {
    +object ConfigCommand extends Logging {
     
       val BrokerDefaultEntityName = ""
       val BrokerLoggerConfigType = "broker-loggers"
    diff --git a/core/src/main/scala/kafka/common/BaseEnum.scala b/core/src/main/scala/kafka/common/BaseEnum.scala
    deleted file mode 100644
    index 9c39466b7f37..000000000000
    --- a/core/src/main/scala/kafka/common/BaseEnum.scala
    +++ /dev/null
    @@ -1,26 +0,0 @@
    -/**
    - * 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 kafka.common
    -
    -/*
    - * We inherit from `Product` and `Serializable` because `case` objects and classes inherit from them and if we don't
    - * do it here, the compiler will infer types that unexpectedly include `Product` and `Serializable`, see
    - * http://underscore.io/blog/posts/2015/06/04/more-on-sealed.html for more information.
    - */
    -trait BaseEnum extends Product with Serializable {
    -  def name: String
    -}
    diff --git a/core/src/main/scala/kafka/common/Config.scala b/core/src/main/scala/kafka/common/Config.scala
    deleted file mode 100644
    index f56cca8bd052..000000000000
    --- a/core/src/main/scala/kafka/common/Config.scala
    +++ /dev/null
    @@ -1,41 +0,0 @@
    -/**
    - * 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 kafka.common
    -
    -import util.matching.Regex
    -import kafka.utils.Logging
    -import org.apache.kafka.common.errors.InvalidConfigurationException
    -
    -trait Config extends Logging {
    -
    -  def validateChars(prop: String, value: String): Unit = {
    -    val legalChars = "[a-zA-Z0-9\\._\\-]"
    -    val rgx = new Regex(legalChars + "*")
    -
    -    rgx.findFirstIn(value) match {
    -      case Some(t) =>
    -        if (!t.equals(value))
    -          throw new InvalidConfigurationException(prop + " " + value + " is illegal, contains a character other than ASCII alphanumerics, '.', '_' and '-'")
    -      case None => throw new InvalidConfigurationException(prop + " " + value + " is illegal, contains a character other than ASCII alphanumerics, '.', '_' and '-'")
    -    }
    -  }
    -}
    -
    -
    -
    -
    
    From 1bdd35d8d8d58238a493c5f5fc94fca413a7d324 Mon Sep 17 00:00:00 2001
    From: RivenSun <91005273+RivenSun2@users.noreply.github.com>
    Date: Fri, 1 Apr 2022 16:23:29 +0800
    Subject: [PATCH 079/447] KAFKA-13786: Add a note
     in`control.plane.listener.name` doc (#11978)
    
    Add a note in `control.plane.listener.name` doc to mention the value can't be identical with `inter.broker.listener.name`.
    
    Reviewers: Luke Chen 
    ---
     core/src/main/scala/kafka/server/KafkaConfig.scala | 3 ++-
     1 file changed, 2 insertions(+), 1 deletion(-)
    
    diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
    index 3282ef6c57c8..7dd9276d6da0 100755
    --- a/core/src/main/scala/kafka/server/KafkaConfig.scala
    +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
    @@ -766,7 +766,8 @@ object KafkaConfig {
         "listener.security.protocol.map = INTERNAL:PLAINTEXT, EXTERNAL:SSL, CONTROLLER:SSL\n" +
         "control.plane.listener.name = CONTROLLER\n" +
         "then controller will use \"broker1.example.com:9094\" with security protocol \"SSL\" to connect to the broker.\n" +
    -    "If not explicitly configured, the default value will be null and there will be no dedicated endpoints for controller connections."
    +    "If not explicitly configured, the default value will be null and there will be no dedicated endpoints for controller connections.\n" +
    +    s"If explicitly configured, the value cannot be the same as the value of $InterBrokerListenerNameProp."
     
       val SocketSendBufferBytesDoc = "The SO_SNDBUF buffer of the socket server sockets. If the value is -1, the OS default will be used."
       val SocketReceiveBufferBytesDoc = "The SO_RCVBUF buffer of the socket server sockets. If the value is -1, the OS default will be used."
    
    From 62ea4c46a9be7388baeaef1c505d3e5798a9066f Mon Sep 17 00:00:00 2001
    From: Colin Patrick McCabe 
    Date: Fri, 1 Apr 2022 10:50:25 -0700
    Subject: [PATCH 080/447] KAFKA-13749: CreateTopics in KRaft must return
     configs (#11941)
    
    Previously, when in KRaft mode, CreateTopics did not return the active configurations for the
    topic(s) it had just created. This PR addresses that gap. We will now return these topic
    configuration(s) when the user has DESCRIBE_CONFIGS permission. (In the case where the user does
    not have this permission, we will omit the configurations and set TopicErrorCode. We will also omit
    the number of partitions and replication factor data as well.)
    
    For historical reasons, we use different names to refer to each topic configuration when it is set
    in the broker context, as opposed to the topic context. For example, the topic configuration
    "segment.ms" corresponds to the broker configuration "log.roll.ms". Additionally, some broker
    configurations have synonyms. For example, the broker configuration "log.roll.hours" can be used to
    set the log roll time instead of "log.roll.ms". In order to track all of this, this PR adds a
    table in LogConfig.scala which maps each topic configuration to an ordered list of ConfigSynonym
    classes. (This table is then passed to KafkaConfigSchema as a constructor argument.)
    
    Some synonyms require transformations. For example, in order to convert from "log.roll.hours" to
    "segment.ms", we must convert hours to milliseconds. (Note that our assumption right now is that
    topic configurations do not have synonyms, only broker configurations. If this changes, we will
    need to add some logic to handle it.)
    
    This PR makes the 8-argument constructor for ConfigEntry public. We need this in order to make full
    use of ConfigEntry outside of the admin namespace. This change is probably inevitable in general
    since otherwise we cannot easily test the output from various admin APIs in junit tests outside the
    admin package.
    
    Testing:
    
    This PR adds PlaintextAdminIntegrationTest#testCreateTopicsReturnsConfigs. This test validates
    some of the configurations that it gets back from the call to CreateTopics, rather than just checking
    if it got back a non-empty map like some of the existing tests. In order to test the
    configuration override logic, testCreateDeleteTopics now sets up some custom static and dynamic
    configurations.
    
    In QuorumTestHarness, we now allow tests to configure what the ID of the controller should be. This
    allows us to set dynamic configurations for the controller in testCreateDeleteTopics. We will have
    a more complete fix for setting dynamic configuations on the controller later.
    
    This PR changes ConfigurationControlManager so that it is created via a Builder. This will make it
    easier to add more parameters to its constructor without having to update every piece of test code
    that uses it. It will also make the test code easier to read.
    
    Reviewers: David Arthur 
    ---
     checkstyle/import-control.xml                 |   1 +
     .../kafka/clients/admin/ConfigEntry.java      |  10 +-
     core/src/main/scala/kafka/log/LogConfig.scala | 108 +++++++++---
     .../scala/kafka/server/ControllerApis.scala   |  19 +-
     .../scala/kafka/server/ControllerServer.scala |   3 +-
     .../scala/kafka/server/KafkaRaftServer.scala  |   2 +-
     .../test/java/kafka/test/MockController.java  |  30 +++-
     .../kafka/api/BaseAdminIntegrationTest.scala  |  26 ++-
     .../api/PlaintextAdminIntegrationTest.scala   |  88 +++++++++-
     .../kafka/server/QuorumTestHarness.scala      |  20 ++-
     .../kafka/server/ControllerApisTest.scala     |  11 +-
     .../ConfigurationControlManager.java          | 101 ++++++++++-
     .../apache/kafka/controller/Controller.java   |   4 +-
     .../kafka/controller/QuorumController.java    |  35 ++--
     .../controller/ReplicationControlManager.java |  60 +++++--
     .../apache/kafka/metadata/ConfigSynonym.java  |  77 +++++++++
     .../kafka/metadata/KafkaConfigSchema.java     | 163 +++++++++++++++++-
     .../ConfigurationControlManagerTest.java      |  59 ++++---
     .../controller/QuorumControllerTest.java      |  57 +++---
     .../ReplicationControlManagerTest.java        |  37 ++--
     .../kafka/metadata/ConfigSynonymTest.java     |  47 +++++
     .../kafka/metadata/KafkaConfigSchemaTest.java | 134 +++++++++++---
     22 files changed, 899 insertions(+), 193 deletions(-)
     create mode 100644 metadata/src/main/java/org/apache/kafka/metadata/ConfigSynonym.java
     create mode 100644 metadata/src/test/java/org/apache/kafka/metadata/ConfigSynonymTest.java
    
    diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
    index 3b744979433f..070d22c14c35 100644
    --- a/checkstyle/import-control.xml
    +++ b/checkstyle/import-control.xml
    @@ -259,6 +259,7 @@
         
         
         
    +    
         
         
         
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java b/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java
    index 30686c93eaee..154fc8e65db3 100644
    --- a/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java
    @@ -61,8 +61,14 @@ public ConfigEntry(String name, String value) {
          * @param isReadOnly whether the config is read-only and cannot be updated
          * @param synonyms Synonym configs in order of precedence
          */
    -    ConfigEntry(String name, String value, ConfigSource source, boolean isSensitive, boolean isReadOnly,
    -                List synonyms, ConfigType type, String documentation) {
    +    public ConfigEntry(String name,
    +            String value,
    +            ConfigSource source,
    +            boolean isSensitive,
    +            boolean isReadOnly,
    +            List synonyms,
    +            ConfigType type,
    +            String documentation) {
             Objects.requireNonNull(name, "name should not be null");
             this.name = name;
             this.value = value;
    diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala
    index 7ad85799c55d..027e47882584 100755
    --- a/core/src/main/scala/kafka/log/LogConfig.scala
    +++ b/core/src/main/scala/kafka/log/LogConfig.scala
    @@ -27,7 +27,10 @@ import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigExceptio
     import org.apache.kafka.common.errors.InvalidConfigurationException
     import org.apache.kafka.common.record.{LegacyRecord, RecordVersion, TimestampType}
     import org.apache.kafka.common.utils.{ConfigUtils, Utils}
    +import org.apache.kafka.metadata.ConfigSynonym
    +import org.apache.kafka.metadata.ConfigSynonym.{HOURS_TO_MILLISECONDS, MINUTES_TO_MILLISECONDS}
     
    +import java.util.Arrays.asList
     import java.util.{Collections, Locale, Properties}
     import scala.annotation.nowarn
     import scala.collection.{Map, mutable}
    @@ -440,37 +443,86 @@ object LogConfig {
       }
     
       /**
    -   * Map topic config to the broker config with highest priority. Some of these have additional synonyms
    -   * that can be obtained using [[kafka.server.DynamicBrokerConfig#brokerConfigSynonyms]]
    +   * Maps topic configurations to their equivalent broker configurations.
    +   *
    +   * Topics can be configured either by setting their dynamic topic configurations, or by
    +   * setting equivalent broker configurations. For historical reasons, the equivalent broker
    +   * configurations have different names. This table maps each topic configuration to its
    +   * equivalent broker configurations.
    +   *
    +   * In some cases, the equivalent broker configurations must be transformed before they
    +   * can be used. For example, log.roll.hours must be converted to milliseconds before it
    +   * can be used as the value of segment.ms.
    +   *
    +   * The broker configurations will be used in the order specified here. In other words, if
    +   * both the first and the second synonyms are configured, we will use only the value of
    +   * the first synonym and ignore the second.
        */
       @nowarn("cat=deprecation")
    -  val TopicConfigSynonyms = Map(
    -    SegmentBytesProp -> KafkaConfig.LogSegmentBytesProp,
    -    SegmentMsProp -> KafkaConfig.LogRollTimeMillisProp,
    -    SegmentJitterMsProp -> KafkaConfig.LogRollTimeJitterMillisProp,
    -    SegmentIndexBytesProp -> KafkaConfig.LogIndexSizeMaxBytesProp,
    -    FlushMessagesProp -> KafkaConfig.LogFlushIntervalMessagesProp,
    -    FlushMsProp -> KafkaConfig.LogFlushIntervalMsProp,
    -    RetentionBytesProp -> KafkaConfig.LogRetentionBytesProp,
    -    RetentionMsProp -> KafkaConfig.LogRetentionTimeMillisProp,
    -    MaxMessageBytesProp -> KafkaConfig.MessageMaxBytesProp,
    -    IndexIntervalBytesProp -> KafkaConfig.LogIndexIntervalBytesProp,
    -    DeleteRetentionMsProp -> KafkaConfig.LogCleanerDeleteRetentionMsProp,
    -    MinCompactionLagMsProp -> KafkaConfig.LogCleanerMinCompactionLagMsProp,
    -    MaxCompactionLagMsProp -> KafkaConfig.LogCleanerMaxCompactionLagMsProp,
    -    FileDeleteDelayMsProp -> KafkaConfig.LogDeleteDelayMsProp,
    -    MinCleanableDirtyRatioProp -> KafkaConfig.LogCleanerMinCleanRatioProp,
    -    CleanupPolicyProp -> KafkaConfig.LogCleanupPolicyProp,
    -    UncleanLeaderElectionEnableProp -> KafkaConfig.UncleanLeaderElectionEnableProp,
    -    MinInSyncReplicasProp -> KafkaConfig.MinInSyncReplicasProp,
    -    CompressionTypeProp -> KafkaConfig.CompressionTypeProp,
    -    PreAllocateEnableProp -> KafkaConfig.LogPreAllocateProp,
    -    MessageFormatVersionProp -> KafkaConfig.LogMessageFormatVersionProp,
    -    MessageTimestampTypeProp -> KafkaConfig.LogMessageTimestampTypeProp,
    -    MessageTimestampDifferenceMaxMsProp -> KafkaConfig.LogMessageTimestampDifferenceMaxMsProp,
    -    MessageDownConversionEnableProp -> KafkaConfig.LogMessageDownConversionEnableProp
    -  )
    +  val AllTopicConfigSynonyms = Map(
    +    SegmentBytesProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogSegmentBytesProp)),
    +    SegmentMsProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogRollTimeMillisProp),
    +      new ConfigSynonym(KafkaConfig.LogRollTimeHoursProp, HOURS_TO_MILLISECONDS)),
    +    SegmentJitterMsProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogRollTimeJitterMillisProp),
    +      new ConfigSynonym(KafkaConfig.LogRollTimeJitterHoursProp, HOURS_TO_MILLISECONDS)),
    +    SegmentIndexBytesProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogIndexSizeMaxBytesProp)),
    +    FlushMessagesProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogFlushIntervalMessagesProp)),
    +    FlushMsProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogFlushSchedulerIntervalMsProp),
    +      new ConfigSynonym(KafkaConfig.LogFlushIntervalMsProp)),
    +    RetentionBytesProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogRetentionBytesProp)),
    +    RetentionMsProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogRetentionTimeMillisProp),
    +      new ConfigSynonym(KafkaConfig.LogRetentionTimeMinutesProp, MINUTES_TO_MILLISECONDS),
    +      new ConfigSynonym(KafkaConfig.LogRetentionTimeHoursProp, HOURS_TO_MILLISECONDS)),
    +    MaxMessageBytesProp -> asList(
    +      new ConfigSynonym(KafkaConfig.MessageMaxBytesProp)),
    +    IndexIntervalBytesProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogIndexIntervalBytesProp)),
    +    DeleteRetentionMsProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogCleanerDeleteRetentionMsProp)),
    +    MinCompactionLagMsProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogCleanerMinCompactionLagMsProp)),
    +    MaxCompactionLagMsProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogCleanerMaxCompactionLagMsProp)),
    +    FileDeleteDelayMsProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogDeleteDelayMsProp)),
    +    MinCleanableDirtyRatioProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogCleanerMinCleanRatioProp)),
    +    CleanupPolicyProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogCleanupPolicyProp)),
    +    UncleanLeaderElectionEnableProp -> asList(
    +      new ConfigSynonym(KafkaConfig.UncleanLeaderElectionEnableProp)),
    +    MinInSyncReplicasProp -> asList(
    +      new ConfigSynonym(KafkaConfig.MinInSyncReplicasProp)),
    +    CompressionTypeProp -> asList(
    +      new ConfigSynonym(KafkaConfig.CompressionTypeProp)),
    +    PreAllocateEnableProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogPreAllocateProp)),
    +    MessageFormatVersionProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogMessageFormatVersionProp)),
    +    MessageTimestampTypeProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogMessageTimestampTypeProp)),
    +    MessageTimestampDifferenceMaxMsProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp)),
    +    MessageDownConversionEnableProp -> asList(
    +      new ConfigSynonym(KafkaConfig.LogMessageDownConversionEnableProp)),
    +  ).asJava
     
    +  /**
    +   * Map topic config to the broker config with highest priority. Some of these have additional synonyms
    +   * that can be obtained using [[kafka.server.DynamicBrokerConfig#brokerConfigSynonyms]]
    +   * or using [[AllTopicConfigSynonyms]]
    +   */
    +  val TopicConfigSynonyms = AllTopicConfigSynonyms.asScala.map {
    +    case (k, v) => k -> v.get(0).name()
    +  }
     
       /**
        * Copy the subset of properties that are relevant to Logs. The individual properties
    diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala
    index c31b205530ea..8b006dc02517 100644
    --- a/core/src/main/scala/kafka/server/ControllerApis.scala
    +++ b/core/src/main/scala/kafka/server/ControllerApis.scala
    @@ -29,7 +29,7 @@ import kafka.server.QuotaFactory.QuotaManagers
     import kafka.utils.Logging
     import org.apache.kafka.clients.admin.AlterConfigOp
     import org.apache.kafka.common.Uuid.ZERO_UUID
    -import org.apache.kafka.common.acl.AclOperation.{ALTER, ALTER_CONFIGS, CLUSTER_ACTION, CREATE, DELETE, DESCRIBE}
    +import org.apache.kafka.common.acl.AclOperation.{ALTER, ALTER_CONFIGS, CLUSTER_ACTION, CREATE, DELETE, DESCRIBE, DESCRIBE_CONFIGS}
     import org.apache.kafka.common.config.ConfigResource
     import org.apache.kafka.common.errors.{ApiException, ClusterAuthorizationException, InvalidRequestException, TopicDeletionDisabledException}
     import org.apache.kafka.common.internals.FatalExitError
    @@ -316,7 +316,9 @@ class ControllerApis(val requestChannel: RequestChannel,
         val createTopicsRequest = request.body[CreateTopicsRequest]
         val future = createTopics(createTopicsRequest.data(),
             authHelper.authorize(request.context, CREATE, CLUSTER, CLUSTER_NAME, logIfDenied = false),
    -        names => authHelper.filterByAuthorized(request.context, CREATE, TOPIC, names)(identity))
    +        names => authHelper.filterByAuthorized(request.context, CREATE, TOPIC, names)(identity),
    +        names => authHelper.filterByAuthorized(request.context, DESCRIBE_CONFIGS, TOPIC,
    +            names, logIfDenied = false)(identity))
         future.whenComplete { (result, exception) =>
           requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => {
             if (exception != null) {
    @@ -329,10 +331,12 @@ class ControllerApis(val requestChannel: RequestChannel,
         }
       }
     
    -  def createTopics(request: CreateTopicsRequestData,
    -                   hasClusterAuth: Boolean,
    -                   getCreatableTopics: Iterable[String] => Set[String])
    -                   : CompletableFuture[CreateTopicsResponseData] = {
    +  def createTopics(
    +    request: CreateTopicsRequestData,
    +    hasClusterAuth: Boolean,
    +    getCreatableTopics: Iterable[String] => Set[String],
    +    getDescribableTopics: Iterable[String] => Set[String]
    +  ): CompletableFuture[CreateTopicsResponseData] = {
         val topicNames = new util.HashSet[String]()
         val duplicateTopicNames = new util.HashSet[String]()
         request.topics().forEach { topicData =>
    @@ -348,6 +352,7 @@ class ControllerApis(val requestChannel: RequestChannel,
         } else {
           getCreatableTopics.apply(topicNames.asScala)
         }
    +    val describableTopicNames = getDescribableTopics.apply(topicNames.asScala).asJava
         val effectiveRequest = request.duplicate()
         val iterator = effectiveRequest.topics().iterator()
         while (iterator.hasNext) {
    @@ -357,7 +362,7 @@ class ControllerApis(val requestChannel: RequestChannel,
             iterator.remove()
           }
         }
    -    controller.createTopics(effectiveRequest).thenApply { response =>
    +    controller.createTopics(effectiveRequest, describableTopicNames).thenApply { response =>
           duplicateTopicNames.forEach { name =>
             response.topics().add(new CreatableTopicResult().
               setName(name).
    diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala
    index 1644c9258d58..072a7721f4a9 100644
    --- a/core/src/main/scala/kafka/server/ControllerServer.scala
    +++ b/core/src/main/scala/kafka/server/ControllerServer.scala
    @@ -183,7 +183,8 @@ class ControllerServer(
               setMetrics(new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry())).
               setCreateTopicPolicy(createTopicPolicy.asJava).
               setAlterConfigPolicy(alterConfigPolicy.asJava).
    -          setConfigurationValidator(new ControllerConfigurationValidator())
    +          setConfigurationValidator(new ControllerConfigurationValidator()).
    +          setStaticConfig(config.originals)
           }
           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 5ec8d3a38aa6..d9629376fd78 100644
    --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala
    +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala
    @@ -184,5 +184,5 @@ object KafkaRaftServer {
       val configSchema = new KafkaConfigSchema(Map(
         ConfigResource.Type.BROKER -> new ConfigDef(KafkaConfig.configDef),
         ConfigResource.Type.TOPIC -> LogConfig.configDefCopy,
    -  ).asJava)
    +  ).asJava, LogConfig.AllTopicConfigSynonyms)
     }
    diff --git a/core/src/test/java/kafka/test/MockController.java b/core/src/test/java/kafka/test/MockController.java
    index acb6f90e7fdc..b4bfb0dde657 100644
    --- a/core/src/test/java/kafka/test/MockController.java
    +++ b/core/src/test/java/kafka/test/MockController.java
    @@ -60,6 +60,7 @@
     import java.util.List;
     import java.util.Map;
     import java.util.Map.Entry;
    +import java.util.Set;
     import java.util.concurrent.CompletableFuture;
     import java.util.concurrent.atomic.AtomicLong;
     
    @@ -113,7 +114,7 @@ public CompletableFuture alterPartition(AlterPartiti
     
         @Override
         synchronized public CompletableFuture
    -            createTopics(CreateTopicsRequestData request) {
    +            createTopics(CreateTopicsRequestData request, Set describable) {
             CreateTopicsResponseData response = new CreateTopicsResponseData();
             for (CreatableTopic topic : request.topics()) {
                 if (topicNameToId.containsKey(topic.name())) {
    @@ -125,13 +126,30 @@ public CompletableFuture alterPartition(AlterPartiti
                     Uuid topicUuid = new Uuid(0, topicId);
                     topicNameToId.put(topic.name(), topicUuid);
                     topics.put(topicUuid, new MockTopic(topic.name(), topicUuid));
    -                response.topics().add(new CreatableTopicResult().
    +                CreatableTopicResult creatableTopicResult = new CreatableTopicResult().
                         setName(topic.name()).
                         setErrorCode(Errors.NONE.code()).
    -                    setTopicId(topicUuid));
    -                // For a better mock, we might want to return configs, replication factor,
    -                // etc.  Right now, the tests that use MockController don't need these
    -                // things.
    +                    setTopicId(topicUuid);
    +                if (describable.contains(topic.name())) {
    +                    // Note: we don't simulate topic configs here yet.
    +                    // Just returning replication factor and numPartitions.
    +                    if (topic.assignments() != null && !topic.assignments().isEmpty()) {
    +                        creatableTopicResult.
    +                            setTopicConfigErrorCode(Errors.NONE.code()).
    +                            setReplicationFactor((short)
    +                                topic.assignments().iterator().next().brokerIds().size()).
    +                            setNumPartitions(topic.assignments().size());
    +                    } else {
    +                        creatableTopicResult.
    +                            setTopicConfigErrorCode(Errors.NONE.code()).
    +                            setReplicationFactor(topic.replicationFactor()).
    +                            setNumPartitions(topic.numPartitions());
    +                    }
    +                } else {
    +                    creatableTopicResult.
    +                        setTopicConfigErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code());
    +                }
    +                response.topics().add(creatableTopicResult);
                 }
             }
             return CompletableFuture.completedFuture(response);
    diff --git a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala
    index e3a79114964a..4f95654d541d 100644
    --- a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala
    +++ b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala
    @@ -48,12 +48,15 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg
       def brokerCount = 3
       override def logDirCount = 2
     
    +  var testInfo: TestInfo = null
    +
       var client: Admin = _
     
       @BeforeEach
       override def setUp(testInfo: TestInfo): Unit = {
    +    this.testInfo = testInfo
         super.setUp(testInfo)
    -    waitUntilBrokerMetadataIsPropagated(servers)
    +    waitUntilBrokerMetadataIsPropagated(brokers)
       }
     
       @AfterEach
    @@ -189,6 +192,15 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg
     
       override def modifyConfigs(configs: Seq[Properties]): Unit = {
         super.modifyConfigs(configs)
    +    // For testCreateTopicsReturnsConfigs, set some static broker configurations so that we can
    +    // verify that they show up in the "configs" output of CreateTopics.
    +    if (testInfo.getTestMethod.toString.contains("testCreateTopicsReturnsConfigs")) {
    +      configs.foreach(config => {
    +        config.setProperty(KafkaConfig.LogRollTimeHoursProp, "2")
    +        config.setProperty(KafkaConfig.LogRetentionTimeMinutesProp, "240")
    +        config.setProperty(KafkaConfig.LogRollTimeJitterMillisProp, "123")
    +      })
    +    }
         configs.foreach { config =>
           config.setProperty(KafkaConfig.DeleteTopicEnableProp, "true")
           config.setProperty(KafkaConfig.GroupInitialRebalanceDelayMsProp, "0")
    @@ -201,6 +213,18 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg
         }
       }
     
    +  override def kraftControllerConfigs(): Seq[Properties] = {
    +    val controllerConfig = new Properties()
    +    if (testInfo.getTestMethod.toString.contains("testCreateTopicsReturnsConfigs")) {
    +      // For testCreateTopicsReturnsConfigs, set the controller's ID to 1 so that the dynamic
    +      // config we set for node 1 will apply to it.
    +      controllerConfig.setProperty(KafkaConfig.NodeIdProp, "1")
    +    }
    +    val controllerConfigs = Seq(controllerConfig)
    +    modifyConfigs(controllerConfigs)
    +    controllerConfigs
    +  }
    +
       def createConfig: util.Map[String, Object] = {
         val config = new util.HashMap[String, Object]
         config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers())
    diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
    index fc14de187c93..b095a6170c38 100644
    --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
    +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
    @@ -33,6 +33,8 @@ import kafka.utils.TestUtils._
     import kafka.utils.{Log4jController, TestUtils}
     import kafka.zk.KafkaZkClient
     import org.apache.kafka.clients.HostResolver
    +import org.apache.kafka.clients.admin.AlterConfigOp.OpType
    +import org.apache.kafka.clients.admin.ConfigEntry.ConfigSource
     import org.apache.kafka.clients.admin._
     import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer}
     import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
    @@ -45,6 +47,8 @@ import org.apache.kafka.common.utils.{Time, Utils}
     import org.apache.kafka.common.{ConsumerGroupState, ElectionType, TopicCollection, TopicPartition, TopicPartitionInfo, TopicPartitionReplica, Uuid}
     import org.junit.jupiter.api.Assertions._
     import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Test, TestInfo}
    +import org.junit.jupiter.params.ParameterizedTest
    +import org.junit.jupiter.params.provider.ValueSource
     import org.slf4j.LoggerFactory
     
     import scala.annotation.nowarn
    @@ -74,7 +78,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
       override def setUp(testInfo: TestInfo): Unit = {
         super.setUp(testInfo)
         brokerLoggerConfigResource = new ConfigResource(
    -      ConfigResource.Type.BROKER_LOGGER, servers.head.config.brokerId.toString)
    +      ConfigResource.Type.BROKER_LOGGER, brokers.head.config.brokerId.toString)
       }
     
       @AfterEach
    @@ -2061,7 +2065,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
         // we expect the log level to be inherited from the first ancestor with a level configured
         assertEquals(kafkaLogLevel, logCleanerLogLevelConfig.value())
         assertEquals("kafka.cluster.Replica", logCleanerLogLevelConfig.name())
    -    assertEquals(ConfigEntry.ConfigSource.DYNAMIC_BROKER_LOGGER_CONFIG, logCleanerLogLevelConfig.source())
    +    assertEquals(ConfigSource.DYNAMIC_BROKER_LOGGER_CONFIG, logCleanerLogLevelConfig.source())
         assertEquals(false, logCleanerLogLevelConfig.isReadOnly)
         assertEquals(false, logCleanerLogLevelConfig.isSensitive)
         assertTrue(logCleanerLogLevelConfig.synonyms().isEmpty)
    @@ -2275,6 +2279,85 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
         }
       }
     
    +  /**
    +   * Test that createTopics returns the dynamic configurations of the topics that were created.
    +   *
    +   * Note: this test requires some custom static broker and controller configurations, which are set up in
    +   * BaseAdminIntegrationTest.modifyConfigs and BaseAdminIntegrationTest.kraftControllerConfigs.
    +   */
    +  @ParameterizedTest
    +  @ValueSource(strings = Array("zk", "kraft"))
    +  def testCreateTopicsReturnsConfigs(quorum: String): Unit = {
    +    client = Admin.create(super.createConfig)
    +
    +    val alterMap = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]
    +    alterMap.put(new ConfigResource(ConfigResource.Type.BROKER, ""), util.Arrays.asList(
    +      new AlterConfigOp(new ConfigEntry(KafkaConfig.LogRetentionTimeMillisProp, "10800000"), OpType.SET)))
    +    (brokers.map(_.config) ++ controllerServers.map(_.config)).foreach { case config =>
    +      alterMap.put(new ConfigResource(ConfigResource.Type.BROKER, config.nodeId.toString()),
    +        util.Arrays.asList(new AlterConfigOp(new ConfigEntry(
    +          KafkaConfig.LogCleanerDeleteRetentionMsProp, "34"), OpType.SET)))
    +    }
    +    client.incrementalAlterConfigs(alterMap).all().get()
    +    waitUntilTrue(() => brokers.forall(_.config.originals.getOrDefault(
    +      KafkaConfig.LogCleanerDeleteRetentionMsProp, "").toString.equals("34")),
    +      s"Timed out waiting for change to ${KafkaConfig.LogCleanerDeleteRetentionMsProp}",
    +      waitTimeMs = 60000L)
    +
    +    val newTopics = Seq(new NewTopic("foo", Map((0: Integer) -> Seq[Integer](1, 2).asJava,
    +      (1: Integer) -> Seq[Integer](2, 0).asJava).asJava).
    +      configs(Collections.singletonMap(LogConfig.IndexIntervalBytesProp, "9999999")),
    +      new NewTopic("bar", 3, 3.toShort),
    +      new NewTopic("baz", Option.empty[Integer].asJava, Option.empty[java.lang.Short].asJava)
    +    )
    +    val result = client.createTopics(newTopics.asJava)
    +    result.all.get()
    +    waitForTopics(client, newTopics.map(_.name()).toList, List())
    +
    +    assertEquals(2, result.numPartitions("foo").get())
    +    assertEquals(2, result.replicationFactor("foo").get())
    +    assertEquals(3, result.numPartitions("bar").get())
    +    assertEquals(3, result.replicationFactor("bar").get())
    +    assertEquals(configs.head.numPartitions, result.numPartitions("baz").get())
    +    assertEquals(configs.head.defaultReplicationFactor, result.replicationFactor("baz").get())
    +
    +    val topicConfigs = result.config("foo").get()
    +
    +    // From the topic configuration defaults.
    +    assertEquals(new ConfigEntry(LogConfig.CleanupPolicyProp, "delete",
    +      ConfigSource.DEFAULT_CONFIG, false, false, Collections.emptyList(), null, null),
    +      topicConfigs.get(LogConfig.CleanupPolicyProp))
    +
    +    // From dynamic cluster config via the synonym LogRetentionTimeHoursProp.
    +    assertEquals(new ConfigEntry(LogConfig.RetentionMsProp, "10800000",
    +      ConfigSource.DYNAMIC_DEFAULT_BROKER_CONFIG, false, false, Collections.emptyList(), null, null),
    +      topicConfigs.get(LogConfig.RetentionMsProp))
    +
    +    // From dynamic broker config via LogCleanerDeleteRetentionMsProp.
    +    assertEquals(new ConfigEntry(LogConfig.DeleteRetentionMsProp, "34",
    +      ConfigSource.DYNAMIC_BROKER_CONFIG, false, false, Collections.emptyList(), null, null),
    +      topicConfigs.get(LogConfig.DeleteRetentionMsProp))
    +
    +    // From static broker config by SegmentJitterMsProp.
    +    assertEquals(new ConfigEntry(LogConfig.SegmentJitterMsProp, "123",
    +      ConfigSource.STATIC_BROKER_CONFIG, false, false, Collections.emptyList(), null, null),
    +      topicConfigs.get(LogConfig.SegmentJitterMsProp))
    +
    +    // From static broker config by the synonym LogRollTimeHoursProp.
    +    val segmentMsPropType = if (isKRaftTest()) {
    +      ConfigSource.STATIC_BROKER_CONFIG
    +    } else {
    +      ConfigSource.DEFAULT_CONFIG
    +    }
    +    assertEquals(new ConfigEntry(LogConfig.SegmentMsProp, "7200000",
    +      segmentMsPropType, false, false, Collections.emptyList(), null, null),
    +      topicConfigs.get(LogConfig.SegmentMsProp))
    +
    +    // From the dynamic topic config.
    +    assertEquals(new ConfigEntry(LogConfig.IndexIntervalBytesProp, "9999999",
    +      ConfigSource.DYNAMIC_TOPIC_CONFIG, false, false, Collections.emptyList(), null, null),
    +      topicConfigs.get(LogConfig.IndexIntervalBytesProp))
    +  }
     }
     
     object PlaintextAdminIntegrationTest {
    @@ -2417,5 +2500,4 @@ object PlaintextAdminIntegrationTest {
     
         assertEquals(Defaults.CompressionType, configs.get(brokerResource).get(KafkaConfig.CompressionTypeProp).value)
       }
    -
     }
    diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
    index 5bfa651ee4c7..c329805e2c03 100755
    --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
    +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
    @@ -239,24 +239,26 @@ abstract class QuorumTestHarness extends Logging {
       }
     
       private def newKRaftQuorum(testInfo: TestInfo): KRaftQuorumImplementation = {
    -    val clusterId = Uuid.randomUuid().toString
    -    val metadataDir = TestUtils.tempDir()
    -    val metaProperties = new MetaProperties(clusterId, 0)
    -    formatDirectories(immutable.Seq(metadataDir.getAbsolutePath()), metaProperties)
    -    val controllerMetrics = new Metrics()
         val propsList = kraftControllerConfigs()
         if (propsList.size != 1) {
           throw new RuntimeException("Only one KRaft controller is supported for now.")
         }
         val props = propsList(0)
         props.setProperty(KafkaConfig.ProcessRolesProp, "controller")
    -    props.setProperty(KafkaConfig.NodeIdProp, "1000")
    +    if (props.getProperty(KafkaConfig.NodeIdProp) == null) {
    +      props.setProperty(KafkaConfig.NodeIdProp, "1000")
    +    }
    +    val nodeId = Integer.parseInt(props.getProperty(KafkaConfig.NodeIdProp))
    +    val metadataDir = TestUtils.tempDir()
    +    val metaProperties = new MetaProperties(Uuid.randomUuid().toString, nodeId)
    +    formatDirectories(immutable.Seq(metadataDir.getAbsolutePath()), metaProperties)
    +    val controllerMetrics = new Metrics()
         props.setProperty(KafkaConfig.MetadataLogDirProp, metadataDir.getAbsolutePath())
         val proto = controllerListenerSecurityProtocol.toString()
         props.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, s"CONTROLLER:${proto}")
         props.setProperty(KafkaConfig.ListenersProp, s"CONTROLLER://localhost:0")
         props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
    -    props.setProperty(KafkaConfig.QuorumVotersProp, "1000@localhost:0")
    +    props.setProperty(KafkaConfig.QuorumVotersProp, s"${nodeId}@localhost:0")
         val config = new KafkaConfig(props)
         val threadNamePrefix = "Controller_" + testInfo.getDisplayName
         val controllerQuorumVotersFuture = new CompletableFuture[util.Map[Integer, AddressSpec]]
    @@ -287,7 +289,7 @@ abstract class QuorumTestHarness extends Logging {
               error("Error completing controller socket server future", e)
               controllerQuorumVotersFuture.completeExceptionally(e)
             } else {
    -          controllerQuorumVotersFuture.complete(Collections.singletonMap(1000,
    +          controllerQuorumVotersFuture.complete(Collections.singletonMap(nodeId,
                 new InetAddressSpec(new InetSocketAddress("localhost", port))))
             }
           })
    @@ -303,7 +305,7 @@ abstract class QuorumTestHarness extends Logging {
           controllerServer,
           metadataDir,
           controllerQuorumVotersFuture,
    -      clusterId,
    +      metaProperties.clusterId,
           this)
       }
     
    diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala
    index c8ce3a160517..65580dc2be7c 100644
    --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala
    +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala
    @@ -497,6 +497,7 @@ class ControllerApisTest {
             new CreatableTopic().setName("bar").setNumPartitions(2).setReplicationFactor(3),
             new CreatableTopic().setName("bar").setNumPartitions(2).setReplicationFactor(3),
             new CreatableTopic().setName("baz").setNumPartitions(2).setReplicationFactor(3),
    +        new CreatableTopic().setName("indescribable").setNumPartitions(2).setReplicationFactor(3),
             new CreatableTopic().setName("quux").setNumPartitions(2).setReplicationFactor(3),
         ).iterator()))
         val expectedResponse = Set(new CreatableTopicResult().setName("foo").
    @@ -507,11 +508,19 @@ class ControllerApisTest {
             setErrorMessage("Duplicate topic name."),
           new CreatableTopicResult().setName("baz").
             setErrorCode(NONE.code()).
    -        setTopicId(new Uuid(0L, 1L)),
    +        setTopicId(new Uuid(0L, 1L)).
    +        setNumPartitions(2).
    +        setReplicationFactor(3).
    +        setTopicConfigErrorCode(NONE.code()),
    +      new CreatableTopicResult().setName("indescribable").
    +        setErrorCode(NONE.code()).
    +        setTopicId(new Uuid(0L, 2L)).
    +        setTopicConfigErrorCode(TOPIC_AUTHORIZATION_FAILED.code()),
           new CreatableTopicResult().setName("quux").
             setErrorCode(TOPIC_AUTHORIZATION_FAILED.code()))
         assertEquals(expectedResponse, controllerApis.createTopics(request,
           false,
    +      _ => Set("baz", "indescribable"),
           _ => Set("baz")).get().topics().asScala.toSet)
       }
     
    diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
    index a16361343b23..9c04e2665608 100644
    --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
    +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
    @@ -18,6 +18,7 @@
     package org.apache.kafka.controller;
     
     import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
    +import org.apache.kafka.clients.admin.ConfigEntry;
     import org.apache.kafka.common.config.ConfigException;
     import org.apache.kafka.common.config.ConfigResource.Type;
     import org.apache.kafka.common.config.ConfigResource;
    @@ -51,6 +52,8 @@
     
     
     public class ConfigurationControlManager {
    +    public static final ConfigResource DEFAULT_NODE = new ConfigResource(Type.BROKER, "");
    +
         private final Logger log;
         private final SnapshotRegistry snapshotRegistry;
         private final KafkaConfigSchema configSchema;
    @@ -58,13 +61,82 @@ public class ConfigurationControlManager {
         private final Optional alterConfigPolicy;
         private final ConfigurationValidator validator;
         private final TimelineHashMap> configData;
    +    private final Map staticConfig;
    +    private final ConfigResource currentController;
    +
    +    static class Builder {
    +        private LogContext logContext = null;
    +        private SnapshotRegistry snapshotRegistry = null;
    +        private KafkaConfigSchema configSchema = KafkaConfigSchema.EMPTY;
    +        private Consumer existenceChecker = __ -> { };
    +        private Optional alterConfigPolicy = Optional.empty();
    +        private ConfigurationValidator validator = ConfigurationValidator.NO_OP;
    +        private Map staticConfig = Collections.emptyMap();
    +        private int nodeId = 0;
    +
    +        Builder setLogContext(LogContext logContext) {
    +            this.logContext = logContext;
    +            return this;
    +        }
    +
    +        Builder setSnapshotRegistry(SnapshotRegistry snapshotRegistry) {
    +            this.snapshotRegistry = snapshotRegistry;
    +            return this;
    +        }
    +
    +        Builder setKafkaConfigSchema(KafkaConfigSchema configSchema) {
    +            this.configSchema = configSchema;
    +            return this;
    +        }
    +
    +        Builder setExistenceChecker(Consumer existenceChecker) {
    +            this.existenceChecker = existenceChecker;
    +            return this;
    +        }
    +
    +        Builder setAlterConfigPolicy(Optional alterConfigPolicy) {
    +            this.alterConfigPolicy = alterConfigPolicy;
    +            return this;
    +        }
    +
    +        Builder setValidator(ConfigurationValidator validator) {
    +            this.validator = validator;
    +            return this;
    +        }
     
    -    ConfigurationControlManager(LogContext logContext,
    -                                SnapshotRegistry snapshotRegistry,
    -                                KafkaConfigSchema configSchema,
    -                                Consumer existenceChecker,
    -                                Optional alterConfigPolicy,
    -                                ConfigurationValidator validator) {
    +        Builder setStaticConfig(Map staticConfig) {
    +            this.staticConfig = staticConfig;
    +            return this;
    +        }
    +
    +        Builder setNodeId(int nodeId) {
    +            this.nodeId = nodeId;
    +            return this;
    +        }
    +
    +        ConfigurationControlManager build() {
    +            if (logContext == null) logContext = new LogContext();
    +            if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext);
    +            return new ConfigurationControlManager(
    +                logContext,
    +                snapshotRegistry,
    +                configSchema,
    +                existenceChecker,
    +                alterConfigPolicy,
    +                validator,
    +                staticConfig,
    +                nodeId);
    +        }
    +    }
    +
    +    private ConfigurationControlManager(LogContext logContext,
    +            SnapshotRegistry snapshotRegistry,
    +            KafkaConfigSchema configSchema,
    +            Consumer existenceChecker,
    +            Optional alterConfigPolicy,
    +            ConfigurationValidator validator,
    +            Map staticConfig,
    +            int nodeId) {
             this.log = logContext.logger(ConfigurationControlManager.class);
             this.snapshotRegistry = snapshotRegistry;
             this.configSchema = configSchema;
    @@ -72,6 +144,8 @@ public class ConfigurationControlManager {
             this.alterConfigPolicy = alterConfigPolicy;
             this.validator = validator;
             this.configData = new TimelineHashMap<>(snapshotRegistry, 0);
    +        this.staticConfig = Collections.unmodifiableMap(new HashMap<>(staticConfig));
    +        this.currentController = new ConfigResource(Type.BROKER, Integer.toString(nodeId));
         }
     
         /**
    @@ -355,6 +429,21 @@ boolean uncleanLeaderElectionEnabledForTopic(String name) {
             return false; // TODO: support configuring unclean leader election.
         }
     
    +    Map computeEffectiveTopicConfigs(Map creationConfigs) {
    +        return configSchema.resolveEffectiveTopicConfigs(staticConfig, clusterConfig(),
    +            currentControllerConfig(), creationConfigs);
    +    }
    +
    +    Map clusterConfig() {
    +        Map result = configData.get(DEFAULT_NODE);
    +        return (result == null) ? Collections.emptyMap() : result;
    +    }
    +
    +    Map currentControllerConfig() {
    +        Map result = configData.get(currentController);
    +        return (result == null) ? Collections.emptyMap() : result;
    +    }
    +
         class ConfigurationControlIterator implements Iterator> {
             private final long epoch;
             private final Iterator>> iterator;
    diff --git a/metadata/src/main/java/org/apache/kafka/controller/Controller.java b/metadata/src/main/java/org/apache/kafka/controller/Controller.java
    index 3d91f67b349e..c5fdefffbf83 100644
    --- a/metadata/src/main/java/org/apache/kafka/controller/Controller.java
    +++ b/metadata/src/main/java/org/apache/kafka/controller/Controller.java
    @@ -47,6 +47,7 @@
     import java.util.Collection;
     import java.util.List;
     import java.util.Map;
    +import java.util.Set;
     import java.util.concurrent.CompletableFuture;
     
     
    @@ -64,11 +65,12 @@ public interface Controller extends AclMutator, AutoCloseable {
          * Create a batch of topics.
          *
          * @param request       The CreateTopicsRequest data.
    +     * @param describable   The topics which we have DESCRIBE permission on.
          *
          * @return              A future yielding the response.
          */
         CompletableFuture
    -        createTopics(CreateTopicsRequestData request);
    +        createTopics(CreateTopicsRequestData request, Set describable);
     
         /**
          * Unregister a broker.
    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 966cfa1dd3ea..7d3c55bd6905 100644
    --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
    +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
    @@ -101,6 +101,7 @@
     import java.util.OptionalInt;
     import java.util.OptionalLong;
     import java.util.Random;
    +import java.util.Set;
     import java.util.concurrent.CompletableFuture;
     import java.util.concurrent.CountDownLatch;
     import java.util.concurrent.RejectedExecutionException;
    @@ -140,7 +141,7 @@ static public class Builder {
             private Time time = Time.SYSTEM;
             private String threadNamePrefix = null;
             private LogContext logContext = null;
    -        private KafkaConfigSchema configSchema = new KafkaConfigSchema(Collections.emptyMap());
    +        private KafkaConfigSchema configSchema = KafkaConfigSchema.EMPTY;
             private RaftClient raftClient = null;
             private Map supportedFeatures = Collections.emptyMap();
             private short defaultReplicationFactor = 3;
    @@ -155,6 +156,7 @@ static public class Builder {
             private Optional alterConfigPolicy = Optional.empty();
             private ConfigurationValidator configurationValidator = ConfigurationValidator.NO_OP;
             private Optional authorizer = Optional.empty();
    +        private Map staticConfig = Collections.emptyMap();
     
             public Builder(int nodeId, String clusterId) {
                 this.nodeId = nodeId;
    @@ -251,6 +253,11 @@ public Builder setAuthorizer(ClusterMetadataAuthorizer authorizer) {
                 return this;
             }
     
    +        public Builder setStaticConfig(Map staticConfig) {
    +            this.staticConfig = staticConfig;
    +            return this;
    +        }
    +
             @SuppressWarnings("unchecked")
             public QuorumController build() throws Exception {
                 if (raftClient == null) {
    @@ -273,7 +280,8 @@ public QuorumController build() throws Exception {
                         configSchema, raftClient, supportedFeatures, defaultReplicationFactor,
                         defaultNumPartitions, isLeaderRecoverySupported, replicaPlacer, snapshotMaxNewRecordBytes,
                         leaderImbalanceCheckIntervalNs, sessionTimeoutNs, controllerMetrics,
    -                    createTopicPolicy, alterConfigPolicy, configurationValidator, authorizer);
    +                    createTopicPolicy, alterConfigPolicy, configurationValidator, authorizer,
    +                    staticConfig);
                 } catch (Exception e) {
                     Utils.closeQuietly(queue, "event queue");
                     throw e;
    @@ -1316,7 +1324,8 @@ private QuorumController(LogContext logContext,
                                  Optional createTopicPolicy,
                                  Optional alterConfigPolicy,
                                  ConfigurationValidator configurationValidator,
    -                             Optional authorizer) {
    +                             Optional authorizer,
    +                             Map staticConfig) {
             this.logContext = logContext;
             this.log = logContext.logger(QuorumController.class);
             this.nodeId = nodeId;
    @@ -1327,12 +1336,16 @@ private QuorumController(LogContext logContext,
             this.snapshotRegistry = new SnapshotRegistry(logContext);
             this.purgatory = new ControllerPurgatory();
             this.resourceExists = new ConfigResourceExistenceChecker();
    -        this.configurationControl = new ConfigurationControlManager(logContext,
    -            snapshotRegistry,
    -            configSchema,
    -            resourceExists,
    -            alterConfigPolicy,
    -            configurationValidator);
    +        this.configurationControl = new ConfigurationControlManager.Builder().
    +            setLogContext(logContext).
    +            setSnapshotRegistry(snapshotRegistry).
    +            setKafkaConfigSchema(configSchema).
    +            setExistenceChecker(resourceExists).
    +            setAlterConfigPolicy(alterConfigPolicy).
    +            setValidator(configurationValidator).
    +            setStaticConfig(staticConfig).
    +            setNodeId(nodeId).
    +            build();
             this.clientQuotaControlManager = new ClientQuotaControlManager(snapshotRegistry);
             this.clusterControl = new ClusterControlManager(logContext, clusterId, time,
                 snapshotRegistry, sessionTimeoutNs, replicaPlacer, controllerMetrics);
    @@ -1367,13 +1380,13 @@ public CompletableFuture alterPartition(AlterPartiti
     
         @Override
         public CompletableFuture
    -            createTopics(CreateTopicsRequestData request) {
    +            createTopics(CreateTopicsRequestData request, Set describable) {
             if (request.topics().isEmpty()) {
                 return CompletableFuture.completedFuture(new CreateTopicsResponseData());
             }
             return appendWriteEvent("createTopics",
                 time.nanoseconds() + NANOSECONDS.convert(request.timeoutMs(), MILLISECONDS),
    -            () -> replicationControl.createTopics(request));
    +            () -> replicationControl.createTopics(request, describable));
         }
     
         @Override
    diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
    index edf627eb51e3..d8005e60ce64 100644
    --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
    +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
    @@ -18,6 +18,7 @@
     package org.apache.kafka.controller;
     
     import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
    +import org.apache.kafka.clients.admin.ConfigEntry;
     import org.apache.kafka.common.ElectionType;
     import org.apache.kafka.common.Uuid;
     import org.apache.kafka.common.config.ConfigResource;
    @@ -50,6 +51,7 @@
     import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
     import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
     import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
    +import org.apache.kafka.common.message.CreateTopicsRequestData.CreateableTopicConfigCollection;
     import org.apache.kafka.common.message.CreateTopicsResponseData;
     import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
     import org.apache.kafka.common.message.ElectLeadersRequestData;
    @@ -71,6 +73,7 @@
     import org.apache.kafka.common.protocol.Errors;
     import org.apache.kafka.common.requests.ApiError;
     import org.apache.kafka.common.utils.LogContext;
    +import org.apache.kafka.metadata.KafkaConfigSchema;
     import org.apache.kafka.server.common.ApiMessageAndVersion;
     import org.apache.kafka.metadata.BrokerHeartbeatReply;
     import org.apache.kafka.metadata.BrokerRegistration;
    @@ -114,7 +117,9 @@
     import static org.apache.kafka.common.protocol.Errors.FENCED_LEADER_EPOCH;
     import static org.apache.kafka.common.protocol.Errors.INVALID_REQUEST;
     import static org.apache.kafka.common.protocol.Errors.INVALID_UPDATE_VERSION;
    +import static org.apache.kafka.common.protocol.Errors.NONE;
     import static org.apache.kafka.common.protocol.Errors.NO_REASSIGNMENT_IN_PROGRESS;
    +import static org.apache.kafka.common.protocol.Errors.TOPIC_AUTHORIZATION_FAILED;
     import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_ID;
     import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_OR_PARTITION;
     import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER;
    @@ -127,7 +132,6 @@
      * of each partition, as well as administrative tasks like creating or deleting topics.
      */
     public class ReplicationControlManager {
    -
         static class TopicControlInfo {
             private final String name;
             private final Uuid id;
    @@ -148,6 +152,15 @@ public Uuid topicId() {
             }
         }
     
    +    /**
    +     * Translate a CreateableTopicConfigCollection to a map from string to string.
    +     */
    +    static Map translateCreationConfigs(CreateableTopicConfigCollection collection) {
    +        HashMap result = new HashMap<>();
    +        collection.forEach(config -> result.put(config.name(), config.value()));
    +        return Collections.unmodifiableMap(result);
    +    }
    +
         private final SnapshotRegistry snapshotRegistry;
         private final Logger log;
     
    @@ -389,7 +402,7 @@ public void replay(RemoveTopicRecord record) {
         }
     
         ControllerResult
    -            createTopics(CreateTopicsRequestData request) {
    +            createTopics(CreateTopicsRequestData request, Set describable) {
             Map topicErrors = new HashMap<>();
             List records = new ArrayList<>();
     
    @@ -420,7 +433,7 @@ public void replay(RemoveTopicRecord record) {
                 if (topicErrors.containsKey(topic.name())) continue;
                 ApiError error;
                 try {
    -                error = createTopic(topic, records, successes);
    +                error = createTopic(topic, records, successes, describable.contains(topic.name()));
                 } catch (ApiException e) {
                     error = ApiError.fromThrowable(e);
                 }
    @@ -462,7 +475,9 @@ public void replay(RemoveTopicRecord record) {
     
         private ApiError createTopic(CreatableTopic topic,
                                      List records,
    -                                 Map successes) {
    +                                 Map successes,
    +                                 boolean authorizedToReturnConfigs) {
    +        Map creationConfigs = translateCreationConfigs(topic.configs());
             Map newParts = new HashMap<>();
             if (!topic.assignments().isEmpty()) {
                 if (topic.replicationFactor() != -1) {
    @@ -499,10 +514,8 @@ private ApiError createTopic(CreatableTopic topic,
                     Map> assignments = new HashMap<>();
                     newParts.entrySet().forEach(e -> assignments.put(e.getKey(),
                         Replicas.toList(e.getValue().replicas)));
    -                Map configs = new HashMap<>();
    -                topic.configs().forEach(config -> configs.put(config.name(), config.value()));
                     return new CreateTopicPolicy.RequestMetadata(
    -                    topic.name(), null, null, assignments, configs);
    +                    topic.name(), null, null, assignments, creationConfigs);
                 });
                 if (error.isFailure()) return error;
             } else if (topic.replicationFactor() < -1 || topic.replicationFactor() == 0) {
    @@ -530,21 +543,38 @@ private ApiError createTopic(CreatableTopic topic,
                             " time(s): " + e.getMessage());
                 }
                 ApiError error = maybeCheckCreateTopicPolicy(() -> {
    -                Map configs = new HashMap<>();
    -                topic.configs().forEach(config -> configs.put(config.name(), config.value()));
                     return new CreateTopicPolicy.RequestMetadata(
    -                    topic.name(), numPartitions, replicationFactor, null, configs);
    +                    topic.name(), numPartitions, replicationFactor, null, creationConfigs);
                 });
                 if (error.isFailure()) return error;
             }
             Uuid topicId = Uuid.randomUuid();
    -        successes.put(topic.name(), new CreatableTopicResult().
    +        CreatableTopicResult result = new CreatableTopicResult().
                 setName(topic.name()).
                 setTopicId(topicId).
    -            setErrorCode((short) 0).
    -            setErrorMessage(null).
    -            setNumPartitions(newParts.size()).
    -            setReplicationFactor((short) newParts.get(0).replicas.length));
    +            setErrorCode(NONE.code()).
    +            setErrorMessage(null);
    +        if (authorizedToReturnConfigs) {
    +            Map effectiveConfig = configurationControl.
    +                computeEffectiveTopicConfigs(creationConfigs);
    +            List configNames = new ArrayList<>(effectiveConfig.keySet());
    +            configNames.sort(String::compareTo);
    +            for (String configName : configNames) {
    +                ConfigEntry entry = effectiveConfig.get(configName);
    +                result.configs().add(new CreateTopicsResponseData.CreatableTopicConfigs().
    +                    setName(entry.name()).
    +                    setValue(entry.isSensitive() ? null : entry.value()).
    +                    setReadOnly(entry.isReadOnly()).
    +                    setConfigSource(KafkaConfigSchema.translateConfigSource(entry.source()).id()).
    +                    setIsSensitive(entry.isSensitive()));
    +            }
    +            result.setNumPartitions(newParts.size());
    +            result.setReplicationFactor((short) newParts.get(0).replicas.length);
    +            result.setTopicConfigErrorCode(NONE.code());
    +        } else {
    +            result.setTopicConfigErrorCode(TOPIC_AUTHORIZATION_FAILED.code());
    +        }
    +        successes.put(topic.name(), result);
             records.add(new ApiMessageAndVersion(new TopicRecord().
                 setName(topic.name()).
                 setTopicId(topicId), TOPIC_RECORD.highestSupportedVersion()));
    diff --git a/metadata/src/main/java/org/apache/kafka/metadata/ConfigSynonym.java b/metadata/src/main/java/org/apache/kafka/metadata/ConfigSynonym.java
    new file mode 100644
    index 000000000000..d331a476c92c
    --- /dev/null
    +++ b/metadata/src/main/java/org/apache/kafka/metadata/ConfigSynonym.java
    @@ -0,0 +1,77 @@
    +/*
    + * 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.metadata;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.concurrent.TimeUnit;
    +import java.util.function.Function;
    +
    +
    +/**
    + * Represents a synonym for a configuration plus a conversion function. The conversion
    + * function is necessary for cases where the synonym is denominated in different units
    + * (hours versus milliseconds, etc.)
    + */
    +public class ConfigSynonym {
    +    private static final Logger log = LoggerFactory.getLogger(ConfigSynonym.class);
    +
    +    public static final Function HOURS_TO_MILLISECONDS = input -> {
    +        int hours = valueToInt(input, 0, "hoursToMilliseconds");
    +        return String.valueOf(TimeUnit.MILLISECONDS.convert(hours, TimeUnit.HOURS));
    +    };
    +
    +    public static final Function MINUTES_TO_MILLISECONDS = input -> {
    +        int hours = valueToInt(input, 0, "minutesToMilliseconds");
    +        return String.valueOf(TimeUnit.MILLISECONDS.convert(hours, TimeUnit.MINUTES));
    +    };
    +
    +    private static int valueToInt(String input, int defaultValue, String what) {
    +        if (input == null) return defaultValue;
    +        String trimmedInput = input.trim();
    +        if (trimmedInput.isEmpty()) {
    +            return defaultValue;
    +        }
    +        try {
    +            return Integer.parseInt(trimmedInput);
    +        } catch (Exception e) {
    +            log.error("{} failed: unable to parse '{}' as an integer.", what, trimmedInput, e);
    +            return defaultValue;
    +        }
    +    }
    +
    +    private final String name;
    +    private final Function converter;
    +
    +    public ConfigSynonym(String name, Function converter) {
    +        this.name = name;
    +        this.converter = converter;
    +    }
    +
    +    public ConfigSynonym(String name) {
    +        this(name, Function.identity());
    +    }
    +
    +    public String name() {
    +        return name;
    +    }
    +
    +    public Function converter() {
    +        return converter;
    +    }
    +}
    diff --git a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java
    index 88bd5e9bfc14..d15d6623e38b 100644
    --- a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java
    +++ b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java
    @@ -17,12 +17,21 @@
     
     package org.apache.kafka.metadata;
     
    +import org.apache.kafka.clients.admin.ConfigEntry;
    +import org.apache.kafka.clients.admin.ConfigEntry.ConfigSource;
     import org.apache.kafka.common.config.ConfigDef;
     import org.apache.kafka.common.config.ConfigResource;
    +import org.apache.kafka.common.config.types.Password;
     import org.apache.kafka.common.metadata.ConfigRecord;
    +import org.apache.kafka.common.requests.DescribeConfigsResponse;
     
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
     import java.util.Map;
    +import java.util.function.Function;
     
    +import static java.util.Collections.emptyList;
     import static java.util.Collections.emptyMap;
     
     
    @@ -32,12 +41,75 @@
      * determining the type of config keys (string, int, password, etc.)
      */
     public class KafkaConfigSchema {
    -    public static final KafkaConfigSchema EMPTY = new KafkaConfigSchema(emptyMap());
    +    public static final KafkaConfigSchema EMPTY = new KafkaConfigSchema(emptyMap(), emptyMap());
    +
    +    private static final ConfigDef EMPTY_CONFIG_DEF = new ConfigDef();
    +
    +    /**
    +     * Translate a ConfigDef.Type to its equivalent for ConfigEntry.ConfigType.
    +     *
    +     * We do not want this code in ConfigEntry, since that is a public-facing API. On the
    +     * other hand, putting this code in ConfigDef.Type would introduce an unwanted dependency
    +     * from org.apache.kafka.common.config to org.apache.kafka.clients.admin. So it
    +     * makes sense to put it here.
    +     */
    +    public static ConfigEntry.ConfigType translateConfigType(ConfigDef.Type type) {
    +        switch (type) {
    +            case BOOLEAN:
    +                return ConfigEntry.ConfigType.BOOLEAN;
    +            case STRING:
    +                return ConfigEntry.ConfigType.STRING;
    +            case INT:
    +                return ConfigEntry.ConfigType.INT;
    +            case SHORT:
    +                return ConfigEntry.ConfigType.SHORT;
    +            case LONG:
    +                return ConfigEntry.ConfigType.LONG;
    +            case DOUBLE:
    +                return ConfigEntry.ConfigType.DOUBLE;
    +            case LIST:
    +                return ConfigEntry.ConfigType.LIST;
    +            case CLASS:
    +                return ConfigEntry.ConfigType.CLASS;
    +            case PASSWORD:
    +                return ConfigEntry.ConfigType.PASSWORD;
    +            default:
    +                return ConfigEntry.ConfigType.UNKNOWN;
    +        }
    +    }
    +
    +    private static final Map TRANSLATE_CONFIG_SOURCE_MAP;
    +
    +    static {
    +        Map map = new HashMap<>();
    +        for (DescribeConfigsResponse.ConfigSource source : DescribeConfigsResponse.ConfigSource.values()) {
    +            map.put(source.source(), source);
    +        }
    +        TRANSLATE_CONFIG_SOURCE_MAP = Collections.unmodifiableMap(map);
    +    }
    +
    +    /**
    +     * Translate a ConfigEntry.ConfigSource enum to its equivalent for DescribeConfigsResponse.
    +     *
    +     * We do not want this code in ConfigEntry, since that is a public-facing API. On the
    +     * other hand, putting this code in DescribeConfigsResponse would introduce an unwanted
    +     * dependency from org.apache.kafka.common.requests to org.apache.kafka.clients.admin.
    +     * So it makes sense to put it here.
    +     */
    +    public static DescribeConfigsResponse.ConfigSource translateConfigSource(ConfigEntry.ConfigSource configSource) {
    +        DescribeConfigsResponse.ConfigSource result = TRANSLATE_CONFIG_SOURCE_MAP.get(configSource);
    +        if (result != null) return result;
    +        return DescribeConfigsResponse.ConfigSource.UNKNOWN;
    +    }
     
         private final Map configDefs;
     
    -    public KafkaConfigSchema(Map configDefs) {
    +    private final Map> logConfigSynonyms;
    +
    +    public KafkaConfigSchema(Map configDefs,
    +                             Map> logConfigSynonyms) {
             this.configDefs = configDefs;
    +        this.logConfigSynonyms = logConfigSynonyms;
         }
     
         /**
    @@ -84,4 +156,91 @@ public String getDefault(ConfigResource.Type type, String key) {
             }
             return ConfigDef.convertToString(configKey.defaultValue, configKey.type);
         }
    +
    +    public Map resolveEffectiveTopicConfigs(
    +            Map staticNodeConfig,
    +            Map dynamicClusterConfigs,
    +            Map dynamicNodeConfigs,
    +            Map dynamicTopicConfigs) {
    +        ConfigDef configDef = configDefs.getOrDefault(ConfigResource.Type.TOPIC, EMPTY_CONFIG_DEF);
    +        HashMap effectiveConfigs = new HashMap<>();
    +        for (ConfigDef.ConfigKey configKey : configDef.configKeys().values()) {
    +            ConfigEntry entry = resolveEffectiveTopicConfig(configKey, staticNodeConfig,
    +                dynamicClusterConfigs, dynamicNodeConfigs, dynamicTopicConfigs);
    +            effectiveConfigs.put(entry.name(), entry);
    +        }
    +        return effectiveConfigs;
    +    }
    +
    +    private ConfigEntry resolveEffectiveTopicConfig(ConfigDef.ConfigKey configKey,
    +            Map staticNodeConfig,
    +            Map dynamicClusterConfigs,
    +            Map dynamicNodeConfigs,
    +            Map dynamicTopicConfigs) {
    +        if (dynamicTopicConfigs.containsKey(configKey.name)) {
    +            return toConfigEntry(configKey,
    +                dynamicTopicConfigs.get(configKey.name),
    +                ConfigSource.DYNAMIC_TOPIC_CONFIG, Function.identity());
    +        }
    +        List synonyms = logConfigSynonyms.getOrDefault(configKey.name, emptyList());
    +        for (ConfigSynonym synonym : synonyms) {
    +            if (dynamicNodeConfigs.containsKey(synonym.name())) {
    +                return toConfigEntry(configKey, dynamicNodeConfigs.get(synonym.name()),
    +                    ConfigSource.DYNAMIC_BROKER_CONFIG, synonym.converter());
    +            }
    +        }
    +        for (ConfigSynonym synonym : synonyms) {
    +            if (dynamicClusterConfigs.containsKey(synonym.name())) {
    +                return toConfigEntry(configKey, dynamicClusterConfigs.get(synonym.name()),
    +                    ConfigSource.DYNAMIC_DEFAULT_BROKER_CONFIG, synonym.converter());
    +            }
    +        }
    +        for (ConfigSynonym synonym : synonyms) {
    +            if (staticNodeConfig.containsKey(synonym.name())) {
    +                return toConfigEntry(configKey, staticNodeConfig.get(synonym.name()),
    +                    ConfigSource.STATIC_BROKER_CONFIG, synonym.converter());
    +            }
    +        }
    +        return toConfigEntry(configKey, configKey.hasDefault() ? configKey.defaultValue : null,
    +            ConfigSource.DEFAULT_CONFIG, Function.identity());
    +    }
    +
    +    private ConfigEntry toConfigEntry(ConfigDef.ConfigKey configKey,
    +                                      Object value,
    +                                      ConfigSource source,
    +                                      Function converter) {
    +        // Convert the value into a nulllable string suitable for storing in ConfigEntry.
    +        String stringValue = null;
    +        if (value != null) {
    +            if (value instanceof String) {
    +                // The value may already be a string if it's coming from a Map.
    +                // Then it doesn't need to be converted.
    +                stringValue = (String) value;
    +            } else if (value instanceof Password) {
    +                // We want the actual value here, not [hidden], which is what we'd get
    +                // from Password#toString. While we don't return sensitive config values
    +                // over the wire to users, we may need the real value internally.
    +                stringValue = ((Password) value).value();
    +            } else {
    +                try {
    +                    // Use the ConfigDef function here which will handle List, Class, etc.
    +                    stringValue = ConfigDef.convertToString(value, configKey.type);
    +                } catch (Exception e) {
    +                    throw new RuntimeException("Unable to convert " + configKey.name + " to string.", e);
    +                }
    +            }
    +        }
    +        if (stringValue != null) {
    +            stringValue = converter.apply(stringValue);
    +        }
    +        return new ConfigEntry(
    +            configKey.name,
    +            stringValue,
    +            source,
    +            configKey.type().isSensitive(),
    +            false, // "readonly" is always false, for now.
    +            emptyList(), // we don't populate synonyms, for now.
    +            translateConfigType(configKey.type()),
    +            configKey.documentation);
    +    }
     }
    diff --git a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java
    index 6a71aba1e679..1ba9591e6a2b 100644
    --- a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java
    +++ b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java
    @@ -24,15 +24,15 @@
     import org.apache.kafka.common.metadata.ConfigRecord;
     import org.apache.kafka.common.protocol.Errors;
     import org.apache.kafka.common.requests.ApiError;
    -import org.apache.kafka.common.utils.LogContext;
    +import org.apache.kafka.metadata.ConfigSynonym;
     import org.apache.kafka.metadata.KafkaConfigSchema;
     import org.apache.kafka.metadata.RecordTestUtils;
     import org.apache.kafka.server.common.ApiMessageAndVersion;
     import org.apache.kafka.server.policy.AlterConfigPolicy;
     import org.apache.kafka.server.policy.AlterConfigPolicy.RequestMetadata;
    -import org.apache.kafka.timeline.SnapshotRegistry;
     
     import java.util.AbstractMap.SimpleImmutableEntry;
    +import java.util.Arrays;
     import java.util.Collections;
     import java.util.HashMap;
     import java.util.LinkedHashMap;
    @@ -53,6 +53,7 @@
     import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SUBTRACT;
     import static org.apache.kafka.common.config.ConfigResource.Type.BROKER;
     import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
    +import static org.apache.kafka.metadata.ConfigSynonym.HOURS_TO_MILLISECONDS;
     import static org.junit.jupiter.api.Assertions.assertEquals;
     
     
    @@ -69,10 +70,19 @@ public class ConfigurationControlManagerTest {
             CONFIGS.put(TOPIC, new ConfigDef().
                 define("abc", ConfigDef.Type.LIST, ConfigDef.Importance.HIGH, "abc").
                 define("def", ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, "def").
    -            define("ghi", ConfigDef.Type.BOOLEAN, true, ConfigDef.Importance.HIGH, "ghi"));
    +            define("ghi", ConfigDef.Type.BOOLEAN, true, ConfigDef.Importance.HIGH, "ghi").
    +            define("quuux", ConfigDef.Type.LONG, ConfigDef.Importance.HIGH, "quux"));
         }
     
    -    static final KafkaConfigSchema SCHEMA = new KafkaConfigSchema(CONFIGS);
    +    public static final Map> SYNONYMS = new HashMap<>();
    +
    +    static {
    +        SYNONYMS.put("abc", Arrays.asList(new ConfigSynonym("foo.bar")));
    +        SYNONYMS.put("def", Arrays.asList(new ConfigSynonym("baz")));
    +        SYNONYMS.put("quuux", Arrays.asList(new ConfigSynonym("quux", HOURS_TO_MILLISECONDS)));
    +    }
    +
    +    static final KafkaConfigSchema SCHEMA = new KafkaConfigSchema(CONFIGS, SYNONYMS);
     
         static final ConfigResource BROKER0 = new ConfigResource(BROKER, "0");
         static final ConfigResource MYTOPIC = new ConfigResource(TOPIC, "mytopic");
    @@ -101,26 +111,11 @@ static  Entry entry(A a, B b) {
             return new SimpleImmutableEntry<>(a, b);
         }
     
    -    static ConfigurationControlManager newConfigurationControlManager() {
    -        return newConfigurationControlManager(Optional.empty());
    -    }
    -
    -    static ConfigurationControlManager newConfigurationControlManager(
    -        Optional alterConfigPolicy
    -    ) {
    -        LogContext logContext = new LogContext();
    -        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
    -        return new ConfigurationControlManager(logContext,
    -                snapshotRegistry,
    -                SCHEMA,
    -                TestExistenceChecker.INSTANCE,
    -                alterConfigPolicy,
    -                ConfigurationValidator.NO_OP);
    -    }
    -
         @Test
         public void testReplay() throws Exception {
    -        ConfigurationControlManager manager = newConfigurationControlManager();
    +        ConfigurationControlManager manager = new ConfigurationControlManager.Builder().
    +            setKafkaConfigSchema(SCHEMA).
    +            build();
             assertEquals(Collections.emptyMap(), manager.getConfigs(BROKER0));
             manager.replay(new ConfigRecord().
                 setResourceType(BROKER.id()).setResourceName("0").
    @@ -151,7 +146,9 @@ public void testReplay() throws Exception {
     
         @Test
         public void testIncrementalAlterConfigs() {
    -        ConfigurationControlManager manager = newConfigurationControlManager();
    +        ConfigurationControlManager manager = new ConfigurationControlManager.Builder().
    +            setKafkaConfigSchema(SCHEMA).
    +            build();
     
             ControllerResult> result = manager.
                 incrementalAlterConfigs(toMap(entry(BROKER0, toMap(
    @@ -180,7 +177,10 @@ public void testIncrementalAlterConfigs() {
     
         @Test
         public void testIncrementalAlterConfigsWithoutExistence() {
    -        ConfigurationControlManager manager = newConfigurationControlManager();
    +        ConfigurationControlManager manager = new ConfigurationControlManager.Builder().
    +            setKafkaConfigSchema(SCHEMA).
    +            setExistenceChecker(TestExistenceChecker.INSTANCE).
    +            build();
             ConfigResource existingTopic = new ConfigResource(TOPIC, "ExistingTopic");
     
             ControllerResult> result = manager.
    @@ -236,9 +236,10 @@ public void testIncrementalAlterConfigsWithPolicy() {
                 new RequestMetadata(MYTOPIC, Collections.emptyMap()),
                 new RequestMetadata(BROKER0, toMap(entry("foo.bar", "123"),
                     entry("quux", "456")))));
    -
    -        ConfigurationControlManager manager = newConfigurationControlManager(Optional.of(policy));
    -
    +        ConfigurationControlManager manager = new ConfigurationControlManager.Builder().
    +            setKafkaConfigSchema(SCHEMA).
    +            setAlterConfigPolicy(Optional.of(policy)).
    +            build();
             assertEquals(ControllerResult.atomicOf(asList(new ApiMessageAndVersion(
                     new ConfigRecord().setResourceType(BROKER.id()).setResourceName("0").
                         setName("foo.bar").setValue("123"), (short) 0), new ApiMessageAndVersion(
    @@ -260,7 +261,9 @@ public void testIncrementalAlterConfigsWithPolicy() {
     
         @Test
         public void testLegacyAlterConfigs() {
    -        ConfigurationControlManager manager = newConfigurationControlManager();
    +        ConfigurationControlManager manager = new ConfigurationControlManager.Builder().
    +            setKafkaConfigSchema(SCHEMA).
    +            build();
             List expectedRecords1 = asList(
                 new ApiMessageAndVersion(new ConfigRecord().
                     setResourceType(TOPIC.id()).setResourceName("mytopic").
    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 963c99eeff53..2ec06de8087b 100644
    --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
    +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
    @@ -233,7 +233,8 @@ public void testFenceMultipleBrokers() throws Throwable {
                     new CreatableTopicCollection(Collections.singleton(
                         new CreatableTopic().setName("foo").setNumPartitions(numberOfPartitions).
                             setReplicationFactor(replicationFactor)).iterator()));
    -            CreateTopicsResponseData createTopicsResponseData = active.createTopics(createTopicsRequestData).get();
    +            CreateTopicsResponseData createTopicsResponseData = active.createTopics(
    +                createTopicsRequestData, Collections.singleton("foo")).get();
                 assertEquals(Errors.NONE, Errors.forCode(createTopicsResponseData.topics().find("foo").errorCode()));
                 Uuid topicIdFoo = createTopicsResponseData.topics().find("foo").topicId();
     
    @@ -322,7 +323,8 @@ public void testBalancePartitionLeaders() throws Throwable {
                     new CreatableTopicCollection(Collections.singleton(
                         new CreatableTopic().setName("foo").setNumPartitions(numberOfPartitions).
                             setReplicationFactor(replicationFactor)).iterator()));
    -            CreateTopicsResponseData createTopicsResponseData = active.createTopics(createTopicsRequestData).get();
    +            CreateTopicsResponseData createTopicsResponseData = active.createTopics(
    +                createTopicsRequestData, Collections.singleton("foo")).get();
                 assertEquals(Errors.NONE, Errors.forCode(createTopicsResponseData.topics().find("foo").errorCode()));
                 Uuid topicIdFoo = createTopicsResponseData.topics().find("foo").topicId();
     
    @@ -432,16 +434,17 @@ public void testUnregisterBroker() throws Throwable {
                                 new CreatableTopic().setName("foo").setNumPartitions(1).
                                     setReplicationFactor((short) 1)).iterator()));
                     assertEquals(Errors.INVALID_REPLICATION_FACTOR.code(), active.createTopics(
    -                    createTopicsRequestData).get().topics().find("foo").errorCode());
    +                    createTopicsRequestData, Collections.singleton("foo")).get().
    +                        topics().find("foo").errorCode());
                     assertEquals("Unable to replicate the partition 1 time(s): All brokers " +
    -                    "are currently fenced.", active.createTopics(
    -                    createTopicsRequestData).get().topics().find("foo").errorMessage());
    +                    "are currently fenced.", active.createTopics(createTopicsRequestData,
    +                        Collections.singleton("foo")).get().topics().find("foo").errorMessage());
                     assertEquals(new BrokerHeartbeatReply(true, false, false, false),
                         active.processBrokerHeartbeat(new BrokerHeartbeatRequestData().
                                 setWantFence(false).setBrokerEpoch(0L).setBrokerId(0).
                                 setCurrentMetadataOffset(100000L)).get());
    -                assertEquals(Errors.NONE.code(), active.createTopics(
    -                    createTopicsRequestData).get().topics().find("foo").errorCode());
    +                assertEquals(Errors.NONE.code(), active.createTopics(createTopicsRequestData,
    +                    Collections.singleton("foo")).get().topics().find("foo").errorCode());
                     CompletableFuture topicPartitionFuture = active.appendReadEvent(
                         "debugGetPartition", () -> {
                             Iterator iterator = active.
    @@ -504,7 +507,8 @@ public void testSnapshotSaveAndLoad() throws Throwable {
                                         new CreatableReplicaAssignment().
                                             setPartitionIndex(1).
                                             setBrokerIds(Arrays.asList(1, 2, 0))).
    -                                            iterator()))).iterator()))).get();
    +                                            iterator()))).iterator())),
    +                    Collections.singleton("foo")).get();
                     fooId = fooData.topics().find("foo").topicId();
                     active.allocateProducerIds(
                         new AllocateProducerIdsRequestData().setBrokerId(0).setBrokerEpoch(brokerEpochs.get(0))).get();
    @@ -573,7 +577,8 @@ public void testSnapshotConfiguration() throws Throwable {
                                         new CreatableReplicaAssignment().
                                             setPartitionIndex(1).
                                             setBrokerIds(Arrays.asList(1, 2, 0))).
    -                                            iterator()))).iterator()))).get();
    +                                            iterator()))).iterator())),
    +                    Collections.singleton("foo")).get();
                     fooId = fooData.topics().find("foo").topicId();
                     active.allocateProducerIds(
                         new AllocateProducerIdsRequestData().setBrokerId(0).setBrokerEpoch(brokerEpochs.get(0))).get();
    @@ -636,7 +641,8 @@ public void testSnapshotOnlyAfterConfiguredMinBytes() throws Throwable {
                                             new CreatableReplicaAssignment().
                                                 setPartitionIndex(1).
                                                 setBrokerIds(Arrays.asList(1, 2, 0))).
    -                                                iterator()))).iterator()))).get();
    +                                                iterator()))).iterator())),
    +                        Collections.singleton(topicName)).get();
                     }
                     logEnv.waitForLatestSnapshot();
                 }
    @@ -772,7 +778,8 @@ public void testTimeouts() throws Throwable {
                     CompletableFuture createFuture =
                         controller.createTopics(new CreateTopicsRequestData().setTimeoutMs(0).
                             setTopics(new CreatableTopicCollection(Collections.singleton(
    -                            new CreatableTopic().setName("foo")).iterator())));
    +                            new CreatableTopic().setName("foo")).iterator())),
    +                        Collections.emptySet());
                     long now = controller.time().nanoseconds();
                     CompletableFuture> deleteFuture =
                         controller.deleteTopics(now, Collections.singletonList(Uuid.ZERO_UUID));
    @@ -827,7 +834,8 @@ public void testEarlyControllerResults() throws Throwable {
                     QuorumController controller = controlEnv.activeController();
                     CountDownLatch countDownLatch = controller.pause();
                     CompletableFuture createFuture =
    -                    controller.createTopics(new CreateTopicsRequestData().setTimeoutMs(120000));
    +                    controller.createTopics(new CreateTopicsRequestData().setTimeoutMs(120000),
    +                        Collections.emptySet());
                     long deadlineMs = controller.time().nanoseconds() + HOURS.toNanos(1);
                     CompletableFuture> deleteFuture =
                         controller.deleteTopics(deadlineMs, Collections.emptyList());
    @@ -877,20 +885,14 @@ public void testMissingInMemorySnapshot() throws Exception {
                     )
                     .collect(Collectors.toList());
     
    -            Uuid topicId = controller.createTopics(
    -                new CreateTopicsRequestData()
    -                    .setTopics(
    -                        new CreatableTopicCollection(
    -                            Collections.singleton(
    -                                new CreatableTopic()
    -                                    .setName(topicName)
    -                                    .setNumPartitions(-1)
    -                                    .setReplicationFactor((short) -1)
    -                                    .setAssignments(new CreatableReplicaAssignmentCollection(partitions.iterator()))
    -                            ).iterator()
    -                        )
    -                    )
    -            ).get().topics().find(topicName).topicId();
    +            Uuid topicId = controller.createTopics(new CreateTopicsRequestData()
    +                    .setTopics(new CreatableTopicCollection(Collections.singleton(new CreatableTopic()
    +                        .setName(topicName)
    +                        .setNumPartitions(-1)
    +                        .setReplicationFactor((short) -1)
    +                        .setAssignments(new CreatableReplicaAssignmentCollection(partitions.iterator()))
    +                    ).iterator())),
    +                Collections.singleton("foo")).get().topics().find(topicName).topicId();
     
                 // Create a lot of alter isr
                 List alterPartitions = IntStream
    @@ -1029,7 +1031,8 @@ public void testConfigResourceExistenceChecker() throws Throwable {
                         setTopics(new CreatableTopicCollection(Collections.singleton(
                             new CreatableTopic().setName("foo").
                                 setReplicationFactor((short) 3).
    -                            setNumPartitions(1)).iterator()))).get();
    +                            setNumPartitions(1)).iterator())),
    +                    Collections.singleton("foo")).get();
                     ConfigResourceExistenceChecker checker =
                         active.new ConfigResourceExistenceChecker();
                     // A ConfigResource with type=BROKER and name=(empty string) represents
    diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
    index 1712d2044466..108f2eca665c 100644
    --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
    +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
    @@ -67,7 +67,6 @@
     import org.apache.kafka.common.utils.Utils;
     import org.apache.kafka.metadata.BrokerHeartbeatReply;
     import org.apache.kafka.metadata.BrokerRegistration;
    -import org.apache.kafka.metadata.KafkaConfigSchema;
     import org.apache.kafka.metadata.LeaderRecoveryState;
     import org.apache.kafka.metadata.PartitionRegistration;
     import org.apache.kafka.metadata.RecordTestUtils;
    @@ -82,6 +81,7 @@
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    +import java.util.Arrays;
     import java.util.concurrent.atomic.AtomicLong;
     import java.util.ArrayList;
     import java.util.Collections;
    @@ -142,13 +142,9 @@ private static class ReplicationControlTestContext {
                 TimeUnit.MILLISECONDS.convert(BROKER_SESSION_TIMEOUT_MS, TimeUnit.NANOSECONDS),
                 new StripedReplicaPlacer(random),
                 metrics);
    -        final ConfigurationControlManager configurationControl = new ConfigurationControlManager(
    -            new LogContext(),
    -            snapshotRegistry,
    -            KafkaConfigSchema.EMPTY,
    -            __ -> { },
    -            Optional.empty(),
    -            (__, ___) -> { });
    +        final ConfigurationControlManager configurationControl = new ConfigurationControlManager.Builder().
    +            setSnapshotRegistry(snapshotRegistry).
    +            build();
             final ReplicationControlManager replicationControl;
     
             void replay(List records) throws Exception {
    @@ -184,7 +180,7 @@ CreatableTopicResult createTestTopic(String name,
                 topic.setNumPartitions(numPartitions).setReplicationFactor(replicationFactor);
                 request.topics().add(topic);
                 ControllerResult result =
    -                replicationControl.createTopics(request);
    +                replicationControl.createTopics(request, Collections.singleton(name));
                 CreatableTopicResult topicResult = result.response().topics().find(name);
                 assertNotNull(topicResult);
                 assertEquals(expectedErrorCode, topicResult.errorCode());
    @@ -219,7 +215,7 @@ CreatableTopicResult createTestTopic(String name, int[][] replicas,
                         setValue(e.getValue())));
                 request.topics().add(topic);
                 ControllerResult result =
    -                replicationControl.createTopics(request);
    +                replicationControl.createTopics(request, Collections.singleton(name));
                 CreatableTopicResult topicResult = result.response().topics().find(name);
                 assertNotNull(topicResult);
                 assertEquals(expectedErrorCode, topicResult.errorCode());
    @@ -402,7 +398,7 @@ public void testCreateTopics() throws Exception {
             request.topics().add(new CreatableTopic().setName("foo").
                 setNumPartitions(-1).setReplicationFactor((short) -1));
             ControllerResult result =
    -            replicationControl.createTopics(request);
    +            replicationControl.createTopics(request, Collections.singleton("foo"));
             CreateTopicsResponseData expectedResponse = new CreateTopicsResponseData();
             expectedResponse.topics().add(new CreatableTopicResult().setName("foo").
                 setErrorCode(Errors.INVALID_REPLICATION_FACTOR.code()).
    @@ -413,7 +409,7 @@ public void testCreateTopics() throws Exception {
             ctx.registerBrokers(0, 1, 2);
             ctx.unfenceBrokers(0, 1, 2);
             ControllerResult result2 =
    -            replicationControl.createTopics(request);
    +            replicationControl.createTopics(request, Collections.singleton("foo"));
             CreateTopicsResponseData expectedResponse2 = new CreateTopicsResponseData();
             expectedResponse2.topics().add(new CreatableTopicResult().setName("foo").
                 setNumPartitions(1).setReplicationFactor((short) 3).
    @@ -426,7 +422,7 @@ public void testCreateTopics() throws Exception {
                 replicationControl.getPartition(
                     ((TopicRecord) result2.records().get(0).message()).topicId(), 0));
             ControllerResult result3 =
    -                replicationControl.createTopics(request);
    +                replicationControl.createTopics(request, Collections.singleton("foo"));
             CreateTopicsResponseData expectedResponse3 = new CreateTopicsResponseData();
             expectedResponse3.topics().add(new CreatableTopicResult().setName("foo").
                     setErrorCode(Errors.TOPIC_ALREADY_EXISTS.code()).
    @@ -488,7 +484,7 @@ public void testCreateTopicsWithValidateOnlyFlag() throws Exception {
             request.topics().add(new CreatableTopic().setName("foo").
                 setNumPartitions(1).setReplicationFactor((short) 3));
             ControllerResult result =
    -            ctx.replicationControl.createTopics(request);
    +            ctx.replicationControl.createTopics(request, Collections.singleton("foo"));
             assertEquals(0, result.records().size());
             CreatableTopicResult topicResult = result.response().topics().find("foo");
             assertEquals((short) 0, topicResult.errorCode());
    @@ -503,7 +499,7 @@ public void testInvalidCreateTopicsWithValidateOnlyFlag() throws Exception {
             request.topics().add(new CreatableTopic().setName("foo").
                 setNumPartitions(1).setReplicationFactor((short) 4));
             ControllerResult result =
    -            ctx.replicationControl.createTopics(request);
    +            ctx.replicationControl.createTopics(request, Collections.singleton("foo"));
             assertEquals(0, result.records().size());
             CreateTopicsResponseData expectedResponse = new CreateTopicsResponseData();
             expectedResponse.topics().add(new CreatableTopicResult().setName("foo").
    @@ -551,7 +547,7 @@ public void testGlobalTopicAndPartitionMetrics() throws Exception {
             List topicsToDelete = new ArrayList<>();
     
             ControllerResult result =
    -            replicationControl.createTopics(request);
    +            replicationControl.createTopics(request, Collections.singleton("foo"));
             topicsToDelete.add(result.response().topics().find("foo").topicId());
     
             RecordTestUtils.replayAll(replicationControl, result.records());
    @@ -562,7 +558,8 @@ public void testGlobalTopicAndPartitionMetrics() throws Exception {
                 setNumPartitions(1).setReplicationFactor((short) -1));
             request.topics().add(new CreatableTopic().setName("baz").
                 setNumPartitions(2).setReplicationFactor((short) -1));
    -        result = replicationControl.createTopics(request);
    +        result = replicationControl.createTopics(request,
    +            new HashSet<>(Arrays.asList("bar", "baz")));
             RecordTestUtils.replayAll(replicationControl, result.records());
             assertEquals(3, ctx.metrics.globalTopicsCount());
             assertEquals(4, ctx.metrics.globalPartitionCount());
    @@ -889,7 +886,7 @@ public void testDeleteTopics() throws Exception {
             ctx.registerBrokers(0, 1);
             ctx.unfenceBrokers(0, 1);
             ControllerResult createResult =
    -            replicationControl.createTopics(request);
    +            replicationControl.createTopics(request, Collections.singleton("foo"));
             CreateTopicsResponseData expectedResponse = new CreateTopicsResponseData();
             Uuid topicId = createResult.response().topics().find("foo").topicId();
             expectedResponse.topics().add(new CreatableTopicResult().setName("foo").
    @@ -961,8 +958,8 @@ public void testCreatePartitions() throws Exception {
                 setNumPartitions(2).setReplicationFactor((short) 2));
             ctx.registerBrokers(0, 1);
             ctx.unfenceBrokers(0, 1);
    -        ControllerResult createTopicResult =
    -            replicationControl.createTopics(request);
    +        ControllerResult createTopicResult = replicationControl.
    +            createTopics(request, new HashSet<>(Arrays.asList("foo", "bar", "quux", "foo2")));
             ctx.replay(createTopicResult.records());
             List topics = new ArrayList<>();
             topics.add(new CreatePartitionsTopic().
    diff --git a/metadata/src/test/java/org/apache/kafka/metadata/ConfigSynonymTest.java b/metadata/src/test/java/org/apache/kafka/metadata/ConfigSynonymTest.java
    new file mode 100644
    index 000000000000..93f63c21091f
    --- /dev/null
    +++ b/metadata/src/test/java/org/apache/kafka/metadata/ConfigSynonymTest.java
    @@ -0,0 +1,47 @@
    +/*
    + * 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.metadata;
    +
    +import org.junit.jupiter.api.Test;
    +import org.junit.jupiter.api.Timeout;
    +
    +import static org.junit.jupiter.api.Assertions.assertEquals;
    +
    +
    +@Timeout(value = 40)
    +public class ConfigSynonymTest {
    +    @Test
    +    public void testHoursToMilliseconds() {
    +        assertEquals("0", ConfigSynonym.HOURS_TO_MILLISECONDS.apply(""));
    +        assertEquals("0", ConfigSynonym.HOURS_TO_MILLISECONDS.apply(" "));
    +        assertEquals("0", ConfigSynonym.HOURS_TO_MILLISECONDS.apply("0"));
    +        assertEquals("442800000", ConfigSynonym.HOURS_TO_MILLISECONDS.apply("123"));
    +        assertEquals("442800000", ConfigSynonym.HOURS_TO_MILLISECONDS.apply(" 123 "));
    +        assertEquals("0", ConfigSynonym.HOURS_TO_MILLISECONDS.apply("not_a_number"));
    +    }
    +
    +    @Test
    +    public void testMinutesToMilliseconds() {
    +        assertEquals("0", ConfigSynonym.MINUTES_TO_MILLISECONDS.apply(""));
    +        assertEquals("0", ConfigSynonym.MINUTES_TO_MILLISECONDS.apply(" "));
    +        assertEquals("0", ConfigSynonym.MINUTES_TO_MILLISECONDS.apply("0"));
    +        assertEquals("7380000", ConfigSynonym.MINUTES_TO_MILLISECONDS.apply("123"));
    +        assertEquals("7380000", ConfigSynonym.MINUTES_TO_MILLISECONDS.apply(" 123 "));
    +        assertEquals("0", ConfigSynonym.MINUTES_TO_MILLISECONDS.apply("not_a_number"));
    +    }
    +}
    diff --git a/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java b/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java
    index fae40e259267..36089d0f9a4f 100644
    --- a/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java
    +++ b/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java
    @@ -17,16 +17,22 @@
     
     package org.apache.kafka.metadata;
     
    +import org.apache.kafka.clients.admin.ConfigEntry;
     import org.apache.kafka.common.config.ConfigDef;
     import org.apache.kafka.common.config.ConfigResource;
    +import org.apache.kafka.common.requests.DescribeConfigsResponse;
     import org.junit.jupiter.api.Test;
     import org.junit.jupiter.api.Timeout;
     
    +import java.util.Arrays;
     import java.util.HashMap;
    +import java.util.List;
     import java.util.Map;
     
    +import static java.util.Collections.emptyList;
     import static org.apache.kafka.common.config.ConfigResource.Type.BROKER;
     import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
    +import static org.apache.kafka.metadata.ConfigSynonym.HOURS_TO_MILLISECONDS;
     import static org.junit.jupiter.api.Assertions.assertEquals;
     import static org.junit.jupiter.api.Assertions.assertFalse;
     import static org.junit.jupiter.api.Assertions.assertTrue;
    @@ -38,42 +44,122 @@ public class KafkaConfigSchemaTest {
     
         static {
             CONFIGS.put(BROKER, new ConfigDef().
    -            define("foo.bar", ConfigDef.Type.LIST, "1", ConfigDef.Importance.HIGH, "foo bar").
    -            define("baz", ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, "baz").
    -            define("quux", ConfigDef.Type.INT, ConfigDef.Importance.HIGH, "quux").
    -            define("quuux", ConfigDef.Type.PASSWORD, ConfigDef.Importance.HIGH, "quuux"));
    +            define("foo.bar", ConfigDef.Type.LIST, "1", ConfigDef.Importance.HIGH, "foo bar doc").
    +            define("baz", ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, "baz doc").
    +            define("quux", ConfigDef.Type.INT, ConfigDef.Importance.HIGH, "quux doc").
    +            define("quuux", ConfigDef.Type.PASSWORD, ConfigDef.Importance.HIGH, "quuux doc").
    +            define("quuux2", ConfigDef.Type.PASSWORD, ConfigDef.Importance.HIGH, "quuux2 doc"));
             CONFIGS.put(TOPIC, new ConfigDef().
    -            define("abc", ConfigDef.Type.LIST, ConfigDef.Importance.HIGH, "abc").
    -            define("def", ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, "def").
    -            define("ghi", ConfigDef.Type.BOOLEAN, true, ConfigDef.Importance.HIGH, "ghi").
    -            define("xyz", ConfigDef.Type.PASSWORD, "thedefault", ConfigDef.Importance.HIGH, "xyz"));
    +            define("abc", ConfigDef.Type.LIST, ConfigDef.Importance.HIGH, "abc doc").
    +            define("def", ConfigDef.Type.LONG, ConfigDef.Importance.HIGH, "def doc").
    +            define("ghi", ConfigDef.Type.BOOLEAN, true, ConfigDef.Importance.HIGH, "ghi doc").
    +            define("xyz", ConfigDef.Type.PASSWORD, "thedefault", ConfigDef.Importance.HIGH, "xyz doc"));
    +    }
    +
    +    public static final Map> SYNONYMS = new HashMap<>();
    +
    +    static {
    +        SYNONYMS.put("abc", Arrays.asList(new ConfigSynonym("foo.bar")));
    +        SYNONYMS.put("def", Arrays.asList(new ConfigSynonym("quux", HOURS_TO_MILLISECONDS)));
    +        SYNONYMS.put("ghi", Arrays.asList(new ConfigSynonym("ghi")));
    +        SYNONYMS.put("xyz", Arrays.asList(new ConfigSynonym("quuux"), new ConfigSynonym("quuux2")));
    +    }
    +
    +    private static final KafkaConfigSchema SCHEMA = new KafkaConfigSchema(CONFIGS, SYNONYMS);
    +
    +    @Test
    +    public void testTranslateConfigTypes() {
    +        testTranslateConfigType(ConfigDef.Type.BOOLEAN, ConfigEntry.ConfigType.BOOLEAN);
    +        testTranslateConfigType(ConfigDef.Type.STRING, ConfigEntry.ConfigType.STRING);
    +        testTranslateConfigType(ConfigDef.Type.INT, ConfigEntry.ConfigType.INT);
    +        testTranslateConfigType(ConfigDef.Type.SHORT, ConfigEntry.ConfigType.SHORT);
    +        testTranslateConfigType(ConfigDef.Type.LONG, ConfigEntry.ConfigType.LONG);
    +        testTranslateConfigType(ConfigDef.Type.DOUBLE, ConfigEntry.ConfigType.DOUBLE);
    +        testTranslateConfigType(ConfigDef.Type.LIST, ConfigEntry.ConfigType.LIST);
    +        testTranslateConfigType(ConfigDef.Type.CLASS, ConfigEntry.ConfigType.CLASS);
    +        testTranslateConfigType(ConfigDef.Type.PASSWORD, ConfigEntry.ConfigType.PASSWORD);
    +    }
    +
    +    private static void testTranslateConfigType(ConfigDef.Type a, ConfigEntry.ConfigType b) {
    +        assertEquals(b, KafkaConfigSchema.translateConfigType(a));
    +    }
    +
    +    @Test
    +    public void testTranslateConfigSources() {
    +        testTranslateConfigSource(ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG,
    +            DescribeConfigsResponse.ConfigSource.TOPIC_CONFIG);
    +        testTranslateConfigSource(ConfigEntry.ConfigSource.DYNAMIC_BROKER_LOGGER_CONFIG,
    +            DescribeConfigsResponse.ConfigSource.DYNAMIC_BROKER_LOGGER_CONFIG);
    +        testTranslateConfigSource(ConfigEntry.ConfigSource.DYNAMIC_BROKER_CONFIG,
    +            DescribeConfigsResponse.ConfigSource.DYNAMIC_BROKER_CONFIG);
    +        testTranslateConfigSource(ConfigEntry.ConfigSource.DYNAMIC_DEFAULT_BROKER_CONFIG,
    +            DescribeConfigsResponse.ConfigSource.DYNAMIC_DEFAULT_BROKER_CONFIG);
    +        testTranslateConfigSource(ConfigEntry.ConfigSource.STATIC_BROKER_CONFIG,
    +            DescribeConfigsResponse.ConfigSource.STATIC_BROKER_CONFIG);
    +        testTranslateConfigSource(ConfigEntry.ConfigSource.DEFAULT_CONFIG,
    +            DescribeConfigsResponse.ConfigSource.DEFAULT_CONFIG);
    +    }
    +
    +    private static void testTranslateConfigSource(ConfigEntry.ConfigSource a,
    +                                                  DescribeConfigsResponse.ConfigSource b) {
    +        assertEquals(b, KafkaConfigSchema.translateConfigSource(a));
         }
     
         @Test
         public void testIsSplittable() {
    -        KafkaConfigSchema schema = new KafkaConfigSchema(CONFIGS);
    -        assertTrue(schema.isSplittable(BROKER, "foo.bar"));
    -        assertFalse(schema.isSplittable(BROKER, "baz"));
    -        assertFalse(schema.isSplittable(BROKER, "foo.baz.quux"));
    -        assertFalse(schema.isSplittable(TOPIC, "baz"));
    -        assertTrue(schema.isSplittable(TOPIC, "abc"));
    +        assertTrue(SCHEMA.isSplittable(BROKER, "foo.bar"));
    +        assertFalse(SCHEMA.isSplittable(BROKER, "baz"));
    +        assertFalse(SCHEMA.isSplittable(BROKER, "foo.baz.quux"));
    +        assertFalse(SCHEMA.isSplittable(TOPIC, "baz"));
    +        assertTrue(SCHEMA.isSplittable(TOPIC, "abc"));
         }
     
         @Test
         public void testGetConfigValueDefault() {
    -        KafkaConfigSchema schema = new KafkaConfigSchema(CONFIGS);
    -        assertEquals("1", schema.getDefault(BROKER, "foo.bar"));
    -        assertEquals(null, schema.getDefault(BROKER, "foo.baz.quux"));
    -        assertEquals(null, schema.getDefault(TOPIC, "abc"));
    -        assertEquals("true", schema.getDefault(TOPIC, "ghi"));
    +        assertEquals("1", SCHEMA.getDefault(BROKER, "foo.bar"));
    +        assertEquals(null, SCHEMA.getDefault(BROKER, "foo.baz.quux"));
    +        assertEquals(null, SCHEMA.getDefault(TOPIC, "abc"));
    +        assertEquals("true", SCHEMA.getDefault(TOPIC, "ghi"));
         }
     
         @Test
         public void testIsSensitive() {
    -        KafkaConfigSchema schema = new KafkaConfigSchema(CONFIGS);
    -        assertFalse(schema.isSensitive(BROKER, "foo.bar"));
    -        assertTrue(schema.isSensitive(BROKER, "quuux"));
    -        assertTrue(schema.isSensitive(BROKER, "unknown.config.key"));
    -        assertFalse(schema.isSensitive(TOPIC, "abc"));
    +        assertFalse(SCHEMA.isSensitive(BROKER, "foo.bar"));
    +        assertTrue(SCHEMA.isSensitive(BROKER, "quuux"));
    +        assertTrue(SCHEMA.isSensitive(BROKER, "quuux2"));
    +        assertTrue(SCHEMA.isSensitive(BROKER, "unknown.config.key"));
    +        assertFalse(SCHEMA.isSensitive(TOPIC, "abc"));
    +    }
    +
    +    @Test
    +    public void testResolveEffectiveTopicConfig() {
    +        Map staticNodeConfig = new HashMap<>();
    +        staticNodeConfig.put("foo.bar", "the,static,value");
    +        staticNodeConfig.put("quux", "123");
    +        staticNodeConfig.put("ghi", "false");
    +        Map dynamicClusterConfigs = new HashMap<>();
    +        dynamicClusterConfigs.put("foo.bar", "the,dynamic,cluster,config,value");
    +        dynamicClusterConfigs.put("quux", "456");
    +        Map dynamicNodeConfigs = new HashMap<>();
    +        dynamicNodeConfigs.put("quux", "789");
    +        Map dynamicTopicConfigs = new HashMap<>();
    +        dynamicTopicConfigs.put("ghi", "true");
    +        Map expected = new HashMap<>();
    +        expected.put("abc", new ConfigEntry("abc", "the,dynamic,cluster,config,value",
    +            ConfigEntry.ConfigSource.DYNAMIC_DEFAULT_BROKER_CONFIG, false, false, emptyList(),
    +                ConfigEntry.ConfigType.LIST, "abc doc"));
    +        expected.put("def", new ConfigEntry("def", "2840400000",
    +            ConfigEntry.ConfigSource.DYNAMIC_BROKER_CONFIG, false, false, emptyList(),
    +            ConfigEntry.ConfigType.LONG, "def doc"));
    +        expected.put("ghi", new ConfigEntry("ghi", "true",
    +            ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, false, false, emptyList(),
    +            ConfigEntry.ConfigType.BOOLEAN, "ghi doc"));
    +        expected.put("xyz", new ConfigEntry("xyz", "thedefault",
    +            ConfigEntry.ConfigSource.DEFAULT_CONFIG, true, false, emptyList(),
    +            ConfigEntry.ConfigType.PASSWORD, "xyz doc"));
    +        assertEquals(expected, SCHEMA.resolveEffectiveTopicConfigs(staticNodeConfig,
    +            dynamicClusterConfigs,
    +            dynamicNodeConfigs,
    +            dynamicTopicConfigs));
         }
     }
    
    From 02a465b0906f3514a0c6182e8cf8f5b75b228088 Mon Sep 17 00:00:00 2001
    From: bozhao12 <102274736+bozhao12@users.noreply.github.com>
    Date: Mon, 4 Apr 2022 15:07:58 +0800
    Subject: [PATCH 081/447] MINOR: fix typo in FetchRequest.json (#11988)
    
    Reviewers: David Jacot 
    ---
     clients/src/main/resources/common/message/FetchRequest.json | 2 +-
     1 file changed, 1 insertion(+), 1 deletion(-)
    
    diff --git a/clients/src/main/resources/common/message/FetchRequest.json b/clients/src/main/resources/common/message/FetchRequest.json
    index df639579bd77..13ab712be3d5 100644
    --- a/clients/src/main/resources/common/message/FetchRequest.json
    +++ b/clients/src/main/resources/common/message/FetchRequest.json
    @@ -92,7 +92,7 @@
         { "name": "ForgottenTopicsData", "type": "[]ForgottenTopic", "versions": "7+", "ignorable": false,
           "about": "In an incremental fetch request, the partitions to remove.", "fields": [
           { "name": "Topic", "type": "string", "versions": "7-12", "entityType": "topicName", "ignorable": true,
    -        "about": "The partition name." },
    +        "about": "The topic name." },
           { "name": "TopicId", "type": "uuid", "versions": "13+", "ignorable": true, "about": "The unique topic ID"},
           { "name": "Partitions", "type": "[]int32", "versions": "7+",
             "about": "The partitions indexes to forget." }
    
    From ae45c59e61c34fcf1abb5d4731a99d400d2c79be Mon Sep 17 00:00:00 2001
    From: Anastasia Vela 
    Date: Mon, 4 Apr 2022 00:10:33 -0700
    Subject: [PATCH 082/447] MINOR: Fix flaky
     testClientDisconnectionUpdatesRequestMetrics() (#11987)
    
    Reviewers: David Jacot 
    ---
     core/src/test/scala/unit/kafka/network/SocketServerTest.scala | 2 +-
     1 file changed, 1 insertion(+), 1 deletion(-)
    
    diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
    index 443edb1a79c3..6c520613087e 100644
    --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
    +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
    @@ -1125,12 +1125,12 @@ class SocketServerTest {
     
           val requestMetrics = channel.metrics(request.header.apiKey.name)
           def totalTimeHistCount(): Long = requestMetrics.totalTimeHist.count
    +      val expectedTotalTimeCount = totalTimeHistCount() + 1
           val send = new NetworkSend(request.context.connectionId, ByteBufferSend.sizePrefixed(ByteBuffer.allocate(responseBufferSize)))
           val headerLog = new ObjectNode(JsonNodeFactory.instance)
           headerLog.set("response", new TextNode("someResponse"))
           channel.sendResponse(new RequestChannel.SendResponse(request, send, Some(headerLog), None))
     
    -      val expectedTotalTimeCount = totalTimeHistCount() + 1
           TestUtils.waitUntilTrue(() => totalTimeHistCount() == expectedTotalTimeCount,
             s"request metrics not updated, expected: $expectedTotalTimeCount, actual: ${totalTimeHistCount()}")
     
    
    From dd4afdeb6d5226e2371d6dfb3ce2eb8c8c7a8c10 Mon Sep 17 00:00:00 2001
    From: Mickael Maison 
    Date: Mon, 4 Apr 2022 14:43:31 +0200
    Subject: [PATCH 083/447] MINOR: Doc updates for Kafka 3.0.1 (#11906)
    
    Reviewers: David Jacot 
    ---
     docs/upgrade.html | 2 +-
     1 file changed, 1 insertion(+), 1 deletion(-)
    
    diff --git a/docs/upgrade.html b/docs/upgrade.html
    index 313057d86672..2ac903819efd 100644
    --- a/docs/upgrade.html
    +++ b/docs/upgrade.html
    @@ -110,7 +110,7 @@ 
    Notable changes in 3 KIP-516. -

    Upgrading to 3.0.0 from any version 0.8.x through 2.8.x

    +

    Upgrading to 3.0.1 from any version 0.8.x through 2.8.x

    If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.

    From 74909e000aaab1f0300ae2e918a6fa361e38078e Mon Sep 17 00:00:00 2001 From: RivenSun <91005273+RivenSun2@users.noreply.github.com> Date: Mon, 4 Apr 2022 23:27:34 +0800 Subject: [PATCH 084/447] MINOR: Fix wrong configuration in Adding and Removing Listeners docs (#11992) Reviewers: Mickael Maison --- docs/configuration.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.html b/docs/configuration.html index 0782c83790eb..ceb671ca7500 100644 --- a/docs/configuration.html +++ b/docs/configuration.html @@ -215,7 +215,7 @@
    Adding and Removing Listeners
  • listener.security.protocol.map
  • Inter-broker listener must be configured using the static broker configuration inter.broker.listener.name - or inter.broker.security.protocol. + or security.inter.broker.protocol.

    3.2 Topic-Level Configs

    From 481cc13a132d33f23e737f88ae28a1aac135afed Mon Sep 17 00:00:00 2001 From: yun-yun <704863678@qq.com> Date: Tue, 5 Apr 2022 15:27:32 +0800 Subject: [PATCH 085/447] KAFKA-13791: Fix potential race condition in FetchResponse#`fetchData` and `forgottenTopics` (#11981) Fix FetchResponse#`fetchData` and `forgottenTopics`: Assignment of lazy-initialized members should be the last step with double-checked locking Reviewers: Luke Chen --- .../kafka/common/requests/FetchRequest.java | 16 +++++++++++----- .../kafka/common/requests/FetchResponse.java | 2 +- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 48ba022610e4..09242bfc4bf0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -351,8 +351,10 @@ public Map fetchData(Map topicNam if (fetchData == null) { synchronized (this) { if (fetchData == null) { - fetchData = new LinkedHashMap<>(); - short version = version(); + // Assigning the lazy-initialized `fetchData` in the last step + // to avoid other threads accessing a half-initialized object. + final LinkedHashMap fetchDataTmp = new LinkedHashMap<>(); + final short version = version(); data.topics().forEach(fetchTopic -> { String name; if (version < 13) { @@ -362,7 +364,7 @@ public Map fetchData(Map topicNam } fetchTopic.partitions().forEach(fetchPartition -> // Topic name may be null here if the topic name was unable to be resolved using the topicNames map. - fetchData.put(new TopicIdPartition(fetchTopic.topicId(), new TopicPartition(name, fetchPartition.partition())), + fetchDataTmp.put(new TopicIdPartition(fetchTopic.topicId(), new TopicPartition(name, fetchPartition.partition())), new PartitionData( fetchTopic.topicId(), fetchPartition.fetchOffset(), @@ -374,6 +376,7 @@ public Map fetchData(Map topicNam ) ); }); + fetchData = fetchDataTmp; } } } @@ -386,7 +389,9 @@ public List forgottenTopics(Map topicNames) { if (toForget == null) { synchronized (this) { if (toForget == null) { - toForget = new ArrayList<>(); + // Assigning the lazy-initialized `toForget` in the last step + // to avoid other threads accessing a half-initialized object. + final List toForgetTmp = new ArrayList<>(); data.forgottenTopicsData().forEach(forgottenTopic -> { String name; if (version() < 13) { @@ -395,8 +400,9 @@ public List forgottenTopics(Map topicNames) { name = topicNames.get(forgottenTopic.topicId()); } // Topic name may be null here if the topic name was unable to be resolved using the topicNames map. - forgottenTopic.partitions().forEach(partitionId -> toForget.add(new TopicIdPartition(forgottenTopic.topicId(), new TopicPartition(name, partitionId)))); + forgottenTopic.partitions().forEach(partitionId -> toForgetTmp.add(new TopicIdPartition(forgottenTopic.topicId(), new TopicPartition(name, partitionId)))); }); + toForget = toForgetTmp; } } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index 0d7049d75514..a4af4ca2a237 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -100,7 +100,7 @@ public LinkedHashMap responseDa if (responseData == null) { synchronized (this) { if (responseData == null) { - // Assigning the lazy-initialized responseData in the last step + // Assigning the lazy-initialized `responseData` in the last step // to avoid other threads accessing a half-initialized object. final LinkedHashMap responseDataTmp = new LinkedHashMap<>(); From 3ceedac79e58a4a457d36d75c1f4c5a5b36af3d2 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 5 Apr 2022 09:48:21 -0700 Subject: [PATCH 086/447] KAFKA-13782; Ensure correct partition added to txn after abort on full batch (#11995) Fixes a regression introduced in https://github.com/apache/kafka/pull/11452. Following [KIP-480](https://cwiki.apache.org/confluence/display/KAFKA/KIP-480%3A+Sticky+Partitioner), the `Partitioner` will receive a callback when a batch has been completed so that it can choose another partition. Because of this, we have to wait until the batch has been successfully appended to the accumulator before adding the partition in `TransactionManager.maybeAddPartition`. This is still safe because the `Sender` cannot dequeue a batch from the accumulator until it has been added to the transaction successfully. Reviewers: Artem Livshits <84364232+artemlivshits@users.noreply.github.com>, David Jacot , Tom Bentley --- .../kafka/clients/producer/KafkaProducer.java | 50 ++- .../producer/internals/RecordAccumulator.java | 2 +- .../clients/producer/KafkaProducerTest.java | 290 +++++++++++++++++- 3 files changed, 333 insertions(+), 9 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 7fa640f43966..96d9125831da 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -440,6 +440,43 @@ public KafkaProducer(Properties properties, Serializer keySerializer, Seriali } } + // visible for testing + KafkaProducer(ProducerConfig config, + LogContext logContext, + Metrics metrics, + Serializer keySerializer, + Serializer valueSerializer, + ProducerMetadata metadata, + RecordAccumulator accumulator, + TransactionManager transactionManager, + Sender sender, + ProducerInterceptors interceptors, + Partitioner partitioner, + Time time, + KafkaThread ioThread) { + this.producerConfig = config; + this.time = time; + this.clientId = config.getString(ProducerConfig.CLIENT_ID_CONFIG); + this.log = logContext.logger(KafkaProducer.class); + this.metrics = metrics; + this.producerMetrics = new KafkaProducerMetrics(metrics); + this.partitioner = partitioner; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + this.interceptors = interceptors; + this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG); + this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG); + this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG)); + this.maxBlockTimeMs = config.getLong(ProducerConfig.MAX_BLOCK_MS_CONFIG); + this.apiVersions = new ApiVersions(); + this.transactionManager = transactionManager; + this.accumulator = accumulator; + this.errors = this.metrics.sensor("errors"); + this.metadata = metadata; + this.sender = sender; + this.ioThread = ioThread; + } + // visible for testing Sender newSender(LogContext logContext, KafkaClient kafkaClient, ProducerMetadata metadata) { int maxInflightRequests = producerConfig.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION); @@ -939,10 +976,6 @@ private Future doSend(ProducerRecord record, Callback call // producer callback will make sure to call both 'callback' and interceptor callback Callback interceptCallback = new InterceptorCallback<>(callback, this.interceptors, tp); - if (transactionManager != null) { - transactionManager.maybeAddPartition(tp); - } - RecordAccumulator.RecordAppendResult result = accumulator.append(tp, timestamp, serializedKey, serializedValue, headers, interceptCallback, remainingWaitMs, true, nowMs); @@ -961,6 +994,15 @@ private Future doSend(ProducerRecord record, Callback call serializedValue, headers, interceptCallback, remainingWaitMs, false, nowMs); } + // Add the partition to the transaction (if in progress) after it has been successfully + // appended to the accumulator. We cannot do it before because the initially selected + // partition may be changed when the batch is closed (as indicated by `abortForNewBatch`). + // Note that the `Sender` will refuse to dequeue batches from the accumulator until they + // have been added to the transaction. + if (transactionManager != null) { + transactionManager.maybeAddPartition(tp); + } + if (result.batchIsFull || result.newBatchCreated) { log.trace("Waking up the sender since topic {} partition {} is either full or getting a new batch", record.topic(), partition); this.sender.wakeup(); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index a47c9d38ad32..7724237e45d7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -63,7 +63,7 @@ * The accumulator uses a bounded amount of memory and append calls will block when that memory is exhausted, unless * this behavior is explicitly disabled. */ -public final class RecordAccumulator { +public class RecordAccumulator { private final Logger log; private volatile boolean closed; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index 81eb6e3f2d69..bc91340a7bf1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -23,9 +23,13 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.producer.internals.FutureRecordMetadata; +import org.apache.kafka.clients.producer.internals.ProduceRequestResult; import org.apache.kafka.clients.producer.internals.ProducerInterceptors; import org.apache.kafka.clients.producer.internals.ProducerMetadata; +import org.apache.kafka.clients.producer.internals.RecordAccumulator; import org.apache.kafka.clients.producer.internals.Sender; +import org.apache.kafka.clients.producer.internals.TransactionManager; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; @@ -51,6 +55,7 @@ import org.apache.kafka.common.network.Selectable; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.requests.AddOffsetsToTxnResponse; import org.apache.kafka.common.requests.EndTxnResponse; @@ -66,6 +71,7 @@ import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.utils.KafkaThread; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; @@ -74,7 +80,9 @@ import org.apache.kafka.test.MockProducerInterceptor; import org.apache.kafka.test.MockSerializer; import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -123,6 +131,7 @@ import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.notNull; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -151,6 +160,8 @@ public class KafkaProducerTest { new PartitionInfo(topic, 2, null, null, null)), Collections.emptySet(), Collections.emptySet()); + private TestInfo testInfo; + private static final int DEFAULT_METADATA_IDLE_MS = 5 * 60 * 1000; private static final Node NODE = new Node(0, "host1", 1000); @@ -161,8 +172,13 @@ private static KafkaProducer kafkaProducer(Map conf KafkaClient kafkaClient, ProducerInterceptors interceptors, Time time) { - return new KafkaProducer<>(new ProducerConfig(ProducerConfig.appendSerializerToConfig(configs, keySerializer, valueSerializer)), - keySerializer, valueSerializer, metadata, kafkaClient, interceptors, time); + return new KafkaProducer(new ProducerConfig(ProducerConfig.appendSerializerToConfig(configs, keySerializer, valueSerializer)), + keySerializer, valueSerializer, metadata, kafkaClient, interceptors, time); + } + + @BeforeEach + public void setup(TestInfo testInfo) { + this.testInfo = testInfo; } @Test @@ -636,7 +652,7 @@ private static KafkaProducer producerWithOverrideNewSender(Map producerWithOverrideNewSender(Map configs, ProducerMetadata metadata, - Time timer) { + Time time) { // let mockClient#leastLoadedNode return the node directly so that we can isolate Metadata calls from KafkaProducer for idempotent producer MockClient mockClient = new MockClient(Time.SYSTEM, metadata) { @Override @@ -647,7 +663,7 @@ public Node leastLoadedNode(long now) { return new KafkaProducer( new ProducerConfig(ProducerConfig.appendSerializerToConfig(configs, new StringSerializer(), new StringSerializer())), - new StringSerializer(), new StringSerializer(), metadata, mockClient, null, timer) { + new StringSerializer(), new StringSerializer(), metadata, mockClient, null, time) { @Override Sender newSender(LogContext logContext, KafkaClient kafkaClient, ProducerMetadata metadata) { // give Sender its own Metadata instance so that we can isolate Metadata calls from KafkaProducer @@ -1944,6 +1960,180 @@ public void negativePartitionShouldThrow() { } } + @Test + public void testPartitionAddedToTransaction() throws Exception { + StringSerializer serializer = new StringSerializer(); + KafkaProducerTestContext ctx = new KafkaProducerTestContext<>(testInfo, serializer); + + String topic = "foo"; + TopicPartition topicPartition = new TopicPartition(topic, 0); + Cluster cluster = TestUtils.singletonCluster(topic, 1); + + when(ctx.sender.isRunning()).thenReturn(true); + when(ctx.metadata.fetch()).thenReturn(cluster); + + long timestamp = ctx.time.milliseconds(); + ProducerRecord record = new ProducerRecord<>(topic, null, timestamp, "key", "value"); + FutureRecordMetadata future = expectAppend(ctx, record, topicPartition, cluster); + + try (KafkaProducer producer = ctx.newKafkaProducer()) { + assertEquals(future, producer.send(record)); + assertFalse(future.isDone()); + verify(ctx.transactionManager).maybeAddPartition(topicPartition); + } + } + + @Test + public void testPartitionAddedToTransactionAfterFullBatchRetry() throws Exception { + StringSerializer serializer = new StringSerializer(); + KafkaProducerTestContext ctx = new KafkaProducerTestContext<>(testInfo, serializer); + + String topic = "foo"; + TopicPartition topicPartition0 = new TopicPartition(topic, 0); + TopicPartition topicPartition1 = new TopicPartition(topic, 1); + Cluster cluster = TestUtils.singletonCluster(topic, 2); + + when(ctx.sender.isRunning()).thenReturn(true); + when(ctx.metadata.fetch()).thenReturn(cluster); + + long timestamp = ctx.time.milliseconds(); + ProducerRecord record = new ProducerRecord<>(topic, null, timestamp, "key", "value"); + + FutureRecordMetadata future = expectAppendWithAbortForNewBatch( + ctx, + record, + topicPartition0, + topicPartition1, + cluster + ); + + try (KafkaProducer producer = ctx.newKafkaProducer()) { + assertEquals(future, producer.send(record)); + assertFalse(future.isDone()); + verify(ctx.partitioner).onNewBatch(topic, cluster, 0); + verify(ctx.transactionManager, never()).maybeAddPartition(topicPartition0); + verify(ctx.transactionManager).maybeAddPartition(topicPartition1); + } + } + + private FutureRecordMetadata expectAppend( + KafkaProducerTestContext ctx, + ProducerRecord record, + TopicPartition initialSelectedPartition, + Cluster cluster + ) throws InterruptedException { + byte[] serializedKey = ctx.serializer.serialize(topic, record.key()); + byte[] serializedValue = ctx.serializer.serialize(topic, record.value()); + long timestamp = record.timestamp() == null ? ctx.time.milliseconds() : record.timestamp(); + + ProduceRequestResult requestResult = new ProduceRequestResult(initialSelectedPartition); + FutureRecordMetadata futureRecordMetadata = new FutureRecordMetadata( + requestResult, + 5, + timestamp, + serializedKey.length, + serializedValue.length, + ctx.time + ); + + when(ctx.partitioner.partition( + initialSelectedPartition.topic(), + record.key(), + serializedKey, + record.value(), + serializedValue, + cluster + )).thenReturn(initialSelectedPartition.partition()); + + when(ctx.accumulator.append( + eq(initialSelectedPartition), + eq(timestamp), + eq(serializedKey), + eq(serializedValue), + eq(Record.EMPTY_HEADERS), + any(Callback.class), + anyLong(), + eq(true), + anyLong() + )).thenReturn(new RecordAccumulator.RecordAppendResult( + futureRecordMetadata, + false, + false, + false + )); + + return futureRecordMetadata; + } + + private FutureRecordMetadata expectAppendWithAbortForNewBatch( + KafkaProducerTestContext ctx, + ProducerRecord record, + TopicPartition initialSelectedPartition, + TopicPartition retrySelectedPartition, + Cluster cluster + ) throws InterruptedException { + byte[] serializedKey = ctx.serializer.serialize(topic, record.key()); + byte[] serializedValue = ctx.serializer.serialize(topic, record.value()); + long timestamp = record.timestamp() == null ? ctx.time.milliseconds() : record.timestamp(); + + ProduceRequestResult requestResult = new ProduceRequestResult(retrySelectedPartition); + FutureRecordMetadata futureRecordMetadata = new FutureRecordMetadata( + requestResult, + 0, + timestamp, + serializedKey.length, + serializedValue.length, + ctx.time + ); + + when(ctx.partitioner.partition( + initialSelectedPartition.topic(), + record.key(), + serializedKey, + record.value(), + serializedValue, + cluster + )).thenReturn(initialSelectedPartition.partition()) + .thenReturn(retrySelectedPartition.partition()); + + when(ctx.accumulator.append( + eq(initialSelectedPartition), + eq(timestamp), + eq(serializedKey), + eq(serializedValue), + eq(Record.EMPTY_HEADERS), + any(Callback.class), + anyLong(), + eq(true), // abortOnNewBatch + anyLong() + )).thenReturn(new RecordAccumulator.RecordAppendResult( + null, + false, + false, + true + )); + + when(ctx.accumulator.append( + eq(retrySelectedPartition), + eq(timestamp), + eq(serializedKey), + eq(serializedValue), + eq(Record.EMPTY_HEADERS), + any(Callback.class), + anyLong(), + eq(false), // abortOnNewBatch + anyLong() + )).thenReturn(new RecordAccumulator.RecordAppendResult( + futureRecordMetadata, + false, + true, + false + )); + + return futureRecordMetadata; + } + + private static final List CLIENT_IDS = new ArrayList<>(); public static class SerializerForClientId implements Serializer { @@ -2012,4 +2202,96 @@ public void close() { public void configure(Map configs) { } } + + private static class KafkaProducerTestContext { + private final TestInfo testInfo; + private final Map configs; + private final Serializer serializer; + private ProducerMetadata metadata = mock(ProducerMetadata.class); + private RecordAccumulator accumulator = mock(RecordAccumulator.class); + private Sender sender = mock(Sender.class); + private TransactionManager transactionManager = mock(TransactionManager.class); + private Partitioner partitioner = mock(Partitioner.class); + private KafkaThread ioThread = mock(KafkaThread.class); + private Time time = new MockTime(); + private Metrics metrics = new Metrics(time); + private List> interceptors = new ArrayList<>(); + + public KafkaProducerTestContext( + TestInfo testInfo, + Serializer serializer + ) { + this(testInfo, new HashMap<>(), serializer); + } + + public KafkaProducerTestContext( + TestInfo testInfo, + Map configs, + Serializer serializer + ) { + this.testInfo = testInfo; + this.configs = configs; + this.serializer = serializer; + + if (!configs.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) { + configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); + } + } + + public KafkaProducerTestContext setProducerMetadata(ProducerMetadata metadata) { + this.metadata = metadata; + return this; + } + + public KafkaProducerTestContext setAccumulator(RecordAccumulator accumulator) { + this.accumulator = accumulator; + return this; + } + + public KafkaProducerTestContext setSender(Sender sender) { + this.sender = sender; + return this; + } + + public KafkaProducerTestContext setTransactionManager(TransactionManager transactionManager) { + this.transactionManager = transactionManager; + return this; + } + + public KafkaProducerTestContext addInterceptor(ProducerInterceptor interceptor) { + this.interceptors.add(interceptor); + return this; + } + + public KafkaProducerTestContext setTime(Time time) { + this.time = time; + return this; + } + + public KafkaProducer newKafkaProducer() { + LogContext logContext = new LogContext("[Producer test=" + testInfo.getDisplayName() + "] "); + + ProducerConfig producerConfig = new ProducerConfig( + ProducerConfig.appendSerializerToConfig(configs, serializer, serializer)); + + ProducerInterceptors interceptors = new ProducerInterceptors<>(this.interceptors); + + return new KafkaProducer<>( + producerConfig, + logContext, + metrics, + serializer, + serializer, + metadata, + accumulator, + transactionManager, + sender, + interceptors, + partitioner, + time, + ioThread + ); + } + } + } From 4218fc61fedb02b78d35c88e56ab253baaf09f39 Mon Sep 17 00:00:00 2001 From: bozhao12 <102274736+bozhao12@users.noreply.github.com> Date: Wed, 6 Apr 2022 00:56:23 +0800 Subject: [PATCH 087/447] KAFKA-13778: Fetch from follower should never run the preferred read replica selection (#11965) The current preferred read replica selection logic relies on `partition.leaderReplicaIdOpt` to determine if the selection must be run. The issue is that `partition.leaderReplicaIdOpt` is defined for both the leader and the followers thus the logic is ran all the time. The impact is not too bad as the leader is selected most of the time when the logic is ran by the follower and the leader is filtered out. However there are cases where the selection on a follower could redirect the consumer to another follower under certain rare conditions. For instance with the `RackAwareReplicaSelector `, the follower must have stale replica states from a previous leadership and must have other followers in the same rack for instance. Other implementation of the selection logic could be more impacted. This patch ensures that the preferred read replica selection is only ran by the leader. Reviewers: David Jacot --- .../main/scala/kafka/cluster/Partition.scala | 5 ++ .../scala/kafka/server/ReplicaManager.scala | 2 +- .../kafka/server/ReplicaManagerTest.scala | 64 ++++++++++++++++++- 3 files changed, 68 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index e692db83ea80..8a6df9ff7f77 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -256,6 +256,7 @@ class Partition(val topicPartition: TopicPartition, // start offset for 'leaderEpoch' above (leader epoch of the current leader for this partition), // defined when this broker is leader for partition @volatile private var leaderEpochStartOffsetOpt: Option[Long] = None + // Replica ID of the leader, defined when this broker is leader or follower for the partition. @volatile var leaderReplicaIdOpt: Option[Int] = None @volatile private[cluster] var partitionState: PartitionState = CommittedPartitionState(Set.empty, LeaderRecoveryState.RECOVERED) @volatile var assignmentState: AssignmentState = SimpleAssignmentState(Seq.empty) @@ -433,6 +434,10 @@ class Partition(val topicPartition: TopicPartition, */ def isLeader: Boolean = leaderReplicaIdOpt.contains(localBrokerId) + def leaderIdIfLocal: Option[Int] = { + leaderReplicaIdOpt.filter(_ == localBrokerId) + } + private def localLogWithEpochOrException(currentLeaderEpoch: Optional[Integer], requireLeader: Boolean): UnifiedLog = { getLocalLog(currentLeaderEpoch, requireLeader) match { diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 4b77a4a56c5d..eff4824c4794 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -1233,7 +1233,7 @@ class ReplicaManager(val config: KafkaConfig, replicaId: Int, fetchOffset: Long, currentTimeMs: Long): Option[Int] = { - partition.leaderReplicaIdOpt.flatMap { leaderReplicaId => + partition.leaderIdIfLocal.flatMap { leaderReplicaId => // Don't look up preferred for follower fetches via normal replication if (Request.isValidBrokerId(replicaId)) None diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index a17c70b72173..3656a91af559 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -21,7 +21,7 @@ import java.io.File import java.net.InetAddress import java.nio.file.Files import java.util -import java.util.concurrent.atomic.AtomicReference +import java.util.concurrent.atomic.{AtomicLong, AtomicReference} import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.stream.IntStream import java.util.{Collections, Optional, Properties} @@ -44,7 +44,7 @@ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record._ -import org.apache.kafka.common.replica.ClientMetadata +import org.apache.kafka.common.replica.{ClientMetadata, PartitionView, ReplicaSelector, ReplicaView} import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse @@ -1300,6 +1300,54 @@ class ReplicaManagerTest { TestUtils.assertNoNonDaemonThreads(this.getClass.getName) } + @Test + def testFetchFromFollowerShouldNotRunPreferLeaderSelect(): Unit = { + val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), + propsModifier = props => props.put(KafkaConfig.ReplicaSelectorClassProp, classOf[MockReplicaSelector].getName)) + try { + val leaderBrokerId = 0 + val followerBrokerId = 1 + val brokerList = Seq[Integer](leaderBrokerId, followerBrokerId).asJava + val topicId = Uuid.randomUuid() + val tp0 = new TopicPartition(topic, 0) + val tidp0 = new TopicIdPartition(topicId, tp0) + initializeLogAndTopicId(replicaManager, tp0, topicId) + + // Make this replica the follower + val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(1) + .setLeaderEpoch(1) + .setIsr(brokerList) + .setZkVersion(0) + .setReplicas(brokerList) + .setIsNew(false)).asJava, + Collections.singletonMap(topic, topicId), + Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() + replicaManager.becomeLeaderOrFollower(1, leaderAndIsrRequest, (_, _) => ()) + + val metadata = new DefaultClientMetadata("rack-a", "client-id", + InetAddress.getLocalHost, KafkaPrincipal.ANONYMOUS, "default") + + val consumerResult = fetchAsConsumer(replicaManager, tidp0, + new PartitionData(Uuid.ZERO_UUID, 0, 0, 100000, + Optional.empty()), clientMetadata = Some(metadata)) + + // Fetch from follower succeeds + assertTrue(consumerResult.isFired) + + // Expect not run the preferred read replica selection + assertEquals(0, replicaManager.replicaSelectorOpt.get.asInstanceOf[MockReplicaSelector].getSelectionCount) + + // Only leader will compute preferred replica + assertTrue(consumerResult.assertFired.preferredReadReplica.isEmpty) + + } finally replicaManager.shutdown(checkpointHW = false) + } + @Test def testFetchShouldReturnImmediatelyWhenPreferredReadReplicaIsDefined(): Unit = { val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), @@ -3719,3 +3767,15 @@ class ReplicaManagerTest { } } } + +class MockReplicaSelector extends ReplicaSelector { + + private val selectionCount = new AtomicLong() + + def getSelectionCount: Long = selectionCount.get + + override def select(topicPartition: TopicPartition, clientMetadata: ClientMetadata, partitionView: PartitionView): Optional[ReplicaView] = { + selectionCount.incrementAndGet() + Optional.of(partitionView.leader) + } +} From f0a2b62b0e7660ca9e0f0159da636463356206c5 Mon Sep 17 00:00:00 2001 From: Xiaoyue Xue <2821566237@qq.com> Date: Wed, 6 Apr 2022 01:03:33 +0800 Subject: [PATCH 088/447] KAFKA-13794; Fix comparator of `inflightBatchesBySequence` in `TransactionManager` (#11991) Fixes a bug in the comparator used to sort producer inflight batches for a topic partition. This can cause batches in the map `inflightBatchesBySequence` to be removed incorrectly: i.e. one batch may be removed by another batch with the same sequence number. This leads to an `IllegalStateException` when the inflight request finally returns. This patch fixes the comparator to check equality of the `ProducerBatch` instances if the base sequences match. Reviewers: Jason Gustafson --- .../internals/TransactionManager.java | 10 ++- .../internals/TransactionManagerTest.java | 62 +++++++++++++++++++ 2 files changed, 70 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index 4362ad68af3d..de93bcaedf1e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -184,16 +184,22 @@ private static class TopicPartitionEntry { // responses which are due to the retention period elapsing, and those which are due to actual lost data. private long lastAckedOffset; + private static final Comparator PRODUCER_BATCH_COMPARATOR = (b1, b2) -> { + if (b1.baseSequence() < b2.baseSequence()) return -1; + else if (b1.baseSequence() > b2.baseSequence()) return 1; + else return b1.equals(b2) ? 0 : 1; + }; + TopicPartitionEntry() { this.producerIdAndEpoch = ProducerIdAndEpoch.NONE; this.nextSequence = 0; this.lastAckedSequence = NO_LAST_ACKED_SEQUENCE_NUMBER; this.lastAckedOffset = ProduceResponse.INVALID_OFFSET; - this.inflightBatchesBySequence = new TreeSet<>(Comparator.comparingInt(ProducerBatch::baseSequence)); + this.inflightBatchesBySequence = new TreeSet<>(PRODUCER_BATCH_COMPARATOR); } void resetSequenceNumbers(Consumer resetSequence) { - TreeSet newInflights = new TreeSet<>(Comparator.comparingInt(ProducerBatch::baseSequence)); + TreeSet newInflights = new TreeSet<>(PRODUCER_BATCH_COMPARATOR); for (ProducerBatch inflightBatch : inflightBatchesBySequence) { resetSequence.accept(inflightBatch); newInflights.add(inflightBatch); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java index 4227db5e61e6..64be3aeaf47b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java @@ -674,6 +674,68 @@ public void testBatchCompletedAfterProducerReset() { assertNull(transactionManager.nextBatchBySequence(tp0)); } + @Test + public void testDuplicateSequenceAfterProducerReset() throws Exception { + initializeTransactionManager(Optional.empty()); + initializeIdempotentProducerId(producerId, epoch); + + Metrics metrics = new Metrics(time); + final int requestTimeout = 10000; + final int deliveryTimeout = 15000; + + RecordAccumulator accumulator = new RecordAccumulator(logContext, 16 * 1024, CompressionType.NONE, 0, 0L, + deliveryTimeout, metrics, "", time, apiVersions, transactionManager, + new BufferPool(1024 * 1024, 16 * 1024, metrics, time, "")); + + Sender sender = new Sender(logContext, this.client, this.metadata, accumulator, false, + MAX_REQUEST_SIZE, ACKS_ALL, MAX_RETRIES, new SenderMetricsRegistry(metrics), this.time, requestTimeout, + 0, transactionManager, apiVersions); + + assertEquals(0, transactionManager.sequenceNumber(tp0).intValue()); + + Future responseFuture1 = accumulator.append(tp0, time.milliseconds(), "1".getBytes(), "1".getBytes(), Record.EMPTY_HEADERS, + null, MAX_BLOCK_TIMEOUT, false, time.milliseconds()).future; + sender.runOnce(); + assertEquals(1, transactionManager.sequenceNumber(tp0).intValue()); + + time.sleep(requestTimeout); + sender.runOnce(); + assertEquals(0, client.inFlightRequestCount()); + assertTrue(transactionManager.hasInflightBatches(tp0)); + assertEquals(1, transactionManager.sequenceNumber(tp0).intValue()); + sender.runOnce(); // retry + assertEquals(1, client.inFlightRequestCount()); + assertTrue(transactionManager.hasInflightBatches(tp0)); + assertEquals(1, transactionManager.sequenceNumber(tp0).intValue()); + + time.sleep(5000); // delivery time out + sender.runOnce(); + + // The retried request will remain inflight until the request timeout + // is reached even though the delivery timeout has expired and the + // future has completed exceptionally. + assertTrue(responseFuture1.isDone()); + TestUtils.assertFutureThrows(responseFuture1, TimeoutException.class); + assertFalse(transactionManager.hasInFlightRequest()); + assertEquals(1, client.inFlightRequestCount()); + + sender.runOnce(); // bump the epoch + assertEquals(epoch + 1, transactionManager.producerIdAndEpoch().epoch); + assertEquals(0, transactionManager.sequenceNumber(tp0).intValue()); + + Future responseFuture2 = accumulator.append(tp0, time.milliseconds(), "2".getBytes(), "2".getBytes(), Record.EMPTY_HEADERS, + null, MAX_BLOCK_TIMEOUT, false, time.milliseconds()).future; + sender.runOnce(); + sender.runOnce(); + assertEquals(0, transactionManager.firstInFlightSequence(tp0)); + assertEquals(1, transactionManager.sequenceNumber(tp0).intValue()); + + time.sleep(5000); // request time out again + sender.runOnce(); + assertTrue(transactionManager.hasInflightBatches(tp0)); // the latter batch failed and retried + assertFalse(responseFuture2.isDone()); + } + private ProducerBatch writeIdempotentBatchWithValue(TransactionManager manager, TopicPartition tp, String value) { From 88e5f133b5a572ba2ee786d10e74360fd62a104c Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 5 Apr 2022 15:42:08 -0400 Subject: [PATCH 089/447] KAFKA-13763: Improve unit testing coverage and flexibility for IncrementalCooperativeAssignor (#11974) Reviewers: Mickael Maison --- .../IncrementalCooperativeAssignorTest.java | 1301 ++++++++--------- 1 file changed, 618 insertions(+), 683 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignorTest.java index 0fe153132eb9..0ad7f63477eb 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignorTest.java @@ -27,6 +27,8 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.mockito.ArgumentCaptor; import org.mockito.Captor; import org.mockito.Mock; @@ -34,25 +36,28 @@ import org.mockito.junit.MockitoJUnit; import org.mockito.junit.MockitoRule; -import java.util.AbstractMap.SimpleEntry; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; +import java.util.stream.Stream; +import static org.apache.kafka.connect.runtime.distributed.ExtendedAssignment.duplicate; import static org.apache.kafka.connect.runtime.distributed.IncrementalCooperativeConnectProtocol.CONNECT_PROTOCOL_V1; import static org.apache.kafka.connect.runtime.distributed.IncrementalCooperativeConnectProtocol.CONNECT_PROTOCOL_V2; import static org.apache.kafka.connect.runtime.distributed.WorkerCoordinator.WorkerLoad; -import static org.hamcrest.CoreMatchers.hasItems; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.runners.Parameterized.Parameter; import static org.junit.runners.Parameterized.Parameters; @@ -64,6 +69,7 @@ import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; +@RunWith(Parameterized.class) public class IncrementalCooperativeAssignorTest { @Rule public MockitoRule rule = MockitoJUnit.rule(); @@ -76,15 +82,14 @@ public class IncrementalCooperativeAssignorTest { @Parameters public static Iterable mode() { - return Arrays.asList(new Object[][] {{CONNECT_PROTOCOL_V1, CONNECT_PROTOCOL_V2}}); + return Arrays.asList(CONNECT_PROTOCOL_V1, CONNECT_PROTOCOL_V2); } @Parameter public short protocolVersion; - private ClusterConfigState configState; + private Map connectors; private Map memberConfigs; - private Map expectedMemberConfigs; private long offset; private String leader; private String leaderUrl; @@ -92,7 +97,6 @@ public static Iterable mode() { private int rebalanceDelay; private IncrementalCooperativeAssignor assignor; private int rebalanceNum; - Map assignments; Map returnedAssignments; @Before @@ -100,11 +104,12 @@ public void setup() { leader = "worker1"; leaderUrl = expectedLeaderUrl(leader); offset = 10; - configState = clusterConfigState(offset, 2, 4); - memberConfigs = memberConfigs(leader, offset, 1, 1); + connectors = new HashMap<>(); + addNewConnector("connector1", 4); + addNewConnector("connector2", 4); + memberConfigs = memberConfigs(leader, offset, "worker1"); time = Time.SYSTEM; rebalanceDelay = DistributedConfig.SCHEDULED_REBALANCE_MAX_DELAY_MS_DEFAULT; - assignments = new HashMap<>(); initAssignor(); } @@ -123,61 +128,38 @@ public void initAssignor() { @Test public void testTaskAssignmentWhenWorkerJoins() { - when(coordinator.configSnapshot()).thenReturn(configState); + updateConfigSnapshot(); doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); // First assignment with 1 worker and 2 connectors configured but not yet assigned - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(2, 8, 0, 0, "worker1"); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker1"); + assertConnectorAllocations(2); + assertTaskAllocations(8); + assertBalancedAndCompleteAllocation(); // Second assignment with a second worker joining and all connectors running on previous worker - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - memberConfigs.put("worker2", new ExtendedWorkerState(leaderUrl, offset, null)); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 1, 4, "worker1", "worker2"); + addNewEmptyWorkers("worker2"); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker1", "worker2"); + assertConnectorAllocations(0, 1); + assertTaskAllocations(0, 4); // Third assignment after revocations - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(1, 4, 0, 0, "worker1", "worker2"); + performStandardRebalance(); + assertDelay(0); + assertConnectorAllocations(1, 1); + assertTaskAllocations(4, 4); + assertBalancedAndCompleteAllocation(); // A fourth rebalance should not change assignments - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 0, "worker1", "worker2"); + performStandardRebalance(); + assertDelay(0); + assertEmptyAssignment(); - verify(coordinator, times(rebalanceNum)).configSnapshot(); - verify(coordinator, times(rebalanceNum)).leaderState(any()); - verify(coordinator, times(2 * rebalanceNum)).generationId(); - verify(coordinator, times(rebalanceNum)).memberId(); - verify(coordinator, times(rebalanceNum)).lastCompletedGenerationId(); + verifyCoordinatorInteractions(); } @Test @@ -186,69 +168,45 @@ public void testTaskAssignmentWhenWorkerLeavesPermanently() { time = new MockTime(); initAssignor(); - when(coordinator.configSnapshot()).thenReturn(configState); + updateConfigSnapshot(); doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); // First assignment with 2 workers and 2 connectors configured but not yet assigned - memberConfigs.put("worker2", new ExtendedWorkerState(leaderUrl, offset, null)); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(2, 8, 0, 0, "worker1", "worker2"); + addNewEmptyWorkers("worker2"); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker1", "worker2"); + assertConnectorAllocations(1, 1); + assertTaskAllocations(4, 4); + assertBalancedAndCompleteAllocation(); // Second assignment with only one worker remaining in the group. The worker that left the // group was a follower. No re-assignments take place immediately and the count // down for the rebalance delay starts - applyAssignments(returnedAssignments); - assignments.remove("worker2"); - memberConfigs = memberConfigs(leader, offset, assignments); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(rebalanceDelay, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 0, "worker1"); + removeWorkers("worker2"); + performStandardRebalance(); + assertDelay(rebalanceDelay); + assertWorkers("worker1"); + assertEmptyAssignment(); time.sleep(rebalanceDelay / 2); // Third (incidental) assignment with still only one worker in the group. Max delay has not // been reached yet - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(rebalanceDelay / 2, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 0, "worker1"); + performStandardRebalance(); + assertDelay(rebalanceDelay / 2); + assertEmptyAssignment(); time.sleep(rebalanceDelay / 2 + 1); // Fourth assignment after delay expired - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(1, 4, 0, 0, "worker1"); + performStandardRebalance(); + assertDelay(0); + assertConnectorAllocations(2); + assertTaskAllocations(8); + assertBalancedAndCompleteAllocation(); - verify(coordinator, times(rebalanceNum)).configSnapshot(); - verify(coordinator, times(rebalanceNum)).leaderState(any()); - verify(coordinator, times(2 * rebalanceNum)).generationId(); - verify(coordinator, times(rebalanceNum)).memberId(); - verify(coordinator, times(rebalanceNum)).lastCompletedGenerationId(); + verifyCoordinatorInteractions(); } @Test @@ -257,86 +215,56 @@ public void testTaskAssignmentWhenWorkerBounces() { time = new MockTime(); initAssignor(); - when(coordinator.configSnapshot()).thenReturn(configState); + updateConfigSnapshot(); doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); // First assignment with 2 workers and 2 connectors configured but not yet assigned - memberConfigs.put("worker2", new ExtendedWorkerState(leaderUrl, offset, null)); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(2, 8, 0, 0, "worker1", "worker2"); + addNewEmptyWorkers("worker2"); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker1", "worker2"); + assertConnectorAllocations(1, 1); + assertTaskAllocations(4, 4); + assertBalancedAndCompleteAllocation(); // Second assignment with only one worker remaining in the group. The worker that left the // group was a follower. No re-assignments take place immediately and the count // down for the rebalance delay starts - applyAssignments(returnedAssignments); - assignments.remove("worker2"); - memberConfigs = memberConfigs(leader, offset, assignments); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(rebalanceDelay, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 0, "worker1"); + removeWorkers("worker2"); + performStandardRebalance(); + assertDelay(rebalanceDelay); + assertWorkers("worker1"); + assertEmptyAssignment(); time.sleep(rebalanceDelay / 2); // Third (incidental) assignment with still only one worker in the group. Max delay has not // been reached yet - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(rebalanceDelay / 2, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 0, "worker1"); + performStandardRebalance(); + assertDelay(rebalanceDelay / 2); + assertEmptyAssignment(); time.sleep(rebalanceDelay / 4); // Fourth assignment with the second worker returning before the delay expires // Since the delay is still active, lost assignments are not reassigned yet - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - memberConfigs.put("worker2", new ExtendedWorkerState(leaderUrl, offset, null)); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(rebalanceDelay / 4, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 0, "worker1", "worker2"); + addNewEmptyWorkers("worker2"); + performStandardRebalance(); + assertDelay(rebalanceDelay / 4); + assertWorkers("worker1", "worker2"); + assertEmptyAssignment(); time.sleep(rebalanceDelay / 4); // Fifth assignment with the same two workers. The delay has expired, so the lost // assignments ought to be assigned to the worker that has appeared as returned. - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(1, 4, 0, 0, "worker1", "worker2"); + performStandardRebalance(); + assertDelay(0); + assertConnectorAllocations(1, 1); + assertTaskAllocations(4, 4); + assertBalancedAndCompleteAllocation(); - verify(coordinator, times(rebalanceNum)).configSnapshot(); - verify(coordinator, times(rebalanceNum)).leaderState(any()); - verify(coordinator, times(2 * rebalanceNum)).generationId(); - verify(coordinator, times(rebalanceNum)).memberId(); - verify(coordinator, times(rebalanceNum)).lastCompletedGenerationId(); + verifyCoordinatorInteractions(); } @Test @@ -345,59 +273,42 @@ public void testTaskAssignmentWhenLeaderLeavesPermanently() { time = new MockTime(); initAssignor(); - when(coordinator.configSnapshot()).thenReturn(configState); + updateConfigSnapshot(); doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); // First assignment with 3 workers and 2 connectors configured but not yet assigned - memberConfigs.put("worker2", new ExtendedWorkerState(leaderUrl, offset, null)); - memberConfigs.put("worker3", new ExtendedWorkerState(leaderUrl, offset, null)); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(2, 8, 0, 0, "worker1", "worker2", "worker3"); + addNewEmptyWorkers("worker2", "worker3"); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker1", "worker2", "worker3"); + assertConnectorAllocations(0, 1, 1); + assertTaskAllocations(2, 3, 3); + assertBalancedAndCompleteAllocation(); // Second assignment with two workers remaining in the group. The worker that left the // group was the leader. The new leader has no previous assignments and is not tracking a // delay upon a leader's exit - applyAssignments(returnedAssignments); - assignments.remove("worker1"); + removeWorkers("worker1"); leader = "worker2"; leaderUrl = expectedLeaderUrl(leader); - memberConfigs = memberConfigs(leader, offset, assignments); // The fact that the leader bounces means that the assignor starts from a clean slate initAssignor(); // Capture needs to be reset to point to the new assignor doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(1, 3, 0, 0, "worker2", "worker3"); - - // Third (incidental) assignment with still only one worker in the group. - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 0, "worker2", "worker3"); - - verify(coordinator, times(rebalanceNum)).configSnapshot(); - verify(coordinator, times(rebalanceNum)).leaderState(any()); - verify(coordinator, times(2 * rebalanceNum)).generationId(); - verify(coordinator, times(rebalanceNum)).memberId(); - verify(coordinator, times(rebalanceNum)).lastCompletedGenerationId(); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker2", "worker3"); + assertConnectorAllocations(1, 1); + assertTaskAllocations(4, 4); + assertBalancedAndCompleteAllocation(); + + // Third (incidental) assignment with still only two workers in the group. + performStandardRebalance(); + assertDelay(0); + assertEmptyAssignment(); + + verifyCoordinatorInteractions(); } @Test @@ -406,74 +317,54 @@ public void testTaskAssignmentWhenLeaderBounces() { time = new MockTime(); initAssignor(); - when(coordinator.configSnapshot()).thenReturn(configState); + updateConfigSnapshot(); doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); // First assignment with 3 workers and 2 connectors configured but not yet assigned - memberConfigs.put("worker2", new ExtendedWorkerState(leaderUrl, offset, null)); - memberConfigs.put("worker3", new ExtendedWorkerState(leaderUrl, offset, null)); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(2, 8, 0, 0, "worker1", "worker2", "worker3"); + addNewEmptyWorkers("worker2", "worker3"); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker1", "worker2", "worker3"); + assertConnectorAllocations(0, 1, 1); + assertTaskAllocations(2, 3, 3); + assertBalancedAndCompleteAllocation(); // Second assignment with two workers remaining in the group. The worker that left the // group was the leader. The new leader has no previous assignments and is not tracking a // delay upon a leader's exit - applyAssignments(returnedAssignments); - assignments.remove("worker1"); + removeWorkers("worker1"); leader = "worker2"; leaderUrl = expectedLeaderUrl(leader); - memberConfigs = memberConfigs(leader, offset, assignments); // The fact that the leader bounces means that the assignor starts from a clean slate initAssignor(); // Capture needs to be reset to point to the new assignor doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(1, 3, 0, 0, "worker2", "worker3"); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker2", "worker3"); + assertConnectorAllocations(1, 1); + assertTaskAllocations(4, 4); + assertBalancedAndCompleteAllocation(); // Third assignment with the previous leader returning as a follower. In this case, the // arrival of the previous leader is treated as an arrival of a new worker. Reassignment // happens immediately, first with a revocation - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - memberConfigs.put("worker1", new ExtendedWorkerState(leaderUrl, offset, null)); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 2, "worker1", "worker2", "worker3"); + addNewEmptyWorkers("worker1"); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker1", "worker2", "worker3"); + assertConnectorAllocations(0, 1, 1); + assertTaskAllocations(0, 3, 3); // Fourth assignment after revocations - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 2, 0, 0, "worker1", "worker2", "worker3"); + performStandardRebalance(); + assertDelay(0); + assertConnectorAllocations(0, 1, 1); + assertTaskAllocations(2, 3, 3); + assertBalancedAndCompleteAllocation(); - verify(coordinator, times(rebalanceNum)).configSnapshot(); - verify(coordinator, times(rebalanceNum)).leaderState(any()); - verify(coordinator, times(2 * rebalanceNum)).generationId(); - verify(coordinator, times(rebalanceNum)).memberId(); - verify(coordinator, times(rebalanceNum)).lastCompletedGenerationId(); + verifyCoordinatorInteractions(); } @Test @@ -482,44 +373,30 @@ public void testTaskAssignmentWhenFirstAssignmentAttemptFails() { time = new MockTime(); initAssignor(); - when(coordinator.configSnapshot()).thenReturn(configState); + updateConfigSnapshot(); doThrow(new RuntimeException("Unable to send computed assignment with SyncGroupRequest")) .when(assignor).serializeAssignments(assignmentsCapture.capture()); // First assignment with 2 workers and 2 connectors configured but not yet assigned - memberConfigs.put("worker2", new ExtendedWorkerState(leaderUrl, offset, null)); - try { - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - } catch (RuntimeException e) { - RequestFuture.failure(e); - } - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); + addNewEmptyWorkers("worker2"); + performFailedRebalance(); // This was the assignment that should have been sent, but didn't make it all the way - assertDelay(0, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(2, 8, 0, 0, "worker1", "worker2"); + assertDelay(0); + assertWorkers("worker1", "worker2"); + assertConnectorAllocations(0, 0); + assertTaskAllocations(0, 0); // Second assignment happens with members returning the same assignments (memberConfigs) // as the first time. The assignor detects that the number of members did not change and // avoids the rebalance delay, treating the lost assignments as new assignments. doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(2, 8, 0, 0, "worker1", "worker2"); + performStandardRebalance(); + assertDelay(0); + assertConnectorAllocations(1, 1); + assertTaskAllocations(4, 4); + assertBalancedAndCompleteAllocation(); - verify(coordinator, times(rebalanceNum)).configSnapshot(); - verify(coordinator, times(rebalanceNum)).leaderState(any()); - verify(coordinator, times(2 * rebalanceNum)).generationId(); - verify(coordinator, times(rebalanceNum)).memberId(); - verify(coordinator, times(rebalanceNum)).lastCompletedGenerationId(); + verifyCoordinatorInteractions(); } @Test @@ -528,60 +405,48 @@ public void testTaskAssignmentWhenSubsequentAssignmentAttemptFails() { time = new MockTime(); initAssignor(); - when(coordinator.configSnapshot()).thenReturn(configState); + updateConfigSnapshot(); doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); // First assignment with 2 workers and 2 connectors configured but not yet assigned - memberConfigs.put("worker2", new ExtendedWorkerState(leaderUrl, offset, null)); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(2, 8, 0, 0, "worker1", "worker2"); - - when(coordinator.configSnapshot()).thenReturn(configState); + addNewEmptyWorkers("worker2"); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker1", "worker2"); + assertConnectorAllocations(1, 1); + assertTaskAllocations(4, 4); + assertBalancedAndCompleteAllocation(); + + updateConfigSnapshot(); doThrow(new RuntimeException("Unable to send computed assignment with SyncGroupRequest")) .when(assignor).serializeAssignments(assignmentsCapture.capture()); // Second assignment triggered by a third worker joining. The computed assignment should // revoke tasks from the existing group. But the assignment won't be correctly delivered. - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - memberConfigs.put("worker3", new ExtendedWorkerState(leaderUrl, offset, null)); - try { - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - } catch (RuntimeException e) { - RequestFuture.failure(e); - } - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); + addNewEmptyWorkers("worker3"); + performFailedRebalance(); // This was the assignment that should have been sent, but didn't make it all the way - assertDelay(0, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 2, "worker1", "worker2", "worker3"); + assertDelay(0); + assertWorkers("worker1", "worker2", "worker3"); + assertConnectorAllocations(0, 1, 1); + assertTaskAllocations(0, 4, 4); // Third assignment happens with members returning the same assignments (memberConfigs) // as the first time. doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertDelay(0, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 2, "worker1", "worker2", "worker3"); + performStandardRebalance(); + assertDelay(0); + assertConnectorAllocations(0, 1, 1); + assertTaskAllocations(0, 3, 3); - verify(coordinator, times(rebalanceNum)).configSnapshot(); - verify(coordinator, times(rebalanceNum)).leaderState(any()); - verify(coordinator, times(2 * rebalanceNum)).generationId(); - verify(coordinator, times(rebalanceNum)).memberId(); - verify(coordinator, times(rebalanceNum)).lastCompletedGenerationId(); + // Fourth assignment after revocations + performStandardRebalance(); + assertDelay(0); + assertConnectorAllocations(0, 1, 1); + assertTaskAllocations(2, 3, 3); + assertBalancedAndCompleteAllocation(); + + verifyCoordinatorInteractions(); } @Test @@ -590,96 +455,73 @@ public void testTaskAssignmentWhenSubsequentAssignmentAttemptFailsOutsideTheAssi time = new MockTime(); initAssignor(); - expectGeneration(); - when(coordinator.configSnapshot()).thenReturn(configState); + updateConfigSnapshot(); doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); // First assignment with 2 workers and 2 connectors configured but not yet assigned - memberConfigs.put("worker2", new ExtendedWorkerState(leaderUrl, offset, null)); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(2, 8, 0, 0, "worker1", "worker2"); + addNewEmptyWorkers("worker2"); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker1", "worker2"); + assertConnectorAllocations(1, 1); + assertTaskAllocations(4, 4); + assertBalancedAndCompleteAllocation(); // Second assignment triggered by a third worker joining. The computed assignment should // revoke tasks from the existing group. But the assignment won't be correctly delivered // and sync group with fail on the leader worker. - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - memberConfigs.put("worker3", new ExtendedWorkerState(leaderUrl, offset, null)); - when(coordinator.generationId()) - .thenReturn(assignor.previousGenerationId + 1) - .thenReturn(assignor.previousGenerationId + 1); - when(coordinator.lastCompletedGenerationId()).thenReturn(assignor.previousGenerationId - 1); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); + addNewEmptyWorkers("worker3"); + performFailedRebalance(); // This was the assignment that should have been sent, but didn't make it all the way - assertDelay(0, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 2, "worker1", "worker2", "worker3"); + assertDelay(0); + assertWorkers("worker1", "worker2", "worker3"); + assertConnectorAllocations(0, 1, 1); + assertTaskAllocations(0, 4, 4); // Third assignment happens with members returning the same assignments (memberConfigs) // as the first time. - when(coordinator.lastCompletedGenerationId()).thenReturn(assignor.previousGenerationId - 1); doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertDelay(0, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 2, "worker1", "worker2", "worker3"); + performRebalanceWithMismatchedGeneration(); + assertDelay(0); + assertConnectorAllocations(0, 1, 1); + assertTaskAllocations(0, 3, 3); - verify(coordinator, times(rebalanceNum)).configSnapshot(); - verify(coordinator, times(rebalanceNum)).leaderState(any()); - verify(coordinator, times(2 * rebalanceNum)).generationId(); - verify(coordinator, times(rebalanceNum)).memberId(); - verify(coordinator, times(rebalanceNum)).lastCompletedGenerationId(); + // Fourth assignment after revocations + performStandardRebalance(); + assertDelay(0); + assertConnectorAllocations(0, 1, 1); + assertTaskAllocations(2, 3, 3); + assertBalancedAndCompleteAllocation(); + + verifyCoordinatorInteractions(); } @Test public void testTaskAssignmentWhenConnectorsAreDeleted() { - configState = clusterConfigState(offset, 3, 4); - when(coordinator.configSnapshot()).thenReturn(configState); + addNewConnector("connector3", 4); + updateConfigSnapshot(); doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); // First assignment with 1 worker and 2 connectors configured but not yet assigned - memberConfigs.put("worker2", new ExtendedWorkerState(leaderUrl, offset, null)); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(3, 12, 0, 0, "worker1", "worker2"); + addNewEmptyWorkers("worker2"); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker1", "worker2"); + assertConnectorAllocations(1, 2); + assertTaskAllocations(6, 6); + assertBalancedAndCompleteAllocation(); // Second assignment with an updated config state that reflects removal of a connector - configState = clusterConfigState(offset + 1, 2, 4); - when(coordinator.configSnapshot()).thenReturn(configState); - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - expectGeneration(); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 1, 4, "worker1", "worker2"); - - verify(coordinator, times(rebalanceNum)).configSnapshot(); - verify(coordinator, times(rebalanceNum)).leaderState(any()); - verify(coordinator, times(2 * rebalanceNum)).generationId(); - verify(coordinator, times(rebalanceNum)).memberId(); - verify(coordinator, times(rebalanceNum)).lastCompletedGenerationId(); + removeConnector("connector3"); + offset++; + updateConfigSnapshot(); + performStandardRebalance(); + assertDelay(0); + assertConnectorAllocations(1, 1); + assertTaskAllocations(4, 4); + assertBalancedAndCompleteAllocation(); + + verifyCoordinatorInteractions(); } @Test @@ -759,7 +601,7 @@ public void testLostAssignmentHandlingWhenWorkerBounces() { configuredAssignment.put("worker0", workerLoad("worker0", 0, 2, 0, 4)); configuredAssignment.put("worker1", workerLoad("worker1", 2, 2, 4, 4)); configuredAssignment.put("worker2", workerLoad("worker2", 4, 2, 8, 4)); - memberConfigs = memberConfigs(leader, offset, 0, 2); + memberConfigs = memberConfigs(leader, offset, "worker0", "worker1", "worker2"); ConnectorsAndTasks newSubmissions = new ConnectorsAndTasks.Builder().build(); @@ -769,16 +611,16 @@ public void testLostAssignmentHandlingWhenWorkerBounces() { new ArrayList<>(configuredAssignment.values()), memberConfigs); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.emptySet(), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.delay); assignor.previousMembers = new HashSet<>(memberConfigs.keySet()); String flakyWorker = "worker1"; WorkerLoad lostLoad = workerLoad(flakyWorker, 2, 2, 4, 4); - memberConfigs.remove(flakyWorker); + removeWorkers(flakyWorker); ConnectorsAndTasks lostAssignments = new ConnectorsAndTasks.Builder() .withCopies(lostLoad.connectors(), lostLoad.tasks()).build(); @@ -787,9 +629,9 @@ public void testLostAssignmentHandlingWhenWorkerBounces() { assignor.handleLostAssignments(lostAssignments, newSubmissions, new ArrayList<>(configuredAssignment.values()), memberConfigs); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.emptySet(), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(rebalanceDelay, assignor.delay); @@ -799,13 +641,13 @@ public void testLostAssignmentHandlingWhenWorkerBounces() { // A new worker (probably returning worker) has joined configuredAssignment.put(flakyWorker, new WorkerLoad.Builder(flakyWorker).build()); - memberConfigs.put(flakyWorker, new ExtendedWorkerState(leaderUrl, offset, null)); + addNewEmptyWorkers(flakyWorker); assignor.handleLostAssignments(lostAssignments, newSubmissions, new ArrayList<>(configuredAssignment.values()), memberConfigs); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.singleton(flakyWorker), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(rebalanceDelay, assignor.delay); @@ -824,9 +666,9 @@ public void testLostAssignmentHandlingWhenWorkerBounces() { configuredAssignment.getOrDefault(flakyWorker, new WorkerLoad.Builder(flakyWorker).build()) .tasks() .containsAll(lostAssignments.tasks())); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.emptySet(), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.delay); } @@ -845,7 +687,7 @@ public void testLostAssignmentHandlingWhenWorkerLeavesPermanently() { configuredAssignment.put("worker0", workerLoad("worker0", 0, 2, 0, 4)); configuredAssignment.put("worker1", workerLoad("worker1", 2, 2, 4, 4)); configuredAssignment.put("worker2", workerLoad("worker2", 4, 2, 8, 4)); - memberConfigs = memberConfigs(leader, offset, 0, 2); + memberConfigs = memberConfigs(leader, offset, "worker0", "worker1", "worker2"); ConnectorsAndTasks newSubmissions = new ConnectorsAndTasks.Builder().build(); @@ -855,16 +697,16 @@ public void testLostAssignmentHandlingWhenWorkerLeavesPermanently() { new ArrayList<>(configuredAssignment.values()), memberConfigs); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.emptySet(), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.delay); assignor.previousMembers = new HashSet<>(memberConfigs.keySet()); String removedWorker = "worker1"; WorkerLoad lostLoad = workerLoad(removedWorker, 2, 2, 4, 4); - memberConfigs.remove(removedWorker); + removeWorkers(removedWorker); ConnectorsAndTasks lostAssignments = new ConnectorsAndTasks.Builder() .withCopies(lostLoad.connectors(), lostLoad.tasks()).build(); @@ -873,9 +715,9 @@ public void testLostAssignmentHandlingWhenWorkerLeavesPermanently() { assignor.handleLostAssignments(lostAssignments, newSubmissions, new ArrayList<>(configuredAssignment.values()), memberConfigs); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.emptySet(), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(rebalanceDelay, assignor.delay); @@ -887,9 +729,9 @@ public void testLostAssignmentHandlingWhenWorkerLeavesPermanently() { assignor.handleLostAssignments(lostAssignments, newSubmissions, new ArrayList<>(configuredAssignment.values()), memberConfigs); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.emptySet(), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(rebalanceDelay, assignor.delay); @@ -902,9 +744,9 @@ public void testLostAssignmentHandlingWhenWorkerLeavesPermanently() { newSubmissions.connectors().containsAll(lostAssignments.connectors())); assertTrue("Wrong assignment of lost tasks", newSubmissions.tasks().containsAll(lostAssignments.tasks())); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.emptySet(), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.delay); } @@ -923,7 +765,7 @@ public void testLostAssignmentHandlingWithMoreThanOneCandidates() { configuredAssignment.put("worker0", workerLoad("worker0", 0, 2, 0, 4)); configuredAssignment.put("worker1", workerLoad("worker1", 2, 2, 4, 4)); configuredAssignment.put("worker2", workerLoad("worker2", 4, 2, 8, 4)); - memberConfigs = memberConfigs(leader, offset, 0, 2); + memberConfigs = memberConfigs(leader, offset, "worker0", "worker1", "worker2"); ConnectorsAndTasks newSubmissions = new ConnectorsAndTasks.Builder().build(); @@ -933,16 +775,16 @@ public void testLostAssignmentHandlingWithMoreThanOneCandidates() { new ArrayList<>(configuredAssignment.values()), memberConfigs); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.emptySet(), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.delay); assignor.previousMembers = new HashSet<>(memberConfigs.keySet()); String flakyWorker = "worker1"; WorkerLoad lostLoad = workerLoad(flakyWorker, 2, 2, 4, 4); - memberConfigs.remove(flakyWorker); + removeWorkers(flakyWorker); String newWorker = "worker3"; ConnectorsAndTasks lostAssignments = new ConnectorsAndTasks.Builder() @@ -950,13 +792,13 @@ public void testLostAssignmentHandlingWithMoreThanOneCandidates() { // Lost assignments detected - A new worker also has joined that is not the returning worker configuredAssignment.put(newWorker, new WorkerLoad.Builder(newWorker).build()); - memberConfigs.put(newWorker, new ExtendedWorkerState(leaderUrl, offset, null)); + addNewEmptyWorkers(newWorker); assignor.handleLostAssignments(lostAssignments, newSubmissions, new ArrayList<>(configuredAssignment.values()), memberConfigs); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.singleton(newWorker), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(rebalanceDelay, assignor.delay); @@ -966,15 +808,15 @@ public void testLostAssignmentHandlingWithMoreThanOneCandidates() { // Now two new workers have joined configuredAssignment.put(flakyWorker, new WorkerLoad.Builder(flakyWorker).build()); - memberConfigs.put(flakyWorker, new ExtendedWorkerState(leaderUrl, offset, null)); + addNewEmptyWorkers(flakyWorker); assignor.handleLostAssignments(lostAssignments, newSubmissions, new ArrayList<>(configuredAssignment.values()), memberConfigs); Set expectedWorkers = new HashSet<>(); expectedWorkers.addAll(Arrays.asList(newWorker, flakyWorker)); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", expectedWorkers, - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(rebalanceDelay, assignor.delay); @@ -1004,9 +846,9 @@ public void testLostAssignmentHandlingWithMoreThanOneCandidates() { listOfConnectorsInLast2Workers.containsAll(lostAssignments.connectors())); assertTrue("Wrong assignment of lost tasks", listOfTasksInLast2Workers.containsAll(lostAssignments.tasks())); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.emptySet(), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.delay); } @@ -1025,7 +867,7 @@ public void testLostAssignmentHandlingWhenWorkerBouncesBackButFinallyLeaves() { configuredAssignment.put("worker0", workerLoad("worker0", 0, 2, 0, 4)); configuredAssignment.put("worker1", workerLoad("worker1", 2, 2, 4, 4)); configuredAssignment.put("worker2", workerLoad("worker2", 4, 2, 8, 4)); - memberConfigs = memberConfigs(leader, offset, 0, 2); + memberConfigs = memberConfigs(leader, offset, "worker0", "worker1", "worker2"); ConnectorsAndTasks newSubmissions = new ConnectorsAndTasks.Builder().build(); @@ -1035,16 +877,16 @@ public void testLostAssignmentHandlingWhenWorkerBouncesBackButFinallyLeaves() { new ArrayList<>(configuredAssignment.values()), memberConfigs); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.emptySet(), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.delay); assignor.previousMembers = new HashSet<>(memberConfigs.keySet()); String veryFlakyWorker = "worker1"; WorkerLoad lostLoad = workerLoad(veryFlakyWorker, 2, 2, 4, 4); - memberConfigs.remove(veryFlakyWorker); + removeWorkers(veryFlakyWorker); ConnectorsAndTasks lostAssignments = new ConnectorsAndTasks.Builder() .withCopies(lostLoad.connectors(), lostLoad.tasks()).build(); @@ -1053,9 +895,9 @@ public void testLostAssignmentHandlingWhenWorkerBouncesBackButFinallyLeaves() { assignor.handleLostAssignments(lostAssignments, newSubmissions, new ArrayList<>(configuredAssignment.values()), memberConfigs); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.emptySet(), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(rebalanceDelay, assignor.delay); @@ -1065,13 +907,13 @@ public void testLostAssignmentHandlingWhenWorkerBouncesBackButFinallyLeaves() { // A new worker (probably returning worker) has joined configuredAssignment.put(veryFlakyWorker, new WorkerLoad.Builder(veryFlakyWorker).build()); - memberConfigs.put(veryFlakyWorker, new ExtendedWorkerState(leaderUrl, offset, null)); + addNewEmptyWorkers(veryFlakyWorker); assignor.handleLostAssignments(lostAssignments, newSubmissions, new ArrayList<>(configuredAssignment.values()), memberConfigs); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.singleton(veryFlakyWorker), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(rebalanceDelay, assignor.delay); @@ -1080,7 +922,7 @@ public void testLostAssignmentHandlingWhenWorkerBouncesBackButFinallyLeaves() { // The returning worker leaves permanently after joining briefly during the delay configuredAssignment.remove(veryFlakyWorker); - memberConfigs.remove(veryFlakyWorker); + removeWorkers(veryFlakyWorker); assignor.handleLostAssignments(lostAssignments, newSubmissions, new ArrayList<>(configuredAssignment.values()), memberConfigs); @@ -1088,160 +930,182 @@ public void testLostAssignmentHandlingWhenWorkerBouncesBackButFinallyLeaves() { newSubmissions.connectors().containsAll(lostAssignments.connectors())); assertTrue("Wrong assignment of lost tasks", newSubmissions.tasks().containsAll(lostAssignments.tasks())); - assertThat("Wrong set of workers for reassignments", + assertEquals("Wrong set of workers for reassignments", Collections.emptySet(), - is(assignor.candidateWorkersForReassignment)); + assignor.candidateWorkersForReassignment); assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.delay); } @Test public void testTaskAssignmentWhenTasksDuplicatedInWorkerAssignment() { - when(coordinator.configSnapshot()).thenReturn(configState); + updateConfigSnapshot(); doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); // First assignment with 1 worker and 2 connectors configured but not yet assigned - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(2, 8, 0, 0, "worker1"); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker1"); + assertConnectorAllocations(2); + assertTaskAllocations(8); + assertBalancedAndCompleteAllocation(); // Second assignment with a second worker with duplicate assignment joining and all connectors running on previous worker - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - ExtendedAssignment duplicatedWorkerAssignment = newExpandableAssignment(); - duplicatedWorkerAssignment.connectors().addAll(newConnectors(1, 2)); - duplicatedWorkerAssignment.tasks().addAll(newTasks("connector1", 0, 4)); - memberConfigs.put("worker2", new ExtendedWorkerState(leaderUrl, offset, duplicatedWorkerAssignment)); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 2, 8, "worker1", "worker2"); + addNewWorker("worker2", newConnectors(1, 2), newTasks("connector1", 0, 4)); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker1", "worker2"); + assertConnectorAllocations(0, 1); + assertTaskAllocations(0, 4); // Third assignment after revocations - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(1, 4, 0, 2, "worker1", "worker2"); + performStandardRebalance(); + assertDelay(0); + assertConnectorAllocations(1, 1); + assertTaskAllocations(2, 4); // fourth rebalance after revocations - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 2, 0, 0, "worker1", "worker2"); + performStandardRebalance(); + assertDelay(0); + assertConnectorAllocations(1, 1); + assertTaskAllocations(4, 4); + assertBalancedAndCompleteAllocation(); // Fifth rebalance should not change assignments - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 0, "worker1", "worker2"); + performStandardRebalance(); + assertDelay(0); + assertEmptyAssignment(); - verify(coordinator, times(rebalanceNum)).configSnapshot(); - verify(coordinator, times(rebalanceNum)).leaderState(any()); - verify(coordinator, times(2 * rebalanceNum)).generationId(); - verify(coordinator, times(rebalanceNum)).memberId(); - verify(coordinator, times(rebalanceNum)).lastCompletedGenerationId(); + verifyCoordinatorInteractions(); } @Test public void testDuplicatedAssignmentHandleWhenTheDuplicatedAssignmentsDeleted() { - when(coordinator.configSnapshot()).thenReturn(configState); + updateConfigSnapshot(); doReturn(Collections.EMPTY_MAP).when(assignor).serializeAssignments(assignmentsCapture.capture()); // First assignment with 1 worker and 2 connectors configured but not yet assigned - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(2, 8, 0, 0, "worker1"); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker1"); + assertConnectorAllocations(2); + assertTaskAllocations(8); + assertBalancedAndCompleteAllocation(); - //delete connector1 - configState = clusterConfigState(offset, 2, 1, 4); - when(coordinator.configSnapshot()).thenReturn(configState); + // Delete connector1 + removeConnector("connector1"); + updateConfigSnapshot(); // Second assignment with a second worker with duplicate assignment joining and the duplicated assignment is deleted at the same time - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - ExtendedAssignment duplicatedWorkerAssignment = newExpandableAssignment(); - duplicatedWorkerAssignment.connectors().addAll(newConnectors(1, 2)); - duplicatedWorkerAssignment.tasks().addAll(newTasks("connector1", 0, 4)); - memberConfigs.put("worker2", new ExtendedWorkerState(leaderUrl, offset, duplicatedWorkerAssignment)); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 2, 8, "worker1", "worker2"); + addNewWorker("worker2", newConnectors(1, 2), newTasks("connector1", 0, 4)); + performStandardRebalance(); + assertDelay(0); + assertWorkers("worker1", "worker2"); + assertConnectorAllocations(0, 1); + assertTaskAllocations(0, 4); // Third assignment after revocations - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 2, "worker1", "worker2"); + performStandardRebalance(); + assertDelay(0); + assertConnectorAllocations(0, 1); + assertTaskAllocations(0, 2); // fourth rebalance after revocations - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); - ++rebalanceNum; - returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 2, 0, 0, "worker1", "worker2"); + performStandardRebalance(); + assertDelay(0); + assertConnectorAllocations(0, 1); + assertTaskAllocations(2, 2); + assertBalancedAndCompleteAllocation(); // Fifth rebalance should not change assignments - applyAssignments(returnedAssignments); - memberConfigs = memberConfigs(leader, offset, assignments); - assignor.performTaskAssignment(leader, offset, memberConfigs, coordinator, protocolVersion); + performStandardRebalance(); + assertDelay(0); + assertEmptyAssignment(); + + verifyCoordinatorInteractions(); + } + + private void performStandardRebalance() { + performRebalance(false, false); + } + + private void performFailedRebalance() { + performRebalance(true, false); + } + + private void performRebalanceWithMismatchedGeneration() { + performRebalance(false, true); + } + + private void performRebalance(boolean assignmentFailure, boolean expectGenerationMismatch) { + expectGeneration(expectGenerationMismatch); + // Member configs are tracked by the assignor; create a deep copy here so that modifications to our own memberConfigs field + // are not accidentally propagated to the one used by the assignor + Map memberConfigsCopy = memberConfigs.entrySet().stream().collect(Collectors.toMap( + Map.Entry::getKey, + e -> { + ExtendedWorkerState originalWorkerState = e.getValue(); + return new ExtendedWorkerState( + originalWorkerState.url(), + originalWorkerState.offset(), + duplicate(originalWorkerState.assignment()) + ); + } + )); + try { + assignor.performTaskAssignment(leader, offset, memberConfigsCopy, coordinator, protocolVersion); + } catch (RuntimeException e) { + if (assignmentFailure) { + RequestFuture.failure(e); + } else { + throw e; + } + } ++rebalanceNum; returnedAssignments = assignmentsCapture.getValue(); - assertDelay(0, returnedAssignments); - expectedMemberConfigs = memberConfigs(leader, offset, returnedAssignments); - assertNoReassignments(memberConfigs, expectedMemberConfigs); - assertAssignment(0, 0, 0, 0, "worker1", "worker2"); + assertNoRedundantAssignments(); + if (!assignmentFailure) { + applyAssignments(leader, offset, returnedAssignments); + } + } - verify(coordinator, times(rebalanceNum)).configSnapshot(); - verify(coordinator, times(rebalanceNum)).leaderState(any()); - verify(coordinator, times(2 * rebalanceNum)).generationId(); - verify(coordinator, times(rebalanceNum)).memberId(); - verify(coordinator, times(rebalanceNum)).lastCompletedGenerationId(); + private void expectGeneration(boolean expectMismatch) { + when(coordinator.generationId()) + .thenReturn(assignor.previousGenerationId + 1); + int lastCompletedGenerationId = expectMismatch ? assignor.previousGenerationId - 1 : assignor.previousGenerationId; + when(coordinator.lastCompletedGenerationId()).thenReturn(lastCompletedGenerationId); } - private WorkerLoad emptyWorkerLoad(String worker) { + private void addNewEmptyWorkers(String... workers) { + for (String worker : workers) { + addNewWorker(worker, Collections.emptyList(), Collections.emptyList()); + } + } + + private void addNewWorker(String worker, List connectors, List tasks) { + ExtendedAssignment assignment = newExpandableAssignment(); + assignment.connectors().addAll(connectors); + assignment.tasks().addAll(tasks); + assertNull( + "Worker " + worker + " already exists", + memberConfigs.put(worker, new ExtendedWorkerState(leaderUrl, offset, assignment)) + ); + } + + private void removeWorkers(String... workers) { + for (String worker : workers) { + assertNotNull( + "Worker " + worker + " does not exist", + memberConfigs.remove(worker) + ); + } + } + + private static WorkerLoad emptyWorkerLoad(String worker) { return new WorkerLoad.Builder(worker).build(); } - private WorkerLoad workerLoad(String worker, int connectorStart, int connectorNum, + private static WorkerLoad workerLoad(String worker, int connectorStart, int connectorNum, int taskStart, int taskNum) { return new WorkerLoad.Builder(worker).with( newConnectors(connectorStart, connectorStart + connectorNum), @@ -1264,100 +1128,81 @@ private static List newTasks(String connectorName, int start, i .collect(Collectors.toList()); } - private static ClusterConfigState clusterConfigState(long offset, - int connectorNum, - int taskNum) { - return clusterConfigState(offset, 1, connectorNum, taskNum); - } - - private static ClusterConfigState clusterConfigState(long offset, - int connectorStart, - int connectorNum, - int taskNum) { - int connectorNumEnd = connectorStart + connectorNum - 1; - return new ClusterConfigState( - offset, - null, - connectorTaskCounts(connectorStart, connectorNumEnd, taskNum), - connectorConfigs(connectorStart, connectorNumEnd), - connectorTargetStates(connectorStart, connectorNumEnd, TargetState.STARTED), - taskConfigs(0, connectorNum, connectorNum * taskNum), - Collections.emptySet()); - } - - private static Map memberConfigs(String givenLeader, - long givenOffset, - Map givenAssignments) { - return givenAssignments.entrySet().stream() - .collect(Collectors.toMap( - Map.Entry::getKey, - e -> new ExtendedWorkerState(expectedLeaderUrl(givenLeader), givenOffset, e.getValue()))); - } - - private static Map memberConfigs(String givenLeader, - long givenOffset, - int start, - int connectorNum) { - return IntStream.range(start, connectorNum + 1) - .mapToObj(i -> new SimpleEntry<>("worker" + i, new ExtendedWorkerState(expectedLeaderUrl(givenLeader), givenOffset, null))) - .collect(Collectors.toMap(SimpleEntry::getKey, SimpleEntry::getValue)); + private void addNewConnector(String connector, int taskCount) { + assertNull( + "Connector " + connector + " already exists", + connectors.put(connector, taskCount) + ); } - private static Map connectorTaskCounts(int start, - int connectorNum, - int taskCounts) { - return IntStream.range(start, connectorNum + 1) - .mapToObj(i -> new SimpleEntry<>("connector" + i, taskCounts)) - .collect(Collectors.toMap(SimpleEntry::getKey, SimpleEntry::getValue)); + private void removeConnector(String connector) { + assertNotNull( + "Connector " + connector + " does not exist", + connectors.remove(connector) + ); } - private static Map> connectorConfigs(int start, int connectorNum) { - return IntStream.range(start, connectorNum + 1) - .mapToObj(i -> new SimpleEntry<>("connector" + i, new HashMap())) - .collect(Collectors.toMap(SimpleEntry::getKey, SimpleEntry::getValue)); + private void updateConfigSnapshot() { + when(coordinator.configSnapshot()).thenReturn(configState()); } - private static Map connectorTargetStates(int start, - int connectorNum, - TargetState state) { - return IntStream.range(start, connectorNum + 1) - .mapToObj(i -> new SimpleEntry<>("connector" + i, state)) - .collect(Collectors.toMap(SimpleEntry::getKey, SimpleEntry::getValue)); + private ClusterConfigState configState() { + Map taskCounts = new HashMap<>(connectors); + Map> connectorConfigs = taskCounts.keySet().stream().collect(Collectors.toMap( + Function.identity(), + connector -> Collections.emptyMap() + )); + Map targetStates = taskCounts.keySet().stream().collect(Collectors.toMap( + Function.identity(), + connector -> TargetState.STARTED + )); + Map> taskConfigs = taskCounts.entrySet().stream() + .flatMap(e -> IntStream.range(0, e.getValue()).mapToObj(i -> new ConnectorTaskId(e.getKey(), i))) + .collect(Collectors.toMap( + Function.identity(), + connectorTaskId -> Collections.emptyMap() + )); + return new ClusterConfigState( + offset, + null, + taskCounts, + connectorConfigs, + targetStates, + taskConfigs, + Collections.emptySet() + ); } - private static Map> taskConfigs(int start, - int connectorNum, - int taskNum) { - return IntStream.range(start, taskNum + 1) - .mapToObj(i -> new SimpleEntry<>( - new ConnectorTaskId("connector" + i / connectorNum + 1, i), - new HashMap()) - ).collect(Collectors.toMap(SimpleEntry::getKey, SimpleEntry::getValue)); + private Map memberConfigs(String givenLeader, long givenOffset, String... workers) { + return Stream.of(workers).collect(Collectors.toMap( + Function.identity(), + w -> new ExtendedWorkerState(expectedLeaderUrl(givenLeader), givenOffset, newExpandableAssignment(givenLeader, givenOffset)) + )); } - private void applyAssignments(Map newAssignments) { - newAssignments.forEach((k, v) -> { - assignments.computeIfAbsent(k, noop -> newExpandableAssignment()) - .connectors() - .removeAll(v.revokedConnectors()); - assignments.computeIfAbsent(k, noop -> newExpandableAssignment()) - .connectors() - .addAll(v.connectors()); - assignments.computeIfAbsent(k, noop -> newExpandableAssignment()) - .tasks() - .removeAll(v.revokedTasks()); - assignments.computeIfAbsent(k, noop -> newExpandableAssignment()) - .tasks() - .addAll(v.tasks()); + private void applyAssignments(String leader, long offset, Map newAssignments) { + newAssignments.forEach((worker, newAssignment) -> { + ExtendedAssignment workerAssignment = Optional.ofNullable(memberConfigs.get(worker)) + .map(ExtendedWorkerState::assignment) + .orElseGet(this::newExpandableAssignment); + workerAssignment.connectors().removeAll(newAssignment.revokedConnectors()); + workerAssignment.connectors().addAll(newAssignment.connectors()); + workerAssignment.tasks().removeAll(newAssignment.revokedTasks()); + workerAssignment.tasks().addAll(newAssignment.tasks()); + memberConfigs.put(worker, new ExtendedWorkerState(expectedLeaderUrl(leader), offset, workerAssignment)); }); } private ExtendedAssignment newExpandableAssignment() { + return newExpandableAssignment(leader, offset); + } + + private ExtendedAssignment newExpandableAssignment(String leader, long offset) { return new ExtendedAssignment( protocolVersion, ConnectProtocol.Assignment.NO_ERROR, leader, - leaderUrl, + expectedLeaderUrl(leader), offset, new ArrayList<>(), new ArrayList<>(), @@ -1370,100 +1215,190 @@ private static String expectedLeaderUrl(String givenLeader) { return "http://" + givenLeader + ":8083"; } - private void assertAssignment(int connectorNum, int taskNum, - int revokedConnectorNum, int revokedTaskNum, - String... workers) { - assertAssignment(leader, connectorNum, taskNum, revokedConnectorNum, revokedTaskNum, workers); + private void assertEmptyAssignment() { + assertEquals( + "No connectors should have been newly assigned during this round", + Collections.emptyList(), + extractFromAssignments(returnedAssignments, ConnectProtocol.Assignment::connectors) + ); + assertEquals( + "No tasks should have been newly assigned during this round", + Collections.emptyList(), + extractFromAssignments(returnedAssignments, ConnectProtocol.Assignment::tasks) + ); + assertEquals( + "No connectors should have been revoked during this round", + Collections.emptyList(), + extractFromAssignments(returnedAssignments, ExtendedAssignment::revokedConnectors) + ); + assertEquals( + "No tasks should have been revoked during this round", + Collections.emptyList(), + extractFromAssignments(returnedAssignments, ExtendedAssignment::revokedTasks) + ); } - private void assertAssignment(String expectedLeader, int connectorNum, int taskNum, - int revokedConnectorNum, int revokedTaskNum, - String... workers) { - assertThat("Wrong number of workers", - expectedMemberConfigs.keySet().size(), - is(workers.length)); - assertThat("Wrong set of workers", - new ArrayList<>(expectedMemberConfigs.keySet()), hasItems(workers)); - assertThat("Wrong number of assigned connectors", - expectedMemberConfigs.values().stream().map(v -> v.assignment().connectors().size()).reduce(0, Integer::sum), - is(connectorNum)); - assertThat("Wrong number of assigned tasks", - expectedMemberConfigs.values().stream().map(v -> v.assignment().tasks().size()).reduce(0, Integer::sum), - is(taskNum)); - assertThat("Wrong number of revoked connectors", - expectedMemberConfigs.values().stream().map(v -> v.assignment().revokedConnectors().size()).reduce(0, Integer::sum), - is(revokedConnectorNum)); - assertThat("Wrong number of revoked tasks", - expectedMemberConfigs.values().stream().map(v -> v.assignment().revokedTasks().size()).reduce(0, Integer::sum), - is(revokedTaskNum)); - assertThat("Wrong leader in assignments", - expectedMemberConfigs.values().stream().map(v -> v.assignment().leader()).distinct().collect(Collectors.joining(", ")), - is(expectedLeader)); - assertThat("Wrong leaderUrl in assignments", - expectedMemberConfigs.values().stream().map(v -> v.assignment().leaderUrl()).distinct().collect(Collectors.joining(", ")), - is(expectedLeaderUrl(expectedLeader))); + private void assertWorkers(String... workers) { + assertEquals( + "Wrong set of workers", + new HashSet<>(Arrays.asList(workers)), + returnedAssignments.keySet() + ); } - private void assertDelay(int expectedDelay, Map newAssignments) { - newAssignments.values().stream() - .forEach(a -> assertEquals( - "Wrong rebalance delay in " + a, expectedDelay, a.delay())); + /** + * Assert that the connector counts for each worker in the cluster match the expected counts. + * For example, calling {@code assertConnectorAllocations(0, 0, 2, 3)} ensures that there are two + * workers in the cluster that are assigned no connectors, one worker that is assigned two connectors, + * and one worker that is assigned three connectors. + */ + private void assertConnectorAllocations(int... connectorCounts) { + assertAllocations("connectors", ExtendedAssignment::connectors, connectorCounts); } - private void assertNoReassignments(Map existingAssignments, - Map newAssignments) { - assertNoDuplicateInAssignment(existingAssignments); - assertNoDuplicateInAssignment(newAssignments); + /** + * Assert that the task counts for each worker in the cluster match the expected counts. + * For example, calling {@code assertTaskAllocations(0, 0, 2, 3)} ensures that there are two + * workers in the cluster that are assigned no tasks, one worker that is assigned two tasks, + * and one worker that is assigned three tasks. + */ + private void assertTaskAllocations(int... taskCounts) { + assertAllocations("tasks", ExtendedAssignment::tasks, taskCounts); + } - List existingConnectors = existingAssignments.values().stream() - .flatMap(a -> a.assignment().connectors().stream()) - .collect(Collectors.toList()); - List newConnectors = newAssignments.values().stream() - .flatMap(a -> a.assignment().connectors().stream()) + private void assertAllocations(String allocated, Function> allocation, int... rawExpectedAllocations) { + List expectedAllocations = IntStream.of(rawExpectedAllocations) + .boxed() + .sorted() .collect(Collectors.toList()); + List actualAllocations = allocations(allocation); + assertEquals( + "Allocation of assigned " + allocated + " across cluster does not match expected counts", + expectedAllocations, + actualAllocations + ); + } - List existingTasks = existingAssignments.values().stream() - .flatMap(a -> a.assignment().tasks().stream()) + private List allocations(Function> allocation) { + return memberConfigs.values().stream() + .map(ExtendedWorkerState::assignment) + .map(allocation) + .map(Collection::size) + .sorted() .collect(Collectors.toList()); + } - List newTasks = newAssignments.values().stream() - .flatMap(a -> a.assignment().tasks().stream()) - .collect(Collectors.toList()); + private void assertDelay(int expectedDelay) { + returnedAssignments.values().forEach(a -> assertEquals( + "Wrong rebalance delay in " + a, expectedDelay, a.delay())); + } + + /** + * Ensure that no connectors or tasks that were already assigned during the previous round are newly assigned in this round, + * and that each newly-assigned connector and task is only assigned to a single worker. + */ + private void assertNoRedundantAssignments() { + Map existingAssignments = memberConfigs.entrySet().stream().collect(Collectors.toMap( + Map.Entry::getKey, + e -> e.getValue().assignment() + )); + + List existingConnectors = extractFromAssignments(existingAssignments, ConnectProtocol.Assignment::connectors); + List newConnectors = extractFromAssignments(returnedAssignments, ConnectProtocol.Assignment::connectors); + List existingTasks = extractFromAssignments(existingAssignments, ConnectProtocol.Assignment::tasks); + List newTasks = extractFromAssignments(returnedAssignments, ConnectProtocol.Assignment::tasks); + + assertNoDuplicates( + newConnectors, + "Connectors should be unique in assignments but duplicates were found; the set of newly-assigned connectors is " + newConnectors + ); + assertNoDuplicates( + newTasks, + "Tasks should be unique in assignments but duplicates were found; the set of newly-assigned tasks is " + newTasks + ); existingConnectors.retainAll(newConnectors); - assertThat("Found connectors in new assignment that already exist in current assignment", + assertEquals("Found connectors in new assignment that already exist in current assignment", Collections.emptyList(), - is(existingConnectors)); + existingConnectors); existingTasks.retainAll(newTasks); - assertThat("Found tasks in new assignment that already exist in current assignment", + assertEquals("Found tasks in new assignment that already exist in current assignment", Collections.emptyList(), - is(existingConnectors)); + existingConnectors); } - private void assertNoDuplicateInAssignment(Map existingAssignment) { - List existingConnectors = existingAssignment.values().stream() - .flatMap(a -> a.assignment().connectors().stream()) - .collect(Collectors.toList()); - Set existingUniqueConnectors = new HashSet<>(existingConnectors); - existingConnectors.removeAll(existingUniqueConnectors); - assertThat("Connectors should be unique in assignments but duplicates where found", - Collections.emptyList(), - is(existingConnectors)); + private void assertBalancedAndCompleteAllocation() { + assertBalancedAllocation(); + assertCompleteAllocation(); + } + + private void assertBalancedAllocation() { + List connectorCounts = allocations(ExtendedAssignment::connectors); + List taskCounts = allocations(ExtendedAssignment::tasks); + + int minConnectors = connectorCounts.get(0); + int maxConnectors = connectorCounts.get(connectorCounts.size() - 1); + + int minTasks = taskCounts.get(0); + int maxTasks = taskCounts.get(taskCounts.size() - 1); - List existingTasks = existingAssignment.values().stream() - .flatMap(a -> a.assignment().tasks().stream()) + assertTrue( + "Assignments are imbalanced. The spread of connectors across each worker is: " + connectorCounts, + maxConnectors - minConnectors <= 1 + ); + assertTrue( + "Assignments are imbalanced. The spread of tasks across each worker is: " + taskCounts, + maxTasks - minTasks <= 1 + ); + } + + private void assertCompleteAllocation() { + List allAssignedConnectors = extractFromAssignments(memberConfigs, e -> e.assignment().connectors()); + assertEquals( + "The set of connectors assigned across the cluster does not match the set of connectors in the config topic", + connectors.keySet(), + new HashSet<>(allAssignedConnectors) + ); + + Map> allAssignedTasks = extractFromAssignments(memberConfigs, e -> e.assignment().tasks()).stream() + .collect(Collectors.groupingBy(ConnectorTaskId::connector, Collectors.toList())); + + connectors.forEach((connector, taskCount) -> { + Set expectedTasks = IntStream.range(0, taskCount) + .mapToObj(i -> new ConnectorTaskId(connector, i)) + .collect(Collectors.toSet()); + assertEquals( + "The set of tasks assigned across the cluster for connector " + connector + " does not match the set of tasks in the config topic", + expectedTasks, + new HashSet<>(allAssignedTasks.get(connector)) + ); + }); + } + + private void verifyCoordinatorInteractions() { + verify(coordinator, times(rebalanceNum)).configSnapshot(); + verify(coordinator, times(rebalanceNum)).leaderState(any()); + verify(coordinator, times(2 * rebalanceNum)).generationId(); + verify(coordinator, times(rebalanceNum)).memberId(); + verify(coordinator, times(rebalanceNum)).lastCompletedGenerationId(); + } + + private static List extractFromAssignments( + Map assignments, + Function> extraction + ) { + return assignments.values().stream() + .map(extraction) + .flatMap(Collection::stream) .collect(Collectors.toList()); - Set existingUniqueTasks = new HashSet<>(existingTasks); - existingTasks.removeAll(existingUniqueTasks); - assertThat("Tasks should be unique in assignments but duplicates where found", - Collections.emptyList(), - is(existingTasks)); } - private void expectGeneration() { - when(coordinator.generationId()) - .thenReturn(assignor.previousGenerationId + 1) - .thenReturn(assignor.previousGenerationId + 1); - when(coordinator.lastCompletedGenerationId()).thenReturn(assignor.previousGenerationId); + private static void assertNoDuplicates(List collection, String assertionMessage) { + assertEquals( + assertionMessage, + new HashSet<>(collection).size(), + collection.size() + ); } + } From 29a6979c54fb835d5c1f252df839023fd3357f97 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 5 Apr 2022 19:23:31 -0700 Subject: [PATCH 090/447] KAFKA-6204 KAFKA-7402 ProducerInterceptor should implement AutoCloseable (#11997) As part of KIP-376 we had ConsumerInterceptor implement AutoCloseable but forgot to do the same for ProducerInterceptor. This fixes the inconsistency and also addresses KAFKA-6204 at the same time. Reviewers: John Roesler --- .../org/apache/kafka/clients/producer/ProducerInterceptor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java index 8f89d6faa9ab..48caf98d44a3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java @@ -34,7 +34,7 @@ *

    * Implement {@link org.apache.kafka.common.ClusterResourceListener} to receive cluster metadata once it's available. Please see the class documentation for ClusterResourceListener for more information. */ -public interface ProducerInterceptor extends Configurable { +public interface ProducerInterceptor extends Configurable, AutoCloseable { /** * This is called from {@link org.apache.kafka.clients.producer.KafkaProducer#send(ProducerRecord)} and * {@link org.apache.kafka.clients.producer.KafkaProducer#send(ProducerRecord, Callback)} methods, before key and value From 9b8f6289bed11b070ec0366bf91d8aa0ec8e7c79 Mon Sep 17 00:00:00 2001 From: Tom Bentley Date: Wed, 6 Apr 2022 10:29:44 +0100 Subject: [PATCH 091/447] MINOR: Mention KAFKA-13748 in release notes (#11994) Reviewers: Mickael Maison , Bruno Cadonna --- docs/upgrade.html | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/upgrade.html b/docs/upgrade.html index 2ac903819efd..5bce37436364 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -32,6 +32,9 @@

    Notable changes in 3 by default in a future major release.
  • Kafka has replaced log4j and slf4j-log4j12 with reload4j and slf4j-reload4j due to security concerns. More information can be found at reload4j.
  • +
  • The example connectors, FileStreamSourceConnector and FileStreamSinkConnector, have been + removed from the default classpath. To use them in Kafka Connect standalone or distributed mode they need to be + explicitly added, for example CLASSPATH=./lib/connect-file-3.2.0.jar ./bin/connect-distributed.sh.
  • Upgrading to 3.1.0 from any version 0.8.x through 3.0.x

    From b964c07a68fa385b1cce715e1814477e1e26ad39 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 6 Apr 2022 07:27:03 -0700 Subject: [PATCH 092/447] MINOR: Upgrade build and test dependencies (#11984) * gradle: 7.3.3 -> 7.4.2 Configuration cache improvements and several other improvements. https://docs.gradle.org/7.4.2/release-notes.html * dependencycheck gradle plugin: 6.5.3 -> 7.0.3 Minor fixes. * spotbugs gradle plugin: 5.0.5 -> 5.0.6 Minor fixes. https://github.com/spotbugs/spotbugs-gradle-plugin/releases/tag/5.0.6 * jmh: 1.34 -> 1.35 Fixes and profiler improvements. https://mail.openjdk.java.net/pipermail/jmh-dev/2022-March/003422.html * jqwik: 1.6.3 -> 1.6.5 Various tweaks and some breaking changes that don't seem to affect us. https://github.com/jlink/jqwik/releases/tag/1.6.4 https://github.com/jlink/jqwik/releases/tag/1.6.5 * mockito: 4.3.1 -> 4.4.0 Add feature to verify static methods calls in order and minor fixes/improvements. https://github.com/mockito/mockito/releases/tag/v4.4.0 Reviewers: Manikumar Reddy --- build.gradle | 4 ++-- gradle/dependencies.gradle | 8 ++++---- gradle/wrapper/gradle-wrapper.properties | 4 ++-- gradlew | 2 +- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/build.gradle b/build.gradle index 5b28b9389b07..fccc8088135c 100644 --- a/build.gradle +++ b/build.gradle @@ -34,10 +34,10 @@ plugins { id 'com.github.ben-manes.versions' version '0.42.0' id 'idea' id 'java-library' - id 'org.owasp.dependencycheck' version '6.5.3' + id 'org.owasp.dependencycheck' version '7.0.3' id 'org.nosphere.apache.rat' version "0.7.0" - id "com.github.spotbugs" version '5.0.5' apply false + id "com.github.spotbugs" version '5.0.6' apply false id 'org.gradle.test-retry' version '1.3.1' apply false id 'org.scoverage' version '7.0.0' apply false id 'com.github.johnrengelman.shadow' version '7.1.2' apply false diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 6910f640728c..066d62bc854f 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.3.3", + gradle: "7.4.2", grgit: "4.1.1", httpclient: "4.5.13", easymock: "4.3", @@ -73,7 +73,7 @@ versions += [ jetty: "9.4.44.v20210927", jersey: "2.34", jline: "3.21.0", - jmh: "1.34", + jmh: "1.35", hamcrest: "2.2", scalaLogging: "3.9.4", jaxb: "2.3.0", @@ -82,7 +82,7 @@ versions += [ jopt: "5.0.4", jose4j: "0.7.9", junit: "5.8.2", - jqwik: "1.6.3", + jqwik: "1.6.5", kafka_0100: "0.10.0.1", kafka_0101: "0.10.1.1", kafka_0102: "0.10.2.2", @@ -103,7 +103,7 @@ versions += [ lz4: "1.8.0", mavenArtifact: "3.8.4", metrics: "2.2.0", - mockito: "4.3.1", + mockito: "4.4.0", netty: "4.1.73.Final", powermock: "2.0.9", reflections: "0.9.12", diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index ee6ba9a3ac42..e1e0c8dc42bf 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=c9490e938b221daf0094982288e4038deed954a3f12fb54cbf270ddf4e37d879 -distributionUrl=https\://services.gradle.org/distributions/gradle-7.3.3-all.zip +distributionSha256Sum=e6d864e3b5bc05cc62041842b306383fc1fefcec359e70cebb1d470a6094ca82 +distributionUrl=https\://services.gradle.org/distributions/gradle-7.4.2-all.zip zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists diff --git a/gradlew b/gradlew index 08c1fc535dce..685cc73b3172 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.3.3/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.4.2/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 From 206ad4d2b530b1c0c776629256352cd044e68197 Mon Sep 17 00:00:00 2001 From: Anastasia Vela Date: Wed, 6 Apr 2022 07:35:25 -0700 Subject: [PATCH 093/447] MINOR: Fix flaky testIdleConnection() test (#11996) The test expects that the connection becomes idle before the mock time is moved forward, but the processor thread runs concurrently and may run some activity on the connection after the mock time is moved forward, thus the connection never expires. The solution is to wait until the message is received on the socket, and only then wait until the connection is unmuted (it's not enough to wait for unmuted without waiting for message being received on the socket, because the channel might have not been muted yet). Reviewers: David Jacot --- core/src/test/scala/unit/kafka/network/SocketServerTest.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 6c520613087e..9b4e81ab39eb 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -478,6 +478,10 @@ class SocketServerTest { processRequest(overrideServer.dataPlaneRequestChannel, request0) assertTrue(openChannel(request0, overrideServer).nonEmpty, "Channel not open") assertEquals(openChannel(request0, overrideServer), openOrClosingChannel(request0, overrideServer)) + // Receive response to make sure activity on socket server processor thread quiesces, otherwise + // it may continue after the mock time sleep, so there would be events that would mark the + // connection as "up-to-date" after the sleep and prevent connection from being idle. + receiveResponse(socket0) TestUtils.waitUntilTrue(() => !openChannel(request0, overrideServer).get.isMuted, "Failed to unmute channel") time.sleep(idleTimeMs + 1) TestUtils.waitUntilTrue(() => openOrClosingChannel(request0, overrideServer).isEmpty, "Failed to close idle channel") From 92305c2cf26764de89fabcbe33401382f43610ff Mon Sep 17 00:00:00 2001 From: sciclon2 <74413315+sciclon2@users.noreply.github.com> Date: Wed, 6 Apr 2022 17:48:43 +0200 Subject: [PATCH 094/447] KAFKA-13687: Limiting the amount of bytes to be read in a segment logs (#11842) This PR allows to limit the output batches while they are inspected via the kafka-dump-log.sh script. The idea is to take samples from the logsegments without affecting a production cluster as the current script will read the whole files, this could create issues related to performance. Please see the KIP-824 Reviewers: Jun Rao --- .../scala/kafka/tools/DumpLogSegments.scala | 15 +++-- .../kafka/tools/DumpLogSegmentsTest.scala | 56 +++++++++++++++++++ 2 files changed, 67 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index 88e59d51ad80..28594cc12bc5 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -59,7 +59,7 @@ object DumpLogSegments { suffix match { case UnifiedLog.LogFileSuffix => dumpLog(file, opts.shouldPrintDataLog, nonConsecutivePairsForLogFilesMap, opts.isDeepIteration, - opts.messageParser, opts.skipRecordMetadata) + opts.messageParser, opts.skipRecordMetadata, opts.maxBytes) case UnifiedLog.IndexFileSuffix => dumpIndex(file, opts.indexSanityOnly, opts.verifyOnly, misMatchesForIndexFilesMap, opts.maxMessageSize) case UnifiedLog.TimeIndexFileSuffix => @@ -246,10 +246,11 @@ object DumpLogSegments { nonConsecutivePairsForLogFilesMap: mutable.Map[String, List[(Long, Long)]], isDeepIteration: Boolean, parser: MessageParser[_, _], - skipRecordMetadata: Boolean): Unit = { + skipRecordMetadata: Boolean, + maxBytes: Int): Unit = { val startOffset = file.getName.split("\\.")(0).toLong println("Starting offset: " + startOffset) - val fileRecords = FileRecords.open(file, false) + val fileRecords = FileRecords.open(file, false).slice(0, maxBytes) try { var validBytes = 0L var lastOffset = -1L @@ -306,7 +307,7 @@ object DumpLogSegments { validBytes += batch.sizeInBytes } val trailingBytes = fileRecords.sizeInBytes - validBytes - if (trailingBytes > 0) + if ( (trailingBytes > 0) && (maxBytes == Integer.MAX_VALUE) ) println(s"Found $trailingBytes invalid bytes at the end of ${file.getName}") } finally fileRecords.closeHandlers() } @@ -430,6 +431,11 @@ object DumpLogSegments { .describedAs("size") .ofType(classOf[java.lang.Integer]) .defaultsTo(5 * 1024 * 1024) + val maxBytesOpt = parser.accepts("max-bytes", "Limit the amount of total batches read in bytes avoiding reading the whole .log file(s).") + .withRequiredArg + .describedAs("size") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(Integer.MAX_VALUE) val deepIterationOpt = parser.accepts("deep-iteration", "if set, uses deep instead of shallow iteration. Automatically set if print-data-log is enabled.") val valueDecoderOpt = parser.accepts("value-decoder-class", "if set, used to deserialize the messages. This class should implement kafka.serializer.Decoder trait. Custom jar should be available in kafka/libs directory.") .withOptionalArg() @@ -473,6 +479,7 @@ object DumpLogSegments { lazy val indexSanityOnly: Boolean = options.has(indexSanityOpt) lazy val files = options.valueOf(filesOpt).split(",") lazy val maxMessageSize = options.valueOf(maxMessageSizeOpt).intValue() + lazy val maxBytes = options.valueOf(maxBytesOpt).intValue() def checkArgs(): Unit = CommandLineUtils.checkRequiredArgs(parser, options, filesOpt) diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 04556aa97af9..edc02d8fd85f 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -39,6 +39,7 @@ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.util.matching.Regex case class BatchInfo(records: Seq[SimpleRecord], hasKeys: Boolean, hasValues: Boolean) @@ -298,6 +299,29 @@ class DumpLogSegmentsTest { outContent.toString } + @Test + def testPrintDataLogPartialBatches(): Unit = { + addSimpleRecords() + val totalBatches = batches.size + val partialBatches = totalBatches / 2 + + // Get all the batches + val output = runDumpLogSegments(Array("--files", logFilePath)) + val lines = util.Arrays.asList(output.split("\n"): _*).listIterator() + + // Get total bytes of the partial batches + val partialBatchesBytes = readPartialBatchesBytes(lines, partialBatches) + + // Request only the partial batches by bytes + val partialOutput = runDumpLogSegments(Array("--max-bytes", partialBatchesBytes.toString, "--files", logFilePath)) + val partialLines = util.Arrays.asList(partialOutput.split("\n"): _*).listIterator() + + // Count the total of partial batches limited by bytes + val partialBatchesCount = countBatches(partialLines) + + assertEquals(partialBatches, partialBatchesCount) + } + private def readBatchMetadata(lines: util.ListIterator[String]): Option[String] = { while (lines.hasNext) { val line = lines.next() @@ -310,6 +334,38 @@ class DumpLogSegmentsTest { None } + // Returns the total bytes of the batches specified + private def readPartialBatchesBytes(lines: util.ListIterator[String], limit: Int): Int = { + val sizePattern: Regex = raw".+?size:\s(\d+).+".r + var batchesBytes = 0 + var batchesCounter = 0 + while (lines.hasNext) { + if (batchesCounter >= limit){ + return batchesBytes + } + val line = lines.next() + if (line.startsWith("baseOffset")) { + line match { + case sizePattern(size) => batchesBytes += size.toInt + case _ => throw new IllegalStateException(s"Failed to parse and find size value for batch line: $line") + } + batchesCounter += 1 + } + } + batchesBytes + } + + private def countBatches(lines: util.ListIterator[String]): Int = { + var countBatches = 0 + while (lines.hasNext) { + val line = lines.next() + if (line.startsWith("baseOffset")) { + countBatches += 1 + } + } + countBatches + } + private def readBatchRecords(lines: util.ListIterator[String]): Seq[String] = { val records = mutable.ArrayBuffer.empty[String] while (lines.hasNext) { From e7cfbad04f0166e6fe4862be2b893a69f3740ba1 Mon Sep 17 00:00:00 2001 From: Xiaoyue Xue <2821566237@qq.com> Date: Thu, 7 Apr 2022 14:24:23 +0800 Subject: [PATCH 095/447] MINOR: Clean up for TransactionManager and RecordAccumulator (#11979) Reviewers: Luke Chen --- .../producer/internals/RecordAccumulator.java | 4 ++-- .../producer/internals/TransactionManager.java | 16 +++------------- 2 files changed, 5 insertions(+), 15 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index 7724237e45d7..017d99f0d0f3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -378,12 +378,12 @@ public int splitAndReenqueue(ProducerBatch bigBatch) { // producer id. We will not attempt to reorder messages if the producer id has changed, we will throw an // IllegalStateException instead. private void insertInSequenceOrder(Deque deque, ProducerBatch batch) { - // When we are requeing and have enabled idempotence, the reenqueued batch must always have a sequence. + // When we are re-enqueueing and have enabled idempotence, the re-enqueued batch must always have a sequence. if (batch.baseSequence() == RecordBatch.NO_SEQUENCE) throw new IllegalStateException("Trying to re-enqueue a batch which doesn't have a sequence even " + "though idempotency is enabled."); - if (transactionManager.nextBatchBySequence(batch.topicPartition) == null) + if (!transactionManager.hasInflightBatches(batch.topicPartition)) throw new IllegalStateException("We are re-enqueueing a batch which is not tracked as part of the in flight " + "requests. batch.topicPartition: " + batch.topicPartition + "; batch.baseSequence: " + batch.baseSequence()); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index de93bcaedf1e..e7aa212eb7af 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -113,16 +113,7 @@ private TopicPartitionEntry getPartition(TopicPartition topicPartition) { } private TopicPartitionEntry getOrCreatePartition(TopicPartition topicPartition) { - TopicPartitionEntry ent = topicPartitions.get(topicPartition); - if (ent == null) { - ent = new TopicPartitionEntry(); - topicPartitions.put(topicPartition, ent); - } - return ent; - } - - private void addPartition(TopicPartition topicPartition) { - this.topicPartitions.putIfAbsent(topicPartition, new TopicPartitionEntry()); + return topicPartitions.computeIfAbsent(topicPartition, tp -> new TopicPartitionEntry()); } private boolean contains(TopicPartition topicPartition) { @@ -450,7 +441,7 @@ public synchronized void maybeAddPartition(TopicPartition topicPartition) { return; } else { log.debug("Begin adding new partition {} to transaction", topicPartition); - topicPartitionBookkeeper.addPartition(topicPartition); + topicPartitionBookkeeper.getOrCreatePartition(topicPartition); newPartitionsInTransaction.add(topicPartition); } } @@ -698,7 +689,7 @@ private void updateLastAckedOffset(ProduceResponse.PartitionResponse response, P // response for this. This can happen only if the producer is only idempotent (not transactional) and in // this case there will be no tracked bookkeeper entry about it, so we have to insert one. if (!lastAckedOffset.isPresent() && !isTransactional()) { - topicPartitionBookkeeper.addPartition(batch.topicPartition); + topicPartitionBookkeeper.getOrCreatePartition(batch.topicPartition); } if (lastOffset > lastAckedOffset.orElse(ProduceResponse.INVALID_OFFSET)) { topicPartitionBookkeeper.getPartition(batch.topicPartition).lastAckedOffset = lastOffset; @@ -796,7 +787,6 @@ private void adjustSequencesDueToFailedBatch(ProducerBatch batch) { throw new IllegalStateException("Sequence number for batch with sequence " + inFlightBatch.baseSequence() + " for partition " + batch.topicPartition + " is going to become negative: " + newSequence); - log.info("Resetting sequence number of batch with current sequence {} for partition {} to {}", inFlightBatch.baseSequence(), batch.topicPartition, newSequence); inFlightBatch.resetProducerState(new ProducerIdAndEpoch(inFlightBatch.producerId(), inFlightBatch.producerEpoch()), newSequence, inFlightBatch.isTransactional()); }); } From 60c0916bfdc87ddf2e01f172d5455bfeaa8e8cc7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 7 Apr 2022 01:13:15 -0700 Subject: [PATCH 096/447] KAFKA-13801: Kafka server does not respect MetricsReporter contract for dynamically configured reporters (#11998) MetricsReporter.contextChange contract states the method should always be called first before MetricsReporter.init is called. This is done correctly for reporters enabled by default (e.g. JmxReporter) but not for metrics reporters configured dynamically. This fixes the call ordering for dynamically configured metrics reporter and updates tests to enforce ordering. Reviewers: David Jacot --- .../main/scala/kafka/server/DynamicBrokerConfig.scala | 5 ++++- .../kafka/server/DynamicBrokerReconfigurationTest.scala | 9 +++++++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 2a4fd9501ba0..f726569e842d 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -800,12 +800,15 @@ class DynamicMetricsReporters(brokerId: Int, server: KafkaBroker) extends Reconf updatedConfigs.forEach { (k, v) => props.put(k, v.asInstanceOf[AnyRef]) } propsOverride.forKeyValue { (k, v) => props.put(k, v) } val reporters = dynamicConfig.currentKafkaConfig.getConfiguredInstances(reporterClasses, classOf[MetricsReporter], props) + // Call notifyMetricsReporters first to satisfy the contract for MetricsReporter.contextChange, + // which provides that MetricsReporter.contextChange must be called before the first call to MetricsReporter.init. + // The first call to MetricsReporter.init is done when we call metrics.addReporter below. + KafkaBroker.notifyMetricsReporters(server.clusterId, server.config, reporters.asScala) reporters.forEach { reporter => metrics.addReporter(reporter) currentReporters += reporter.getClass.getName -> reporter } KafkaBroker.notifyClusterListeners(server.clusterId, reporters.asScala) - KafkaBroker.notifyMetricsReporters(server.clusterId, server.config, reporters.asScala) } private def removeReporter(className: String): Unit = { diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 24d1eb969e7b..d42947171018 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -52,8 +52,7 @@ import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.config.provider.FileConfigProvider import org.apache.kafka.common.errors.{AuthenticationException, InvalidRequestException} import org.apache.kafka.common.internals.Topic -import org.apache.kafka.common.metrics.Quota -import org.apache.kafka.common.metrics.{KafkaMetric, MetricsReporter} +import org.apache.kafka.common.metrics.{KafkaMetric, MetricsContext, MetricsReporter, Quota} import org.apache.kafka.common.network.{ListenerName, Mode} import org.apache.kafka.common.network.CertStores.{KEYSTORE_PROPS, TRUSTSTORE_PROPS} import org.apache.kafka.common.record.TimestampType @@ -1813,6 +1812,7 @@ class TestMetricsReporter extends MetricsReporter with Reconfigurable with Close import TestMetricsReporter._ val kafkaMetrics = ArrayBuffer[KafkaMetric]() @volatile var initializeCount = 0 + @volatile var contextChangeCount = 0 @volatile var configureCount = 0 @volatile var reconfigureCount = 0 @volatile var closeCount = 0 @@ -1820,7 +1820,12 @@ class TestMetricsReporter extends MetricsReporter with Reconfigurable with Close @volatile var pollingInterval: Int = -1 testReporters.add(this) + override def contextChange(metricsContext: MetricsContext): Unit = { + contextChangeCount += 1 + } + override def init(metrics: util.List[KafkaMetric]): Unit = { + assertTrue(contextChangeCount > 0, "contextChange must be called before init") kafkaMetrics ++= metrics.asScala initializeCount += 1 } From a2149c41786b6769df079e18d1f8078ef3c75832 Mon Sep 17 00:00:00 2001 From: bozhao12 <102274736+bozhao12@users.noreply.github.com> Date: Thu, 7 Apr 2022 18:16:12 +0800 Subject: [PATCH 097/447] MINOR: Fix method javadoc and typo in comments (#12007) Reviewers: Luke Chen --- .../org/apache/kafka/clients/consumer/internals/Fetcher.java | 2 +- .../kafka/clients/consumer/internals/SubscriptionState.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 54f70cafd72c..fa7073cf0dc0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -1201,7 +1201,7 @@ private Map prepareFetchRequests() { continue; } - // Use the preferred read replica if set, otherwise the position's leader + // Use the preferred read replica if set, otherwise the partition's leader Node node = selectReadReplica(partition, leaderOpt.get(), currentTimeMs); if (client.isUnavailable(node)) { client.maybeThrowAuthFailure(node); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index 416468d945fc..0225822a1946 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals; -import java.util.TreeSet; import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NodeApiVersions; @@ -43,6 +42,7 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.TreeSet; import java.util.function.LongSupplier; import java.util.function.Predicate; import java.util.regex.Pattern; @@ -617,7 +617,7 @@ public synchronized Optional preferredReadReplica(TopicPartition tp, lo * Unset the preferred read replica. This causes the fetcher to go back to the leader for fetches. * * @param tp The topic partition - * @return true if the preferred read replica was set, false otherwise. + * @return the removed preferred read replica if set, None otherwise. */ public synchronized Optional clearPreferredReadReplica(TopicPartition tp) { return assignedState(tp).clearPreferredReadReplica(); From ce4f2ad6060c76120d2a2a6a7ef3e6135b2c9932 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 8 Apr 2022 08:55:52 -0700 Subject: [PATCH 098/447] MINOR: Fix support for custom commit ids in the build (#12014) This regressed in ca375d8004c1 due to a typo. We need tests for our builds. :) I verified that passing the commitId via `-PcommitId=123` works correctly. Reviewers: Ismael Juma --- build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index fccc8088135c..7ecfa3e07e7e 100644 --- a/build.gradle +++ b/build.gradle @@ -154,8 +154,8 @@ allprojects { def determineCommitId() { def takeFromHash = 16 - if (project.hasProperty('commitId2')) { - commitId2.take(takeFromHash) + if (project.hasProperty('commitId')) { + commitId.take(takeFromHash) } else if (file("$rootDir/.git/HEAD").exists()) { def headRef = file("$rootDir/.git/HEAD").text if (headRef.contains('ref: ')) { From 7a5f0cfaefb57cefa70c81d63c60316075c6dd97 Mon Sep 17 00:00:00 2001 From: Alok Nikhil Date: Fri, 8 Apr 2022 12:54:09 -0700 Subject: [PATCH 099/447] MINOR: Fix DescribeLogDirs API error handling for older API versions (#12017) With KAFKA-13527 / KIP-784 we introduced a new top-level error code for the DescribeLogDirs API for versions 3 and above. However, the change regressed the error handling for versions less than 3 since the response converter fails to write the non-zero error code out (rightly) for versions lower than 3 and drops the response to the client which eventually times out instead of receiving an empty log dirs response and processing that as a Cluster Auth failure. With this change, the API conditionally propagates the error code out to the client if the request API version is 3 and above. This keeps the semantics of the error handling the same for all versions and restores the behavior for older versions. See current behavior in the broker log: ```bash ERROR] 2022-04-08 01:22:56,406 [data-plane-kafka-request-handler-10] kafka.server.KafkaApis - [KafkaApi-0] Unexpected error handling request RequestHeader(apiKey=DESCRIBE_LOG_DIRS, apiVersion=0, clientId=sarama, correlationId=1) -- DescribeLogDirsRequestData(topics=null) org.apache.kafka.common.errors.UnsupportedVersionException: Attempted to write a non-default errorCode at version 0 [ERROR] 2022-04-08 01:22:56,407 [data-plane-kafka-request-handler-10] kafka.server.KafkaRequestHandler - [Kafka Request Handler 10 on Broker 0], Exception when handling request org.apache.kafka.common.errors.UnsupportedVersionException: Attempted to write a non-default errorCode at version 0 ``` Reviewers: Ismael Juma --- .../message/DescribeLogDirsResponse.json | 3 +- .../network/RequestConvertToJsonTest.scala | 30 +++++++++++++++++-- 2 files changed, 30 insertions(+), 3 deletions(-) diff --git a/clients/src/main/resources/common/message/DescribeLogDirsResponse.json b/clients/src/main/resources/common/message/DescribeLogDirsResponse.json index 0171a16481ff..c79e756aada1 100644 --- a/clients/src/main/resources/common/message/DescribeLogDirsResponse.json +++ b/clients/src/main/resources/common/message/DescribeLogDirsResponse.json @@ -25,7 +25,8 @@ "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, - { "name": "ErrorCode", "type": "int16", "versions": "3+", "about": "The error code, or 0 if there was no error." }, + { "name": "ErrorCode", "type": "int16", "versions": "3+", + "ignorable": true, "about": "The error code, or 0 if there was no error." }, { "name": "Results", "type": "[]DescribeLogDirsResult", "versions": "0+", "about": "The log directories.", "fields": [ { "name": "ErrorCode", "type": "int16", "versions": "0+", diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index 09dceac15102..0ce8448a4f26 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -19,14 +19,13 @@ package kafka.network import java.net.InetAddress import java.nio.ByteBuffer - import com.fasterxml.jackson.databind.node.{BooleanNode, DoubleNode, JsonNodeFactory, LongNode, ObjectNode, TextNode} import kafka.network import kafka.network.RequestConvertToJson.requestHeaderNode import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.message._ import org.apache.kafka.common.network.{ClientInformation, ListenerName, NetworkSend} -import org.apache.kafka.common.protocol.{ApiKeys, MessageUtil} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, MessageUtil} import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.junit.jupiter.api.Assertions.assertEquals @@ -61,6 +60,33 @@ class RequestConvertToJsonTest { assertEquals(ArrayBuffer.empty, unhandledKeys, "Unhandled request keys") } + @Test + def testAllApiVersionsResponseHandled(): Unit = { + + ApiKeys.values().foreach { key => { + val unhandledVersions = ArrayBuffer[java.lang.Short]() + key.allVersions().forEach { version => { + val message = key match { + // Specify top-level error handling for verifying compatibility across versions + case ApiKeys.DESCRIBE_LOG_DIRS => + ApiMessageType.fromApiKey(key.id).newResponse().asInstanceOf[DescribeLogDirsResponseData] + .setErrorCode(Errors.CLUSTER_AUTHORIZATION_FAILED.code()) + case _ => + ApiMessageType.fromApiKey(key.id).newResponse() + } + + val bytes = MessageUtil.toByteBuffer(message, version) + val response = AbstractResponse.parseResponse(key, bytes, version) + try { + RequestConvertToJson.response(response, version) + } catch { + case _ : IllegalStateException => unhandledVersions += version + }} + } + assertEquals(ArrayBuffer.empty, unhandledVersions, s"API: ${key.toString} - Unhandled request versions") + }} + } + @Test def testAllResponseTypesHandled(): Unit = { val unhandledKeys = ArrayBuffer[String]() From 9596c7b9cfacb6aba961a328023c6572e8800837 Mon Sep 17 00:00:00 2001 From: Xiaoyue Xue <2821566237@qq.com> Date: Sat, 9 Apr 2022 10:58:16 +0800 Subject: [PATCH 100/447] KAFKA-13794: Follow up to fix producer batch comparator (#12006) In comparator, objects that are not equal need to have a stable order otherwise, binary search may not find the objects. Improve the producer batch comparator --- .../kafka/clients/producer/internals/TransactionManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index e7aa212eb7af..961e4e4f5a9f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -178,7 +178,7 @@ private static class TopicPartitionEntry { private static final Comparator PRODUCER_BATCH_COMPARATOR = (b1, b2) -> { if (b1.baseSequence() < b2.baseSequence()) return -1; else if (b1.baseSequence() > b2.baseSequence()) return 1; - else return b1.equals(b2) ? 0 : 1; + else return Integer.compare(b1.hashCode(), b2.hashCode()); }; TopicPartitionEntry() { From 6e9cd0c7f58a24b5ecef9746881ebcccd7633a7e Mon Sep 17 00:00:00 2001 From: David Jacot Date: Sat, 9 Apr 2022 11:42:42 +0200 Subject: [PATCH 101/447] MINOR: A few code cleanups in DynamicBrokerConfig (#12015) Reviewers: Luke Chen --- .../kafka/server/DynamicBrokerConfig.scala | 30 +++++++++++-------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index f726569e842d..18eeea62709a 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -182,16 +182,18 @@ object DynamicBrokerConfig { private[server] def dynamicConfigUpdateModes: util.Map[String, String] = { AllDynamicConfigs.map { name => val mode = if (PerBrokerConfigs.contains(name)) "per-broker" else "cluster-wide" - (name -> mode) + name -> mode }.toMap.asJava } private[server] def resolveVariableConfigs(propsOriginal: Properties): Properties = { val props = new Properties val config = new AbstractConfig(new ConfigDef(), propsOriginal, false) - config.originals.asScala.filter(!_._1.startsWith(AbstractConfig.CONFIG_PROVIDERS_CONFIG)).foreach {case (key: String, value: Object) => { - props.put(key, value) - }} + config.originals.forEach { (key, value) => + if (!key.startsWith(AbstractConfig.CONFIG_PROVIDERS_CONFIG)) { + props.put(key, value) + } + } props } } @@ -253,7 +255,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging } addReconfigurable(kafkaServer.kafkaYammerMetrics) addReconfigurable(new DynamicMetricsReporters(kafkaConfig.brokerId, kafkaServer)) - addReconfigurable(new DynamicClientQuotaCallback(kafkaConfig.brokerId, kafkaServer)) + addReconfigurable(new DynamicClientQuotaCallback(kafkaServer)) addBrokerReconfigurable(new DynamicThreadPool(kafkaServer)) addBrokerReconfigurable(new DynamicLogConfig(kafkaServer.logManager, kafkaServer)) @@ -421,7 +423,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging debug(s"Dynamic password config $configName could not be decoded using old secret, new secret will be used.") None } - decoded.foreach { value => props.put(configName, passwordEncoder.encode(new Password(value))) } + decoded.foreach(value => props.put(configName, passwordEncoder.encode(new Password(value)))) } } adminZkClient.changeBrokerConfig(Some(kafkaConfig.brokerId), props) @@ -487,8 +489,8 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging * Returns the change in configurations between the new props and current props by returning a * map of the changed configs, as well as the set of deleted keys */ - private def updatedConfigs(newProps: java.util.Map[String, _], currentProps: java.util.Map[String, _]): - (mutable.Map[String, _], Set[String]) = { + private def updatedConfigs(newProps: java.util.Map[String, _], + currentProps: java.util.Map[String, _]): (mutable.Map[String, _], Set[String]) = { val changeMap = newProps.asScala.filter { case (k, v) => v != currentProps.get(k) } @@ -597,7 +599,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging newCustomConfigs: util.Map[String, Object], validateOnly: Boolean): Unit = { val newConfigs = new util.HashMap[String, Object] - allNewConfigs.forEach { (k, v) => newConfigs.put(k, v.asInstanceOf[AnyRef]) } + allNewConfigs.forEach((k, v) => newConfigs.put(k, v.asInstanceOf[AnyRef])) newConfigs.putAll(newCustomConfigs) try { reconfigurable.validateReconfiguration(newConfigs) @@ -651,7 +653,9 @@ class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends Brok logManager.allLogs.foreach { log => val props = mutable.Map.empty[Any, Any] props ++= newBrokerDefaults - props ++= log.config.originals.asScala.filter { case (k, _) => log.config.overriddenConfigs.contains(k) } + props ++= log.config.originals.asScala.filter { case (k, _) => + log.config.overriddenConfigs.contains(k) + } val logConfig = LogConfig(props.asJava, log.config.overriddenConfigs) log.updateConfig(logConfig) @@ -797,8 +801,8 @@ class DynamicMetricsReporters(brokerId: Int, server: KafkaBroker) extends Reconf private def createReporters(reporterClasses: util.List[String], updatedConfigs: util.Map[String, _]): Unit = { val props = new util.HashMap[String, AnyRef] - updatedConfigs.forEach { (k, v) => props.put(k, v.asInstanceOf[AnyRef]) } - propsOverride.forKeyValue { (k, v) => props.put(k, v) } + updatedConfigs.forEach((k, v) => props.put(k, v.asInstanceOf[AnyRef])) + propsOverride.forKeyValue((k, v) => props.put(k, v)) val reporters = dynamicConfig.currentKafkaConfig.getConfiguredInstances(reporterClasses, classOf[MetricsReporter], props) // Call notifyMetricsReporters first to satisfy the contract for MetricsReporter.contextChange, // which provides that MetricsReporter.contextChange must be called before the first call to MetricsReporter.init. @@ -871,7 +875,7 @@ object DynamicListenerConfig { ) } -class DynamicClientQuotaCallback(brokerId: Int, server: KafkaBroker) extends Reconfigurable { +class DynamicClientQuotaCallback(server: KafkaBroker) extends Reconfigurable { override def configure(configs: util.Map[String, _]): Unit = {} From 4ad439c56db551697ea366dc4986eee5fc288706 Mon Sep 17 00:00:00 2001 From: RivenSun <91005273+RivenSun2@users.noreply.github.com> Date: Mon, 11 Apr 2022 10:06:56 +0800 Subject: [PATCH 102/447] MINOR: Change the log output information in the KafkaConsumer assign method (#12026) Reviewers: Luke Chen --- .../java/org/apache/kafka/clients/consumer/KafkaConsumer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index d0ef1a0cbebc..a49c89560f2c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1120,7 +1120,7 @@ public void assign(Collection partitions) { if (coordinator != null) this.coordinator.maybeAutoCommitOffsetsAsync(time.milliseconds()); - log.info("Subscribed to partition(s): {}", Utils.join(partitions, ", ")); + log.info("Assigned to partition(s): {}", Utils.join(partitions, ", ")); if (this.subscriptions.assignFromUser(new HashSet<>(partitions))) metadata.requestUpdateForNewTopics(); } From 317fff9bb05fd7c7932f481fd200e56bff761459 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 11 Apr 2022 13:37:47 -0700 Subject: [PATCH 103/447] MINOR: Re-use counter in mocking of LogSegment.size (#12021) When migrating from Easymock to Mockito, the mockito implemetnation didn't have the same semantic as the Easymock implementation. Without this fix the mocking of LogSegment.size() always returns 0 because a new AtomicInteger was getting created for each invocation of LogSegment.size() Reviewers: Mickael Maison --- .../unit/kafka/server/LogOffsetTest.scala | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index e143539fd489..51f75b2ded00 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -27,6 +27,8 @@ import org.apache.kafka.common.{IsolationLevel, TopicPartition} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.mockito.Mockito.{mock, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer import java.io.File import java.util.concurrent.atomic.AtomicInteger @@ -258,10 +260,9 @@ class LogOffsetTest extends BaseRequestTest { def testFetchOffsetsBeforeWithChangingSegmentSize(): Unit = { val log: UnifiedLog = mock(classOf[UnifiedLog]) val logSegment: LogSegment = mock(classOf[LogSegment]) - when(logSegment.size).thenAnswer(_ => { - val value = new AtomicInteger(0) - def answer: Int = value.getAndIncrement() - answer + when(logSegment.size).thenAnswer(new Answer[Int] { + private[this] val value = new AtomicInteger(0) + override def answer(invocation: InvocationOnMock): Int = value.getAndIncrement() }) val logSegments = Seq(logSegment) when(log.logSegments).thenReturn(logSegments) @@ -274,13 +275,12 @@ class LogOffsetTest extends BaseRequestTest { def testFetchOffsetsBeforeWithChangingSegments(): Unit = { val log: UnifiedLog = mock(classOf[UnifiedLog]) val logSegment: LogSegment = mock(classOf[LogSegment]) - when(log.logSegments).thenAnswer { _ => - def answer = new Iterable[LogSegment] { + when(log.logSegments).thenReturn( + new Iterable[LogSegment] { override def size = 2 - def iterator = Seq(logSegment).iterator + override def iterator = Seq(logSegment).iterator } - answer - } + ) log.legacyFetchOffsetsBefore(System.currentTimeMillis, 100) } From 0d518aaed158896ee9ee6949b8f38128d1d73634 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Mon, 11 Apr 2022 22:17:01 +0100 Subject: [PATCH 104/447] MINOR: Fix SessionStore#fetchSession parameter names (#11999) Fixes a small copy/paste error from #10390 that changed the parameter names for fetchSession from the singular session form (eg `startTime`) to the range form (eg `earliestSessionStartTime`). Reviewers: John Roesler --- .../kafka/streams/state/ReadOnlySessionStore.java | 8 ++++---- .../org/apache/kafka/streams/state/SessionStore.java | 12 +++++++----- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java index 4d4469199230..7fe11a6bea0e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java @@ -322,9 +322,9 @@ default KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, /** * Get the value of key from a single session. * - * @param key the key to fetch + * @param key the key to fetch * @param sessionStartTime start timestamp of the session - * @param sessionEndTime end timestamp of the session + * @param sessionEndTime end timestamp of the session * @return The value or {@code null} if no session with the exact start and end timestamp exists * for the given key * @throws NullPointerException If {@code null} is used for any key. @@ -339,9 +339,9 @@ default AGG fetchSession(final K key, /** * Get the value of key from a single session. * - * @param key the key to fetch + * @param key the key to fetch * @param sessionStartTime start timestamp of the session - * @param sessionEndTime end timestamp of the session + * @param sessionEndTime end timestamp of the session * @return The value or {@code null} if no session with the exact start and end timestamp exists * for the given key * @throws NullPointerException If {@code null} is used for any key. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java index 926cddc4d2a4..cbc1cc5b9668 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java @@ -89,12 +89,14 @@ default KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, prepareMillisCheckFailMsgPrefix(latestSessionStartTime, "latestSessionStartTime"))); } - default AGG fetchSession(final K key, final Instant earliestSessionEndTime, final Instant latestSessionStartTime) { + default AGG fetchSession(final K key, + final Instant sessionStartTime, + final Instant sessionEndTime) { return fetchSession(key, - ApiUtils.validateMillisecondInstant(earliestSessionEndTime, - prepareMillisCheckFailMsgPrefix(earliestSessionEndTime, "startTime")), - ApiUtils.validateMillisecondInstant(latestSessionStartTime, - prepareMillisCheckFailMsgPrefix(latestSessionStartTime, "endTime"))); + ApiUtils.validateMillisecondInstant(sessionStartTime, + prepareMillisCheckFailMsgPrefix(sessionStartTime, "sessionStartTime")), + ApiUtils.validateMillisecondInstant(sessionEndTime, + prepareMillisCheckFailMsgPrefix(sessionEndTime, "sessionEndTime"))); } /** From 1df232c839f4568718a52c04aad72b69beb52026 Mon Sep 17 00:00:00 2001 From: RivenSun <91005273+RivenSun2@users.noreply.github.com> Date: Wed, 13 Apr 2022 12:24:57 +0800 Subject: [PATCH 105/447] MINOR: Supplement the description of `Valid Values` in the documentation of `compression.type` (#11985) Because a validator is added to ProducerConfig.COMPRESSION_TYPE_CONFIG and KafkaConfig.CompressionTypeProp, the corresponding testCase is improved to verify whether the wrong value of compression.type will throw a ConfigException. Reviewers: Mickael Maison , Guozhang Wang --- .../apache/kafka/clients/producer/ProducerConfig.java | 4 +++- .../apache/kafka/common/record/CompressionType.java | 6 ++++++ .../kafka/clients/producer/ProducerConfigTest.java | 11 +++++++++++ core/src/main/scala/kafka/server/KafkaConfig.scala | 6 +++--- .../scala/unit/kafka/server/KafkaConfigTest.scala | 2 +- 5 files changed, 24 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index afc1e55cdfda..8fec07a29749 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -26,7 +26,9 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.SecurityConfig; import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -329,7 +331,7 @@ public class ProducerConfig extends AbstractConfig { in("all", "-1", "0", "1"), Importance.LOW, ACKS_DOC) - .define(COMPRESSION_TYPE_CONFIG, Type.STRING, "none", Importance.HIGH, COMPRESSION_TYPE_DOC) + .define(COMPRESSION_TYPE_CONFIG, Type.STRING, CompressionType.NONE.name, in(Utils.enumOptions(CompressionType.class)), Importance.HIGH, COMPRESSION_TYPE_DOC) .define(BATCH_SIZE_CONFIG, Type.INT, 16384, atLeast(0), Importance.MEDIUM, BATCH_SIZE_DOC) .define(LINGER_MS_CONFIG, Type.LONG, 0, atLeast(0), Importance.MEDIUM, LINGER_MS_DOC) .define(DELIVERY_TIMEOUT_MS_CONFIG, Type.INT, 120 * 1000, atLeast(0), Importance.MEDIUM, DELIVERY_TIMEOUT_MS_DOC) diff --git a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java index 1b9754ffabbb..c526929b72e9 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java +++ b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java @@ -190,4 +190,10 @@ else if (ZSTD.name.equals(name)) else throw new IllegalArgumentException("Unknown compression name: " + name); } + + @Override + public String toString() { + return name; + } + } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java index a2f318bebc7a..ae9de7b70a6b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java @@ -19,12 +19,14 @@ import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.config.ConfigException; import org.junit.jupiter.api.Test; import java.util.HashMap; import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; public class ProducerConfigTest { @@ -59,4 +61,13 @@ public void testAppendSerializerToConfig() { assertEquals(newConfigs.get(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG), keySerializerClass); assertEquals(newConfigs.get(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG), valueSerializerClass); } + + @Test + public void testInvalidCompressionType() { + Map configs = new HashMap<>(); + configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); + configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); + configs.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, "abc"); + assertThrows(ConfigException.class, () -> new ProducerConfig(configs)); + } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 7dd9276d6da0..6fe0acaa9d3c 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -25,7 +25,7 @@ import kafka.coordinator.group.OffsetConfig import kafka.coordinator.transaction.{TransactionLog, TransactionStateManager} import kafka.log.LogConfig import kafka.log.LogConfig.MessageFormatVersion -import kafka.message.{BrokerCompressionCodec, CompressionCodec, ZStdCompressionCodec} +import kafka.message.{BrokerCompressionCodec, CompressionCodec, ProducerCompressionCodec, ZStdCompressionCodec} import kafka.security.authorizer.AuthorizerUtils import kafka.server.KafkaConfig.{ControllerListenerNamesProp, ListenerSecurityProtocolMapProp} import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole, ProcessRole} @@ -227,7 +227,7 @@ object Defaults { val DeleteTopicEnable = true - val CompressionType = "producer" + val CompressionType = ProducerCompressionCodec.name val MaxIdMapSnapshots = 2 /** ********* Kafka Metrics Configuration ***********/ @@ -1257,7 +1257,7 @@ object KafkaConfig { .define(OffsetCommitTimeoutMsProp, INT, Defaults.OffsetCommitTimeoutMs, atLeast(1), HIGH, OffsetCommitTimeoutMsDoc) .define(OffsetCommitRequiredAcksProp, SHORT, Defaults.OffsetCommitRequiredAcks, HIGH, OffsetCommitRequiredAcksDoc) .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DeleteTopicEnable, HIGH, DeleteTopicEnableDoc) - .define(CompressionTypeProp, STRING, Defaults.CompressionType, HIGH, CompressionTypeDoc) + .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionCodec.brokerCompressionOptions:_*), HIGH, CompressionTypeDoc) /** ********* Transaction management configuration ***********/ .define(TransactionalIdExpirationMsProp, INT, Defaults.TransactionalIdExpirationMs, atLeast(1), HIGH, TransactionalIdExpirationMsDoc) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index a6597d881587..ed31dba41a16 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -657,7 +657,7 @@ class KafkaConfigTest { def testInvalidCompressionType(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) props.put(KafkaConfig.CompressionTypeProp, "abc") - assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)) + assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @Test From b9fc8935468e3eaa0385d35e2892d9df228701ec Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 13 Apr 2022 06:51:34 -0400 Subject: [PATCH 106/447] MINOR: Correct Connect docs on connector/task states (#11914) The `DESTROYED` state is represented internally as a tombstone record when running in distributed mode and by the removal of the connector/task from the in-memory status map when running in standalone mode. As a result, it will never appear to users of the REST API, and we should remove mention of it from our docs so that developers creating tooling against the REST API don't write unnecessary logic to account for that state. Reviewers: Mickael Maison --- .../apache/kafka/connect/runtime/ConnectMetricsRegistry.java | 4 ++-- docs/connect.html | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetricsRegistry.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetricsRegistry.java index f301439da835..d8579d44fc65 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetricsRegistry.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetricsRegistry.java @@ -129,7 +129,7 @@ public ConnectMetricsRegistry(Set tags) { connectorStatus = createTemplate("status", CONNECTOR_GROUP_NAME, "The status of the connector. One of 'unassigned', 'running', 'paused', 'failed', or " + - "'destroyed'.", + "'restarting'.", connectorTags); connectorType = createTemplate("connector-type", CONNECTOR_GROUP_NAME, "The type of the connector. One of 'source' or 'sink'.", connectorTags); @@ -144,7 +144,7 @@ public ConnectMetricsRegistry(Set tags) { taskStatus = createTemplate("status", TASK_GROUP_NAME, "The status of the connector task. One of 'unassigned', 'running', 'paused', 'failed', or " + - "'destroyed'.", + "'restarting'.", workerTaskTags); taskRunningRatio = createTemplate("running-ratio", TASK_GROUP_NAME, "The fraction of time this task has spent in the running state.", workerTaskTags); diff --git a/docs/connect.html b/docs/connect.html index 1251c3ce683e..be6a2ac46137 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -711,7 +711,7 @@

    Kafka Connect
  • RUNNING: The connector/task is running.
  • PAUSED: The connector/task has been administratively paused.
  • FAILED: The connector/task has failed (usually by raising an exception, which is reported in the status output).
  • -
  • DESTROYED: The connector/task has been administratively removed and will stop appearing in the Connect cluster.
  • +
  • RESTARTING: The connector/task is either actively restarting or is expected to restart soon
  • From 4eeb7071076fc5028cc0e48410819dea95fb1ba8 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Wed, 13 Apr 2022 13:42:27 +0200 Subject: [PATCH 107/447] KAFKA-13828; Ensure reasons sent by the consumer are small (#12043) This PR reworks the reasons used in the ConsumerCoordinator to ensure that they remain reasonably short. Reviewers: Bruno Cadonna --- .../internals/AbstractCoordinator.java | 23 +++++++++++++++---- .../internals/ConsumerCoordinator.java | 16 ++++++------- 2 files changed, 26 insertions(+), 13 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 5b9712f34667..b2f944ad5d2e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -1022,15 +1022,28 @@ synchronized void resetGenerationOnLeaveGroup() { resetStateAndRejoin("consumer pro-actively leaving the group", true); } - public synchronized void requestRejoinIfNecessary(final String reason) { + public synchronized void requestRejoinIfNecessary(final String shortReason, + final String fullReason) { if (!this.rejoinNeeded) { - requestRejoin(reason); + requestRejoin(shortReason, fullReason); } } - public synchronized void requestRejoin(final String reason) { - log.info("Request joining group due to: {}", reason); - this.rejoinReason = reason; + public synchronized void requestRejoin(final String shortReason) { + requestRejoin(shortReason, shortReason); + } + + /** + * Request to rejoin the group. + * + * @param shortReason This is the reason passed up to the group coordinator. It must be + * reasonably small. + * @param fullReason This is the reason logged locally. + */ + public synchronized void requestRejoin(final String shortReason, + final String fullReason) { + log.info("Request joining group due to: {}", fullReason); + this.rejoinReason = shortReason; this.rejoinNeeded = true; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 10939b2a0e48..51fa0b62ed19 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -401,10 +401,10 @@ protected void onJoinComplete(int generation, assignedPartitions.addAll(assignment.partitions()); if (!subscriptions.checkAssignmentMatchedSubscription(assignedPartitions)) { - final String reason = String.format("received assignment %s does not match the current subscription %s; " + + final String fullReason = String.format("received assignment %s does not match the current subscription %s; " + "it is likely that the subscription has changed since we joined the group, will re-join with current subscription", assignment.partitions(), subscriptions.prettyString()); - requestRejoin(reason); + requestRejoin("received assignment does not match the current subscription", fullReason); return; } @@ -437,9 +437,9 @@ protected void onJoinComplete(int generation, firstException.compareAndSet(null, invokePartitionsRevoked(revokedPartitions)); // If revoked any partitions, need to re-join the group afterwards - final String reason = String.format("need to revoke partitions %s as indicated " + + final String fullReason = String.format("need to revoke partitions %s as indicated " + "by the current assignment and re-join", revokedPartitions); - requestRejoin(reason); + requestRejoin("need to revoke partitions and re-join", fullReason); } } @@ -851,17 +851,17 @@ public boolean rejoinNeededOrPending() { // we need to rejoin if we performed the assignment and metadata has changed; // also for those owned-but-no-longer-existed partitions we should drop them as lost if (assignmentSnapshot != null && !assignmentSnapshot.matches(metadataSnapshot)) { - final String reason = String.format("cached metadata has changed from %s at the beginning of the rebalance to %s", + final String fullReason = String.format("cached metadata has changed from %s at the beginning of the rebalance to %s", assignmentSnapshot, metadataSnapshot); - requestRejoinIfNecessary(reason); + requestRejoinIfNecessary("cached metadata has changed", fullReason); return true; } // we need to join if our subscription has changed since the last join if (joinedSubscription != null && !joinedSubscription.equals(subscriptions.subscription())) { - final String reason = String.format("subscription has changed from %s at the beginning of the rebalance to %s", + final String fullReason = String.format("subscription has changed from %s at the beginning of the rebalance to %s", joinedSubscription, subscriptions.subscription()); - requestRejoinIfNecessary(reason); + requestRejoinIfNecessary("subscription has changed", fullReason); return true; } From c93b717836b6d92d0f2e9fb101ea6ff3e823ffca Mon Sep 17 00:00:00 2001 From: Hao Li <1127478+lihaosky@users.noreply.github.com> Date: Wed, 13 Apr 2022 04:49:31 -0700 Subject: [PATCH 108/447] KAFKA-13542: Add rebalance reason in Kafka Streams (#12018) Reviewers: Bruno Cadonna , David Jacot --- .../kafka/streams/processor/internals/StreamThread.java | 6 +++--- .../kafka/streams/processor/internals/StreamThreadTest.java | 5 ++--- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index ab4b09469688..15e4903e6363 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -594,7 +594,7 @@ boolean runLoop() { runOnce(); if (nextProbingRebalanceMs.get() < time.milliseconds()) { log.info("Triggering the followup rebalance scheduled for {} ms.", nextProbingRebalanceMs.get()); - mainConsumer.enforceRebalance(); + mainConsumer.enforceRebalance("Scheduled probing rebalance"); nextProbingRebalanceMs.set(Long.MAX_VALUE); } } catch (final TaskCorruptedException e) { @@ -606,7 +606,7 @@ boolean runLoop() { final boolean enforceRebalance = taskManager.handleCorruption(e.corruptedTasks()); if (enforceRebalance && eosEnabled) { log.info("Active task(s) got corrupted. Triggering a rebalance."); - mainConsumer.enforceRebalance(); + mainConsumer.enforceRebalance("Active tasks corrupted"); } } catch (final TaskMigratedException taskMigrated) { handleTaskMigrated(taskMigrated); @@ -648,7 +648,7 @@ public void maybeSendShutdown() { if (assignmentErrorCode.get() == AssignorError.SHUTDOWN_REQUESTED.code()) { log.warn("Detected that shutdown was requested. " + "All clients in this app will now begin to shutdown"); - mainConsumer.enforceRebalance(); + mainConsumer.enforceRebalance("Shutdown requested"); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index af02e555114a..11bd1d42a788 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -516,6 +516,7 @@ public void shouldEnforceRebalanceAfterNextScheduledProbingRebalanceTime() throw final EasyMockConsumerClientSupplier mockClientSupplier = new EasyMockConsumerClientSupplier(mockConsumer); mockClientSupplier.setCluster(createCluster()); + mockConsumer.enforceRebalance("Scheduled probing rebalance"); EasyMock.replay(mockConsumer); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); topologyMetadata.buildAndRewriteTopology(); @@ -538,8 +539,6 @@ public void shouldEnforceRebalanceAfterNextScheduledProbingRebalanceTime() throw null ); - mockConsumer.enforceRebalance(); - mockClientSupplier.nextRebalanceMs().set(mockTime.milliseconds() - 1L); thread.start(); @@ -2430,7 +2429,7 @@ public void shouldEnforceRebalanceWhenTaskCorruptedExceptionIsThrownForAnActiveT expect(task2.id()).andReturn(taskId2).anyTimes(); expect(taskManager.handleCorruption(corruptedTasks)).andReturn(true); - consumer.enforceRebalance(); + consumer.enforceRebalance("Active tasks corrupted"); expectLastCall(); EasyMock.replay(task1, task2, taskManager, consumer); From a6d86b9998b4cdcdda40cedc5da2f7e695e39ee1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 13 Apr 2022 09:05:27 -0700 Subject: [PATCH 109/447] MINOR: Verify stopReplica if broker epoch not stale (#12040) Verify that ReplicaManager.stopReplica is called if the stop replica request doesn't result in a stale broker epoch error. Reviewers: Mickael Maison --- core/src/test/scala/unit/kafka/server/KafkaApisTest.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 2668303cfe81..b72deafdf17e 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -3192,6 +3192,14 @@ class KafkaApisTest { ) val stopReplicaResponse = capturedResponse.getValue assertEquals(expectedError, stopReplicaResponse.error()) + if (expectedError != Errors.STALE_BROKER_EPOCH) { + verify(replicaManager).stopReplicas( + ArgumentMatchers.eq(request.context.correlationId), + ArgumentMatchers.eq(controllerId), + ArgumentMatchers.eq(controllerEpoch), + ArgumentMatchers.eq(stopReplicaRequest.partitionStates().asScala) + ) + } } @Test From f97646488cff1984455ffb1fe9a147a522e6ac76 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 13 Apr 2022 10:33:15 -0700 Subject: [PATCH 110/447] KAFKA-13651; Add audit logging to `StandardAuthorizer` (#12031) This patch adds audit support through the kafka.authorizer.logger logger to StandardAuthorizer. It follows the same conventions as AclAuthorizer with a similarly formatted log message. When logIfAllowed is set in the Action, then the log message is at DEBUG level; otherwise, we log at trace. When logIfDenied is set, then the log message is at INFO level; otherwise, we again log at TRACE. Reviewers: Colin P. McCabe --- build.gradle | 2 + .../common/resource/ResourcePattern.java | 2 +- .../metadata/authorizer/StandardAcl.java | 12 + .../authorizer/StandardAuthorizerData.java | 306 ++++++++++++++---- .../authorizer/StandardAuthorizerTest.java | 94 +++++- 5 files changed, 347 insertions(+), 69 deletions(-) diff --git a/build.gradle b/build.gradle index 7ecfa3e07e7e..f7fe0ea046dc 100644 --- a/build.gradle +++ b/build.gradle @@ -1136,6 +1136,8 @@ project(':metadata') { compileOnly libs.log4j testImplementation libs.junitJupiter testImplementation libs.hamcrest + testImplementation libs.mockitoCore + testImplementation libs.mockitoInline testImplementation libs.slf4jlog4j testImplementation project(':clients').sourceSets.test.output testImplementation project(':raft').sourceSets.test.output diff --git a/clients/src/main/java/org/apache/kafka/common/resource/ResourcePattern.java b/clients/src/main/java/org/apache/kafka/common/resource/ResourcePattern.java index 2b7504f70a59..b3dfc4937f03 100644 --- a/clients/src/main/java/org/apache/kafka/common/resource/ResourcePattern.java +++ b/clients/src/main/java/org/apache/kafka/common/resource/ResourcePattern.java @@ -89,7 +89,7 @@ public ResourcePatternFilter toFilter() { @Override public String toString() { - return "ResourcePattern(resourceType=" + resourceType + ", name=" + ((name == null) ? "" : name) + ", patternType=" + patternType + ")"; + return "ResourcePattern(resourceType=" + resourceType + ", name=" + name + ", patternType=" + patternType + ")"; } /** diff --git a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAcl.java b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAcl.java index fd3e0f029e48..3fe8ac70da57 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAcl.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAcl.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourceType; +import org.apache.kafka.common.security.auth.KafkaPrincipal; import java.util.Objects; @@ -96,6 +97,17 @@ public String principal() { return principal; } + public KafkaPrincipal kafkaPrincipal() { + int colonIndex = principal.indexOf(":"); + if (colonIndex == -1) { + throw new IllegalStateException("Could not parse principal from `" + principal + "` " + + "(no colon is present separating the principal type from the principal name)"); + } + String principalType = principal.substring(0, colonIndex); + String principalName = principal.substring(colonIndex + 1); + return new KafkaPrincipal(principalType, principalName); + } + public String host() { return host; } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java index a70fa8ca45dc..27cca4271b80 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java @@ -22,12 +22,18 @@ import org.apache.kafka.common.acl.AclBindingFilter; import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclPermissionType; +import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePattern; +import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.SecurityUtils; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.authorizer.Action; import org.apache.kafka.server.authorizer.AuthorizableRequestContext; import org.apache.kafka.server.authorizer.AuthorizationResult; import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Collection; import java.util.Collections; @@ -69,12 +75,18 @@ public class StandardAuthorizerData { * The principal entry used in ACLs that match any principal. */ public static final String WILDCARD_PRINCIPAL = "User:*"; + public static final KafkaPrincipal WILDCARD_KAFKA_PRINCIPAL = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*"); /** * The logger to use. */ final Logger log; + /** + * Logger to use for auditing. + */ + final Logger auditLog; + /** * The current AclMutator. */ @@ -88,7 +100,7 @@ public class StandardAuthorizerData { /** * The result to return if no ACLs match. */ - private final AuthorizationResult defaultResult; + private final DefaultRule defaultRule; /** * Contains all of the current ACLs sorted by (resource type, resource name). @@ -104,6 +116,10 @@ private static Logger createLogger(int nodeId) { return new LogContext("[StandardAuthorizer " + nodeId + "] ").logger(StandardAuthorizerData.class); } + private static Logger auditLogger() { + return LoggerFactory.getLogger("kafka.authorizer.logger"); + } + static StandardAuthorizerData createEmpty() { return new StandardAuthorizerData(createLogger(-1), null, @@ -119,18 +135,20 @@ private StandardAuthorizerData(Logger log, ConcurrentSkipListSet aclsByResource, ConcurrentHashMap aclsById) { this.log = log; + this.auditLog = auditLogger(); this.aclMutator = aclMutator; this.superUsers = superUsers; - this.defaultResult = defaultResult; + this.defaultRule = new DefaultRule(defaultResult); this.aclsByResource = aclsByResource; this.aclsById = aclsById; } StandardAuthorizerData copyWithNewAclMutator(AclMutator newAclMutator) { - return new StandardAuthorizerData(log, + return new StandardAuthorizerData( + log, newAclMutator, superUsers, - defaultResult, + defaultRule.result, aclsByResource, aclsById); } @@ -152,7 +170,7 @@ StandardAuthorizerData copyWithNewAcls(Collection> aclE log, aclMutator, superUsers, - defaultResult, + defaultRule.result, new ConcurrentSkipListSet<>(), new ConcurrentHashMap<>()); for (Entry entry : aclEntries) { @@ -206,18 +224,13 @@ Set superUsers() { } AuthorizationResult defaultResult() { - return defaultResult; + return defaultRule.result; } int aclCount() { return aclsById.size(); } - static class AuthorizationResultBuilder { - boolean foundDeny = false; - boolean foundAllow = false; - } - /** * Authorize an action based on the current set of ACLs. * @@ -227,18 +240,98 @@ static class AuthorizationResultBuilder { * result. In general it makes more sense to configure the default result to be * DENY, but some people (and unit tests) configure it as ALLOW. */ - AuthorizationResult authorize(AuthorizableRequestContext requestContext, - Action action) { + public AuthorizationResult authorize( + AuthorizableRequestContext requestContext, + Action action + ) { + KafkaPrincipal principal = baseKafkaPrincipal(requestContext); + final MatchingRule rule; + // Superusers are authorized to do anything. - if (superUsers.contains(requestContext.principal().toString())) { - if (log.isTraceEnabled()) { - log.trace("authorize(requestContext=" + requestContext + ", action=" + action + - "): ALLOWED because " + requestContext.principal().toString() + - " is a superuser"); + if (superUsers.contains(principal.toString())) { + rule = SuperUserRule.INSTANCE; + } else { + MatchingAclRule aclRule = findAclRule( + matchingPrincipals(requestContext), + requestContext.clientAddress().getHostAddress(), + action + ); + + if (aclRule != null) { + rule = aclRule; + } else { + // If nothing matched, we return the default result. + rule = defaultRule; } - return ALLOWED; } + logAuditMessage(principal, requestContext, action, rule); + return rule.result(); + } + + private String buildAuditMessage( + KafkaPrincipal principal, + AuthorizableRequestContext context, + Action action, + MatchingRule rule + ) { + StringBuilder bldr = new StringBuilder(); + bldr.append("Principal = ").append(principal); + bldr.append(" is ").append(rule.result() == ALLOWED ? "Allowed" : "Denied"); + bldr.append(" operation = ").append(action.operation()); + bldr.append(" from host = ").append(context.clientAddress().getHostAddress()); + bldr.append(" on resource = "); + appendResourcePattern(action.resourcePattern(), bldr); + bldr.append(" for request = ").append(ApiKeys.forId(context.requestType()).name); + bldr.append(" with resourceRefCount = ").append(action.resourceReferenceCount()); + bldr.append(" based on rule ").append(rule); + return bldr.toString(); + } + + private void appendResourcePattern(ResourcePattern resourcePattern, StringBuilder bldr) { + bldr.append(SecurityUtils.resourceTypeName(resourcePattern.resourceType())) + .append(":") + .append(resourcePattern.patternType()) + .append(":") + .append(resourcePattern.name()); + } + + private void logAuditMessage( + KafkaPrincipal principal, + AuthorizableRequestContext requestContext, + Action action, + MatchingRule rule + ) { + switch (rule.result()) { + case ALLOWED: + // logIfAllowed is true if access is granted to the resource as a result of this authorization. + // In this case, log at debug level. If false, no access is actually granted, the result is used + // only to determine authorized operations. So log only at trace level. + if (action.logIfAllowed() && auditLog.isDebugEnabled()) { + auditLog.debug(buildAuditMessage(principal, requestContext, action, rule)); + } else if (auditLog.isTraceEnabled()) { + auditLog.trace(buildAuditMessage(principal, requestContext, action, rule)); + } + return; + + case DENIED: + // logIfDenied is true if access to the resource was explicitly requested. Since this is an attempt + // to access unauthorized resources, log at info level. If false, this is either a request to determine + // authorized operations or a filter (e.g for regex subscriptions) to filter out authorized resources. + // In this case, log only at trace level. + if (action.logIfDenied()) { + auditLog.info(buildAuditMessage(principal, requestContext, action, rule)); + } else if (auditLog.isTraceEnabled()) { + auditLog.trace(buildAuditMessage(principal, requestContext, action, rule)); + } + } + } + + private MatchingAclRule findAclRule( + Set matchingPrincipals, + String host, + Action action + ) { // This code relies on the ordering of StandardAcl within the NavigableMap. // Entries are sorted by resource type first, then REVERSE resource name. // Therefore, we can find all the applicable ACLs by starting at @@ -255,7 +348,7 @@ AuthorizationResult authorize(AuthorizableRequestContext requestContext, // 5. rs=TOPIC rn=eeee pt=LITERAL // // Once we reached element 5, we would stop scanning. - AuthorizationResultBuilder builder = new AuthorizationResultBuilder(); + MatchingAclBuilder matchingAclBuilder = new MatchingAclBuilder(); StandardAcl exemplar = new StandardAcl( action.resourcePattern().resourceType(), action.resourcePattern().name(), @@ -264,8 +357,10 @@ AuthorizationResult authorize(AuthorizableRequestContext requestContext, "", AclOperation.UNKNOWN, AclPermissionType.UNKNOWN); - checkSection(action, exemplar, requestContext, builder); - if (builder.foundDeny) return DENIED; + checkSection(action, exemplar, matchingPrincipals, host, matchingAclBuilder); + if (matchingAclBuilder.foundDeny()) { + return matchingAclBuilder.build(); + } // In addition to ACLs for this specific resource name, there can also be wildcard // ACLs that match any resource name. These are stored as type = LITERAL, @@ -278,30 +373,17 @@ AuthorizationResult authorize(AuthorizableRequestContext requestContext, "", AclOperation.UNKNOWN, AclPermissionType.UNKNOWN); - checkSection(action, exemplar, requestContext, builder); - if (builder.foundDeny) return DENIED; - - // If we found ALLOW ACLs, the action is allowed. - if (builder.foundAllow) { - if (log.isTraceEnabled()) { - log.trace("authorize(requestContext=" + requestContext + ", action=" + - action + "): ALLOWED"); - } - return ALLOWED; - } - - // If nothing matched, we return the default result. - if (log.isTraceEnabled()) { - log.trace("authorize(requestContext=" + requestContext + ", action=" + - action + "): returning default result " + defaultResult); - } - return defaultResult; + checkSection(action, exemplar, matchingPrincipals, host, matchingAclBuilder); + return matchingAclBuilder.build(); } - void checkSection(Action action, - StandardAcl exemplar, - AuthorizableRequestContext requestContext, - AuthorizationResultBuilder builder) { + private void checkSection( + Action action, + StandardAcl exemplar, + Set matchingPrincipals, + String host, + MatchingAclBuilder matchingAclBuilder + ) { NavigableSet tailSet = aclsByResource.tailSet(exemplar, true); String resourceName = action.resourcePattern().name(); for (Iterator iterator = tailSet.iterator(); @@ -325,15 +407,11 @@ void checkSection(Action action, // stepped outside of the section we care about and should stop scanning. break; } - AuthorizationResult result = findResult(action, requestContext, acl); + AuthorizationResult result = findResult(action, matchingPrincipals, host, acl); if (ALLOWED == result) { - builder.foundAllow = true; + matchingAclBuilder.allowAcl = acl; } else if (DENIED == result) { - if (log.isTraceEnabled()) { - log.trace("authorize(requestContext=" + requestContext + ", action=" + - action + "): DENIED because of " + acl); - } - builder.foundDeny = true; + matchingAclBuilder.denyAcl = acl; return; } } @@ -351,30 +429,55 @@ void checkSection(Action action, private static final Set IMPLIES_DESCRIBE_CONFIGS = Collections.unmodifiableSet( EnumSet.of(DESCRIBE_CONFIGS, ALTER_CONFIGS)); + static AuthorizationResult findResult(Action action, + AuthorizableRequestContext requestContext, + StandardAcl acl) { + return findResult( + action, + matchingPrincipals(requestContext), + requestContext.clientAddress().getHostAddress(), + acl + ); + } + + static KafkaPrincipal baseKafkaPrincipal(AuthorizableRequestContext context) { + KafkaPrincipal sessionPrincipal = context.principal(); + return sessionPrincipal.getClass().equals(KafkaPrincipal.class) + ? sessionPrincipal + : new KafkaPrincipal(sessionPrincipal.getPrincipalType(), sessionPrincipal.getName()); + } + + static Set matchingPrincipals(AuthorizableRequestContext context) { + KafkaPrincipal sessionPrincipal = context.principal(); + KafkaPrincipal basePrincipal = sessionPrincipal.getClass().equals(KafkaPrincipal.class) + ? sessionPrincipal + : new KafkaPrincipal(sessionPrincipal.getPrincipalType(), sessionPrincipal.getName()); + return Utils.mkSet(basePrincipal, WILDCARD_KAFKA_PRINCIPAL); + } + /** * Determine what the result of applying an ACL to the given action and request * context should be. Note that this function assumes that the resource name matches; * the resource name is not checked here. * - * @param action The input action. - * @param requestContext The input request context. - * @param acl The input ACL. - * @return null if the ACL does not match. The authorization result - * otherwise. + * @param action The input action. + * @param matchingPrincipals The set of input matching principals + * @param host The input host. + * @param acl The input ACL. + * @return null if the ACL does not match. The authorization result + * otherwise. */ static AuthorizationResult findResult(Action action, - AuthorizableRequestContext requestContext, + Set matchingPrincipals, + String host, StandardAcl acl) { // Check if the principal matches. If it doesn't, return no result (null). - if (!acl.principal().equals(WILDCARD_PRINCIPAL)) { - if (!acl.principal().equals(requestContext.principal().toString())) return null; + if (!matchingPrincipals.contains(acl.kafkaPrincipal())) { + return null; } // Check if the host matches. If it doesn't, return no result (null). - // The hostname should be cached in the InetAddress object, so calling this more - // than once shouldn't be too expensive. - if (!acl.host().equals(WILDCARD)) { - String host = requestContext.clientAddress().getHostAddress(); - if (!acl.host().equals(host)) return null; + if (!acl.host().equals(WILDCARD) && !acl.host().equals(host)) { + return null; } // Check if the operation field matches. Here we hit a slight complication. // ACLs for various operations (READ, WRITE, DELETE, ALTER), "imply" the presence @@ -456,4 +559,79 @@ public AclBinding next() { return result; } } + + private interface MatchingRule { + AuthorizationResult result(); + } + + private static class SuperUserRule implements MatchingRule { + private static final SuperUserRule INSTANCE = new SuperUserRule(); + + @Override + public AuthorizationResult result() { + return ALLOWED; + } + + @Override + public String toString() { + return "SuperUser"; + } + } + + private static class DefaultRule implements MatchingRule { + private final AuthorizationResult result; + + private DefaultRule(AuthorizationResult result) { + this.result = result; + } + + @Override + public AuthorizationResult result() { + return result; + } + + @Override + public String toString() { + return result == ALLOWED ? "DefaultAllow" : "DefaultDeny"; + } + } + + private static class MatchingAclRule implements MatchingRule { + private final StandardAcl acl; + private final AuthorizationResult result; + + private MatchingAclRule(StandardAcl acl, AuthorizationResult result) { + this.acl = acl; + this.result = result; + } + + @Override + public AuthorizationResult result() { + return result; + } + + @Override + public String toString() { + return "MatchingAcl(acl=" + acl + ")"; + } + } + + private static class MatchingAclBuilder { + private StandardAcl denyAcl; + private StandardAcl allowAcl; + + boolean foundDeny() { + return denyAcl != null; + } + + MatchingAclRule build() { + if (denyAcl != null) { + return new MatchingAclRule(denyAcl, DENIED); + } else if (allowAcl != null) { + return new MatchingAclRule(allowAcl, ALLOWED); + } else { + return null; + } + } + } } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java b/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java index ee09bb4c1273..7ed37785c423 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java @@ -32,6 +32,12 @@ import org.apache.kafka.server.authorizer.AuthorizableRequestContext; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.MockedStatic; +import org.mockito.Mockito; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.net.InetAddress; import java.util.Arrays; @@ -40,7 +46,6 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.concurrent.atomic.AtomicLong; import static java.util.Arrays.asList; import static java.util.Collections.singletonList; @@ -119,8 +124,6 @@ static Action newAction(AclOperation aclOperation, new ResourcePattern(resourceType, resourceName, LITERAL), 1, false, false); } - private final static AtomicLong NEXT_ID = new AtomicLong(0); - static StandardAcl newFooAcl(AclOperation op, AclPermissionType permission) { return new StandardAcl( TOPIC, @@ -428,4 +431,87 @@ public void testAuthorizationWithManyAcls() throws Exception { newAction(WRITE, GROUP, "arbitrary"), newAction(READ, TOPIC, "ala")))); } -} \ No newline at end of file + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testDenyAuditLogging(boolean logIfDenied) throws Exception { + try (MockedStatic mockedLoggerFactory = Mockito.mockStatic(LoggerFactory.class)) { + Logger otherLog = Mockito.mock(Logger.class); + Logger auditLog = Mockito.mock(Logger.class); + mockedLoggerFactory + .when(() -> LoggerFactory.getLogger("kafka.authorizer.logger")) + .thenReturn(auditLog); + + mockedLoggerFactory + .when(() -> LoggerFactory.getLogger(Mockito.any(Class.class))) + .thenReturn(otherLog); + + Mockito.when(auditLog.isDebugEnabled()).thenReturn(true); + Mockito.when(auditLog.isTraceEnabled()).thenReturn(true); + + StandardAuthorizer authorizer = createAuthorizerWithManyAcls(); + ResourcePattern topicResource = new ResourcePattern(TOPIC, "alpha", LITERAL); + Action action = new Action(READ, topicResource, 1, false, logIfDenied); + MockAuthorizableRequestContext requestContext = new MockAuthorizableRequestContext.Builder() + .setPrincipal(new KafkaPrincipal(USER_TYPE, "bob")) + .setClientAddress(InetAddress.getByName("127.0.0.1")) + .build(); + + assertEquals(singletonList(DENIED), authorizer.authorize(requestContext, singletonList(action))); + + String expectedAuditLog = "Principal = User:bob is Denied operation = READ " + + "from host = 127.0.0.1 on resource = Topic:LITERAL:alpha for request = Fetch " + + "with resourceRefCount = 1 based on rule MatchingAcl(acl=StandardAcl(resourceType=TOPIC, " + + "resourceName=alp, patternType=PREFIXED, principal=User:bob, host=*, operation=READ, " + + "permissionType=DENY))"; + + if (logIfDenied) { + Mockito.verify(auditLog).info(expectedAuditLog); + } else { + Mockito.verify(auditLog).trace(expectedAuditLog); + } + } + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testAllowAuditLogging(boolean logIfAllowed) throws Exception { + try (MockedStatic mockedLoggerFactory = Mockito.mockStatic(LoggerFactory.class)) { + Logger otherLog = Mockito.mock(Logger.class); + Logger auditLog = Mockito.mock(Logger.class); + mockedLoggerFactory + .when(() -> LoggerFactory.getLogger("kafka.authorizer.logger")) + .thenReturn(auditLog); + + mockedLoggerFactory + .when(() -> LoggerFactory.getLogger(Mockito.any(Class.class))) + .thenReturn(otherLog); + + Mockito.when(auditLog.isDebugEnabled()).thenReturn(true); + Mockito.when(auditLog.isTraceEnabled()).thenReturn(true); + + StandardAuthorizer authorizer = createAuthorizerWithManyAcls(); + ResourcePattern topicResource = new ResourcePattern(TOPIC, "green1", LITERAL); + Action action = new Action(READ, topicResource, 1, logIfAllowed, false); + MockAuthorizableRequestContext requestContext = new MockAuthorizableRequestContext.Builder() + .setPrincipal(new KafkaPrincipal(USER_TYPE, "bob")) + .setClientAddress(InetAddress.getByName("127.0.0.1")) + .build(); + + assertEquals(singletonList(ALLOWED), authorizer.authorize(requestContext, singletonList(action))); + + String expectedAuditLog = "Principal = User:bob is Allowed operation = READ " + + "from host = 127.0.0.1 on resource = Topic:LITERAL:green1 for request = Fetch " + + "with resourceRefCount = 1 based on rule MatchingAcl(acl=StandardAcl(resourceType=TOPIC, " + + "resourceName=green, patternType=PREFIXED, principal=User:bob, host=*, operation=READ, " + + "permissionType=ALLOW))"; + + if (logIfAllowed) { + Mockito.verify(auditLog).debug(expectedAuditLog); + } else { + Mockito.verify(auditLog).trace(expectedAuditLog); + } + } + } + +} From 87aa8259ddf5d4e5ed75aa41c4ba2ad65e2624a6 Mon Sep 17 00:00:00 2001 From: dengziming Date: Thu, 17 Mar 2022 16:25:45 +0800 Subject: [PATCH 111/447] KAFKA-13743: Prevent topics with conflicting metrics names from being created in KRaft mode #11910 In ZK mode, the topic "foo_bar" will conflict with "foo.bar" because of limitations in metric names. We should implement this in KRaft mode. This PR also changes TopicCommandIntegrationTest to support KRaft mode. Reviewers: Colin P. McCabe --- .../apache/kafka/common/internals/Topic.java | 13 +- .../kafka/common/internals/TopicTest.java | 9 + .../admin/TopicCommandIntegrationTest.scala | 306 +++++++++++------- .../controller/ReplicationControlManager.java | 51 ++- .../ReplicationControlManagerTest.java | 21 +- 5 files changed, 278 insertions(+), 122 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/internals/Topic.java b/clients/src/main/java/org/apache/kafka/common/internals/Topic.java index 7a5fefb3d9f2..3c93ef87b5c9 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/Topic.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/Topic.java @@ -67,6 +67,17 @@ public static boolean hasCollisionChars(String topic) { return topic.contains("_") || topic.contains("."); } + /** + * Unify topic name with a period ('.') or underscore ('_'), this is only used to check collision and will not + * be used to really change topic name. + * + * @param topic A topic to unify + * @return A unified topic name + */ + public static String unifyCollisionChars(String topic) { + return topic.replace('.', '_'); + } + /** * Returns true if the topicNames collide due to a period ('.') or underscore ('_') in the same position. * @@ -75,7 +86,7 @@ public static boolean hasCollisionChars(String topic) { * @return true if the topics collide */ public static boolean hasCollision(String topicA, String topicB) { - return topicA.replace('.', '_').equals(topicB.replace('.', '_')); + return unifyCollisionChars(topicA).equals(unifyCollisionChars(topicB)); } /** diff --git a/clients/src/test/java/org/apache/kafka/common/internals/TopicTest.java b/clients/src/test/java/org/apache/kafka/common/internals/TopicTest.java index 9bf237fb1b31..03c0811fa450 100644 --- a/clients/src/test/java/org/apache/kafka/common/internals/TopicTest.java +++ b/clients/src/test/java/org/apache/kafka/common/internals/TopicTest.java @@ -24,6 +24,7 @@ import java.util.Collections; import java.util.List; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; @@ -81,6 +82,14 @@ public void testTopicHasCollisionChars() { assertTrue(Topic.hasCollisionChars(topic)); } + @Test + public void testUnifyCollisionChars() { + assertEquals("topic", Topic.unifyCollisionChars("topic")); + assertEquals("_topic", Topic.unifyCollisionChars(".topic")); + assertEquals("_topic", Topic.unifyCollisionChars("_topic")); + assertEquals("__topic", Topic.unifyCollisionChars("_.topic")); + } + @Test public void testTopicHasCollision() { List periodFirstMiddleLastNone = Arrays.asList(".topic", "to.pic", "topic.", "topic"); diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandIntegrationTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandIntegrationTest.scala index 9a1fe378f6cf..ee7e64957ebb 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandIntegrationTest.scala @@ -17,23 +17,24 @@ package kafka.admin import java.util.{Collection, Collections, Optional, Properties} - import kafka.admin.TopicCommand.{TopicCommandOptions, TopicService} import kafka.integration.KafkaServerTestHarness -import kafka.server.{ConfigType, KafkaConfig} -import kafka.utils.{Logging, TestUtils} +import kafka.server.KafkaConfig +import kafka.utils.{Logging, TestInfoUtils, TestUtils} import kafka.zk.{ConfigEntityChangeNotificationZNode, DeleteTopicsTopicZNode} import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.clients.admin._ import org.apache.kafka.common.config.{ConfigException, ConfigResource, TopicConfig} -import org.apache.kafka.common.errors.{ClusterAuthorizationException, ThrottlingQuotaExceededException, TopicExistsException} +import org.apache.kafka.common.errors.{ClusterAuthorizationException, InvalidTopicException, ThrottlingQuotaExceededException, TopicExistsException} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.{Node, TopicPartition, TopicPartitionInfo} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import org.mockito.ArgumentMatcher import org.mockito.ArgumentMatchers.{eq => eqThat, _} import org.mockito.Mockito._ @@ -54,7 +55,7 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi */ override def generateConfigs: Seq[KafkaConfig] = TestUtils.createBrokerConfigs( numConfigs = 6, - zkConnect = zkConnect, + zkConnect = zkConnectOrNull, rackInfo = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack1", 4 -> "rack3", 5 -> "rack3"), numPartitions = numPartitions, defaultReplicationFactor = defaultReplicationFactor, @@ -76,7 +77,7 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi } private[this] def waitForTopicCreated(topicName: String, timeout: Int = 10000): Unit = { - TestUtils.waitForPartitionMetadata(servers, topicName, partition = 0, timeout) + TestUtils.waitForPartitionMetadata(brokers, topicName, partition = 0, timeout) } @BeforeEach @@ -98,16 +99,18 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi topicService.close() } - @Test - def testCreate(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreate(quorum: String): Unit = { createAndWaitTopic(new TopicCommandOptions( Array("--partitions", "2", "--replication-factor", "1", "--topic", testTopicName))) adminClient.listTopics().names().get().contains(testTopicName) } - @Test - def testCreateWithDefaults(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateWithDefaults(quorum: String): Unit = { createAndWaitTopic(new TopicCommandOptions(Array("--topic", testTopicName))) val partitions = adminClient @@ -120,8 +123,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi assertEquals(partitions.get(0).replicas().size(), defaultReplicationFactor) } - @Test - def testCreateWithDefaultReplication(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateWithDefaultReplication(quorum: String): Unit = { createAndWaitTopic(new TopicCommandOptions( Array("--topic", testTopicName, "--partitions", "2"))) @@ -135,8 +139,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi assertEquals(partitions.get(0).replicas().size(), defaultReplicationFactor) } - @Test - def testCreateWithDefaultPartitions(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateWithDefaultPartitions(quorum: String): Unit = { createAndWaitTopic(new TopicCommandOptions( Array("--topic", testTopicName, "--replication-factor", "2"))) @@ -151,8 +156,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi assertEquals(partitions.get(0).replicas().size(), 2) } - @Test - def testCreateWithConfigs(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateWithConfigs(quorum: String): Unit = { val configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName) createAndWaitTopic(new TopicCommandOptions( Array("--partitions", "2", "--replication-factor", "2", "--topic", testTopicName, "--config", "delete.retention.ms=1000"))) @@ -163,8 +169,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi assertEquals(1000, Integer.valueOf(configs.get("delete.retention.ms").value())) } - @Test - def testCreateWhenAlreadyExists(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateWhenAlreadyExists(quorum: String): Unit = { val numPartitions = 1 // create the topic @@ -176,15 +183,17 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi assertThrows(classOf[TopicExistsException], () => topicService.createTopic(createOpts)) } - @Test - def testCreateWhenAlreadyExistsWithIfNotExists(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateWhenAlreadyExistsWithIfNotExists(quorum: String): Unit = { val createOpts = new TopicCommandOptions(Array("--topic", testTopicName, "--if-not-exists")) createAndWaitTopic(createOpts) topicService.createTopic(createOpts) } - @Test - def testCreateWithReplicaAssignment(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateWithReplicaAssignment(quorum: String): Unit = { // create the topic val createOpts = new TopicCommandOptions( Array("--replica-assignment", "5:4,3:2,1:0", "--topic", testTopicName)) @@ -202,37 +211,42 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi assertEquals(List(1, 0), partitions.get(2).replicas().asScala.map(_.id())) } - @Test - def testCreateWithInvalidReplicationFactor(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateWithInvalidReplicationFactor(quorum: String): Unit = { assertThrows(classOf[IllegalArgumentException], () => topicService.createTopic(new TopicCommandOptions( Array("--partitions", "2", "--replication-factor", (Short.MaxValue+1).toString, "--topic", testTopicName)))) } - @Test - def testCreateWithNegativeReplicationFactor(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateWithNegativeReplicationFactor(quorum: String): Unit = { assertThrows(classOf[IllegalArgumentException], () => topicService.createTopic(new TopicCommandOptions( Array("--partitions", "2", "--replication-factor", "-1", "--topic", testTopicName)))) } - @Test - def testCreateWithNegativePartitionCount(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateWithNegativePartitionCount(quorum: String): Unit = { assertThrows(classOf[IllegalArgumentException], () => topicService.createTopic(new TopicCommandOptions( Array("--partitions", "-1", "--replication-factor", "1", "--topic", testTopicName)))) } - @Test - def testInvalidTopicLevelConfig(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testInvalidTopicLevelConfig(quorum: String): Unit = { val createOpts = new TopicCommandOptions( Array("--partitions", "1", "--replication-factor", "1", "--topic", testTopicName, "--config", "message.timestamp.type=boom")) assertThrows(classOf[ConfigException], () => topicService.createTopic(createOpts)) } - @Test - def testListTopics(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testListTopics(quorum: String): Unit = { createAndWaitTopic(new TopicCommandOptions( Array("--partitions", "1", "--replication-factor", "1", "--topic", testTopicName))) @@ -242,8 +256,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi assertTrue(output.contains(testTopicName)) } - @Test - def testListTopicsWithIncludeList(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testListTopicsWithIncludeList(quorum: String): Unit = { val topic1 = "kafka.testTopic1" val topic2 = "kafka.testTopic2" val topic3 = "oooof.testTopic1" @@ -264,8 +279,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi assertFalse(output.contains(topic3)) } - @Test - def testListTopicsWithExcludeInternal(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testListTopicsWithExcludeInternal(quorum: String): Unit = { val topic1 = "kafka.testTopic1" adminClient.createTopics( List(new NewTopic(topic1, 2, 2.toShort), @@ -280,8 +296,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi assertFalse(output.contains(Topic.GROUP_METADATA_TOPIC_NAME)) } - @Test - def testAlterPartitionCount(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testAlterPartitionCount(quorum: String): Unit = { adminClient.createTopics( List(new NewTopic(testTopicName, 2, 2.toShort)).asJavaCollection).all().get() waitForTopicCreated(testTopicName) @@ -289,12 +306,16 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi topicService.alterTopic(new TopicCommandOptions( Array("--topic", testTopicName, "--partitions", "3"))) + TestUtils.waitUntilTrue( + () => brokers.forall(_.metadataCache.getTopicPartitions(testTopicName).size == 3), + "Timeout waiting new assignment propagate to broker") val topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get() assertTrue(topicDescription.partitions().size() == 3) } - @Test - def testAlterAssignment(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testAlterAssignment(quorum: String): Unit = { adminClient.createTopics( Collections.singletonList(new NewTopic(testTopicName, 2, 2.toShort))).all().get() waitForTopicCreated(testTopicName) @@ -307,8 +328,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi assertEquals(List(4,2), topicDescription.partitions().get(2).replicas().asScala.map(_.id())) } - @Test - def testAlterAssignmentWithMoreAssignmentThanPartitions(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testAlterAssignmentWithMoreAssignmentThanPartitions(quorum: String): Unit = { adminClient.createTopics( List(new NewTopic(testTopicName, 2, 2.toShort)).asJavaCollection).all().get() waitForTopicCreated(testTopicName) @@ -318,8 +340,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi Array("--topic", testTopicName, "--replica-assignment", "5:3,3:1,4:2,3:2", "--partitions", "3")))) } - @Test - def testAlterAssignmentWithMorePartitionsThanAssignment(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testAlterAssignmentWithMorePartitionsThanAssignment(quorum: String): Unit = { adminClient.createTopics( List(new NewTopic(testTopicName, 2, 2.toShort)).asJavaCollection).all().get() waitForTopicCreated(testTopicName) @@ -329,8 +352,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi Array("--topic", testTopicName, "--replica-assignment", "5:3,3:1,4:2", "--partitions", "6")))) } - @Test - def testAlterWithInvalidPartitionCount(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testAlterWithInvalidPartitionCount(quorum: String): Unit = { createAndWaitTopic(new TopicCommandOptions( Array("--partitions", "1", "--replication-factor", "1", "--topic", testTopicName))) @@ -339,22 +363,25 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi Array("--partitions", "-1", "--topic", testTopicName)))) } - @Test - def testAlterWhenTopicDoesntExist(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testAlterWhenTopicDoesntExist(quorum: String): Unit = { // alter a topic that does not exist without --if-exists val alterOpts = new TopicCommandOptions(Array("--topic", testTopicName, "--partitions", "1")) val topicService = TopicService(adminClient) assertThrows(classOf[IllegalArgumentException], () => topicService.alterTopic(alterOpts)) } - @Test - def testAlterWhenTopicDoesntExistWithIfExists(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testAlterWhenTopicDoesntExistWithIfExists(quorum: String): Unit = { topicService.alterTopic(new TopicCommandOptions( Array("--topic", testTopicName, "--partitions", "1", "--if-exists"))) } - @Test - def testCreateAlterTopicWithRackAware(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateAlterTopicWithRackAware(quorum: String): Unit = { val rackInfo = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack1", 4 -> "rack3", 5 -> "rack3") val numPartitions = 18 @@ -365,9 +392,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi "--topic", testTopicName)) createAndWaitTopic(createOpts) - var assignment = zkClient.getReplicaAssignmentForTopics(Set(testTopicName)).map { case (tp, replicas) => - tp.partition -> replicas - } + var assignment = adminClient.describeTopics(Collections.singletonList(testTopicName)) + .allTopicNames().get().get(testTopicName).partitions() + .asScala.map(info => info.partition() -> info.replicas().asScala.map(_.id())).toMap checkReplicaDistribution(assignment, rackInfo, rackInfo.size, numPartitions, replicationFactor) val alteredNumPartitions = 36 @@ -376,14 +403,19 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi "--partitions", alteredNumPartitions.toString, "--topic", testTopicName)) topicService.alterTopic(alterOpts) - assignment = zkClient.getReplicaAssignmentForTopics(Set(testTopicName)).map { case (tp, replicas) => - tp.partition -> replicas - } + + TestUtils.waitUntilTrue( + () => brokers.forall(_.metadataCache.getTopicPartitions(testTopicName).size == alteredNumPartitions), + "Timeout waiting new assignment propagate to broker") + assignment = adminClient.describeTopics(Collections.singletonList(testTopicName)) + .allTopicNames().get().get(testTopicName).partitions() + .asScala.map(info => info.partition() -> info.replicas().asScala.map(_.id())).toMap checkReplicaDistribution(assignment, rackInfo, rackInfo.size, alteredNumPartitions, replicationFactor) } - @Test - def testConfigPreservationAcrossPartitionAlteration(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testConfigPreservationAcrossPartitionAlteration(quorum: String): Unit = { val numPartitionsOriginal = 1 val cleanupKey = "cleanup.policy" val cleanupVal = "compact" @@ -395,25 +427,30 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi "--config", cleanupKey + "=" + cleanupVal, "--topic", testTopicName)) createAndWaitTopic(createOpts) - val props = adminZkClient.fetchEntityConfig(ConfigType.Topic, testTopicName) - assertTrue(props.containsKey(cleanupKey), "Properties after creation don't contain " + cleanupKey) - assertTrue(props.getProperty(cleanupKey).equals(cleanupVal), "Properties after creation have incorrect value") + val configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName) + val props = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource) + // val props = adminZkClient.fetchEntityConfig(ConfigType.Topic, testTopicName) + assertNotNull(props.get(cleanupKey), "Properties after creation don't contain " + cleanupKey) + assertEquals(cleanupVal, props.get(cleanupKey).value(), "Properties after creation have incorrect value") // pre-create the topic config changes path to avoid a NoNodeException - zkClient.makeSurePersistentPathExists(ConfigEntityChangeNotificationZNode.path) + if (!isKRaftTest()) { + zkClient.makeSurePersistentPathExists(ConfigEntityChangeNotificationZNode.path) + } // modify the topic to add new partitions val numPartitionsModified = 3 val alterOpts = new TopicCommandOptions( Array("--partitions", numPartitionsModified.toString, "--topic", testTopicName)) topicService.alterTopic(alterOpts) - val newProps = adminZkClient.fetchEntityConfig(ConfigType.Topic, testTopicName) - assertTrue(newProps.containsKey(cleanupKey), "Updated properties do not contain " + cleanupKey) - assertTrue(newProps.getProperty(cleanupKey).equals(cleanupVal), "Updated properties have incorrect value") + val newProps = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource) + assertNotNull(newProps.get(cleanupKey), "Updated properties do not contain " + cleanupKey) + assertEquals(cleanupVal, newProps.get(cleanupKey).value(), "Updated properties have incorrect value") } - @Test - def testTopicDeletion(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testTopicDeletion(quorum: String): Unit = { // create the NormalTopic val createOpts = new TopicCommandOptions(Array("--partitions", "1", "--replication-factor", "1", @@ -423,14 +460,17 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi // delete the NormalTopic val deleteOpts = new TopicCommandOptions(Array("--topic", testTopicName)) - val deletePath = DeleteTopicsTopicZNode.path(testTopicName) - assertFalse(zkClient.pathExists(deletePath), "Delete path for topic shouldn't exist before deletion.") + if (!isKRaftTest()) { + val deletePath = DeleteTopicsTopicZNode.path(testTopicName) + assertFalse(zkClient.pathExists(deletePath), "Delete path for topic shouldn't exist before deletion.") + } topicService.deleteTopic(deleteOpts) - TestUtils.verifyTopicDeletion(zkClient, testTopicName, 1, servers) + TestUtils.verifyTopicDeletion(zkClientOrNull, testTopicName, 1, brokers) } - @Test - def testDeleteInternalTopic(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDeleteInternalTopic(quorum: String): Unit = { // create the offset topic val createOffsetTopicOpts = new TopicCommandOptions(Array("--partitions", "1", "--replication-factor", "1", @@ -443,25 +483,30 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi val deleteOffsetTopicOpts = new TopicCommandOptions( Array("--topic", Topic.GROUP_METADATA_TOPIC_NAME)) val deleteOffsetTopicPath = DeleteTopicsTopicZNode.path(Topic.GROUP_METADATA_TOPIC_NAME) - assertFalse(zkClient.pathExists(deleteOffsetTopicPath), "Delete path for topic shouldn't exist before deletion.") + if (!isKRaftTest()) { + assertFalse(zkClient.pathExists(deleteOffsetTopicPath), "Delete path for topic shouldn't exist before deletion.") + } topicService.deleteTopic(deleteOffsetTopicOpts) - TestUtils.verifyTopicDeletion(zkClient, Topic.GROUP_METADATA_TOPIC_NAME, 1, servers) + TestUtils.verifyTopicDeletion(zkClientOrNull, Topic.GROUP_METADATA_TOPIC_NAME, 1, brokers) } - @Test - def testDeleteWhenTopicDoesntExist(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDeleteWhenTopicDoesntExist(quorum: String): Unit = { // delete a topic that does not exist val deleteOpts = new TopicCommandOptions(Array("--topic", testTopicName)) assertThrows(classOf[IllegalArgumentException], () => topicService.deleteTopic(deleteOpts)) } - @Test - def testDeleteWhenTopicDoesntExistWithIfExists(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDeleteWhenTopicDoesntExistWithIfExists(quorum: String): Unit = { topicService.deleteTopic(new TopicCommandOptions(Array("--topic", testTopicName, "--if-exists"))) } - @Test - def testDescribe(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDescribe(quorum: String): Unit = { adminClient.createTopics( Collections.singletonList(new NewTopic(testTopicName, 2, 2.toShort))).all().get() waitForTopicCreated(testTopicName) @@ -473,19 +518,22 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi assertTrue(rows(0).startsWith(s"Topic: $testTopicName")) } - @Test - def testDescribeWhenTopicDoesntExist(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDescribeWhenTopicDoesntExist(quorum: String): Unit = { assertThrows(classOf[IllegalArgumentException], () => topicService.describeTopic(new TopicCommandOptions(Array("--topic", testTopicName)))) } - @Test - def testDescribeWhenTopicDoesntExistWithIfExists(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDescribeWhenTopicDoesntExistWithIfExists(quorum: String): Unit = { topicService.describeTopic(new TopicCommandOptions(Array("--topic", testTopicName, "--if-exists"))) } - @Test - def testDescribeUnavailablePartitions(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDescribeUnavailablePartitions(quorum: String): Unit = { adminClient.createTopics( Collections.singletonList(new NewTopic(testTopicName, 6, 1.toShort))).all().get() waitForTopicCreated(testTopicName) @@ -500,7 +548,7 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi // wait until the topic metadata for the test topic is propagated to each alive broker TestUtils.waitUntilTrue(() => { - servers + brokers .filterNot(_.config.brokerId == 0) .foldLeft(true) { (result, server) => { @@ -527,15 +575,16 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi } } - @Test - def testDescribeUnderReplicatedPartitions(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDescribeUnderReplicatedPartitions(quorum: String): Unit = { adminClient.createTopics( Collections.singletonList(new NewTopic(testTopicName, 1, 6.toShort))).all().get() waitForTopicCreated(testTopicName) try { killBroker(0) - val aliveServers = servers.filterNot(_.config.brokerId == 0) + val aliveServers = brokers.filterNot(_.config.brokerId == 0) TestUtils.waitForPartitionMetadata(aliveServers, testTopicName, 0) val output = TestUtils.grabConsoleOutput( topicService.describeTopic(new TopicCommandOptions(Array("--under-replicated-partitions")))) @@ -546,8 +595,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi } } - @Test - def testDescribeUnderMinIsrPartitions(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDescribeUnderMinIsrPartitions(quorum: String): Unit = { val configMap = new java.util.HashMap[String, String]() configMap.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "6") @@ -557,7 +607,7 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi try { killBroker(0) - val aliveServers = servers.filterNot(_.config.brokerId == 0) + val aliveServers = brokers.filterNot(_.config.brokerId == 0) TestUtils.waitForPartitionMetadata(aliveServers, testTopicName, 0) val output = TestUtils.grabConsoleOutput( topicService.describeTopic(new TopicCommandOptions(Array("--under-min-isr-partitions")))) @@ -568,8 +618,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi } } - @Test - def testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress(quorum: String): Unit = { val configMap = new java.util.HashMap[String, String]() val replicationFactor: Short = 1 val partitions = 1 @@ -580,12 +631,12 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi waitForTopicCreated(testTopicName) // Produce multiple batches. - TestUtils.generateAndProduceMessages(servers, testTopicName, numMessages = 10, acks = -1) - TestUtils.generateAndProduceMessages(servers, testTopicName, numMessages = 10, acks = -1) + TestUtils.generateAndProduceMessages(brokers, testTopicName, numMessages = 10, acks = -1) + TestUtils.generateAndProduceMessages(brokers, testTopicName, numMessages = 10, acks = -1) // Enable throttling. Note the broker config sets the replica max fetch bytes to `1` upon to minimize replication // throughput so the reassignment doesn't complete quickly. - val brokerIds = servers.map(_.config.brokerId) + val brokerIds = brokers.map(_.config.brokerId) TestUtils.setReplicationThrottleForPartitions(adminClient, brokerIds, Set(tp), throttleBytes = 1) val testTopicDesc = adminClient.describeTopics(Collections.singleton(testTopicName)).allTopicNames().get().get(testTopicName) @@ -622,8 +673,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi TestUtils.waitForAllReassignmentsToComplete(adminClient) } - @Test - def testDescribeAtMinIsrPartitions(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDescribeAtMinIsrPartitions(quorum: String): Unit = { val configMap = new java.util.HashMap[String, String]() configMap.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "4") @@ -653,8 +705,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi * * Output should only display the (1) topic with partition under min ISR count and (3) topic with offline partition */ - @Test - def testDescribeUnderMinIsrPartitionsMixed(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDescribeUnderMinIsrPartitionsMixed(quorum: String): Unit = { val underMinIsrTopic = "under-min-isr-topic" val notUnderMinIsrTopic = "not-under-min-isr-topic" val offlineTopic = "offline-topic" @@ -677,7 +730,7 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi try { killBroker(0) - val aliveServers = servers.filterNot(_.config.brokerId == 0) + val aliveServers = brokers.filterNot(_.config.brokerId == 0) TestUtils.waitForPartitionMetadata(aliveServers, underMinIsrTopic, 0) val output = TestUtils.grabConsoleOutput( topicService.describeTopic(new TopicCommandOptions(Array("--under-min-isr-partitions")))) @@ -690,8 +743,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi } } - @Test - def testDescribeReportOverriddenConfigs(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDescribeReportOverriddenConfigs(quorum: String): Unit = { val config = "file.delete.delay.ms=1000" createAndWaitTopic(new TopicCommandOptions( Array("--partitions", "2", "--replication-factor", "2", "--topic", testTopicName, "--config", config))) @@ -700,8 +754,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi assertTrue(output.contains(config), s"Describe output should have contained $config") } - @Test - def testDescribeAndListTopicsWithoutInternalTopics(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDescribeAndListTopicsWithoutInternalTopics(quorum: String): Unit = { createAndWaitTopic( new TopicCommandOptions(Array("--partitions", "1", "--replication-factor", "1", "--topic", testTopicName))) // create a internal topic @@ -720,8 +775,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi assertFalse(output.contains(Topic.GROUP_METADATA_TOPIC_NAME)) } - @Test - def testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized(quorum: String): Unit = { adminClient = spy(adminClient) topicService = TopicService(adminClient) @@ -746,8 +802,20 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi assertTrue(rows(0).startsWith(s"Topic: $testTopicName")) } - @Test - def testCreateTopicDoesNotRetryThrottlingQuotaExceededException(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateWithTopicNameCollision(quorum: String): Unit = { + adminClient.createTopics( + Collections.singletonList(new NewTopic("foo_bar", 1, 6.toShort))).all().get() + waitForTopicCreated("foo_bar") + + assertThrows(classOf[InvalidTopicException], + () => topicService.createTopic(new TopicCommandOptions(Array("--topic", "foo.bar")))) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateTopicDoesNotRetryThrottlingQuotaExceededException(quorum: String): Unit = { val adminClient = mock(classOf[Admin]) val topicService = TopicService(adminClient) @@ -766,8 +834,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi ) } - @Test - def testDeleteTopicDoesNotRetryThrottlingQuotaExceededException(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testDeleteTopicDoesNotRetryThrottlingQuotaExceededException(quorum: String): Unit = { val adminClient = mock(classOf[Admin]) val topicService = TopicService(adminClient) @@ -787,8 +856,9 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi ) } - @Test - def testCreatePartitionsDoesNotRetryThrottlingQuotaExceededException(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreatePartitionsDoesNotRetryThrottlingQuotaExceededException(quorum: String): Unit = { val adminClient = mock(classOf[Admin]) val topicService = TopicService(adminClient) diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index d8005e60ce64..f104364d1a6d 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -216,6 +216,23 @@ static Map translateCreationConfigs(CreateableTopicConfigCollect */ private final TimelineHashMap topicsByName; + /** + * We try to prevent topics from being created if their names would collide with + * existing topics when periods in the topic name are replaced with underscores. + * The reason for this is that some per-topic metrics do replace periods with + * underscores, and would therefore be ambiguous otherwise. + * + * This map is from normalized topic name to a set of topic names. So if we had two + * topics named foo.bar and foo_bar this map would contain + * a mapping from foo_bar to a set containing foo.bar and foo_bar. + * + * Since we reject topic creations that would collide, under normal conditions the + * sets in this map should only have a size of 1. However, if the cluster was + * upgraded from a version prior to KAFKA-13743, it may be possible to have more + * values here, since collidiing topic names will be "grandfathered in." + */ + private final TimelineHashMap> topicsWithCollisionChars; + /** * Maps topic UUIDs to structures containing topic information, including partitions. */ @@ -258,6 +275,7 @@ static Map translateCreationConfigs(CreateableTopicConfigCollect this.clusterControl = clusterControl; this.globalPartitionCount = new TimelineInteger(snapshotRegistry); this.topicsByName = new TimelineHashMap<>(snapshotRegistry, 0); + this.topicsWithCollisionChars = new TimelineHashMap<>(snapshotRegistry, 0); this.topics = new TimelineHashMap<>(snapshotRegistry, 0); this.brokersToIsrs = new BrokersToIsrs(snapshotRegistry); this.reassigningTopics = new TimelineHashMap<>(snapshotRegistry, 0); @@ -266,6 +284,15 @@ static Map translateCreationConfigs(CreateableTopicConfigCollect public void replay(TopicRecord record) { topicsByName.put(record.name(), record.topicId()); + if (Topic.hasCollisionChars(record.name())) { + String normalizedName = Topic.unifyCollisionChars(record.name()); + TimelineHashSet topicNames = topicsWithCollisionChars.get(normalizedName); + if (topicNames == null) { + topicNames = new TimelineHashSet<>(snapshotRegistry, 1); + topicsWithCollisionChars.put(normalizedName, topicNames); + } + topicNames.add(record.name()); + } topics.put(record.topicId(), new TopicControlInfo(record.name(), snapshotRegistry, record.topicId())); controllerMetrics.setGlobalTopicsCount(topics.size()); @@ -374,6 +401,16 @@ public void replay(RemoveTopicRecord record) { " to remove."); } topicsByName.remove(topic.name); + if (Topic.hasCollisionChars(topic.name)) { + String normalizedName = Topic.unifyCollisionChars(topic.name); + TimelineHashSet colliding = topicsWithCollisionChars.get(normalizedName); + if (colliding != null) { + colliding.remove(topic.name); + if (colliding.isEmpty()) { + topicsWithCollisionChars.remove(topic.name); + } + } + } reassigningTopics.remove(record.topicId()); // Delete the configurations associated with this topic. @@ -407,7 +444,7 @@ public void replay(RemoveTopicRecord record) { List records = new ArrayList<>(); // Check the topic names. - validateNewTopicNames(topicErrors, request.topics()); + validateNewTopicNames(topicErrors, request.topics(), topicsWithCollisionChars); // Identify topics that already exist and mark them with the appropriate error request.topics().stream().filter(creatableTopic -> topicsByName.containsKey(creatableTopic.name())) @@ -598,7 +635,8 @@ private ApiError maybeCheckCreateTopicPolicy(Supplier topicErrors, - CreatableTopicCollection topics) { + CreatableTopicCollection topics, + Map> topicsWithCollisionChars) { for (CreatableTopic topic : topics) { if (topicErrors.containsKey(topic.name())) continue; try { @@ -607,6 +645,15 @@ static void validateNewTopicNames(Map topicErrors, topicErrors.put(topic.name(), new ApiError(Errors.INVALID_TOPIC_EXCEPTION, e.getMessage())); } + if (Topic.hasCollisionChars(topic.name())) { + String normalizedName = Topic.unifyCollisionChars(topic.name()); + Set colliding = topicsWithCollisionChars.get(normalizedName); + if (colliding != null) { + topicErrors.put(topic.name(), new ApiError(Errors.INVALID_TOPIC_EXCEPTION, + "Topic '" + topic.name() + "' collides with existing topic: " + + colliding.iterator().next())); + } + } } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index 108f2eca665c..d095a9fe3729 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -82,6 +82,7 @@ import org.slf4j.LoggerFactory; import java.util.Arrays; +import java.util.TreeSet; import java.util.concurrent.atomic.AtomicLong; import java.util.ArrayList; import java.util.Collections; @@ -641,7 +642,7 @@ public void testValidateNewTopicNames() { topics.add(new CreatableTopic().setName("")); topics.add(new CreatableTopic().setName("woo")); topics.add(new CreatableTopic().setName(".")); - ReplicationControlManager.validateNewTopicNames(topicErrors, topics); + ReplicationControlManager.validateNewTopicNames(topicErrors, topics, Collections.emptyMap()); Map expectedTopicErrors = new HashMap<>(); expectedTopicErrors.put("", new ApiError(INVALID_TOPIC_EXCEPTION, "Topic name is illegal, it can't be empty")); @@ -650,6 +651,24 @@ public void testValidateNewTopicNames() { assertEquals(expectedTopicErrors, topicErrors); } + @Test + public void testTopicNameCollision() { + Map topicErrors = new HashMap<>(); + CreatableTopicCollection topics = new CreatableTopicCollection(); + topics.add(new CreatableTopic().setName("foo.bar")); + topics.add(new CreatableTopic().setName("woo.bar_foo")); + Map> collisionMap = new HashMap<>(); + collisionMap.put("foo_bar", new TreeSet<>(Arrays.asList("foo_bar"))); + collisionMap.put("woo_bar_foo", new TreeSet<>(Arrays.asList("woo.bar.foo", "woo_bar.foo"))); + ReplicationControlManager.validateNewTopicNames(topicErrors, topics, collisionMap); + Map expectedTopicErrors = new HashMap<>(); + expectedTopicErrors.put("foo.bar", new ApiError(INVALID_TOPIC_EXCEPTION, + "Topic 'foo.bar' collides with existing topic: foo_bar")); + expectedTopicErrors.put("woo.bar_foo", new ApiError(INVALID_TOPIC_EXCEPTION, + "Topic 'woo.bar_foo' collides with existing topic: woo.bar.foo")); + assertEquals(expectedTopicErrors, topicErrors); + } + @Test public void testRemoveLeaderships() throws Exception { ReplicationControlTestContext ctx = new ReplicationControlTestContext(); From 01e4ceba52b7e2a4046b595f21b233c65060192e Mon Sep 17 00:00:00 2001 From: Lee Dongjin Date: Thu, 14 Apr 2022 18:58:57 +0900 Subject: [PATCH 112/447] KAFKA-12613: Fix inconsistent validation logic between KafkaConfig and LogConfig (#10472) Reviewers: Mickael Maison --- core/src/main/scala/kafka/log/LogConfig.scala | 4 ++-- core/src/main/scala/kafka/server/KafkaConfig.scala | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 027e47882584..6e3cf459c364 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -331,9 +331,9 @@ object LogConfig { KafkaConfig.LogRollTimeMillisProp) .define(SegmentJitterMsProp, LONG, Defaults.SegmentJitterMs, atLeast(0), MEDIUM, SegmentJitterMsDoc, KafkaConfig.LogRollTimeJitterMillisProp) - .define(SegmentIndexBytesProp, INT, Defaults.MaxIndexSize, atLeast(0), MEDIUM, MaxIndexSizeDoc, + .define(SegmentIndexBytesProp, INT, Defaults.MaxIndexSize, atLeast(4), MEDIUM, MaxIndexSizeDoc, KafkaConfig.LogIndexSizeMaxBytesProp) - .define(FlushMessagesProp, LONG, Defaults.FlushInterval, atLeast(0), MEDIUM, FlushIntervalDoc, + .define(FlushMessagesProp, LONG, Defaults.FlushInterval, atLeast(1), MEDIUM, FlushIntervalDoc, KafkaConfig.LogFlushIntervalMessagesProp) .define(FlushMsProp, LONG, Defaults.FlushMs, atLeast(0), MEDIUM, FlushMsDoc, KafkaConfig.LogFlushIntervalMsProp) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 6fe0acaa9d3c..1dd162613919 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -1185,11 +1185,11 @@ object KafkaConfig { .define(LogCleanerIoBufferSizeProp, INT, Defaults.LogCleanerIoBufferSize, atLeast(0), MEDIUM, LogCleanerIoBufferSizeDoc) .define(LogCleanerDedupeBufferLoadFactorProp, DOUBLE, Defaults.LogCleanerDedupeBufferLoadFactor, MEDIUM, LogCleanerDedupeBufferLoadFactorDoc) .define(LogCleanerBackoffMsProp, LONG, Defaults.LogCleanerBackoffMs, atLeast(0), MEDIUM, LogCleanerBackoffMsDoc) - .define(LogCleanerMinCleanRatioProp, DOUBLE, Defaults.LogCleanerMinCleanRatio, MEDIUM, LogCleanerMinCleanRatioDoc) + .define(LogCleanerMinCleanRatioProp, DOUBLE, Defaults.LogCleanerMinCleanRatio, between(0, 1), MEDIUM, LogCleanerMinCleanRatioDoc) .define(LogCleanerEnableProp, BOOLEAN, Defaults.LogCleanerEnable, MEDIUM, LogCleanerEnableDoc) - .define(LogCleanerDeleteRetentionMsProp, LONG, Defaults.LogCleanerDeleteRetentionMs, MEDIUM, LogCleanerDeleteRetentionMsDoc) - .define(LogCleanerMinCompactionLagMsProp, LONG, Defaults.LogCleanerMinCompactionLagMs, MEDIUM, LogCleanerMinCompactionLagMsDoc) - .define(LogCleanerMaxCompactionLagMsProp, LONG, Defaults.LogCleanerMaxCompactionLagMs, MEDIUM, LogCleanerMaxCompactionLagMsDoc) + .define(LogCleanerDeleteRetentionMsProp, LONG, Defaults.LogCleanerDeleteRetentionMs, atLeast(0), MEDIUM, LogCleanerDeleteRetentionMsDoc) + .define(LogCleanerMinCompactionLagMsProp, LONG, Defaults.LogCleanerMinCompactionLagMs, atLeast(0), MEDIUM, LogCleanerMinCompactionLagMsDoc) + .define(LogCleanerMaxCompactionLagMsProp, LONG, Defaults.LogCleanerMaxCompactionLagMs, atLeast(1), MEDIUM, LogCleanerMaxCompactionLagMsDoc) .define(LogIndexSizeMaxBytesProp, INT, Defaults.LogIndexSizeMaxBytes, atLeast(4), MEDIUM, LogIndexSizeMaxBytesDoc) .define(LogIndexIntervalBytesProp, INT, Defaults.LogIndexIntervalBytes, atLeast(0), MEDIUM, LogIndexIntervalBytesDoc) .define(LogFlushIntervalMessagesProp, LONG, Defaults.LogFlushIntervalMessages, atLeast(1), HIGH, LogFlushIntervalMessagesDoc) @@ -1204,7 +1204,7 @@ object KafkaConfig { .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), HIGH, MinInSyncReplicasDoc) .define(LogMessageFormatVersionProp, STRING, Defaults.LogMessageFormatVersion, ApiVersionValidator, MEDIUM, LogMessageFormatVersionDoc) .define(LogMessageTimestampTypeProp, STRING, Defaults.LogMessageTimestampType, in("CreateTime", "LogAppendTime"), MEDIUM, LogMessageTimestampTypeDoc) - .define(LogMessageTimestampDifferenceMaxMsProp, LONG, Defaults.LogMessageTimestampDifferenceMaxMs, MEDIUM, LogMessageTimestampDifferenceMaxMsDoc) + .define(LogMessageTimestampDifferenceMaxMsProp, LONG, Defaults.LogMessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, LogMessageTimestampDifferenceMaxMsDoc) .define(CreateTopicPolicyClassNameProp, CLASS, null, LOW, CreateTopicPolicyClassNameDoc) .define(AlterConfigPolicyClassNameProp, CLASS, null, LOW, AlterConfigPolicyClassNameDoc) .define(LogMessageDownConversionEnableProp, BOOLEAN, Defaults.MessageDownConversionEnable, LOW, LogMessageDownConversionEnableDoc) From 55ff5d360381af370fe5b3a215831beac49571a4 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 14 Apr 2022 13:04:32 -0400 Subject: [PATCH 113/447] KAFKA-13823 Feature flag changes from KIP-778 (#12036) This PR includes the changes to feature flags that were outlined in KIP-778. Specifically, it changes UpdateFeatures and FeatureLevelRecord to remove the maximum version level. It also adds dry-run to the RPC so the controller can actually attempt the upgrade (rather than the client). It introduces an upgrade type enum, which supersedes the allowDowngrade boolean. Because FeatureLevelRecord was unused previously, we do not need to introduce a new version. The kafka-features.sh tool was overhauled in KIP-778 and now includes the describe, upgrade, downgrade, and disable sub-commands. Refer to [KIP-778](https://cwiki.apache.org/confluence/display/KAFKA/KIP-778%3A+KRaft+Upgrades) for more details on the new command structure. Reviewers: Colin P. McCabe , dengziming --- checkstyle/suppressions.xml | 4 +- .../kafka/clients/admin/FeatureUpdate.java | 75 ++- .../kafka/clients/admin/KafkaAdminClient.java | 3 +- .../clients/admin/UpdateFeaturesOptions.java | 20 + .../apache/kafka/common/protocol/ApiKeys.java | 2 +- .../requests/UpdateFeaturesRequest.java | 54 +- .../common/message/UpdateFeaturesRequest.json | 14 +- .../message/UpdateFeaturesResponse.json | 2 +- .../clients/admin/KafkaAdminClientTest.java | 14 +- .../kafka/common/protocol/ApiKeysTest.java | 2 +- .../requests/UpdateFeaturesRequestTest.java | 90 +++ .../scala/kafka/admin/FeatureCommand.scala | 581 ++++++++---------- .../kafka/controller/KafkaController.scala | 57 +- .../scala/kafka/server/BrokerFeatures.scala | 12 +- .../scala/kafka/server/BrokerServer.scala | 20 +- .../scala/kafka/server/ControllerApis.scala | 19 +- .../scala/kafka/server/ControllerServer.scala | 9 +- .../kafka/server/FinalizedFeatureCache.scala | 6 +- .../scala/kafka/server/KafkaRaftServer.scala | 3 +- core/src/main/scala/kafka/server/Server.scala | 6 - .../test/java/kafka/test/MockController.java | 11 +- .../kafka/testkit/KafkaClusterTestKit.java | 4 +- .../kafka/server/QuorumTestHarness.scala | 3 +- .../unit/kafka/admin/FeatureCommandTest.scala | 182 +----- .../kafka/server/BrokerFeaturesTest.scala | 3 +- .../kafka/server/ControllerApisTest.scala | 1 - .../unit/kafka/server/RequestQuotaTest.scala | 2 +- .../kafka/server/UpdateFeaturesTest.scala | 62 +- .../controller/ClusterControlManager.java | 24 +- .../apache/kafka/controller/Controller.java | 15 +- .../controller/FeatureControlManager.java | 115 ++-- .../controller/ProducerIdControlManager.java | 24 +- .../kafka/controller/QuorumController.java | 50 +- .../kafka/controller/QuorumFeatures.java | 51 ++ .../org/apache/kafka/image/FeaturesDelta.java | 19 +- .../org/apache/kafka/image/FeaturesImage.java | 22 +- .../kafka/metadata/BrokerRegistration.java | 2 +- .../org/apache/kafka/metadata/FeatureMap.java | 67 -- ....java => FinalizedControllerFeatures.java} | 30 +- .../apache/kafka/metadata/VersionRange.java | 22 +- .../common/metadata/FeatureLevelRecord.json | 6 +- .../controller/ClusterControlManagerTest.java | 5 +- .../controller/FeatureControlManagerTest.java | 119 ++-- .../controller/QuorumControllerTest.java | 10 +- .../controller/QuorumControllerTestEnv.java | 1 + .../apache/kafka/image/ClusterImageTest.java | 8 +- .../apache/kafka/image/FeaturesImageTest.java | 15 +- .../metadata/BrokerRegistrationTest.java | 6 +- .../kafka/metadata/VersionRangeTest.java | 19 +- .../kafka/raft/internals/RecordsIterator.java | 1 + 50 files changed, 1027 insertions(+), 865 deletions(-) create mode 100644 metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/FeatureMap.java rename metadata/src/main/java/org/apache/kafka/metadata/{FeatureMapAndEpoch.java => FinalizedControllerFeatures.java} (61%) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 0824e0b60b73..c1d3cb078d8c 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -291,12 +291,12 @@ files="(QuorumController|QuorumControllerTest|ReplicationControlManager|ReplicationControlManagerTest).java"/> - + files="(ClientQuotasImage|KafkaEventQueue|ReplicationControlManager|FeatureControlManager).java"/> { + private boolean validateOnly = false; + + @Deprecated + public boolean dryRun() { + return validateOnly; + } + + public boolean validateOnly() { + return validateOnly; + } + + @Deprecated + public UpdateFeaturesOptions dryRun(boolean dryRun) { + return validateOnly(dryRun); + } + + public UpdateFeaturesOptions validateOnly(boolean validateOnly) { + this.validateOnly = validateOnly; + return this; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index ad96cdfa428b..399b631d767b 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -98,7 +98,7 @@ public enum ApiKeys { END_QUORUM_EPOCH(ApiMessageType.END_QUORUM_EPOCH, true, RecordBatch.MAGIC_VALUE_V0, false), DESCRIBE_QUORUM(ApiMessageType.DESCRIBE_QUORUM, true, RecordBatch.MAGIC_VALUE_V0, true), ALTER_PARTITION(ApiMessageType.ALTER_PARTITION, true), - UPDATE_FEATURES(ApiMessageType.UPDATE_FEATURES), + UPDATE_FEATURES(ApiMessageType.UPDATE_FEATURES, true, true), ENVELOPE(ApiMessageType.ENVELOPE, true, RecordBatch.MAGIC_VALUE_V0, false), FETCH_SNAPSHOT(ApiMessageType.FETCH_SNAPSHOT, false, RecordBatch.MAGIC_VALUE_V0, false), DESCRIBE_CLUSTER(ApiMessageType.DESCRIBE_CLUSTER), diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java index 7a6bf66cd9de..27cddfadcaa7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java @@ -16,15 +16,46 @@ */ package org.apache.kafka.common.requests; +import org.apache.kafka.clients.admin.FeatureUpdate; import org.apache.kafka.common.message.UpdateFeaturesRequestData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; import java.nio.ByteBuffer; +import java.util.Collection; import java.util.Collections; +import java.util.stream.Collectors; public class UpdateFeaturesRequest extends AbstractRequest { + public static class FeatureUpdateItem { + private final String featureName; + private final short featureLevel; + private final FeatureUpdate.UpgradeType upgradeType; + + public FeatureUpdateItem(String featureName, short featureLevel, FeatureUpdate.UpgradeType upgradeType) { + this.featureName = featureName; + this.featureLevel = featureLevel; + this.upgradeType = upgradeType; + } + + public String feature() { + return featureName; + } + + public short versionLevel() { + return featureLevel; + } + + public FeatureUpdate.UpgradeType upgradeType() { + return upgradeType; + } + + public boolean isDeleteRequest() { + return featureLevel < 1 && !upgradeType.equals(FeatureUpdate.UpgradeType.UPGRADE); + } + } + public static class Builder extends AbstractRequest.Builder { private final UpdateFeaturesRequestData data; @@ -52,6 +83,25 @@ public UpdateFeaturesRequest(UpdateFeaturesRequestData data, short version) { this.data = data; } + public FeatureUpdateItem getFeature(String name) { + UpdateFeaturesRequestData.FeatureUpdateKey update = data.featureUpdates().find(name); + if (super.version() == 0) { + if (update.allowDowngrade()) { + return new FeatureUpdateItem(update.feature(), update.maxVersionLevel(), FeatureUpdate.UpgradeType.SAFE_DOWNGRADE); + } else { + return new FeatureUpdateItem(update.feature(), update.maxVersionLevel(), FeatureUpdate.UpgradeType.UPGRADE); + } + } else { + return new FeatureUpdateItem(update.feature(), update.maxVersionLevel(), FeatureUpdate.UpgradeType.fromCode(update.upgradeType())); + } + } + + public Collection featureUpdates() { + return data.featureUpdates().stream() + .map(update -> getFeature(update.feature())) + .collect(Collectors.toList()); + } + @Override public UpdateFeaturesResponse getErrorResponse(int throttleTimeMs, Throwable e) { return UpdateFeaturesResponse.createWithErrors( @@ -69,8 +119,4 @@ public UpdateFeaturesRequestData data() { public static UpdateFeaturesRequest parse(ByteBuffer buffer, short version) { return new UpdateFeaturesRequest(new UpdateFeaturesRequestData(new ByteBufferAccessor(buffer), version), version); } - - public static boolean isDeleteRequest(UpdateFeaturesRequestData.FeatureUpdateKey update) { - return update.maxVersionLevel() < 1 && update.allowDowngrade(); - } } diff --git a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json index 2b3181362d7c..27ed8420fbb0 100644 --- a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json +++ b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json @@ -16,9 +16,9 @@ { "apiKey": 57, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["zkBroker", "broker", "controller"], "name": "UpdateFeaturesRequest", - "validVersions": "0", + "validVersions": "0-1", "flexibleVersions": "0+", "fields": [ { "name": "timeoutMs", "type": "int32", "versions": "0+", "default": "60000", @@ -29,8 +29,12 @@ "about": "The name of the finalized feature to be updated."}, {"name": "MaxVersionLevel", "type": "int16", "versions": "0+", "about": "The new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature."}, - {"name": "AllowDowngrade", "type": "bool", "versions": "0+", - "about": "When set to true, the finalized feature version level is allowed to be downgraded/deleted. The downgrade request will fail if the new maximum version level is a value that's not lower than the existing maximum finalized version level."} - ]} + {"name": "AllowDowngrade", "type": "bool", "versions": "0", + "about": "DEPRECATED in version 1 (see DowngradeType). When set to true, the finalized feature version level is allowed to be downgraded/deleted. The downgrade request will fail if the new maximum version level is a value that's not lower than the existing maximum finalized version level."}, + {"name": "UpgradeType", "type": "int8", "versions": "1+", "default": 1, + "about": "Determine which type of upgrade will be performed: 1 will perform an upgrade only (default), 2 is safe downgrades only (lossless), 3 is unsafe downgrades (lossy)."} + ]}, + {"name": "ValidateOnly", "type": "bool", "versions": "1+", "default": false, + "about": "True if we should validate the request, but not perform the upgrade or downgrade."} ] } diff --git a/clients/src/main/resources/common/message/UpdateFeaturesResponse.json b/clients/src/main/resources/common/message/UpdateFeaturesResponse.json index 63e84ff9683d..033926b801e3 100644 --- a/clients/src/main/resources/common/message/UpdateFeaturesResponse.json +++ b/clients/src/main/resources/common/message/UpdateFeaturesResponse.json @@ -17,7 +17,7 @@ "apiKey": 57, "type": "response", "name": "UpdateFeaturesResponse", - "validVersions": "0", + "validVersions": "0-1", "flexibleVersions": "0+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index c337831b0722..eb4681856e27 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -4685,8 +4685,8 @@ public void testListOffsetsNonMaxTimestampDowngradedImmediately() throws Excepti private Map makeTestFeatureUpdates() { return Utils.mkMap( - Utils.mkEntry("test_feature_1", new FeatureUpdate((short) 2, false)), - Utils.mkEntry("test_feature_2", new FeatureUpdate((short) 3, true))); + Utils.mkEntry("test_feature_1", new FeatureUpdate((short) 2, FeatureUpdate.UpgradeType.UPGRADE)), + Utils.mkEntry("test_feature_2", new FeatureUpdate((short) 3, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE))); } private Map makeTestFeatureUpdateErrors(final Map updates, final Errors error) { @@ -4782,8 +4782,8 @@ public void testUpdateFeaturesHandleNotControllerException() throws Exception { env.cluster().nodeById(controllerId)); final KafkaFuture future = env.adminClient().updateFeatures( Utils.mkMap( - Utils.mkEntry("test_feature_1", new FeatureUpdate((short) 2, false)), - Utils.mkEntry("test_feature_2", new FeatureUpdate((short) 3, true))), + Utils.mkEntry("test_feature_1", new FeatureUpdate((short) 2, FeatureUpdate.UpgradeType.UPGRADE)), + Utils.mkEntry("test_feature_2", new FeatureUpdate((short) 3, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE))), new UpdateFeaturesOptions().timeoutMs(10000) ).all(); future.get(); @@ -4806,8 +4806,8 @@ public void testUpdateFeaturesShouldFailRequestForInvalidFeatureName() { assertThrows( IllegalArgumentException.class, () -> env.adminClient().updateFeatures( - Utils.mkMap(Utils.mkEntry("feature", new FeatureUpdate((short) 2, false)), - Utils.mkEntry("", new FeatureUpdate((short) 2, false))), + Utils.mkMap(Utils.mkEntry("feature", new FeatureUpdate((short) 2, FeatureUpdate.UpgradeType.UPGRADE)), + Utils.mkEntry("", new FeatureUpdate((short) 2, FeatureUpdate.UpgradeType.UPGRADE))), new UpdateFeaturesOptions())); } } @@ -4816,7 +4816,7 @@ public void testUpdateFeaturesShouldFailRequestForInvalidFeatureName() { public void testUpdateFeaturesShouldFailRequestInClientWhenDowngradeFlagIsNotSetDuringDeletion() { assertThrows( IllegalArgumentException.class, - () -> new FeatureUpdate((short) 0, false)); + () -> new FeatureUpdate((short) 0, FeatureUpdate.UpgradeType.UPGRADE)); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java index ec6259ae725d..7e0a6e438e95 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java @@ -62,7 +62,7 @@ public void testAlterIsrIsClusterAction() { public void testResponseThrottleTime() { Set authenticationKeys = EnumSet.of(ApiKeys.SASL_HANDSHAKE, ApiKeys.SASL_AUTHENTICATE); // Newer protocol apis include throttle time ms even for cluster actions - Set clusterActionsWithThrottleTimeMs = EnumSet.of(ApiKeys.ALTER_PARTITION, ApiKeys.ALLOCATE_PRODUCER_IDS); + Set clusterActionsWithThrottleTimeMs = EnumSet.of(ApiKeys.ALTER_PARTITION, ApiKeys.ALLOCATE_PRODUCER_IDS, ApiKeys.UPDATE_FEATURES); for (ApiKeys apiKey: ApiKeys.zkBrokerApis()) { Schema responseSchema = apiKey.messageType.responseSchemas()[apiKey.latestVersion()]; BoundField throttleTimeField = responseSchema.get("throttle_time_ms"); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/UpdateFeaturesRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/UpdateFeaturesRequestTest.java index 1b63aecd0161..cf267da5574c 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/UpdateFeaturesRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/UpdateFeaturesRequestTest.java @@ -16,14 +16,20 @@ */ package org.apache.kafka.common.requests; +import org.apache.kafka.clients.admin.FeatureUpdate; import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.UpdateFeaturesRequestData; import org.apache.kafka.common.protocol.Errors; import org.junit.jupiter.api.Test; +import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; public class UpdateFeaturesRequestTest { @@ -53,4 +59,88 @@ public void testGetErrorResponse() { assertEquals(Collections.singletonMap(Errors.UNKNOWN_SERVER_ERROR, 1), response.errorCounts()); } + @Test + public void testUpdateFeaturesV0() { + UpdateFeaturesRequestData.FeatureUpdateKeyCollection features = + new UpdateFeaturesRequestData.FeatureUpdateKeyCollection(); + + features.add(new UpdateFeaturesRequestData.FeatureUpdateKey() + .setFeature("foo") + .setMaxVersionLevel((short) 1) + .setAllowDowngrade(true) + ); + + features.add(new UpdateFeaturesRequestData.FeatureUpdateKey() + .setFeature("bar") + .setMaxVersionLevel((short) 3) + ); + + UpdateFeaturesRequest request = new UpdateFeaturesRequest( + new UpdateFeaturesRequestData().setFeatureUpdates(features), + UpdateFeaturesRequestData.LOWEST_SUPPORTED_VERSION + ); + ByteBuffer buffer = request.serialize(); + request = UpdateFeaturesRequest.parse(buffer, UpdateFeaturesRequestData.LOWEST_SUPPORTED_VERSION); + + List updates = new ArrayList<>(request.featureUpdates()); + assertEquals(updates.size(), 2); + assertEquals(updates.get(0).upgradeType(), FeatureUpdate.UpgradeType.SAFE_DOWNGRADE); + assertEquals(updates.get(1).upgradeType(), FeatureUpdate.UpgradeType.UPGRADE); + } + + @Test + public void testUpdateFeaturesV1() { + UpdateFeaturesRequestData.FeatureUpdateKeyCollection features = + new UpdateFeaturesRequestData.FeatureUpdateKeyCollection(); + + features.add(new UpdateFeaturesRequestData.FeatureUpdateKey() + .setFeature("foo") + .setMaxVersionLevel((short) 1) + .setUpgradeType(FeatureUpdate.UpgradeType.SAFE_DOWNGRADE.code()) + ); + + features.add(new UpdateFeaturesRequestData.FeatureUpdateKey() + .setFeature("bar") + .setMaxVersionLevel((short) 3) + ); + + UpdateFeaturesRequest request = new UpdateFeaturesRequest( + new UpdateFeaturesRequestData().setFeatureUpdates(features), + UpdateFeaturesRequestData.HIGHEST_SUPPORTED_VERSION + ); + + ByteBuffer buffer = request.serialize(); + request = UpdateFeaturesRequest.parse(buffer, UpdateFeaturesRequestData.HIGHEST_SUPPORTED_VERSION); + + List updates = new ArrayList<>(request.featureUpdates()); + assertEquals(updates.size(), 2); + assertEquals(updates.get(0).upgradeType(), FeatureUpdate.UpgradeType.SAFE_DOWNGRADE); + assertEquals(updates.get(1).upgradeType(), FeatureUpdate.UpgradeType.UPGRADE); + + } + + @Test + public void testUpdateFeaturesV1OldBoolean() { + UpdateFeaturesRequestData.FeatureUpdateKeyCollection features = + new UpdateFeaturesRequestData.FeatureUpdateKeyCollection(); + + features.add(new UpdateFeaturesRequestData.FeatureUpdateKey() + .setFeature("foo") + .setMaxVersionLevel((short) 1) + .setAllowDowngrade(true) + ); + + features.add(new UpdateFeaturesRequestData.FeatureUpdateKey() + .setFeature("bar") + .setMaxVersionLevel((short) 3) + ); + + UpdateFeaturesRequest request = new UpdateFeaturesRequest( + new UpdateFeaturesRequestData().setFeatureUpdates(features), + UpdateFeaturesRequestData.HIGHEST_SUPPORTED_VERSION + ); + assertThrows(UnsupportedVersionException.class, request::serialize, + "This should fail since allowDowngrade is not supported in v1 of this RPC"); + } + } diff --git a/core/src/main/scala/kafka/admin/FeatureCommand.scala b/core/src/main/scala/kafka/admin/FeatureCommand.scala index 4b299652a6f2..c5c62648f4ed 100644 --- a/core/src/main/scala/kafka/admin/FeatureCommand.scala +++ b/core/src/main/scala/kafka/admin/FeatureCommand.scala @@ -17,374 +17,317 @@ package kafka.admin -import kafka.server.BrokerFeatures -import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit} +import kafka.tools.TerseFailure +import kafka.utils.Exit +import net.sourceforge.argparse4j.ArgumentParsers +import net.sourceforge.argparse4j.impl.Arguments.{append, fileType, storeTrue} +import net.sourceforge.argparse4j.inf.{Namespace, Subparsers} import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.clients.admin.{Admin, FeatureUpdate, UpdateFeaturesOptions} -import org.apache.kafka.common.feature.{Features, SupportedVersionRange} +import org.apache.kafka.clients.admin.FeatureUpdate.UpgradeType +import org.apache.kafka.clients.admin.{Admin, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult} import org.apache.kafka.common.utils.Utils -import java.util.Properties +import java.io.File +import java.util.Properties import scala.collection.Seq -import scala.collection.immutable.ListMap -import scala.jdk.CollectionConverters._ -import joptsimple.OptionSpec - import scala.concurrent.ExecutionException +import scala.jdk.CollectionConverters._ object FeatureCommand { def main(args: Array[String]): Unit = { - val opts = new FeatureCommandOptions(args) - val featureApis = new FeatureApis(opts) - var exitCode = 0 + val res = mainNoExit(args) + Exit.exit(res) + } + + // This is used for integration tests in order to avoid killing the test with Exit.exit + def mainNoExit(args: Array[String]): Int = { + val parser = ArgumentParsers.newArgumentParser("kafka-features") + .defaultHelp(true) + .description("This tool manages feature flags in Kafka.") + parser.addArgument("--bootstrap-server") + .help("A comma-separated list of host:port pairs to use for establishing the connection to the Kafka cluster.") + .required(true) + + parser.addArgument("--command-config") + .`type`(fileType()) + .help("Property file containing configs to be passed to Admin Client.") + val subparsers = parser.addSubparsers().dest("command") + addDescribeParser(subparsers) + addUpgradeParser(subparsers) + addDowngradeParser(subparsers) + addDisableParser(subparsers) + try { - featureApis.execute() + val namespace = parser.parseArgsOrFail(args) + val command = namespace.getString("command") + + val commandConfig = namespace.get[File]("command_config") + val props = if (commandConfig != null) { + if (!commandConfig.exists()) { + throw new TerseFailure(s"Properties file ${commandConfig.getPath} does not exists!") + } + Utils.loadProps(commandConfig.getPath) + } else { + new Properties() + } + props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, namespace.getString("bootstrap_server")) + val admin = Admin.create(props) + + command match { + case "describe" => handleDescribe(namespace, admin) + case "upgrade" => handleUpgrade(namespace, admin) + case "downgrade" => handleDowngrade(namespace, admin) + case "disable" => handleDisable(namespace, admin) + } + admin.close() + 0 } catch { - case e: IllegalArgumentException => - printException(e) - opts.parser.printHelpOn(System.err) - exitCode = 1 - case _: UpdateFeaturesException => - exitCode = 1 - case e: ExecutionException => - val cause = if (e.getCause == null) e else e.getCause - printException(cause) - exitCode = 1 - case e: Throwable => - printException(e) - exitCode = 1 - } finally { - featureApis.close() - Exit.exit(exitCode) + case e: TerseFailure => + System.err.println(e.getMessage) + 1 } } - private def printException(exception: Throwable): Unit = { - System.err.println("\nError encountered when executing command: " + Utils.stackTrace(exception)) - } -} + def addDescribeParser(subparsers: Subparsers): Unit = { + val describeParser = subparsers.addParser("describe") + .help("Describe one or more feature flags.") -class UpdateFeaturesException(message: String) extends RuntimeException(message) - -/** - * A class that provides necessary APIs to bridge feature APIs provided by the Admin client with - * the requirements of the CLI tool. - * - * @param opts the CLI options - */ -class FeatureApis(private var opts: FeatureCommandOptions) { - private var supportedFeatures = BrokerFeatures.createDefault().supportedFeatures - private var adminClient = FeatureApis.createAdminClient(opts) + val featureArgs = describeParser.addArgumentGroup("Specific Features") + featureArgs.addArgument("--feature") + .action(append()) + .help("A specific feature to describe. This option may be repeated for describing multiple feature flags.") - private def pad(op: String): String = { - f"$op%11s" + val releaseArgs = describeParser.addArgumentGroup("All Features for release") + releaseArgs.addArgument("--release") } - private val addOp = pad("[Add]") - private val upgradeOp = pad("[Upgrade]") - private val deleteOp = pad("[Delete]") - private val downgradeOp = pad("[Downgrade]") - - // For testing only. - private[admin] def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = { - supportedFeatures = newFeatures + def addUpgradeParser(subparsers: Subparsers): Unit = { + val upgradeParser = subparsers.addParser("upgrade") + .help("Upgrade one or more feature flags.") + + val featureArgs = upgradeParser.addArgumentGroup("Upgrade specific features") + featureArgs.addArgument("--feature") + .action(append()) + .help("A feature flag to upgrade. This option may be repeated for upgrading multiple feature flags.") + featureArgs.addArgument("--version") + .`type`(classOf[Short]) + .help("The version to upgrade to.") + .action(append()) + + val releaseArgs = upgradeParser.addArgumentGroup("Upgrade to feature level defined for a given release") + releaseArgs.addArgument("--release") + + upgradeParser.addArgument("--dry-run") + .help("Perform a dry-run of this upgrade operation.") + .action(storeTrue()) } - // For testing only. - private[admin] def setOptions(newOpts: FeatureCommandOptions): Unit = { - adminClient.close() - adminClient = FeatureApis.createAdminClient(newOpts) - opts = newOpts + def addDowngradeParser(subparsers: Subparsers): Unit = { + val downgradeParser = subparsers.addParser("downgrade") + .help("Upgrade one or more feature flags.") + + downgradeParser.addArgument("--feature") + .help("A feature flag to downgrade. This option may be repeated for downgrade multiple feature flags.") + .required(true) + .action(append()) + downgradeParser.addArgument("--version") + .`type`(classOf[Short]) + .help("The version to downgrade to.") + .required(true) + .action(append()) + downgradeParser.addArgument("--unsafe") + .help("Perform this downgrade even if it considered unsafe. Refer to specific feature flag documentation for details.") + .action(storeTrue()) + downgradeParser.addArgument("--dry-run") + .help("Perform a dry-run of this downgrade operation.") + .action(storeTrue()) } - /** - * Describes the supported and finalized features. The request is issued to any of the provided - * bootstrap servers. - */ - def describeFeatures(): Unit = { - val result = adminClient.describeFeatures.featureMetadata.get - val features = result.supportedFeatures.asScala.keys.toSet ++ result.finalizedFeatures.asScala.keys.toSet - - features.toList.sorted.foreach { - feature => - val output = new StringBuilder() - output.append(s"Feature: $feature") - - val (supportedMinVersion, supportedMaxVersion) = { - val supportedVersionRange = result.supportedFeatures.get(feature) - if (supportedVersionRange == null) { - ("-", "-") - } else { - (supportedVersionRange.minVersion, supportedVersionRange.maxVersion) - } - } - output.append(s"\tSupportedMinVersion: $supportedMinVersion") - output.append(s"\tSupportedMaxVersion: $supportedMaxVersion") + def addDisableParser(subparsers: Subparsers): Unit = { + val disableParser = subparsers.addParser("disable") + .help("Disable one or more feature flags. This is the same as downgrading the version to zero.") + + disableParser.addArgument("--feature") + .help("A feature flag to disable. This option may be repeated for disable multiple feature flags.") + .required(true) + .action(append()) + disableParser.addArgument("--unsafe") + .help("Disable the feature flag(s) even if it considered unsafe. Refer to specific feature flag documentation for details.") + .action(storeTrue()) + disableParser.addArgument("--dry-run") + .help("Perform a dry-run of this disable operation.") + .action(storeTrue()) + } - val (finalizedMinVersionLevel, finalizedMaxVersionLevel) = { - val finalizedVersionRange = result.finalizedFeatures.get(feature) - if (finalizedVersionRange == null) { - ("-", "-") - } else { - (finalizedVersionRange.minVersionLevel, finalizedVersionRange.maxVersionLevel) - } - } - output.append(s"\tFinalizedMinVersionLevel: $finalizedMinVersionLevel") - output.append(s"\tFinalizedMaxVersionLevel: $finalizedMaxVersionLevel") + def handleDescribe(namespace: Namespace, admin: Admin): Unit = { + val featureFilter = parseFeaturesOrRelease(namespace) match { + case Neither() => (_: String) => true + case Features(featureNames) => (feature: String) => featureNames.contains(feature) + case Release(release) => + // Special case, print the versions associated with the given release + printReleaseFeatures(release) + return + case Both() => throw new TerseFailure("Only one of --release or --feature may be specified with describe sub-command.") + } - val epoch = { - if (result.finalizedFeaturesEpoch.isPresent) { - result.finalizedFeaturesEpoch.get.toString + val featureMetadata = admin.describeFeatures().featureMetadata().get() + val featureEpoch = featureMetadata.finalizedFeaturesEpoch() + val epochString = if (featureEpoch.isPresent) { + s"Epoch: ${featureEpoch.get}" + } else { + "Epoch: -" + } + val finalized = featureMetadata.finalizedFeatures().asScala + featureMetadata.supportedFeatures().asScala.foreach { + case (feature, range) => + if (featureFilter.apply(feature)) { + if (finalized.contains(feature)) { + println(s"Feature: $feature\tSupportedMinVersion: ${range.minVersion()}\t" + + s"SupportedMaxVersion: ${range.maxVersion()}\tFinalizedVersionLevel: ${finalized(feature).maxVersionLevel()}\t$epochString") } else { - "-" + println(s"Feature: $feature\tSupportedMinVersion: ${range.minVersion()}\t" + + s"SupportedMaxVersion: ${range.maxVersion()}\tFinalizedVersionLevel: -\t$epochString") } } - output.append(s"\tEpoch: $epoch") - - println(output) } } - /** - * Upgrades all features known to this tool to their highest max version levels. The method may - * add new finalized features if they were not finalized previously, but it does not delete - * any existing finalized feature. The results of the feature updates are written to STDOUT. - * - * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature - * updates to STDOUT, without applying them. - * - * @throws UpdateFeaturesException if at least one of the feature updates failed - */ - def upgradeAllFeatures(): Unit = { - val metadata = adminClient.describeFeatures.featureMetadata.get - val existingFinalizedFeatures = metadata.finalizedFeatures - val updates = supportedFeatures.features.asScala.map { - case (feature, targetVersionRange) => - val existingVersionRange = existingFinalizedFeatures.get(feature) - if (existingVersionRange == null) { - val updateStr = - addOp + - s"\tFeature: $feature" + - s"\tExistingFinalizedMaxVersion: -" + - s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}" - (feature, Some((updateStr, new FeatureUpdate(targetVersionRange.max, false)))) - } else { - if (targetVersionRange.max > existingVersionRange.maxVersionLevel) { - val updateStr = - upgradeOp + - s"\tFeature: $feature" + - s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" + - s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}" - (feature, Some((updateStr, new FeatureUpdate(targetVersionRange.max, false)))) - } else { - (feature, Option.empty) - } - } - }.filter { - case(_, updateInfo) => updateInfo.isDefined - }.map { - case(feature, updateInfo) => (feature, updateInfo.get) - }.toMap + def printReleaseFeatures(release: String): Unit = { + println(s"Default feature versions for release $release:") + } - if (updates.nonEmpty) { - maybeApplyFeatureUpdates(updates) + def handleUpgrade(namespace: Namespace, admin: Admin): Unit = { + val featuresToUpgrade = parseFeaturesOrRelease(namespace) match { + case Features(featureNames) => parseVersions(featureNames, namespace) + case Release(release) => featuresForRelease(release) + case Neither() => throw new TerseFailure("Must specify either --release or at least one --feature and --version with upgrade sub-command.") + case Both() => throw new TerseFailure("Cannot specify both --release and --feature with upgrade sub-command.") } - } - /** - * Downgrades existing finalized features to the highest max version levels known to this tool. - * The method may delete existing finalized features if they are no longer seen to be supported, - * but it does not add a feature that was not finalized previously. The results of the feature - * updates are written to STDOUT. - * - * NOTE: if the --dry-run CLI option is provided, this method only prints the expected feature - * updates to STDOUT, without applying them. - * - * @throws UpdateFeaturesException if at least one of the feature updates failed - */ - def downgradeAllFeatures(): Unit = { - val metadata = adminClient.describeFeatures.featureMetadata.get - val existingFinalizedFeatures = metadata.finalizedFeatures - val supportedFeaturesMap = supportedFeatures.features - val updates = existingFinalizedFeatures.asScala.map { - case (feature, existingVersionRange) => - val targetVersionRange = supportedFeaturesMap.get(feature) - if (targetVersionRange == null) { - val updateStr = - deleteOp + - s"\tFeature: $feature" + - s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" + - s"\tNewFinalizedMaxVersion: -" - (feature, Some(updateStr, new FeatureUpdate(0, true))) - } else { - if (targetVersionRange.max < existingVersionRange.maxVersionLevel) { - val updateStr = - downgradeOp + - s"\tFeature: $feature" + - s"\tExistingFinalizedMaxVersion: ${existingVersionRange.maxVersionLevel}" + - s"\tNewFinalizedMaxVersion: ${targetVersionRange.max}" - (feature, Some(updateStr, new FeatureUpdate(targetVersionRange.max, true))) - } else { - (feature, Option.empty) - } - } - }.filter { - case(_, updateInfo) => updateInfo.isDefined - }.map { - case(feature, updateInfo) => (feature, updateInfo.get) - }.toMap + val dryRun = namespace.getBoolean("dry_run") + val updateResult = admin.updateFeatures(featuresToUpgrade.map { case (feature, version) => + feature -> new FeatureUpdate(version, UpgradeType.UPGRADE) + }.asJava, new UpdateFeaturesOptions().validateOnly(dryRun)) + handleUpdateFeaturesResponse(updateResult, featuresToUpgrade, dryRun, "upgrade") + } - if (updates.nonEmpty) { - maybeApplyFeatureUpdates(updates) + def handleDowngrade(namespace: Namespace, admin: Admin): Unit = { + val featuresToDowngrade = parseFeaturesOrRelease(namespace) match { + case Features(featureNames) => parseVersions(featureNames, namespace) + case Neither() => throw new TerseFailure("Must specify at least one --feature and --version with downgrade sub-command.") + case _ => throw new IllegalStateException() } - } - /** - * Applies the provided feature updates. If the --dry-run CLI option is provided, the method - * only prints the expected feature updates to STDOUT without applying them. - * - * @param updates the feature updates to be applied via the admin client - * - * @throws UpdateFeaturesException if at least one of the feature updates failed - */ - private def maybeApplyFeatureUpdates(updates: Map[String, (String, FeatureUpdate)]): Unit = { - if (opts.hasDryRunOption) { - println("Expected feature updates:" + ListMap( - updates - .toSeq - .sortBy { case(feature, _) => feature} :_*) - .map { case(_, (updateStr, _)) => updateStr} - .mkString("\n")) - } else { - val result = adminClient.updateFeatures( - updates - .map { case(feature, (_, update)) => (feature, update)} - .asJava, - new UpdateFeaturesOptions()) - val resultSortedByFeature = ListMap( - result - .values - .asScala - .toSeq - .sortBy { case(feature, _) => feature} :_*) - val failures = resultSortedByFeature.map { - case (feature, updateFuture) => - val (updateStr, _) = updates(feature) - try { - updateFuture.get - println(updateStr + "\tResult: OK") - 0 - } catch { - case e: ExecutionException => - val cause = if (e.getCause == null) e else e.getCause - println(updateStr + "\tResult: FAILED due to " + cause) - 1 - case e: Throwable => - println(updateStr + "\tResult: FAILED due to " + e) - 1 - } - }.sum - if (failures > 0) { - throw new UpdateFeaturesException(s"$failures feature updates failed!") + val dryRun = namespace.getBoolean("dry_run") + val unsafe = namespace.getBoolean("unsafe") + val updateResult = admin.updateFeatures(featuresToDowngrade.map { case (feature, version) => + if (unsafe) { + feature -> new FeatureUpdate(version, UpgradeType.UNSAFE_DOWNGRADE) + } else { + feature -> new FeatureUpdate(version, UpgradeType.SAFE_DOWNGRADE) } - } + }.asJava, new UpdateFeaturesOptions().validateOnly(dryRun)) + + handleUpdateFeaturesResponse(updateResult, featuresToDowngrade, dryRun, "downgrade") } - def execute(): Unit = { - if (opts.hasDescribeOption) { - describeFeatures() - } else if (opts.hasUpgradeAllOption) { - upgradeAllFeatures() - } else if (opts.hasDowngradeAllOption) { - downgradeAllFeatures() - } else { - throw new IllegalStateException("Unexpected state: no CLI command could be executed.") + def handleDisable(namespace: Namespace, admin: Admin): Unit = { + val featuresToDisable = parseFeaturesOrRelease(namespace) match { + case Features(featureNames) => featureNames + case Neither() => throw new TerseFailure("Must specify at least one --feature and --version with downgrade sub-command.") + case _ => throw new IllegalStateException() } - } - def close(): Unit = { - adminClient.close() + val dryRun = namespace.getBoolean("dry_run") + val unsafe = namespace.getBoolean("unsafe") + val updateResult = admin.updateFeatures(featuresToDisable.map { feature => + if (unsafe) { + feature -> new FeatureUpdate(0.toShort, UpgradeType.UNSAFE_DOWNGRADE) + } else { + feature -> new FeatureUpdate(0.toShort, UpgradeType.SAFE_DOWNGRADE) + } + }.toMap.asJava, new UpdateFeaturesOptions().validateOnly(dryRun)) + + handleUpdateFeaturesResponse(updateResult, featuresToDisable.map { + feature => feature -> 0.toShort + }.toMap, dryRun, "disable") } -} -class FeatureCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) { - private val bootstrapServerOpt = parser.accepts( - "bootstrap-server", - "REQUIRED: A comma-separated list of host:port pairs to use for establishing the connection" + - " to the Kafka cluster.") - .withRequiredArg - .describedAs("server to connect to") - .ofType(classOf[String]) - private val commandConfigOpt = parser.accepts( - "command-config", - "Property file containing configs to be passed to Admin Client." + - " This is used with --bootstrap-server option when required.") - .withOptionalArg - .describedAs("command config property file") - .ofType(classOf[String]) - private val describeOpt = parser.accepts( - "describe", - "Describe supported and finalized features from a random broker.") - private val upgradeAllOpt = parser.accepts( - "upgrade-all", - "Upgrades all finalized features to the maximum version levels known to the tool." + - " This command finalizes new features known to the tool that were never finalized" + - " previously in the cluster, but it is guaranteed to not delete any existing feature.") - private val downgradeAllOpt = parser.accepts( - "downgrade-all", - "Downgrades all finalized features to the maximum version levels known to the tool." + - " This command deletes unknown features from the list of finalized features in the" + - " cluster, but it is guaranteed to not add a new feature.") - private val dryRunOpt = parser.accepts( - "dry-run", - "Performs a dry-run of upgrade/downgrade mutations to finalized feature without applying them.") - - options = parser.parse(args : _*) - - checkArgs() - - def has(builder: OptionSpec[_]): Boolean = options.has(builder) - - def hasDescribeOption: Boolean = has(describeOpt) - - def hasDryRunOption: Boolean = has(dryRunOpt) - - def hasUpgradeAllOption: Boolean = has(upgradeAllOpt) - - def hasDowngradeAllOption: Boolean = has(downgradeAllOpt) - - def commandConfig: Properties = { - if (has(commandConfigOpt)) - Utils.loadProps(options.valueOf(commandConfigOpt)) - else - new Properties() + def handleUpdateFeaturesResponse(updateResult: UpdateFeaturesResult, + updatedFeatures: Map[String, Short], + dryRun: Boolean, + op: String): Unit = { + val errors = updateResult.values().asScala.map { case (feature, future) => + try { + future.get() + feature -> None + } catch { + case e: ExecutionException => feature -> Some(e.getCause) + case t: Throwable => feature -> Some(t) + } + } + + errors.foreach { case (feature, maybeThrowable) => + if (maybeThrowable.isDefined) { + if (dryRun) { + System.out.println(s"Can not $op feature '$feature' to ${updatedFeatures(feature)}. ${maybeThrowable.get.getMessage}") + } else { + System.out.println(s"Could not $op feature '$feature' to ${updatedFeatures(feature)}. ${maybeThrowable.get.getMessage}") + } + } else { + if (dryRun) { + System.out.println(s"Feature '$feature' can be ${op}d to ${updatedFeatures(feature)}.") + } else { + System.out.println(s"Feature '$feature' was ${op}d to ${updatedFeatures(feature)}.") + } + } + } } - def bootstrapServers: String = options.valueOf(bootstrapServerOpt) + sealed trait ReleaseOrFeatures { } + case class Neither() extends ReleaseOrFeatures + case class Release(release: String) extends ReleaseOrFeatures + case class Features(featureNames: Seq[String]) extends ReleaseOrFeatures + case class Both() extends ReleaseOrFeatures + + def parseFeaturesOrRelease(namespace: Namespace): ReleaseOrFeatures = { + val release = namespace.getString("release") + val features = namespace.getList[String]("feature").asScala + + if (release != null && features != null) { + Both() + } else if (release == null && features == null) { + Neither() + } else if (release != null) { + Release(release) + } else { + Features(features) + } + } - def checkArgs(): Unit = { - CommandLineUtils.printHelpAndExitIfNeeded(this, "This tool describes and updates finalized features.") - val numActions = Seq(describeOpt, upgradeAllOpt, downgradeAllOpt).count(has) - if (numActions != 1) { - CommandLineUtils.printUsageAndDie( - parser, - "Command must include exactly one action: --describe, --upgrade-all, --downgrade-all.") + def parseVersions(features: Seq[String], namespace: Namespace): Map[String, Short] = { + val versions = namespace.getList[Short]("version").asScala + if (versions == null) { + throw new TerseFailure("Must specify --version when using --feature argument(s).") } - CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt) - if (hasDryRunOption && !hasUpgradeAllOption && !hasDowngradeAllOption) { - CommandLineUtils.printUsageAndDie( - parser, - "Command can contain --dry-run option only when either --upgrade-all or --downgrade-all actions are provided.") + if (versions.size != features.size) { + if (versions.size > features.size) { + throw new TerseFailure("Too many --version arguments given. For each --feature argument there should be one --version argument.") + } else { + throw new TerseFailure("Too many --feature arguments given. For each --feature argument there should be one --version argument.") + } } + features.zip(versions).map { case (feature, version) => + feature -> version + }.toMap + } + + def defaultFeatures(): Map[String, Short] = { + Map.empty } -} -object FeatureApis { - private def createAdminClient(opts: FeatureCommandOptions): Admin = { - val props = new Properties() - props.putAll(opts.commandConfig) - props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.bootstrapServers) - Admin.create(props) + def featuresForRelease(release: String): Map[String, Short] = { + Map.empty } } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 7aeeded87fbd..c29d937ce0ce 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -33,12 +33,13 @@ import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult import kafka.zk.TopicZNode.TopicIdReplicaAssignment import kafka.zk.{FeatureZNodeStatus, _} import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler, ZNodeChildChangeHandler} +import org.apache.kafka.clients.admin.FeatureUpdate.UpgradeType import org.apache.kafka.common.ElectionType import org.apache.kafka.common.KafkaException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{BrokerNotAvailableException, ControllerMovedException, StaleBrokerEpochException} import org.apache.kafka.common.feature.{Features, FinalizedVersionRange} -import org.apache.kafka.common.message.{AllocateProducerIdsRequestData, AllocateProducerIdsResponseData, AlterPartitionRequestData, AlterPartitionResponseData, UpdateFeaturesRequestData} +import org.apache.kafka.common.message.{AllocateProducerIdsRequestData, AllocateProducerIdsResponseData, AlterPartitionRequestData, AlterPartitionResponseData} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AbstractControlRequest, ApiError, LeaderAndIsrResponse, UpdateFeaturesRequest, UpdateMetadataResponse} @@ -1928,8 +1929,9 @@ class KafkaController(val config: KafkaConfig, * * @return the new FinalizedVersionRange or error, as described above. */ - private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = { - if (UpdateFeaturesRequest.isDeleteRequest(update)) { + private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequest.FeatureUpdateItem): + Either[FinalizedVersionRange, ApiError] = { + if (update.isDeleteRequest) { throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update") } @@ -1941,7 +1943,7 @@ class KafkaController(val config: KafkaConfig, } else { var newVersionRange: FinalizedVersionRange = null try { - newVersionRange = new FinalizedVersionRange(supportedVersionRange.min, update.maxVersionLevel) + newVersionRange = new FinalizedVersionRange(update.versionLevel(), update.versionLevel()) } catch { case _: IllegalArgumentException => { // This exception means the provided maxVersionLevel is invalid. It is handled below @@ -1951,7 +1953,7 @@ class KafkaController(val config: KafkaConfig, if (newVersionRange == null) { Right(new ApiError(Errors.INVALID_REQUEST, "Could not apply finalized feature update because the provided" + - s" maxVersionLevel:${update.maxVersionLevel} is lower than the" + + s" maxVersionLevel:${update.versionLevel} is lower than the" + s" supported minVersion:${supportedVersionRange.min}.")) } else { val newFinalizedFeature = @@ -1985,9 +1987,9 @@ class KafkaController(val config: KafkaConfig, * @return the new FinalizedVersionRange to be updated into ZK or error * as described above. */ - private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey, + private def validateFeatureUpdate(update: UpdateFeaturesRequest.FeatureUpdateItem, existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = { - def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = { + def newVersionRangeOrError(update: UpdateFeaturesRequest.FeatureUpdateItem): Either[Option[FinalizedVersionRange], ApiError] = { newFinalizedVersionRangeOrIncompatibilityError(update) .fold(versionRange => Left(Some(versionRange)), error => Right(error)) } @@ -1995,9 +1997,12 @@ class KafkaController(val config: KafkaConfig, if (update.feature.isEmpty) { // Check that the feature name is not empty. Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty.")) + } else if (update.upgradeType.equals(UpgradeType.UNKNOWN)) { + Right(new ApiError(Errors.INVALID_REQUEST, "Received unknown upgrade type.")) } else { + // We handle deletion requests separately from non-deletion requests. - if (UpdateFeaturesRequest.isDeleteRequest(update)) { + if (update.isDeleteRequest) { if (existingVersionRange.isEmpty) { // Disallow deletion of a non-existing finalized feature. Right(new ApiError(Errors.INVALID_REQUEST, @@ -2005,39 +2010,33 @@ class KafkaController(val config: KafkaConfig, } else { Left(Option.empty) } - } else if (update.maxVersionLevel() < 1) { - // Disallow deletion of a finalized feature without allowDowngrade flag set. + } else if (update.versionLevel() < 1) { + // Disallow deletion of a finalized feature without SAFE downgrade type. Right(new ApiError(Errors.INVALID_REQUEST, - s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" + - s" than 1 without setting the allowDowngrade flag to true in the request.")) + s"Can not provide maxVersionLevel: ${update.versionLevel} less" + + s" than 1 without setting the SAFE downgradeType in the request.")) } else { existingVersionRange.map(existing => - if (update.maxVersionLevel == existing.max) { + if (update.versionLevel == existing.max) { // Disallow a case where target maxVersionLevel matches existing maxVersionLevel. Right(new ApiError(Errors.INVALID_REQUEST, - s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" + + s"Can not ${if (update.upgradeType.equals(UpgradeType.SAFE_DOWNGRADE)) "downgrade" else "upgrade"}" + s" a finalized feature from existing maxVersionLevel:${existing.max}" + " to the same value.")) - } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) { - // Disallow downgrade of a finalized feature without the allowDowngrade flag set. + } else if (update.versionLevel < existing.max && !update.upgradeType.equals(UpgradeType.SAFE_DOWNGRADE)) { + // Disallow downgrade of a finalized feature without the downgradeType set. Right(new ApiError(Errors.INVALID_REQUEST, s"Can not downgrade finalized feature from existing" + s" maxVersionLevel:${existing.max} to provided" + - s" maxVersionLevel:${update.maxVersionLevel} without setting the" + - " allowDowngrade flag in the request.")) - } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) { - // Disallow a request that sets allowDowngrade flag without specifying a + s" maxVersionLevel:${update.versionLevel} without setting the" + + " downgradeType to SAFE in the request.")) + } else if (!update.upgradeType.equals(UpgradeType.UPGRADE) && update.versionLevel > existing.max) { + // Disallow a request that sets downgradeType without specifying a // maxVersionLevel that's lower than the existing maxVersionLevel. Right(new ApiError(Errors.INVALID_REQUEST, - s"When the allowDowngrade flag set in the request, the provided" + - s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" + + s"When the downgradeType is set to SAFE set in the request, the provided" + + s" maxVersionLevel:${update.versionLevel} can not be greater than" + s" existing maxVersionLevel:${existing.max}.")) - } else if (update.maxVersionLevel < existing.min) { - // Disallow downgrade of a finalized feature below the existing finalized - // minVersionLevel. - Right(new ApiError(Errors.INVALID_REQUEST, - s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" + - s" because it's lower than the existing minVersionLevel:${existing.min}.")) } else { newVersionRangeOrError(update) } @@ -2057,7 +2056,7 @@ class KafkaController(val config: KafkaConfig, private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest, callback: UpdateFeaturesCallback): Unit = { - val updates = request.data.featureUpdates + val updates = request.featureUpdates val existingFeatures = featureCache.get .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala) .getOrElse(Map[String, FinalizedVersionRange]()) diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index dd84f9e73e70..4f0fe379898f 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -19,8 +19,8 @@ package kafka.server import kafka.utils.Logging import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange} -import org.apache.kafka.common.feature.Features._ +import java.util import scala.jdk.CollectionConverters._ /** @@ -32,7 +32,9 @@ import scala.jdk.CollectionConverters._ class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange]) { // For testing only. def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = { - supportedFeatures = newFeatures + val combined = new util.HashMap[String, SupportedVersionRange](supportedFeatures.features()) + combined.putAll(newFeatures.features()) + supportedFeatures = Features.supportedFeatures(combined) } /** @@ -43,7 +45,7 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte Features.finalizedFeatures( supportedFeatures.features.asScala.map { case(name, versionRange) => ( - name, new FinalizedVersionRange(versionRange.min, versionRange.max)) + name, new FinalizedVersionRange(versionRange.max, versionRange.max)) }.asJava) } @@ -70,9 +72,7 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte object BrokerFeatures extends Logging { def createDefault(): BrokerFeatures = { - // The arguments are currently empty, but, in the future as we define features we should - // populate the required values here. - new BrokerFeatures(emptySupportedFeatures) + new BrokerFeatures(Features.emptySupportedFeatures()) } /** diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 05aa0f954c04..a5166e5e66a7 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -33,6 +33,7 @@ import kafka.security.CredentialProvider import kafka.server.KafkaRaftServer.ControllerRole import kafka.server.metadata.{BrokerMetadataListener, BrokerMetadataPublisher, BrokerMetadataSnapshotter, ClientQuotaMetadataManager, KRaftMetadataCache, SnapshotWriterBuilder} import kafka.utils.{CoreUtils, KafkaScheduler} +import org.apache.kafka.common.feature.SupportedVersionRange import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.message.BrokerRegistrationRequestData.{Listener, ListenerCollection} import org.apache.kafka.common.metrics.Metrics @@ -80,8 +81,7 @@ class BrokerServer( val metrics: Metrics, val threadNamePrefix: Option[String], val initialOfflineDirs: Seq[String], - val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], - val supportedFeatures: util.Map[String, VersionRange] + val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]] ) extends KafkaBroker { override def brokerState: BrokerState = lifecycleManager.state @@ -141,10 +141,6 @@ class BrokerServer( @volatile var brokerTopicStats: BrokerTopicStats = null - val brokerFeatures: BrokerFeatures = BrokerFeatures.createDefault() - - val featureCache: FinalizedFeatureCache = new FinalizedFeatureCache(brokerFeatures) - val clusterId: String = metaProps.clusterId var metadataSnapshotter: Option[BrokerMetadataSnapshotter] = None @@ -153,6 +149,8 @@ class BrokerServer( var metadataPublisher: BrokerMetadataPublisher = null + val brokerFeatures: BrokerFeatures = BrokerFeatures.createDefault() + def kafkaYammerMetrics: KafkaYammerMetrics = KafkaYammerMetrics.INSTANCE private def maybeChangeStatus(from: ProcessStatus, to: ProcessStatus): Boolean = { @@ -223,6 +221,8 @@ class BrokerServer( clientToControllerChannelManager.start() forwardingManager = new ForwardingManagerImpl(clientToControllerChannelManager) + val featureCache: FinalizedFeatureCache = new FinalizedFeatureCache(brokerFeatures) + val apiVersionManager = ApiVersionManager( ListenerType.BROKER, config, @@ -332,10 +332,16 @@ class BrokerServer( setPort(if (ep.port == 0) socketServer.boundPort(ep.listenerName) else ep.port). setSecurityProtocol(ep.securityProtocol.id)) } + + val featuresRemapped = brokerFeatures.supportedFeatures.features().asScala.map { + case (k: String, v: SupportedVersionRange) => + k -> VersionRange.of(v.min, v.max) + }.asJava + lifecycleManager.start(() => metadataListener.highestMetadataOffset, BrokerToControllerChannelManager(controllerNodeProvider, time, metrics, config, "heartbeat", threadNamePrefix, config.brokerSessionTimeoutMs.toLong), - metaProps.clusterId, networkListeners, supportedFeatures) + metaProps.clusterId, networkListeners, featuresRemapped) // Register a listener with the Raft layer to receive metadata event notifications raftManager.register(metadataListener) diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala index 8b006dc02517..e4b40bc60362 100644 --- a/core/src/main/scala/kafka/server/ControllerApis.scala +++ b/core/src/main/scala/kafka/server/ControllerApis.scala @@ -22,7 +22,6 @@ import java.util.Collections import java.util.Map.Entry import java.util.concurrent.TimeUnit.{MILLISECONDS, NANOSECONDS} import java.util.concurrent.{CompletableFuture, ExecutionException} - import kafka.network.RequestChannel import kafka.raft.RaftManager import kafka.server.QuotaFactory.QuotaManagers @@ -48,7 +47,7 @@ import org.apache.kafka.common.resource.ResourceType.{CLUSTER, TOPIC} import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{Node, Uuid} import org.apache.kafka.controller.Controller -import org.apache.kafka.metadata.{BrokerHeartbeatReply, BrokerRegistrationReply, VersionRange} +import org.apache.kafka.metadata.{BrokerHeartbeatReply, BrokerRegistrationReply} import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.ApiMessageAndVersion @@ -62,7 +61,6 @@ class ControllerApis(val requestChannel: RequestChannel, val authorizer: Option[Authorizer], val quotas: QuotaManagers, val time: Time, - val supportedFeatures: Map[String, VersionRange], val controller: Controller, val raftManager: RaftManager[ApiMessageAndVersion], val config: KafkaConfig, @@ -108,6 +106,7 @@ class ControllerApis(val requestChannel: RequestChannel, case ApiKeys.CREATE_ACLS => aclApis.handleCreateAcls(request) case ApiKeys.DELETE_ACLS => aclApis.handleDeleteAcls(request) case ApiKeys.ELECT_LEADERS => handleElectLeaders(request) + case ApiKeys.UPDATE_FEATURES => handleUpdateFeatures(request) case _ => throw new ApiException(s"Unsupported ApiKey ${request.context.header.apiKey}") } } catch { @@ -784,4 +783,18 @@ class ControllerApis(val requestChannel: RequestChannel, } }) } + + def handleUpdateFeatures(request: RequestChannel.Request): Unit = { + val updateFeaturesRequest = request.body[UpdateFeaturesRequest] + authHelper.authorizeClusterOperation(request, ALTER) + controller.updateFeatures(updateFeaturesRequest.data) + .whenComplete((response, exception) => { + if (exception != null) { + requestHelper.handleError(request, exception) + } else { + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + new UpdateFeaturesResponse(response.setThrottleTimeMs(requestThrottleMs))) + } + }) + } } diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 072a7721f4a9..049ad5b8c2d5 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -37,8 +37,8 @@ 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.{Controller, QuorumController, QuorumControllerMetrics} -import org.apache.kafka.metadata.{KafkaConfigSchema, VersionRange} +import org.apache.kafka.controller.{Controller, QuorumController, QuorumControllerMetrics, QuorumFeatures} +import org.apache.kafka.metadata.KafkaConfigSchema import org.apache.kafka.raft.RaftConfig import org.apache.kafka.raft.RaftConfig.AddressSpec import org.apache.kafka.server.authorizer.Authorizer @@ -79,7 +79,6 @@ class ControllerServer( var createTopicPolicy: Option[CreateTopicPolicy] = None var alterConfigPolicy: Option[AlterConfigPolicy] = None var controller: Controller = null - val supportedFeatures: Map[String, VersionRange] = Map() var quotaManagers: QuotaManagers = null var controllerApis: ControllerApis = null var controllerApisHandlerPool: KafkaRequestHandlerPool = null @@ -161,6 +160,8 @@ class ControllerServer( alterConfigPolicy = Option(config. getConfiguredInstance(AlterConfigPolicyClassNameProp, classOf[AlterConfigPolicy])) + val quorumFeatures = QuorumFeatures.create(config.nodeId, QuorumFeatures.defaultFeatureMap()) + val controllerBuilder = { val leaderImbalanceCheckIntervalNs = if (config.autoLeaderRebalanceEnable) { OptionalLong.of(TimeUnit.NANOSECONDS.convert(config.leaderImbalanceCheckIntervalSeconds, TimeUnit.SECONDS)) @@ -173,6 +174,7 @@ class ControllerServer( setThreadNamePrefix(threadNamePrefixAsString). setConfigSchema(configSchema). setRaftClient(raftManager.client). + setQuorumFeatures(quorumFeatures). setDefaultReplicationFactor(config.defaultReplicationFactor.toShort). setDefaultNumPartitions(config.numPartitions.intValue()). setIsLeaderRecoverySupported(config.interBrokerProtocolVersion >= KAFKA_3_2_IV0). @@ -198,7 +200,6 @@ class ControllerServer( authorizer, quotaManagers, time, - supportedFeatures, controller, raftManager, config, diff --git a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala index 88addb76c4d5..ee7337653c73 100644 --- a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala +++ b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala @@ -138,10 +138,10 @@ class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends val newFeatures = new util.HashMap[String, FinalizedVersionRange]() newFeatures.putAll(features.features.features()) featuresDelta.changes().entrySet().forEach { e => - e.getValue().asScala match { + e.getValue.asScala match { case None => newFeatures.remove(e.getKey) - case Some(feature) => newFeatures.put(e.getKey, - new FinalizedVersionRange(feature.min(), feature.max())) + case Some(version) => newFeatures.put(e.getKey, + new FinalizedVersionRange(version, version)) } } featuresAndEpoch = Some(FinalizedFeaturesAndEpoch(Features.finalizedFeatures( diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index d9629376fd78..a0dd19559c41 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -86,8 +86,7 @@ class KafkaRaftServer( metrics, threadNamePrefix, offlineDirs, - controllerQuorumVotersFuture, - Server.SUPPORTED_FEATURES + controllerQuorumVotersFuture )) } else { None diff --git a/core/src/main/scala/kafka/server/Server.scala b/core/src/main/scala/kafka/server/Server.scala index c395df47e63f..5d902c583129 100644 --- a/core/src/main/scala/kafka/server/Server.scala +++ b/core/src/main/scala/kafka/server/Server.scala @@ -16,15 +16,12 @@ */ package kafka.server -import java.util.Collections import java.util.concurrent.TimeUnit import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.metrics.{JmxReporter, KafkaMetricsContext, MetricConfig, Metrics, MetricsReporter, Sensor} import org.apache.kafka.common.utils.Time -import org.apache.kafka.metadata.VersionRange -import scala.jdk.CollectionConverters._ trait Server { def startup(): Unit @@ -99,7 +96,4 @@ object Server { case object STARTING extends ProcessStatus case object STARTED extends ProcessStatus case object SHUTTING_DOWN extends ProcessStatus - - val SUPPORTED_FEATURES = Collections. - unmodifiableMap[String, VersionRange](Map[String, VersionRange]().asJava) } diff --git a/core/src/test/java/kafka/test/MockController.java b/core/src/test/java/kafka/test/MockController.java index b4bfb0dde657..d3bda0736185 100644 --- a/core/src/test/java/kafka/test/MockController.java +++ b/core/src/test/java/kafka/test/MockController.java @@ -41,6 +41,8 @@ import org.apache.kafka.common.message.ElectLeadersResponseData; import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData; import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; +import org.apache.kafka.common.message.UpdateFeaturesRequestData; +import org.apache.kafka.common.message.UpdateFeaturesResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaEntity; @@ -49,7 +51,7 @@ import org.apache.kafka.controller.ResultOrError; import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistrationReply; -import org.apache.kafka.metadata.FeatureMapAndEpoch; +import org.apache.kafka.metadata.FinalizedControllerFeatures; import org.apache.kafka.server.authorizer.AclCreateResult; import org.apache.kafka.server.authorizer.AclDeleteResult; @@ -246,7 +248,7 @@ public CompletableFuture electLeaders(ElectLeadersRequ } @Override - public CompletableFuture finalizedFeatures() { + public CompletableFuture finalizedFeatures() { throw new UnsupportedOperationException(); } @@ -349,6 +351,11 @@ public CompletableFuture allocateProducerIds(Al throw new UnsupportedOperationException(); } + @Override + public CompletableFuture updateFeatures(UpdateFeaturesRequestData request) { + throw new UnsupportedOperationException(); + } + @Override synchronized public CompletableFuture> createPartitions(long deadlineNs, List topicList) { diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index cbfe4ff34e49..2263b09116be 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -24,7 +24,6 @@ import kafka.server.KafkaConfig$; import kafka.server.KafkaRaftServer; import kafka.server.MetaProperties; -import kafka.server.Server; import kafka.tools.StorageTool; import kafka.utils.Logging; import org.apache.kafka.clients.CommonClientConfigs; @@ -238,8 +237,7 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS new Metrics(), Option.apply(threadNamePrefix), JavaConverters.asScalaBuffer(Collections.emptyList()).toSeq(), - connectFutureManager.future, - Server.SUPPORTED_FEATURES() + connectFutureManager.future ); brokers.put(node.id(), broker); raftManagers.put(node.id(), raftManager); diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index c329805e2c03..9f00c0564cf2 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -85,8 +85,7 @@ class KRaftQuorumImplementation(val raftManager: KafkaRaftManager[ApiMessageAndV metrics = new Metrics(), threadNamePrefix = Some("Broker%02d_".format(config.nodeId)), initialOfflineDirs = Seq(), - controllerQuorumVotersFuture = controllerQuorumVotersFuture, - supportedFeatures = Collections.emptyMap()) + controllerQuorumVotersFuture = controllerQuorumVotersFuture) if (startup) broker.startup() broker } diff --git a/core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala b/core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala index 93c22eb7a0fc..d75714818259 100644 --- a/core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.common.utils.Utils import java.util.Properties -import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue, assertThrows} +import org.junit.jupiter.api.Assertions.assertTrue import org.junit.jupiter.api.Test class FeatureCommandTest extends BaseRequestTest { @@ -75,167 +75,37 @@ class FeatureCommandTest extends BaseRequestTest { @Test def testDescribeFeaturesSuccess(): Unit = { updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures) - val featureApis = new FeatureApis(new FeatureCommandOptions(Array("--bootstrap-server", bootstrapServers(), "--describe"))) - featureApis.setSupportedFeatures(defaultSupportedFeatures) - try { - val initialDescribeOutput = TestUtils.grabConsoleOutput(featureApis.describeFeatures()) - val expectedInitialDescribeOutput = - "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedMinVersionLevel: -\tFinalizedMaxVersionLevel: -\tEpoch: 0\n" + - "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedMinVersionLevel: -\tFinalizedMaxVersionLevel: -\tEpoch: 0\n" - assertEquals(expectedInitialDescribeOutput, initialDescribeOutput) - featureApis.upgradeAllFeatures() - val finalDescribeOutput = TestUtils.grabConsoleOutput(featureApis.describeFeatures()) - val expectedFinalDescribeOutput = - "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedMinVersionLevel: 1\tFinalizedMaxVersionLevel: 3\tEpoch: 1\n" + - "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedMinVersionLevel: 1\tFinalizedMaxVersionLevel: 5\tEpoch: 1\n" - assertEquals(expectedFinalDescribeOutput, finalDescribeOutput) - } finally { - featureApis.close() - } - } - /** - * Tests if the FeatureApis#upgradeAllFeatures API works as expected during a success case. - */ - @Test - def testUpgradeAllFeaturesSuccess(): Unit = { - val upgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", bootstrapServers(), "--upgrade-all")) - val featureApis = new FeatureApis(upgradeOpts) - try { - // Step (1): - // - Update the supported features across all brokers. - // - Upgrade non-existing feature_1 to maxVersionLevel: 2. - // - Verify results. - val initialSupportedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2)))) - updateSupportedFeaturesInAllBrokers(initialSupportedFeatures) - featureApis.setSupportedFeatures(initialSupportedFeatures) - var output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures()) - var expected = - " [Add]\tFeature: feature_1\tExistingFinalizedMaxVersion: -\tNewFinalizedMaxVersion: 2\tResult: OK\n" - assertEquals(expected, output) + val initialDescribeOutput = TestUtils.grabConsoleOutput(FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), "describe"))) + val expectedInitialDescribeOutputs = Seq( + "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedVersionLevel: -", + "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedVersionLevel: -" + ) - // Step (2): - // - Update the supported features across all brokers. - // - Upgrade existing feature_1 to maxVersionLevel: 3. - // - Upgrade non-existing feature_2 to maxVersionLevel: 5. - // - Verify results. - updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures) - featureApis.setSupportedFeatures(defaultSupportedFeatures) - output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures()) - expected = - " [Upgrade]\tFeature: feature_1\tExistingFinalizedMaxVersion: 2\tNewFinalizedMaxVersion: 3\tResult: OK\n" + - " [Add]\tFeature: feature_2\tExistingFinalizedMaxVersion: -\tNewFinalizedMaxVersion: 5\tResult: OK\n" - assertEquals(expected, output) - - // Step (3): - // - Perform an upgrade of all features again. - // - Since supported features have not changed, expect that the above action does not yield - // any results. - output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures()) - assertTrue(output.isEmpty) - featureApis.setOptions(upgradeOpts) - output = TestUtils.grabConsoleOutput(featureApis.upgradeAllFeatures()) - assertTrue(output.isEmpty) - } finally { - featureApis.close() + expectedInitialDescribeOutputs.foreach { expectedOutput => + assertTrue(initialDescribeOutput.contains(expectedOutput)) } - } - - /** - * Tests if the FeatureApis#downgradeAllFeatures API works as expected during a success case. - */ - @Test - def testDowngradeFeaturesSuccess(): Unit = { - val downgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", bootstrapServers(), "--downgrade-all")) - val upgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", bootstrapServers(), "--upgrade-all")) - val featureApis = new FeatureApis(upgradeOpts) - try { - // Step (1): - // - Update the supported features across all brokers. - // - Upgrade non-existing feature_1 to maxVersionLevel: 3. - // - Upgrade non-existing feature_2 to maxVersionLevel: 5. - updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures) - featureApis.setSupportedFeatures(defaultSupportedFeatures) - featureApis.upgradeAllFeatures() - - // Step (2): - // - Downgrade existing feature_1 to maxVersionLevel: 2. - // - Delete feature_2 since it is no longer supported by the FeatureApis object. - // - Verify results. - val downgradedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2)))) - featureApis.setSupportedFeatures(downgradedFeatures) - featureApis.setOptions(downgradeOpts) - var output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures()) - var expected = - "[Downgrade]\tFeature: feature_1\tExistingFinalizedMaxVersion: 3\tNewFinalizedMaxVersion: 2\tResult: OK\n" + - " [Delete]\tFeature: feature_2\tExistingFinalizedMaxVersion: 5\tNewFinalizedMaxVersion: -\tResult: OK\n" - assertEquals(expected, output) - // Step (3): - // - Perform a downgrade of all features again. - // - Since supported features have not changed, expect that the above action does not yield - // any results. - updateSupportedFeaturesInAllBrokers(downgradedFeatures) - output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures()) - assertTrue(output.isEmpty) - - // Step (4): - // - Delete feature_1 since it is no longer supported by the FeatureApis object. - // - Verify results. - featureApis.setSupportedFeatures(Features.emptySupportedFeatures()) - output = TestUtils.grabConsoleOutput(featureApis.downgradeAllFeatures()) - expected = - " [Delete]\tFeature: feature_1\tExistingFinalizedMaxVersion: 2\tNewFinalizedMaxVersion: -\tResult: OK\n" - assertEquals(expected, output) - } finally { - featureApis.close() + FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), "upgrade", + "--feature", "feature_1", "--version", "3", "--feature", "feature_2", "--version", "5")) + val upgradeDescribeOutput = TestUtils.grabConsoleOutput(FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), "describe"))) + val expectedUpgradeDescribeOutput = Seq( + "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedVersionLevel: 3", + "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedVersionLevel: 5" + ) + expectedUpgradeDescribeOutput.foreach { expectedOutput => + assertTrue(upgradeDescribeOutput.contains(expectedOutput)) } - } - - /** - * Tests if the FeatureApis#upgradeAllFeatures API works as expected during a partial failure case. - */ - @Test - def testUpgradeFeaturesFailure(): Unit = { - val upgradeOpts = new FeatureCommandOptions(Array("--bootstrap-server", bootstrapServers(), "--upgrade-all")) - val featureApis = new FeatureApis(upgradeOpts) - try { - // Step (1): Update the supported features across all brokers. - updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures) - // Step (2): - // - Intentionally setup the FeatureApis object such that it contains incompatible target - // features (viz. feature_2 and feature_3). - // - Upgrade non-existing feature_1 to maxVersionLevel: 4. Expect the operation to fail with - // an incompatibility failure. - // - Upgrade non-existing feature_2 to maxVersionLevel: 5. Expect the operation to succeed. - // - Upgrade non-existing feature_3 to maxVersionLevel: 3. Expect the operation to fail - // since the feature is not supported. - val targetFeaturesWithIncompatibilities = - Features.supportedFeatures( - Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 4)), - Utils.mkEntry("feature_2", new SupportedVersionRange(1, 5)), - Utils.mkEntry("feature_3", new SupportedVersionRange(1, 3)))) - featureApis.setSupportedFeatures(targetFeaturesWithIncompatibilities) - val output = TestUtils.grabConsoleOutput({ - val exception = assertThrows(classOf[UpdateFeaturesException], () => featureApis.upgradeAllFeatures()) - assertEquals("2 feature updates failed!", exception.getMessage) - }) - val expected = - " [Add]\tFeature: feature_1\tExistingFinalizedMaxVersion: -" + - "\tNewFinalizedMaxVersion: 4\tResult: FAILED due to" + - " org.apache.kafka.common.errors.InvalidRequestException: Could not apply finalized" + - " feature update because brokers were found to have incompatible versions for the" + - " feature.\n" + - " [Add]\tFeature: feature_2\tExistingFinalizedMaxVersion: -" + - "\tNewFinalizedMaxVersion: 5\tResult: OK\n" + - " [Add]\tFeature: feature_3\tExistingFinalizedMaxVersion: -" + - "\tNewFinalizedMaxVersion: 3\tResult: FAILED due to" + - " org.apache.kafka.common.errors.InvalidRequestException: Could not apply finalized" + - " feature update because the provided feature is not supported.\n" - assertEquals(expected, output) - } finally { - featureApis.close() + FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), "downgrade", + "--feature", "feature_1", "--version", "2", "--feature", "feature_2", "--version", "2")) + val downgradeDescribeOutput = TestUtils.grabConsoleOutput(FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), "describe"))) + val expectedFinalDescribeOutput = Seq( + "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedVersionLevel: 2", + "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedVersionLevel: 2" + ) + expectedFinalDescribeOutput.foreach { expectedOutput => + assertTrue(downgradeDescribeOutput.contains(expectedOutput)) } } } diff --git a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala index c4cc52c27c9c..10d69e2cd614 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala @@ -19,7 +19,7 @@ package kafka.server import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} -import org.junit.jupiter.api.Test +import org.junit.jupiter.api.{Disabled, Test} import scala.jdk.CollectionConverters._ @@ -89,6 +89,7 @@ class BrokerFeaturesTest { } @Test + @Disabled("Need to remove or rewrite this test after we fully remove FinalizedVersionRange") def testDefaultFinalizedFeatures(): Unit = { val brokerFeatures = BrokerFeatures.createDefault() val supportedFeatures = Features.supportedFeatures(Map[String, SupportedVersionRange]( diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 65580dc2be7c..5fcf763d7f20 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -101,7 +101,6 @@ class ControllerApisTest { authorizer, quotas, time, - Map.empty, controller, raftManager, new KafkaConfig(props), diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 43ba5bae4853..0b38a02a1734 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -763,7 +763,7 @@ class RequestQuotaTest extends BaseRequestTest { object RequestQuotaTest { val ClusterActions = ApiKeys.zkBrokerApis.asScala.filter(_.clusterAction).toSet - val ClusterActionsWithThrottle = Set(ApiKeys.ALLOCATE_PRODUCER_IDS) + val ClusterActionsWithThrottle = Set(ApiKeys.ALLOCATE_PRODUCER_IDS, ApiKeys.UPDATE_FEATURES) val SaslActions = Set(ApiKeys.SASL_HANDSHAKE, ApiKeys.SASL_AUTHENTICATE) val ClientActions = ApiKeys.zkBrokerApis.asScala.toSet -- ClusterActions -- SaslActions diff --git a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala index 92ba0425dcb2..a7085850b265 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -213,8 +213,8 @@ class UpdateFeaturesTest extends BaseRequestTest { val targetMaxVersionLevel = (defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short] testWithInvalidFeatureUpdate[InvalidRequestException]( "feature_1", - new FeatureUpdate(targetMaxVersionLevel,false), - ".*Can not downgrade finalized feature.*allowDowngrade.*".r) + new FeatureUpdate(targetMaxVersionLevel, FeatureUpdate.UpgradeType.UPGRADE), + ".*Can not downgrade finalized feature.*".r) } /** @@ -226,8 +226,8 @@ class UpdateFeaturesTest extends BaseRequestTest { val targetMaxVersionLevel = (defaultFinalizedFeatures().get("feature_1").max() + 1).asInstanceOf[Short] testWithInvalidFeatureUpdate[InvalidRequestException]( "feature_1", - new FeatureUpdate(targetMaxVersionLevel, true), - ".*When the allowDowngrade flag set in the request, the provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r) + new FeatureUpdate(targetMaxVersionLevel, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), + ".*When the downgradeType is set to SAFE set in the request, the provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r) } /** @@ -264,7 +264,7 @@ class UpdateFeaturesTest extends BaseRequestTest { assertEquals(Errors.INVALID_REQUEST, Errors.forCode(result.errorCode)) assertNotNull(result.errorMessage) assertFalse(result.errorMessage.isEmpty) - val exceptionMsgPattern = ".*Can not provide maxVersionLevel: 0 less than 1.*allowDowngrade.*".r + val exceptionMsgPattern = ".*Can not provide maxVersionLevel: 0 less than 1.*".r assertTrue(exceptionMsgPattern.findFirstIn(result.errorMessage).isDefined, result.errorMessage) checkFeatures( adminClient, @@ -282,7 +282,7 @@ class UpdateFeaturesTest extends BaseRequestTest { def testShouldFailRequestDuringDeletionOfNonExistingFeature(): Unit = { testWithInvalidFeatureUpdate[InvalidRequestException]( "feature_non_existing", - new FeatureUpdate(3, true), + new FeatureUpdate(3.toShort, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), ".*Could not apply finalized feature update because the provided feature is not supported.*".r) } @@ -295,7 +295,7 @@ class UpdateFeaturesTest extends BaseRequestTest { val targetMaxVersionLevel = defaultFinalizedFeatures().get("feature_1").max() testWithInvalidFeatureUpdate[InvalidRequestException]( "feature_1", - new FeatureUpdate(targetMaxVersionLevel, false), + new FeatureUpdate(targetMaxVersionLevel, FeatureUpdate.UpgradeType.UPGRADE), ".*Can not upgrade a finalized feature.*to the same value.*".r) } @@ -331,7 +331,7 @@ class UpdateFeaturesTest extends BaseRequestTest { ).getOrElse(Features.emptyFinalizedFeatures()) val versionBefore = updateFeatureZNode(initialFinalizedFeatures) - val invalidUpdate = new FeatureUpdate(supportedVersionRange.max(), false) + val invalidUpdate = new FeatureUpdate(supportedVersionRange.max(), FeatureUpdate.UpgradeType.UPGRADE) val nodeBefore = getFeatureZNode() val adminClient = createAdminClient() val result = adminClient.updateFeatures( @@ -393,10 +393,10 @@ class UpdateFeaturesTest extends BaseRequestTest { val targetFinalizedFeatures = Features.finalizedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)), - Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3)))) - val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) - val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), false) + Utils.mkEntry("feature_1", new FinalizedVersionRange(3, 3)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(3, 3)))) + val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), FeatureUpdate.UpgradeType.UPGRADE) + val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), FeatureUpdate.UpgradeType.UPGRADE) val adminClient = createAdminClient() adminClient.updateFeatures( @@ -427,8 +427,8 @@ class UpdateFeaturesTest extends BaseRequestTest { updateSupportedFeaturesInAllBrokers(supportedFeatures) val initialFinalizedFeatures = Features.finalizedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2)), - Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 4)))) + Utils.mkEntry("feature_1", new FinalizedVersionRange(2, 2)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(4, 4)))) val versionBefore = updateFeatureZNode(initialFinalizedFeatures) // Below we aim to do the following: @@ -436,10 +436,10 @@ class UpdateFeaturesTest extends BaseRequestTest { // - Valid downgrade of feature_2 maxVersionLevel from 4 to 3 val targetFinalizedFeatures = Features.finalizedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)), - Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3)))) - val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) - val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), true) + Utils.mkEntry("feature_1", new FinalizedVersionRange(3, 3)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(3, 3)))) + val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), FeatureUpdate.UpgradeType.UPGRADE) + val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), FeatureUpdate.UpgradeType.SAFE_DOWNGRADE) val adminClient = createAdminClient() adminClient.updateFeatures( @@ -471,8 +471,8 @@ class UpdateFeaturesTest extends BaseRequestTest { updateSupportedFeaturesInAllBrokers(supportedFeatures) val initialFinalizedFeatures = Features.finalizedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2)), - Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 4)))) + Utils.mkEntry("feature_1", new FinalizedVersionRange(2, 2)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(4, 4)))) val versionBefore = updateFeatureZNode(initialFinalizedFeatures) // Below we aim to do the following: @@ -481,10 +481,10 @@ class UpdateFeaturesTest extends BaseRequestTest { // (because we intentionally do not set the allowDowngrade flag) val targetFinalizedFeatures = Features.finalizedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)), - Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3)))) - val validUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) - val invalidUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), false) + Utils.mkEntry("feature_1", new FinalizedVersionRange(3, 3)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(3, 3)))) + val validUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), FeatureUpdate.UpgradeType.UPGRADE) + val invalidUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), FeatureUpdate.UpgradeType.UPGRADE) val adminClient = createAdminClient() val result = adminClient.updateFeatures( @@ -495,7 +495,7 @@ class UpdateFeaturesTest extends BaseRequestTest { result.values().get("feature_1").get() // Expect update for "feature_2" to have failed. checkException[InvalidRequestException]( - result, Map("feature_2" -> ".*Can not downgrade finalized feature.*allowDowngrade.*".r)) + result, Map("feature_2" -> ".*Can not downgrade finalized feature.*".r)) val expectedFeatures = Features.finalizedFeatures( Utils.mkMap( Utils.mkEntry("feature_1", targetFinalizedFeatures.get("feature_1")), @@ -539,8 +539,8 @@ class UpdateFeaturesTest extends BaseRequestTest { val initialFinalizedFeatures = Features.finalizedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2)), - Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 4)))) + Utils.mkEntry("feature_1", new FinalizedVersionRange(2, 2)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(4, 4)))) val versionBefore = updateFeatureZNode(initialFinalizedFeatures) // Below we aim to do the following: @@ -549,10 +549,10 @@ class UpdateFeaturesTest extends BaseRequestTest { // - Valid downgrade of feature_2 maxVersionLevel from 4 to 3 val targetFinalizedFeatures = Features.finalizedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)), - Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3)))) - val invalidUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) - val validUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), true) + Utils.mkEntry("feature_1", new FinalizedVersionRange(3, 3)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(3, 3)))) + val invalidUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), FeatureUpdate.UpgradeType.UPGRADE) + val validUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), FeatureUpdate.UpgradeType.SAFE_DOWNGRADE) val adminClient = createAdminClient() val result = adminClient.updateFeatures( diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index fb3844f23bd5..29b41c797b8d 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -36,7 +36,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.metadata.BrokerRegistration; import org.apache.kafka.metadata.BrokerRegistrationReply; -import org.apache.kafka.metadata.FeatureMapAndEpoch; +import org.apache.kafka.metadata.FinalizedControllerFeatures; import org.apache.kafka.metadata.VersionRange; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.timeline.SnapshotRegistry; @@ -178,6 +178,13 @@ Map brokerRegistrations() { return brokerRegistrations; } + Map> brokerSupportedVersions() { + return brokerRegistrations() + .entrySet() + .stream() + .collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().supportedFeatures())); + } + Set fencedBrokerIds() { return brokerRegistrations.values() .stream() @@ -192,7 +199,7 @@ Set fencedBrokerIds() { public ControllerResult registerBroker( BrokerRegistrationRequestData request, long brokerEpoch, - FeatureMapAndEpoch finalizedFeatures) { + FinalizedControllerFeatures finalizedFeatures) { if (heartbeatManager == null) { throw new RuntimeException("ClusterControlManager is not active."); } @@ -229,13 +236,14 @@ public ControllerResult registerBroker( setSecurityProtocol(listener.securityProtocol())); } for (BrokerRegistrationRequestData.Feature feature : request.features()) { - Optional finalized = finalizedFeatures.map().get(feature.name()); + Optional finalized = finalizedFeatures.get(feature.name()); if (finalized.isPresent()) { - if (!finalized.get().contains(new VersionRange(feature.minSupportedVersion(), - feature.maxSupportedVersion()))) { + if (!VersionRange.of(feature.minSupportedVersion(), feature.maxSupportedVersion()).contains(finalized.get())) { throw new UnsupportedVersionException("Unable to register because " + - "the broker has an unsupported version of " + feature.name()); + "the broker has an unsupported version of " + feature.name()); } + } else { + log.warn("Broker registered with feature {} that is unknown to the controller", feature.name()); } record.features().add(new BrokerFeature(). setName(feature.name()). @@ -265,10 +273,10 @@ public void replay(RegisterBrokerRecord record) { } Map features = new HashMap<>(); for (BrokerFeature feature : record.features()) { - features.put(feature.name(), new VersionRange( + features.put(feature.name(), VersionRange.of( feature.minSupportedVersion(), feature.maxSupportedVersion())); } - + // Update broker registrations. BrokerRegistration prevRegistration = brokerRegistrations.put(brokerId, new BrokerRegistration(brokerId, record.brokerEpoch(), diff --git a/metadata/src/main/java/org/apache/kafka/controller/Controller.java b/metadata/src/main/java/org/apache/kafka/controller/Controller.java index c5fdefffbf83..6f9976542420 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/Controller.java +++ b/metadata/src/main/java/org/apache/kafka/controller/Controller.java @@ -36,12 +36,14 @@ import org.apache.kafka.common.message.ElectLeadersResponseData; import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData; import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; +import org.apache.kafka.common.message.UpdateFeaturesRequestData; +import org.apache.kafka.common.message.UpdateFeaturesResponseData; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaEntity; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistrationReply; -import org.apache.kafka.metadata.FeatureMapAndEpoch; +import org.apache.kafka.metadata.FinalizedControllerFeatures; import org.apache.kafka.metadata.authorizer.AclMutator; import java.util.Collection; @@ -152,7 +154,7 @@ CompletableFuture> deleteTopics(long deadlineNs, * * @return A future yielding the feature ranges. */ - CompletableFuture finalizedFeatures(); + CompletableFuture finalizedFeatures(); /** * Perform some incremental configuration changes. @@ -247,6 +249,15 @@ CompletableFuture allocateProducerIds( AllocateProducerIdsRequestData request ); + /** + * Update a set of feature flags + * @param request The update features request + * @return A future which yields the result of the action + */ + CompletableFuture updateFeatures( + UpdateFeaturesRequestData request + ); + /** * Begin writing a controller snapshot. If there was already an ongoing snapshot, it * simply returns information about that snapshot rather than starting a new one. diff --git a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java index ed7c98cbb6d2..307a0ce09df8 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java @@ -25,109 +25,140 @@ import java.util.Map.Entry; import java.util.Map; import java.util.NoSuchElementException; -import java.util.Set; import java.util.TreeMap; +import org.apache.kafka.clients.admin.FeatureUpdate; import org.apache.kafka.common.metadata.FeatureLevelRecord; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.metadata.FinalizedControllerFeatures; import org.apache.kafka.server.common.ApiMessageAndVersion; -import org.apache.kafka.metadata.FeatureMap; -import org.apache.kafka.metadata.FeatureMapAndEpoch; import org.apache.kafka.metadata.VersionRange; import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.TimelineHashMap; +import org.slf4j.Logger; import static org.apache.kafka.common.metadata.MetadataRecordType.FEATURE_LEVEL_RECORD; public class FeatureControlManager { + private final Logger log; + /** * An immutable map containing the features supported by this controller's software. */ - private final Map supportedFeatures; + private final QuorumFeatures quorumFeatures; /** * Maps feature names to finalized version ranges. */ - private final TimelineHashMap finalizedVersions; + private final TimelineHashMap finalizedVersions; + - FeatureControlManager(Map supportedFeatures, + FeatureControlManager(LogContext logContext, + QuorumFeatures quorumFeatures, SnapshotRegistry snapshotRegistry) { - this.supportedFeatures = supportedFeatures; + this.log = logContext.logger(FeatureControlManager.class); + this.quorumFeatures = quorumFeatures; this.finalizedVersions = new TimelineHashMap<>(snapshotRegistry, 0); } ControllerResult> updateFeatures( - Map updates, Set downgradeables, - Map> brokerFeatures) { + Map updates, + Map upgradeTypes, + Map> brokerFeatures, + boolean validateOnly) { TreeMap results = new TreeMap<>(); List records = new ArrayList<>(); - for (Entry entry : updates.entrySet()) { + for (Entry entry : updates.entrySet()) { results.put(entry.getKey(), updateFeature(entry.getKey(), entry.getValue(), - downgradeables.contains(entry.getKey()), brokerFeatures, records)); + upgradeTypes.getOrDefault(entry.getKey(), FeatureUpdate.UpgradeType.UPGRADE), brokerFeatures, records)); + } + + if (validateOnly) { + return ControllerResult.of(Collections.emptyList(), results); + } else { + return ControllerResult.atomicOf(records, results); } + } + + boolean canSupportVersion(String featureName, short versionRange) { + return quorumFeatures.localSupportedFeature(featureName) + .filter(localRange -> localRange.contains(versionRange)) + .isPresent(); + } - return ControllerResult.atomicOf(records, results); + boolean featureExists(String featureName) { + return quorumFeatures.localSupportedFeature(featureName).isPresent(); } private ApiError updateFeature(String featureName, - VersionRange newRange, - boolean downgradeable, - Map> brokerFeatures, + short newVersion, + FeatureUpdate.UpgradeType upgradeType, + Map> brokersAndFeatures, List records) { - if (newRange.min() <= 0) { + if (!featureExists(featureName)) { + return new ApiError(Errors.INVALID_UPDATE_VERSION, + "The controller does not support the given feature."); + } + + if (upgradeType.equals(FeatureUpdate.UpgradeType.UNKNOWN)) { return new ApiError(Errors.INVALID_UPDATE_VERSION, - "The lower value for the new range cannot be less than 1."); + "The controller does not support the given upgrade type."); } - if (newRange.max() <= 0) { + + final Short currentVersion = finalizedVersions.get(featureName); + + if (newVersion <= 0) { return new ApiError(Errors.INVALID_UPDATE_VERSION, "The upper value for the new range cannot be less than 1."); } - VersionRange localRange = supportedFeatures.get(featureName); - if (localRange == null || !localRange.contains(newRange)) { + + if (!canSupportVersion(featureName, newVersion)) { return new ApiError(Errors.INVALID_UPDATE_VERSION, "The controller does not support the given feature range."); } - for (Entry> brokerEntry : - brokerFeatures.entrySet()) { + + for (Entry> brokerEntry : brokersAndFeatures.entrySet()) { VersionRange brokerRange = brokerEntry.getValue().get(featureName); - if (brokerRange == null || !brokerRange.contains(newRange)) { + if (brokerRange == null || !brokerRange.contains(newVersion)) { return new ApiError(Errors.INVALID_UPDATE_VERSION, "Broker " + brokerEntry.getKey() + " does not support the given " + "feature range."); } } - VersionRange currentRange = finalizedVersions.get(featureName); - if (currentRange != null && currentRange.max() > newRange.max()) { - if (!downgradeable) { + + if (currentVersion != null && newVersion < currentVersion) { + if (upgradeType.equals(FeatureUpdate.UpgradeType.UPGRADE)) { return new ApiError(Errors.INVALID_UPDATE_VERSION, - "Can't downgrade the maximum version of this feature without " + - "setting downgradable to true."); + "Can't downgrade the maximum version of this feature without setting the upgrade type to safe or unsafe downgrade."); } } + records.add(new ApiMessageAndVersion( - new FeatureLevelRecord().setName(featureName). - setMinFeatureLevel(newRange.min()).setMaxFeatureLevel(newRange.max()), + new FeatureLevelRecord() + .setName(featureName) + .setFeatureLevel(newVersion), FEATURE_LEVEL_RECORD.highestSupportedVersion())); return ApiError.NONE; } - FeatureMapAndEpoch finalizedFeatures(long lastCommittedOffset) { - Map features = new HashMap<>(); - for (Entry entry : finalizedVersions.entrySet(lastCommittedOffset)) { + FinalizedControllerFeatures finalizedFeatures(long lastCommittedOffset) { + Map features = new HashMap<>(); + for (Entry entry : finalizedVersions.entrySet(lastCommittedOffset)) { features.put(entry.getKey(), entry.getValue()); } - return new FeatureMapAndEpoch(new FeatureMap(features), lastCommittedOffset); + return new FinalizedControllerFeatures(features, lastCommittedOffset); } public void replay(FeatureLevelRecord record) { - finalizedVersions.put(record.name(), - new VersionRange(record.minFeatureLevel(), record.maxFeatureLevel())); + log.info("Setting feature {} to {}", record.name(), record.featureLevel()); + finalizedVersions.put(record.name(), record.featureLevel()); } class FeatureControlIterator implements Iterator> { - private final Iterator> iterator; + private final Iterator> iterator; FeatureControlIterator(long epoch) { this.iterator = finalizedVersions.entrySet(epoch).iterator(); @@ -141,12 +172,10 @@ public boolean hasNext() { @Override public List next() { if (!hasNext()) throw new NoSuchElementException(); - Entry entry = iterator.next(); - VersionRange versions = entry.getValue(); - return Collections.singletonList(new ApiMessageAndVersion(new FeatureLevelRecord(). - setName(entry.getKey()). - setMinFeatureLevel(versions.min()). - setMaxFeatureLevel(versions.max()), FEATURE_LEVEL_RECORD.highestSupportedVersion())); + Entry entry = iterator.next(); + return Collections.singletonList(new ApiMessageAndVersion(new FeatureLevelRecord() + .setName(entry.getKey()) + .setFeatureLevel(entry.getValue()), FEATURE_LEVEL_RECORD.highestSupportedVersion())); } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java index d6491e2c1405..178ef46bdb55 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java @@ -23,6 +23,7 @@ import org.apache.kafka.server.common.ProducerIdsBlock; import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.TimelineLong; +import org.apache.kafka.timeline.TimelineObject; import java.util.ArrayList; import java.util.Collections; @@ -33,17 +34,19 @@ public class ProducerIdControlManager { private final ClusterControlManager clusterControlManager; - private final TimelineLong nextProducerId; // Initializes to 0 + private final TimelineObject nextProducerBlock; + private final TimelineLong brokerEpoch; ProducerIdControlManager(ClusterControlManager clusterControlManager, SnapshotRegistry snapshotRegistry) { this.clusterControlManager = clusterControlManager; - this.nextProducerId = new TimelineLong(snapshotRegistry); + this.nextProducerBlock = new TimelineObject<>(snapshotRegistry, ProducerIdsBlock.EMPTY); + this.brokerEpoch = new TimelineLong(snapshotRegistry); } ControllerResult generateNextProducerId(int brokerId, long brokerEpoch) { clusterControlManager.checkBrokerEpoch(brokerId, brokerEpoch); - long firstProducerIdInBlock = nextProducerId.get(); + long firstProducerIdInBlock = nextProducerBlock.get().firstProducerId(); if (firstProducerIdInBlock > Long.MAX_VALUE - ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) { throw new UnknownServerException("Exhausted all producerIds as the next block's end producerId " + "has exceeded the int64 type limit"); @@ -60,25 +63,26 @@ ControllerResult generateNextProducerId(int brokerId, long bro } void replay(ProducerIdsRecord record) { - long currentNextProducerId = nextProducerId.get(); + long currentNextProducerId = nextProducerBlock.get().firstProducerId(); if (record.nextProducerId() <= currentNextProducerId) { throw new RuntimeException("Next Producer ID from replayed record (" + record.nextProducerId() + ")" + " is not greater than current next Producer ID (" + currentNextProducerId + ")"); } else { - nextProducerId.set(record.nextProducerId()); + nextProducerBlock.set(new ProducerIdsBlock(record.brokerId(), record.nextProducerId(), ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE)); + brokerEpoch.set(record.brokerEpoch()); } } Iterator> iterator(long epoch) { List records = new ArrayList<>(1); - long producerId = nextProducerId.get(epoch); - if (producerId > 0) { + ProducerIdsBlock producerIdBlock = nextProducerBlock.get(epoch); + if (producerIdBlock.firstProducerId() > 0) { records.add(new ApiMessageAndVersion( new ProducerIdsRecord() - .setNextProducerId(producerId) - .setBrokerId(0) - .setBrokerEpoch(0L), + .setNextProducerId(producerIdBlock.firstProducerId()) + .setBrokerId(producerIdBlock.assignedBrokerId()) + .setBrokerEpoch(brokerEpoch.get(epoch)), (short) 0)); } return Collections.singleton(records).iterator(); 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 7d3c55bd6905..e42799fd1f69 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -18,6 +18,7 @@ package org.apache.kafka.controller; import org.apache.kafka.clients.admin.AlterConfigOp.OpType; +import org.apache.kafka.clients.admin.FeatureUpdate; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.acl.AclBinding; import org.apache.kafka.common.acl.AclBindingFilter; @@ -44,6 +45,8 @@ import org.apache.kafka.common.message.ElectLeadersResponseData; import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData; import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; +import org.apache.kafka.common.message.UpdateFeaturesRequestData; +import org.apache.kafka.common.message.UpdateFeaturesResponseData; import org.apache.kafka.common.metadata.AccessControlEntryRecord; import org.apache.kafka.common.metadata.ConfigRecord; import org.apache.kafka.common.metadata.ClientQuotaRecord; @@ -74,8 +77,7 @@ import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistrationReply; -import org.apache.kafka.metadata.FeatureMapAndEpoch; -import org.apache.kafka.metadata.VersionRange; +import org.apache.kafka.metadata.FinalizedControllerFeatures; import org.apache.kafka.queue.EventQueue; import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction; import org.apache.kafka.queue.KafkaEventQueue; @@ -94,6 +96,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map.Entry; import java.util.Map; @@ -143,7 +146,7 @@ static public class Builder { private LogContext logContext = null; private KafkaConfigSchema configSchema = KafkaConfigSchema.EMPTY; private RaftClient raftClient = null; - private Map supportedFeatures = Collections.emptyMap(); + private QuorumFeatures quorumFeatures = null; private short defaultReplicationFactor = 3; private int defaultNumPartitions = 1; private boolean isLeaderRecoverySupported = false; @@ -188,8 +191,8 @@ public Builder setRaftClient(RaftClient logManager) { return this; } - public Builder setSupportedFeatures(Map supportedFeatures) { - this.supportedFeatures = supportedFeatures; + public Builder setQuorumFeatures(QuorumFeatures quorumFeatures) { + this.quorumFeatures = quorumFeatures; return this; } @@ -263,6 +266,9 @@ public QuorumController build() throws Exception { if (raftClient == null) { throw new RuntimeException("You must set a raft client."); } + if (quorumFeatures == null) { + throw new RuntimeException("You must specify the quorum features"); + } if (threadNamePrefix == null) { threadNamePrefix = String.format("Node%d_", nodeId); } @@ -273,11 +279,12 @@ public QuorumController build() throws Exception { controllerMetrics = (ControllerMetrics) Class.forName( "org.apache.kafka.controller.MockControllerMetrics").getConstructor().newInstance(); } + KafkaEventQueue queue = null; try { queue = new KafkaEventQueue(time, logContext, threadNamePrefix + "QuorumController"); return new QuorumController(logContext, nodeId, clusterId, queue, time, - configSchema, raftClient, supportedFeatures, defaultReplicationFactor, + configSchema, raftClient, quorumFeatures, defaultReplicationFactor, defaultNumPartitions, isLeaderRecoverySupported, replicaPlacer, snapshotMaxNewRecordBytes, leaderImbalanceCheckIntervalNs, sessionTimeoutNs, controllerMetrics, createTopicPolicy, alterConfigPolicy, configurationValidator, authorizer, @@ -1312,7 +1319,7 @@ private QuorumController(LogContext logContext, Time time, KafkaConfigSchema configSchema, RaftClient raftClient, - Map supportedFeatures, + QuorumFeatures quorumFeatures, short defaultReplicationFactor, int defaultNumPartitions, boolean isLeaderRecoverySupported, @@ -1349,7 +1356,7 @@ private QuorumController(LogContext logContext, this.clientQuotaControlManager = new ClientQuotaControlManager(snapshotRegistry); this.clusterControl = new ClusterControlManager(logContext, clusterId, time, snapshotRegistry, sessionTimeoutNs, replicaPlacer, controllerMetrics); - this.featureControl = new FeatureControlManager(supportedFeatures, snapshotRegistry); + this.featureControl = new FeatureControlManager(logContext, quorumFeatures, snapshotRegistry); this.producerIdControlManager = new ProducerIdControlManager(clusterControl, snapshotRegistry); this.snapshotMaxNewRecordBytes = snapshotMaxNewRecordBytes; this.leaderImbalanceCheckIntervalNs = leaderImbalanceCheckIntervalNs; @@ -1446,7 +1453,7 @@ public CompletableFuture> deleteTopics(long deadlineNs, } @Override - public CompletableFuture finalizedFeatures() { + public CompletableFuture finalizedFeatures() { return appendReadEvent("getFinalizedFeatures", () -> featureControl.finalizedFeatures(lastCommittedOffset)); } @@ -1575,6 +1582,31 @@ public CompletableFuture allocateProducerIds( .setProducerIdLen(result.size())); } + @Override + public CompletableFuture updateFeatures( + UpdateFeaturesRequestData request) { + return appendWriteEvent("updateFeatures", () -> { + Map updates = new HashMap<>(); + Map upgradeTypes = new HashMap<>(); + request.featureUpdates().forEach(featureUpdate -> { + String featureName = featureUpdate.feature(); + upgradeTypes.put(featureName, FeatureUpdate.UpgradeType.fromCode(featureUpdate.upgradeType())); + updates.put(featureName, featureUpdate.maxVersionLevel()); + }); + return featureControl.updateFeatures(updates, upgradeTypes, clusterControl.brokerSupportedVersions(), + request.validateOnly()); + }).thenApply(result -> { + UpdateFeaturesResponseData responseData = new UpdateFeaturesResponseData(); + responseData.setResults(new UpdateFeaturesResponseData.UpdatableFeatureResultCollection(result.size())); + result.forEach((featureName, error) -> responseData.results().add( + new UpdateFeaturesResponseData.UpdatableFeatureResult() + .setFeature(featureName) + .setErrorCode(error.error().code()) + .setErrorMessage(error.message()))); + return responseData; + }); + } + @Override public CompletableFuture> createPartitions(long deadlineNs, List topics) { diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java new file mode 100644 index 000000000000..0ee27bd4f382 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java @@ -0,0 +1,51 @@ +/* + * 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.controller; + +import org.apache.kafka.metadata.VersionRange; + +import java.util.Collections; +import java.util.Map; +import java.util.Optional; + +/** + * A holder class of the local node's supported feature flags. + */ +public class QuorumFeatures { + private final int nodeId; + private final Map supportedFeatures; + + QuorumFeatures(int nodeId, + Map supportedFeatures) { + this.nodeId = nodeId; + this.supportedFeatures = Collections.unmodifiableMap(supportedFeatures); + } + + public static QuorumFeatures create(int nodeId, + Map supportedFeatures) { + return new QuorumFeatures(nodeId, supportedFeatures); + } + + public static Map defaultFeatureMap() { + return Collections.emptyMap(); + } + + Optional localSupportedFeature(String featureName) { + return Optional.ofNullable(supportedFeatures.get(featureName)); + } +} diff --git a/metadata/src/main/java/org/apache/kafka/image/FeaturesDelta.java b/metadata/src/main/java/org/apache/kafka/image/FeaturesDelta.java index 781c496f19b6..ca472322d664 100644 --- a/metadata/src/main/java/org/apache/kafka/image/FeaturesDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/FeaturesDelta.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.metadata.FeatureLevelRecord; import org.apache.kafka.common.metadata.RemoveFeatureLevelRecord; -import org.apache.kafka.metadata.VersionRange; import java.util.HashMap; import java.util.Map; @@ -33,13 +32,13 @@ public final class FeaturesDelta { private final FeaturesImage image; - private final Map> changes = new HashMap<>(); + private final Map> changes = new HashMap<>(); public FeaturesDelta(FeaturesImage image) { this.image = image; } - public Map> changes() { + public Map> changes() { return changes; } @@ -52,8 +51,7 @@ public void finishSnapshot() { } public void replay(FeatureLevelRecord record) { - changes.put(record.name(), Optional.of( - new VersionRange(record.minFeatureLevel(), record.maxFeatureLevel()))); + changes.put(record.name(), Optional.of(record.featureLevel())); } public void replay(RemoveFeatureLevelRecord record) { @@ -61,26 +59,27 @@ public void replay(RemoveFeatureLevelRecord record) { } public FeaturesImage apply() { - Map newFinalizedVersions = + Map newFinalizedVersions = new HashMap<>(image.finalizedVersions().size()); - for (Entry entry : image.finalizedVersions().entrySet()) { + for (Entry entry : image.finalizedVersions().entrySet()) { String name = entry.getKey(); - Optional change = changes.get(name); + Optional change = changes.get(name); if (change == null) { newFinalizedVersions.put(name, entry.getValue()); } else if (change.isPresent()) { newFinalizedVersions.put(name, change.get()); } } - for (Entry> entry : changes.entrySet()) { + for (Entry> entry : changes.entrySet()) { String name = entry.getKey(); - Optional change = entry.getValue(); + Optional change = entry.getValue(); if (!newFinalizedVersions.containsKey(name)) { if (change.isPresent()) { newFinalizedVersions.put(name, change.get()); } } } + return new FeaturesImage(newFinalizedVersions); } diff --git a/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java b/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java index f5f372936a1b..7e0f7fb4357c 100644 --- a/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java @@ -18,7 +18,6 @@ package org.apache.kafka.image; import org.apache.kafka.common.metadata.FeatureLevelRecord; -import org.apache.kafka.metadata.VersionRange; import org.apache.kafka.server.common.ApiMessageAndVersion; import java.util.ArrayList; @@ -28,7 +27,6 @@ import java.util.Map.Entry; import java.util.Optional; import java.util.function.Consumer; -import java.util.stream.Collectors; import static org.apache.kafka.common.metadata.MetadataRecordType.FEATURE_LEVEL_RECORD; @@ -41,9 +39,9 @@ public final class FeaturesImage { public static final FeaturesImage EMPTY = new FeaturesImage(Collections.emptyMap()); - private final Map finalizedVersions; + private final Map finalizedVersions; - public FeaturesImage(Map finalizedVersions) { + public FeaturesImage(Map finalizedVersions) { this.finalizedVersions = Collections.unmodifiableMap(finalizedVersions); } @@ -51,22 +49,20 @@ public boolean isEmpty() { return finalizedVersions.isEmpty(); } - Map finalizedVersions() { + Map finalizedVersions() { return finalizedVersions; } - private Optional finalizedVersion(String feature) { + private Optional finalizedVersion(String feature) { return Optional.ofNullable(finalizedVersions.get(feature)); } public void write(Consumer> out) { List batch = new ArrayList<>(); - for (Entry entry : finalizedVersions.entrySet()) { + for (Entry entry : finalizedVersions.entrySet()) { batch.add(new ApiMessageAndVersion(new FeatureLevelRecord(). setName(entry.getKey()). - setMinFeatureLevel(entry.getValue().min()). - setMaxFeatureLevel(entry.getValue().max()), - FEATURE_LEVEL_RECORD.highestSupportedVersion())); + setFeatureLevel(entry.getValue()), FEATURE_LEVEL_RECORD.highestSupportedVersion())); } out.accept(batch); } @@ -83,9 +79,11 @@ public boolean equals(Object o) { return finalizedVersions.equals(other.finalizedVersions); } + @Override public String toString() { - return finalizedVersions.entrySet().stream(). - map(e -> e.getKey() + ":" + e.getValue()).collect(Collectors.joining(", ")); + return "FeaturesImage{" + + "finalizedVersions=" + finalizedVersions + + '}'; } } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java b/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java index fd5eb65d1a2b..cc8ed9b4aaeb 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java @@ -104,7 +104,7 @@ public static BrokerRegistration fromRecord(RegisterBrokerRecord record) { } Map supportedFeatures = new HashMap<>(); for (BrokerFeature feature : record.features()) { - supportedFeatures.put(feature.name(), new VersionRange( + supportedFeatures.put(feature.name(), VersionRange.of( feature.minSupportedVersion(), feature.maxSupportedVersion())); } return new BrokerRegistration(record.brokerId(), diff --git a/metadata/src/main/java/org/apache/kafka/metadata/FeatureMap.java b/metadata/src/main/java/org/apache/kafka/metadata/FeatureMap.java deleted file mode 100644 index 272c87d21383..000000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/FeatureMap.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.metadata; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; - - -/** - * A map of feature names to their supported versions. - */ -public class FeatureMap { - private final Map features; - - public FeatureMap(Map features) { - this.features = Collections.unmodifiableMap(new HashMap<>(features)); - } - - public Optional get(String name) { - return Optional.ofNullable(features.get(name)); - } - - public Map features() { - return features; - } - - @Override - public int hashCode() { - return features.hashCode(); - } - - @Override - public boolean equals(Object o) { - if (!(o instanceof FeatureMap)) return false; - FeatureMap other = (FeatureMap) o; - return features.equals(other.features); - } - - @Override - public String toString() { - StringBuilder bld = new StringBuilder(); - bld.append("{"); - bld.append(features.keySet().stream().sorted(). - map(k -> k + ": " + features.get(k)). - collect(Collectors.joining(", "))); - bld.append("}"); - return bld.toString(); - } -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/FeatureMapAndEpoch.java b/metadata/src/main/java/org/apache/kafka/metadata/FinalizedControllerFeatures.java similarity index 61% rename from metadata/src/main/java/org/apache/kafka/metadata/FeatureMapAndEpoch.java rename to metadata/src/main/java/org/apache/kafka/metadata/FinalizedControllerFeatures.java index 26096ea7a333..2ebce9e3e6ce 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/FeatureMapAndEpoch.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/FinalizedControllerFeatures.java @@ -17,23 +17,31 @@ package org.apache.kafka.metadata; +import java.util.Collections; +import java.util.Map; import java.util.Objects; +import java.util.Optional; +import java.util.Set; /** * A map of feature names to their supported versions. */ -public class FeatureMapAndEpoch { - private final FeatureMap map; +public class FinalizedControllerFeatures { + private final Map featureMap; private final long epoch; - public FeatureMapAndEpoch(FeatureMap map, long epoch) { - this.map = map; + public FinalizedControllerFeatures(Map featureMap, long epoch) { + this.featureMap = Collections.unmodifiableMap(featureMap); this.epoch = epoch; } - public FeatureMap map() { - return map; + public Optional get(String name) { + return Optional.ofNullable(featureMap.get(name)); + } + + public Set featureNames() { + return featureMap.keySet(); } public long epoch() { @@ -42,21 +50,21 @@ public long epoch() { @Override public int hashCode() { - return Objects.hash(map, epoch); + return Objects.hash(featureMap, epoch); } @Override public boolean equals(Object o) { - if (!(o instanceof FeatureMapAndEpoch)) return false; - FeatureMapAndEpoch other = (FeatureMapAndEpoch) o; - return map.equals(other.map) && epoch == other.epoch; + if (!(o instanceof FinalizedControllerFeatures)) return false; + FinalizedControllerFeatures other = (FinalizedControllerFeatures) o; + return featureMap.equals(other.featureMap) && epoch == other.epoch; } @Override public String toString() { StringBuilder bld = new StringBuilder(); bld.append("{"); - bld.append("map=").append(map.toString()); + bld.append("featureMap=").append(featureMap.toString()); bld.append(", epoch=").append(epoch); bld.append("}"); return bld.toString(); diff --git a/metadata/src/main/java/org/apache/kafka/metadata/VersionRange.java b/metadata/src/main/java/org/apache/kafka/metadata/VersionRange.java index f171ea14bc42..178d338344d0 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/VersionRange.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/VersionRange.java @@ -23,16 +23,20 @@ * An immutable class which represents version ranges. */ public class VersionRange { - public final static VersionRange ALL = new VersionRange((short) 0, Short.MAX_VALUE); + public final static VersionRange ALL = of((short) 0, Short.MAX_VALUE); private final short min; private final short max; - public VersionRange(short min, short max) { + private VersionRange(short min, short max) { this.min = min; this.max = max; } + public static VersionRange of(short min, short max) { + return new VersionRange(min, max); + } + public short min() { return min; } @@ -41,8 +45,18 @@ public short max() { return max; } - public boolean contains(VersionRange other) { - return other.min >= min && other.max <= max; + /** + * Check if a given version is fully contained within this range + */ + public boolean contains(short version) { + return version >= min && version <= max; + } + + /** + * Check if a given version range has overlap with this one + */ + public boolean intersects(VersionRange other) { + return other.min <= max && other.max >= min; } @Override diff --git a/metadata/src/main/resources/common/metadata/FeatureLevelRecord.json b/metadata/src/main/resources/common/metadata/FeatureLevelRecord.json index ac112f15c2fd..03ff347eb82d 100644 --- a/metadata/src/main/resources/common/metadata/FeatureLevelRecord.json +++ b/metadata/src/main/resources/common/metadata/FeatureLevelRecord.json @@ -22,9 +22,7 @@ "fields": [ { "name": "Name", "type": "string", "versions": "0+", "about": "The feature name." }, - { "name": "MinFeatureLevel", "type": "int16", "versions": "0+", - "about": "The current finalized minimum feature level of this feature for the cluster." }, - { "name": "MaxFeatureLevel", "type": "int16", "versions": "0+", - "about": "The current finalized maximum feature level of this feature for the cluster." } + { "name": "FeatureLevel", "type": "int16", "versions": "0+", + "about": "The current finalized feature level of this feature for the cluster." } ] } diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java index 124cb3d5f354..ea223d7de814 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java @@ -37,8 +37,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.metadata.BrokerRegistration; -import org.apache.kafka.metadata.FeatureMap; -import org.apache.kafka.metadata.FeatureMapAndEpoch; +import org.apache.kafka.metadata.FinalizedControllerFeatures; import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.timeline.SnapshotRegistry; @@ -103,7 +102,7 @@ public void testRegistrationWithIncorrectClusterId() throws Exception { setRack(null). setIncarnationId(Uuid.fromString("0H4fUu1xQEKXFYwB1aBjhg")), 123L, - new FeatureMapAndEpoch(new FeatureMap(Collections.emptyMap()), 456L))); + new FinalizedControllerFeatures(Collections.emptyMap(), 456L))); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java index 680253c71254..f53b493a9777 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java @@ -23,12 +23,13 @@ import java.util.HashMap; import java.util.List; import java.util.Map; + +import org.apache.kafka.clients.admin.FeatureUpdate; import org.apache.kafka.common.metadata.FeatureLevelRecord; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.metadata.FeatureMap; -import org.apache.kafka.metadata.FeatureMapAndEpoch; +import org.apache.kafka.metadata.FinalizedControllerFeatures; import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.metadata.VersionRange; import org.apache.kafka.server.common.ApiMessageAndVersion; @@ -48,59 +49,87 @@ private static Map rangeMap(Object... args) { String feature = (String) args[i]; Integer low = (Integer) args[i + 1]; Integer high = (Integer) args[i + 2]; - result.put(feature, new VersionRange(low.shortValue(), high.shortValue())); + result.put(feature, VersionRange.of(low.shortValue(), high.shortValue())); + } + return result; + } + + private static Map versionMap(Object... args) { + Map result = new HashMap<>(); + for (int i = 0; i < args.length; i += 2) { + String feature = (String) args[i]; + Integer ver = (Integer) args[i + 1]; + result.put(feature, ver.shortValue()); + } + return result; + } + + public static QuorumFeatures features(Object... args) { + return QuorumFeatures.create(0, rangeMap(args)); + } + + private static Map updateMap(Object... args) { + Map result = new HashMap<>(); + for (int i = 0; i < args.length; i += 2) { + String feature = (String) args[i]; + Integer ver = (Integer) args[i + 1]; + result.put(feature, ver.shortValue()); } return result; } @Test public void testUpdateFeatures() { - SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); + LogContext logContext = new LogContext(); + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); snapshotRegistry.getOrCreateSnapshot(-1); - FeatureControlManager manager = new FeatureControlManager( - rangeMap("foo", 1, 2), snapshotRegistry); - assertEquals(new FeatureMapAndEpoch(new FeatureMap(Collections.emptyMap()), -1), + FeatureControlManager manager = new FeatureControlManager(logContext, + features("foo", 1, 2), snapshotRegistry); + assertEquals(new FinalizedControllerFeatures(Collections.emptyMap(), -1), manager.finalizedFeatures(-1)); assertEquals(ControllerResult.atomicOf(Collections.emptyList(), Collections. singletonMap("foo", new ApiError(Errors.INVALID_UPDATE_VERSION, "The controller does not support the given feature range."))), - manager.updateFeatures(rangeMap("foo", 1, 3), - Collections.singleton("foo"), - Collections.emptyMap())); + manager.updateFeatures(updateMap("foo", 3), + Collections.singletonMap("foo", FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), + Collections.emptyMap(), false)); ControllerResult> result = manager.updateFeatures( - rangeMap("foo", 1, 2, "bar", 1, 1), Collections.emptySet(), - Collections.emptyMap()); + updateMap("foo", 2, "bar", 1), Collections.emptyMap(), + Collections.emptyMap(), false); Map expectedMap = new HashMap<>(); expectedMap.put("foo", ApiError.NONE); expectedMap.put("bar", new ApiError(Errors.INVALID_UPDATE_VERSION, - "The controller does not support the given feature range.")); + "The controller does not support the given feature.")); assertEquals(expectedMap, result.response()); List expectedMessages = new ArrayList<>(); expectedMessages.add(new ApiMessageAndVersion(new FeatureLevelRecord(). - setName("foo").setMinFeatureLevel((short) 1).setMaxFeatureLevel((short) 2), + setName("foo").setFeatureLevel((short) 2), (short) 0)); assertEquals(expectedMessages, result.records()); } @Test public void testReplay() { + LogContext logContext = new LogContext(); + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); FeatureLevelRecord record = new FeatureLevelRecord(). - setName("foo").setMinFeatureLevel((short) 1).setMaxFeatureLevel((short) 2); - SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); + setName("foo").setFeatureLevel((short) 2); + snapshotRegistry.getOrCreateSnapshot(-1); - FeatureControlManager manager = new FeatureControlManager( - rangeMap("foo", 1, 2), snapshotRegistry); + FeatureControlManager manager = new FeatureControlManager(logContext, + features("foo", 1, 2), snapshotRegistry); manager.replay(record); snapshotRegistry.getOrCreateSnapshot(123); - assertEquals(new FeatureMapAndEpoch(new FeatureMap(rangeMap("foo", 1, 2)), 123), + assertEquals(new FinalizedControllerFeatures(versionMap("foo", 2), 123), manager.finalizedFeatures(123)); } @Test public void testUpdateFeaturesErrorCases() { - SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); - FeatureControlManager manager = new FeatureControlManager( - rangeMap("foo", 1, 5, "bar", 1, 2), snapshotRegistry); + LogContext logContext = new LogContext(); + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); + FeatureControlManager manager = new FeatureControlManager(logContext, + features("foo", 1, 5, "bar", 1, 2), snapshotRegistry); assertEquals( ControllerResult.atomicOf( @@ -114,24 +143,24 @@ public void testUpdateFeaturesErrorCases() { ) ), manager.updateFeatures( - rangeMap("foo", 1, 3), - Collections.singleton("foo"), - Collections.singletonMap(5, rangeMap()) - ) + updateMap("foo", 3), + Collections.singletonMap("foo", FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), + Collections.singletonMap(5, rangeMap()), + false) ); ControllerResult> result = manager.updateFeatures( - rangeMap("foo", 1, 3), Collections.emptySet(), Collections.emptyMap()); + updateMap("foo", 3), Collections.emptyMap(), Collections.emptyMap(), false); assertEquals(Collections.singletonMap("foo", ApiError.NONE), result.response()); manager.replay((FeatureLevelRecord) result.records().get(0).message()); snapshotRegistry.getOrCreateSnapshot(3); assertEquals(ControllerResult.atomicOf(Collections.emptyList(), Collections. singletonMap("foo", new ApiError(Errors.INVALID_UPDATE_VERSION, - "Can't downgrade the maximum version of this feature without " + - "setting downgradable to true."))), - manager.updateFeatures(rangeMap("foo", 1, 2), - Collections.emptySet(), Collections.emptyMap())); + "Can't downgrade the maximum version of this feature without setting the upgrade type to " + + "safe or unsafe downgrade."))), + manager.updateFeatures(updateMap("foo", 2), + Collections.emptyMap(), Collections.emptyMap(), false)); assertEquals( ControllerResult.atomicOf( @@ -139,39 +168,37 @@ public void testUpdateFeaturesErrorCases() { new ApiMessageAndVersion( new FeatureLevelRecord() .setName("foo") - .setMinFeatureLevel((short) 1) - .setMaxFeatureLevel((short) 2), + .setFeatureLevel((short) 2), (short) 0 ) ), Collections.singletonMap("foo", ApiError.NONE) ), manager.updateFeatures( - rangeMap("foo", 1, 2), - Collections.singleton("foo"), - Collections.emptyMap() - ) + updateMap("foo", 2), + Collections.singletonMap("foo", FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), + Collections.emptyMap(), + false) ); } @Test public void testFeatureControlIterator() throws Exception { - SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); - FeatureControlManager manager = new FeatureControlManager( - rangeMap("foo", 1, 5, "bar", 1, 2), snapshotRegistry); + LogContext logContext = new LogContext(); + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); + FeatureControlManager manager = new FeatureControlManager(logContext, + features("foo", 1, 5, "bar", 1, 2), snapshotRegistry); ControllerResult> result = manager. - updateFeatures(rangeMap("foo", 1, 5, "bar", 1, 1), - Collections.emptySet(), Collections.emptyMap()); + updateFeatures(updateMap("foo", 5, "bar", 1), + Collections.emptyMap(), Collections.emptyMap(), false); RecordTestUtils.replayAll(manager, result.records()); RecordTestUtils.assertBatchIteratorContains(Arrays.asList( Arrays.asList(new ApiMessageAndVersion(new FeatureLevelRecord(). setName("foo"). - setMinFeatureLevel((short) 1). - setMaxFeatureLevel((short) 5), (short) 0)), + setFeatureLevel((short) 5), (short) 0)), Arrays.asList(new ApiMessageAndVersion(new FeatureLevelRecord(). setName("bar"). - setMinFeatureLevel((short) 1). - setMaxFeatureLevel((short) 1), (short) 0))), + setFeatureLevel((short) 1), (short) 0))), manager.iterator(Long.MAX_VALUE)); } } 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 2ec06de8087b..40b7274f0259 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -178,7 +178,7 @@ public void testDelayedConfigurationOperations() throws Throwable { private void testDelayedConfigurationOperations(LocalLogManagerTestEnv logEnv, QuorumController controller) throws Throwable { - logEnv.logManagers().forEach(m -> m.setMaxReadOffset(0L)); + logEnv.logManagers().forEach(m -> m.setMaxReadOffset(2L)); CompletableFuture> future1 = controller.incrementalAlterConfigs(Collections.singletonMap( BROKER0, Collections.singletonMap("baz", entry(SET, "123"))), false); @@ -187,7 +187,7 @@ private void testDelayedConfigurationOperations(LocalLogManagerTestEnv logEnv, new ResultOrError<>(Collections.emptyMap())), controller.describeConfigs(Collections.singletonMap( BROKER0, Collections.emptyList())).get()); - logEnv.logManagers().forEach(m -> m.setMaxReadOffset(2L)); + logEnv.logManagers().forEach(m -> m.setMaxReadOffset(3L)); assertEquals(Collections.singletonMap(BROKER0, ApiError.NONE), future1.get()); } @@ -426,6 +426,7 @@ public void testUnregisterBroker() throws Throwable { setBrokerId(0). setClusterId(active.clusterId()). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwBA")). + setFeatures(brokerFeatures()). setListeners(listeners)); assertEquals(0L, reply.get().epoch()); CreateTopicsRequestData createTopicsRequestData = @@ -466,6 +467,11 @@ public void testUnregisterBroker() throws Throwable { } } + private BrokerRegistrationRequestData.FeatureCollection brokerFeatures() { + BrokerRegistrationRequestData.FeatureCollection features = new BrokerRegistrationRequestData.FeatureCollection(); + return features; + } + @Test public void testSnapshotSaveAndLoad() throws Throwable { final int numBrokers = 4; 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 43346fe1fbd6..b7bff3883b85 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java @@ -61,6 +61,7 @@ public QuorumControllerTestEnv( for (int i = 0; i < numControllers; i++) { QuorumController.Builder builder = new QuorumController.Builder(i, logEnv.clusterId()); builder.setRaftClient(logEnv.logManagers().get(i)); + builder.setQuorumFeatures(new QuorumFeatures(i, QuorumFeatures.defaultFeatureMap())); sessionTimeoutMillis.ifPresent(timeout -> { builder.setSessionTimeoutNs(NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS)); }); diff --git a/metadata/src/test/java/org/apache/kafka/image/ClusterImageTest.java b/metadata/src/test/java/org/apache/kafka/image/ClusterImageTest.java index 6908cf2a78b6..e10d4a5971d5 100644 --- a/metadata/src/test/java/org/apache/kafka/image/ClusterImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/ClusterImageTest.java @@ -60,14 +60,14 @@ public class ClusterImageTest { 1000, Uuid.fromString("vZKYST0pSA2HO5x_6hoO2Q"), Arrays.asList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9092)), - Collections.singletonMap("foo", new VersionRange((short) 1, (short) 3)), + Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 3)), Optional.empty(), true)); map1.put(1, new BrokerRegistration(1, 1001, Uuid.fromString("U52uRe20RsGI0RvpcTx33Q"), Arrays.asList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9093)), - Collections.singletonMap("foo", new VersionRange((short) 1, (short) 3)), + Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 3)), Optional.empty(), false)); map1.put(2, new BrokerRegistration(2, @@ -96,14 +96,14 @@ public class ClusterImageTest { 1000, Uuid.fromString("vZKYST0pSA2HO5x_6hoO2Q"), Arrays.asList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9092)), - Collections.singletonMap("foo", new VersionRange((short) 1, (short) 3)), + Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 3)), Optional.empty(), false)); map2.put(1, new BrokerRegistration(1, 1001, Uuid.fromString("U52uRe20RsGI0RvpcTx33Q"), Arrays.asList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9093)), - Collections.singletonMap("foo", new VersionRange((short) 1, (short) 3)), + Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 3)), Optional.empty(), true)); IMAGE2 = new ClusterImage(map2); diff --git a/metadata/src/test/java/org/apache/kafka/image/FeaturesImageTest.java b/metadata/src/test/java/org/apache/kafka/image/FeaturesImageTest.java index 720086f87b94..52388fbf33fa 100644 --- a/metadata/src/test/java/org/apache/kafka/image/FeaturesImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/FeaturesImageTest.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.metadata.FeatureLevelRecord; import org.apache.kafka.common.metadata.RemoveFeatureLevelRecord; import org.apache.kafka.metadata.RecordTestUtils; -import org.apache.kafka.metadata.VersionRange; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -43,15 +42,15 @@ public class FeaturesImageTest { final static FeaturesImage IMAGE2; static { - Map map1 = new HashMap<>(); - map1.put("foo", new VersionRange((short) 1, (short) 2)); - map1.put("bar", new VersionRange((short) 1, (short) 1)); - map1.put("baz", new VersionRange((short) 1, (short) 8)); + Map map1 = new HashMap<>(); + map1.put("foo", (short) 2); + map1.put("bar", (short) 1); + map1.put("baz", (short) 8); IMAGE1 = new FeaturesImage(map1); DELTA1_RECORDS = new ArrayList<>(); DELTA1_RECORDS.add(new ApiMessageAndVersion(new FeatureLevelRecord(). - setName("foo").setMinFeatureLevel((short) 1).setMaxFeatureLevel((short) 3), + setName("foo").setFeatureLevel((short) 3), FEATURE_LEVEL_RECORD.highestSupportedVersion())); DELTA1_RECORDS.add(new ApiMessageAndVersion(new RemoveFeatureLevelRecord(). setName("bar"), REMOVE_FEATURE_LEVEL_RECORD.highestSupportedVersion())); @@ -61,8 +60,8 @@ public class FeaturesImageTest { DELTA1 = new FeaturesDelta(IMAGE1); RecordTestUtils.replayAll(DELTA1, DELTA1_RECORDS); - Map map2 = new HashMap<>(); - map2.put("foo", new VersionRange((short) 1, (short) 3)); + Map map2 = new HashMap<>(); + map2.put("foo", (short) 3); IMAGE2 = new FeaturesImage(map2); } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java b/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java index 0f350c46221b..d9622b8e4cc4 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java @@ -40,15 +40,15 @@ public class BrokerRegistrationTest { private static final List REGISTRATIONS = Arrays.asList( new BrokerRegistration(0, 0, Uuid.fromString("pc1GhUlBS92cGGaKXl6ipw"), Arrays.asList(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9090)), - Collections.singletonMap("foo", new VersionRange((short) 1, (short) 2)), + Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 2)), Optional.empty(), false), new BrokerRegistration(1, 0, Uuid.fromString("3MfdxWlNSn2UDYsmDP1pYg"), Arrays.asList(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9091)), - Collections.singletonMap("foo", new VersionRange((short) 1, (short) 2)), + Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 2)), Optional.empty(), false), new BrokerRegistration(2, 0, Uuid.fromString("eY7oaG1RREie5Kk9uy1l6g"), Arrays.asList(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9092)), - Collections.singletonMap("foo", new VersionRange((short) 2, (short) 3)), + Collections.singletonMap("foo", VersionRange.of((short) 2, (short) 3)), Optional.of("myrack"), false)); @Test diff --git a/metadata/src/test/java/org/apache/kafka/metadata/VersionRangeTest.java b/metadata/src/test/java/org/apache/kafka/metadata/VersionRangeTest.java index 88082a6f5533..d31e8f813962 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/VersionRangeTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/VersionRangeTest.java @@ -21,6 +21,7 @@ import org.junit.jupiter.api.Timeout; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(value = 40) @@ -31,24 +32,24 @@ private static VersionRange v(int a, int b) { assertTrue(a >= Short.MIN_VALUE); assertTrue(b <= Short.MAX_VALUE); assertTrue(b >= Short.MIN_VALUE); - return new VersionRange((short) a, (short) b); + return VersionRange.of((short) a, (short) b); } @Test public void testEquality() { assertEquals(v(1, 1), v(1, 1)); - assertFalse(v(1, 1).equals(v(1, 2))); - assertFalse(v(2, 1).equals(v(1, 2))); - assertFalse(v(2, 1).equals(v(2, 2))); + assertNotEquals(v(1, 2), v(1, 1)); + assertNotEquals(v(1, 2), v(2, 1)); + assertNotEquals(v(2, 2), v(2, 1)); } @Test public void testContains() { - assertTrue(v(1, 1).contains(v(1, 1))); - assertFalse(v(1, 1).contains(v(1, 2))); - assertTrue(v(1, 2).contains(v(1, 1))); - assertFalse(v(4, 10).contains(v(3, 8))); - assertTrue(v(2, 12).contains(v(3, 11))); + assertTrue(v(1, 1).contains((short) 1)); + assertFalse(v(1, 1).contains((short) 2)); + assertTrue(v(1, 2).contains((short) 1)); + assertFalse(v(4, 10).contains((short) 3)); + assertTrue(v(2, 12).contains((short) 11)); } @Test 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 b36d4f156347..866f541fb248 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 @@ -240,6 +240,7 @@ private T readRecord(Readable input) { throw new IllegalArgumentException(); } + // Read the metadata record body from the file input reader T record = serde.read(input, valueSize); int numHeaders = input.readVarint(); From a3a4323a5a2ea4ba1820da58a8c31532e8864f0d Mon Sep 17 00:00:00 2001 From: Bruno Cadonna Date: Thu, 14 Apr 2022 22:23:52 +0200 Subject: [PATCH 114/447] MINOR: Update LICENSE-binary (#12051) Updates the license file. Reviewer: Bill Bejeck --- LICENSE-binary | 77 ++++++++++++++++++++++++++------------------------ 1 file changed, 40 insertions(+), 37 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index dfe7d44f0815..c2400694f252 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -208,50 +208,53 @@ License Version 2.0: audience-annotations-0.5.0 commons-cli-1.4 commons-lang3-3.8.1 -jackson-annotations-2.12.3 -jackson-core-2.12.3 +jackson-annotations-2.12.6 +jackson-core-2.12.6 jackson-databind-2.12.6.1 -jackson-dataformat-csv-2.12.3 -jackson-datatype-jdk8-2.12.3 -jackson-jaxrs-base-2.12.3 -jackson-jaxrs-json-provider-2.12.3 -jackson-module-jaxb-annotations-2.12.3 +jackson-dataformat-csv-2.12.6 +jackson-datatype-jdk8-2.12.6 +jackson-jaxrs-base-2.12.6 +jackson-jaxrs-json-provider-2.12.6 +jackson-module-jaxb-annotations-2.12.6 jackson-module-paranamer-2.10.5 -jackson-module-scala_2.13-2.12.3 +jackson-module-scala_2.13-2.12.6 jakarta.validation-api-2.0.2 javassist-3.27.0-GA -jetty-client-9.4.43.v20210629 -jetty-continuation-9.4.43.v20210629 -jetty-http-9.4.43.v20210629 -jetty-io-9.4.43.v20210629 -jetty-security-9.4.43.v20210629 -jetty-server-9.4.43.v20210629 -jetty-servlet-9.4.43.v20210629 -jetty-servlets-9.4.43.v20210629 -jetty-util-9.4.43.v20210629 -jetty-util-ajax-9.4.43.v20210629 +jetty-client-9.4.44.v20210927 +jetty-continuation-9.4.44.v20210927 +jetty-http-9.4.44.v20210927 +jetty-io-9.4.44.v20210927 +jetty-security-9.4.44.v20210927 +jetty-server-9.4.44.v20210927 +jetty-servlet-9.4.44.v20210927 +jetty-servlets-9.4.44.v20210927 +jetty-util-9.4.44.v20210927 +jetty-util-ajax-9.4.44.v20210927 jersey-common-2.34 jersey-server-2.34 -jose4j-0.7.8 +jose4j-0.7.9 lz4-java-1.8.0 -maven-artifact-3.8.1 +maven-artifact-3.8.4 metrics-core-4.1.12.1 -netty-buffer-4.1.68.Final -netty-codec-4.1.68.Final -netty-common-4.1.68.Final -netty-handler-4.1.68.Final -netty-resolver-4.1.68.Final -netty-transport-4.1.68.Final -netty-transport-native-epoll-4.1.68.Final -netty-transport-native-unix-common-4.1.68.Final -plexus-utils-3.2.1 +metrics-core-2.2.0 +netty-buffer-4.1.73.Final +netty-codec-4.1.73.Final +netty-common-4.1.73.Final +netty-handler-4.1.73.Final +netty-resolver-4.1.73.Final +netty-tcnative-classes-2.0.46.Final +netty-transport-4.1.73.Final +netty-transport-classes-epoll-4.1.73.Final +netty-transport-native-epoll-4.1.73.Final +netty-transport-native-unix-common-4.1.73.Final +plexus-utils-3.3.0 reload4j-1.2.19 -rocksdbjni-6.22.1.1 -scala-collection-compat_2.13-2.4.4 -scala-library-2.13.6 -scala-logging_2.13-3.9.3 -scala-reflect-2.13.6 -scala-java8-compat_2.13-1.0.0 +rocksdbjni-6.29.4.1 +scala-collection-compat_2.13-2.6.0 +scala-library-2.13.8 +scala-logging_2.13-3.9.4 +scala-reflect-2.13.8 +scala-java8-compat_2.13-1.0.2 snappy-java-1.1.8.4 zookeeper-3.6.3 zookeeper-jute-3.6.3 @@ -306,12 +309,12 @@ slf4j-reload4j-1.7.36, see: licenses/slf4j-MIT --------------------------------------- BSD 2-Clause -zstd-jni-1.5.0-4 see: licenses/zstd-jni-BSD-2-clause +zstd-jni-1.5.2-1 see: licenses/zstd-jni-BSD-2-clause --------------------------------------- BSD 3-Clause -jline-3.12.1, see: licenses/jline-BSD-3-clause +jline-3.21.0, see: licenses/jline-BSD-3-clause paranamer-2.8, see: licenses/paranamer-BSD-3-clause --------------------------------------- From f49cff412d3e15c5e7427b7ec09a89b3d157b2ec Mon Sep 17 00:00:00 2001 From: RivenSun <91005273+RivenSun2@users.noreply.github.com> Date: Fri, 15 Apr 2022 08:25:37 +0800 Subject: [PATCH 115/447] MINOR: Remove redundant conditional judgments in Selector.clear() (#12048) Condition 'sendFailed' is always 'false' when reached. Reviewers: Guozhang Wang --- .../src/main/java/org/apache/kafka/common/network/Selector.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index c3b488833908..bd1175a8ee0a 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -845,7 +845,7 @@ private void clear() { boolean hasPending = false; if (!sendFailed) hasPending = maybeReadFromClosingChannel(channel); - if (!hasPending || sendFailed) { + if (!hasPending) { doClose(channel, true); it.remove(); } From cf5e714a8bea8bb1de75201d0769bb1c246b9334 Mon Sep 17 00:00:00 2001 From: RivenSun <91005273+RivenSun2@users.noreply.github.com> Date: Fri, 15 Apr 2022 08:30:43 +0800 Subject: [PATCH 116/447] MINOR: ignore unused configuration when ConsumerCoordinator is not constructed (#12041) Following PR #11940, ignore unused config when ConsumerCoordinator is not constructed. Reviewers: Guozhang Wang --- .../org/apache/kafka/clients/consumer/KafkaConsumer.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index a49c89560f2c..6ffb772915d6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -774,8 +774,12 @@ public KafkaConsumer(Map configs, ); // no coordinator will be constructed for the default (null) group id - this.coordinator = !groupId.isPresent() ? null : - new ConsumerCoordinator(groupRebalanceConfig, + if (!groupId.isPresent()) { + config.ignore(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG); + config.ignore(ConsumerConfig.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED); + this.coordinator = null; + } else { + this.coordinator = new ConsumerCoordinator(groupRebalanceConfig, logContext, this.client, assignors, @@ -788,6 +792,7 @@ public KafkaConsumer(Map configs, config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG), this.interceptors, config.getBoolean(ConsumerConfig.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); + } this.fetcher = new Fetcher<>( logContext, this.client, From b7f2d979d6e9be1ca542d42c6d79e3a9e14770ef Mon Sep 17 00:00:00 2001 From: Bruno Cadonna Date: Fri, 15 Apr 2022 10:21:05 +0200 Subject: [PATCH 117/447] [MINOR] Update upgrade documentation for 3.2 (#12055) Reviewer: Bruno Cadonna --- docs/documentation.html | 3 ++- docs/upgrade.html | 42 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 44 insertions(+), 1 deletion(-) diff --git a/docs/documentation.html b/docs/documentation.html index d13f691209f3..07014db7af4b 100644 --- a/docs/documentation.html +++ b/docs/documentation.html @@ -33,7 +33,7 @@

    Documentation

    -

    Kafka 3.1 Documentation

    +

    Kafka 3.2 Documentation

    Prior releases:
    0.7.x, 0.8.0, 0.8.1.X, @@ -55,6 +55,7 @@

    Kafka 3.1 Documentation

    2.7.X, 2.8.X, 3.0.X. + 3.1.X.

    1. Getting Started

    1.1 Introduction

    diff --git a/docs/upgrade.html b/docs/upgrade.html index 5bce37436364..033c8b9ffb28 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -19,6 +19,48 @@