-
Notifications
You must be signed in to change notification settings - Fork 14.5k
KAFKA-10357: Add explicit internal topic initialization for Kafka Streams #19913
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: trunk
Are you sure you want to change the base?
Changes from 18 commits
0a3f8e9
1768707
12f67af
160025d
8dedaf9
71d32cb
e081b05
e8aeaeb
3757672
00c847b
44b3860
a888e30
f139ab1
7496e51
0402ba7
8cb10b8
2911371
ec63b89
013fe7d
62f13e5
1ba84bb
f4070d1
9d0aedf
ee9a89f
ffbcfa7
fed9c39
6a9d6f1
168e9c9
26e26b1
25a07e9
01debbc
72c6d51
2a90edc
b41b361
44c888a
8cb778a
5eca29a
b11bb29
782d36c
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -45,6 +45,10 @@ | |
import org.apache.kafka.common.utils.Utils; | ||
import org.apache.kafka.streams.errors.InvalidStateStoreException; | ||
import org.apache.kafka.streams.errors.InvalidStateStorePartitionException; | ||
import org.apache.kafka.streams.errors.MissingSourceTopicException; | ||
import org.apache.kafka.streams.errors.MisconfiguredInternalTopicException; | ||
import org.apache.kafka.streams.errors.MissingInternalTopicsException; | ||
import org.apache.kafka.streams.errors.InternalTopicsAlreadySetupException; | ||
import org.apache.kafka.streams.errors.ProcessorStateException; | ||
import org.apache.kafka.streams.errors.StreamsException; | ||
import org.apache.kafka.streams.errors.StreamsNotStartedException; | ||
|
@@ -61,6 +65,9 @@ | |
import org.apache.kafka.streams.processor.TaskId; | ||
import org.apache.kafka.streams.processor.internals.ClientUtils; | ||
import org.apache.kafka.streams.processor.internals.GlobalStreamThread; | ||
import org.apache.kafka.streams.processor.internals.InternalTopicManager; | ||
import org.apache.kafka.streams.processor.internals.InternalTopicConfig; | ||
import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; | ||
import org.apache.kafka.streams.processor.internals.StateDirectory; | ||
import org.apache.kafka.streams.processor.internals.StreamThread; | ||
import org.apache.kafka.streams.processor.internals.StreamsMetadataState; | ||
|
@@ -69,6 +76,7 @@ | |
import org.apache.kafka.streams.processor.internals.TopologyMetadata; | ||
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; | ||
import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopology; | ||
import org.apache.kafka.streams.processor.internals.InternalTopicManager.ValidationResult; | ||
import org.apache.kafka.streams.query.FailureReason; | ||
import org.apache.kafka.streams.query.PositionBound; | ||
import org.apache.kafka.streams.query.QueryConfig; | ||
|
@@ -111,6 +119,7 @@ | |
import java.util.stream.Collectors; | ||
|
||
import static org.apache.kafka.streams.StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG; | ||
import static org.apache.kafka.streams.StreamsConfig.DEFAULT_INIT_TIMEOUT_MS; | ||
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; | ||
|
@@ -293,6 +302,168 @@ public boolean isValidTransition(final State newState) { | |
private final Object stateLock = new Object(); | ||
protected volatile State state = State.CREATED; | ||
|
||
/** | ||
* Initializes broker-side state. | ||
* | ||
* @throws MissingSourceTopicException if a source topic is missing | ||
* @throws MissingInternalTopicsException if some but not all of the internal topics are missing | ||
* @throws MisconfiguredInternalTopicException if an internal topics is misconfigured | ||
* @throws InternalTopicsAlreadySetupException if all internal topics are already setup | ||
*/ | ||
public void init() { | ||
k-apol marked this conversation as resolved.
Show resolved
Hide resolved
|
||
this.init(DEFAULT_INIT_TIMEOUT_MS); | ||
k-apol marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
|
||
/** | ||
* Initializes broker-side state. | ||
* | ||
* @throws MissingSourceTopicException if a source topic is missing | ||
* @throws MissingInternalTopicsException if some but not all of the internal topics are missing | ||
* @throws MisconfiguredInternalTopicException if an internal topics is misconfigured | ||
* @throws InternalTopicsAlreadySetupException if all internal topics are already setup | ||
* @throws TimeoutException if initialization exceeds the given timeout | ||
*/ | ||
|
||
public void init(final Duration timeout) { | ||
final InitParameters initParameters = InitParameters.initParameters(); | ||
initParameters.setTimeout(timeout); | ||
|
||
this.doInit(InitParameters.initParameters()); | ||
} | ||
|
||
/** | ||
* Initializes broker-side state. | ||
* | ||
* This methods takes parameters that specify which internal topics to setup if some | ||
* but not all of them are absent. | ||
* | ||
* @throws MissingSourceTopicException if a source topic is missing | ||
* @throws MissingInternalTopicsException if some but not all of the internal topics are missing | ||
* and the given initialization parameters do not specify to setup them | ||
* @throws MisconfiguredInternalTopicException if an internal topics is misconfigured | ||
* @throws InternalTopicsAlreadySetupException if all internal topics are already setup | ||
*/ | ||
|
||
public void init(final InitParameters initParameters) { | ||
this.doInit(initParameters); | ||
} | ||
|
||
/** | ||
* Initializes broker-side state. | ||
* | ||
* This methods takes parameters that specify which internal topics to setup if some | ||
* but not all of them are absent. | ||
* | ||
* @throws MissingSourceTopicException if a source topic is missing | ||
* @throws MissingInternalTopicsException if some but not all of the internal topics are missing | ||
* and the given initialization parameters do not specify to setup them | ||
* @throws MisconfiguredInternalTopicException if an internal topics is misconfigured | ||
* @throws InternalTopicsAlreadySetupException if all internal topics are already setup | ||
* @throws TimeoutException if initialization exceeds the given timeout | ||
*/ | ||
public void init(final InitParameters initParameters, final Duration timeout) { | ||
initParameters.enableTimeout(); | ||
initParameters.setTimeout(timeout); | ||
|
||
this.doInit(initParameters); | ||
} | ||
|
||
private void doInit(final InitParameters initParameters) { | ||
final InternalTopicManager internalTopicManager = new InternalTopicManager(time, adminClient, applicationConfigs); | ||
if (initParameters.hasTimeoutEnabled()) { | ||
internalTopicManager.setInitTimeout(initParameters.getTimeout()); | ||
} | ||
|
||
final Map<String, InternalTopicConfig> allInternalTopics = new HashMap<>(); | ||
final Set<String> allSourceTopics = new HashSet<>(); | ||
for (final Map<TopologyMetadata.Subtopology, InternalTopologyBuilder.TopicsInfo> subtopologyMap : topologyMetadata.topologyToSubtopologyTopicsInfoMap().values()) { | ||
for (final InternalTopologyBuilder.TopicsInfo topicsInfo : subtopologyMap.values()) { | ||
allInternalTopics.putAll(topicsInfo.stateChangelogTopics); | ||
allInternalTopics.putAll(topicsInfo.repartitionSourceTopics); | ||
allSourceTopics.addAll(topicsInfo.sourceTopics); | ||
} | ||
} | ||
try { | ||
k-apol marked this conversation as resolved.
Show resolved
Hide resolved
|
||
final ValidationResult validationResult = internalTopicManager.validate(allInternalTopics); // can throw timeout | ||
|
||
final boolean noInternalTopicsExist = allInternalTopics.keySet() == validationResult.missingTopics(); | ||
k-apol marked this conversation as resolved.
Show resolved
Hide resolved
|
||
final boolean internalTopicsMisconfigured = !validationResult.misconfigurationsForTopics().isEmpty(); | ||
final boolean allInternalTopicsExist = validationResult.missingTopics().isEmpty(); | ||
final boolean missingSourceTopics = !Collections.disjoint(validationResult.missingTopics(), allSourceTopics); | ||
|
||
if (internalTopicsMisconfigured) { | ||
throw new MisconfiguredInternalTopicException("Misconfigured Internal Topics: " + validationResult.misconfigurationsForTopics()); | ||
} | ||
if (missingSourceTopics) { | ||
allSourceTopics.retainAll(validationResult.missingTopics()); | ||
throw new MissingSourceTopicException("Missing source topics: " + allSourceTopics); | ||
} | ||
if (noInternalTopicsExist) { | ||
internalTopicManager.setup(allInternalTopics); | ||
} else if (allInternalTopicsExist) { | ||
throw new InternalTopicsAlreadySetupException("All internal topics have already been setup"); | ||
} else { | ||
if (initParameters.setupInternalTopicsIfIncompleteEnabled()) { | ||
final Map<String, InternalTopicConfig> topicsToCreate = new HashMap<>(); | ||
for (final String missingTopic : validationResult.missingTopics()) { | ||
topicsToCreate.put(missingTopic, allInternalTopics.get(missingTopic)); | ||
} | ||
internalTopicManager.makeReady(topicsToCreate); | ||
} else { | ||
throw new MissingInternalTopicsException("Missing Internal Topics: ", new ArrayList<>(validationResult.missingTopics())); | ||
} | ||
} | ||
} catch (final TimeoutException timeoutException) { | ||
throw new TimeoutException(timeoutException.getMessage(), timeoutException); | ||
} catch (final StreamsException streamsException) { | ||
throw new StreamsException(streamsException.getMessage(), streamsException); | ||
} | ||
k-apol marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
|
||
public static class InitParameters { | ||
private boolean timeoutEnabled; | ||
private Duration timeout; | ||
private final boolean setupInternalTopicsIfIncomplete; | ||
|
||
private InitParameters(final boolean setupInternalTopicsIfIncomplete) { | ||
this.setupInternalTopicsIfIncomplete = setupInternalTopicsIfIncomplete; | ||
} | ||
|
||
// Default: don't create missing topics if only some are missing | ||
public static InitParameters initParameters() { | ||
return new InitParameters(false); | ||
} | ||
|
||
public InitParameters enableSetupInternalTopicsIfIncomplete() { | ||
return new InitParameters(true); | ||
} | ||
|
||
public InitParameters disableSetupInternalTopicsIfIncomplete() { | ||
return new InitParameters(false); | ||
} | ||
|
||
public boolean setupInternalTopicsIfIncompleteEnabled() { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. While the KIP does specify this getter, I am wondering why? We usually don't use getters on the main config-classes, but use sub-classes. Might be good to re-read the KIP discussion (and if need be, update the KIP to "fix" this) |
||
return setupInternalTopicsIfIncomplete; | ||
} | ||
|
||
public final void enableTimeout() { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This method was not defined on the KIP -- so we should not add it -- or we need to update the KIP. Looking into KIP-1092 and KIP-1153 which introduce There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Makes sense, I will update as this and the other timeout-related code as conversation evolves. There may be a better way to approach handling a timeout here. |
||
this.timeoutEnabled = true; | ||
} | ||
|
||
public final boolean hasTimeoutEnabled() { | ||
return timeoutEnabled; | ||
} | ||
|
||
public final void setTimeout(final Duration timeout) { | ||
this.timeout = timeout; | ||
} | ||
|
||
public final Duration getTimeout() { | ||
return this.timeout; | ||
} | ||
|
||
} | ||
|
||
private boolean waitOnStates(final long waitMs, final State... targetStates) { | ||
final Set<State> targetStateSet = Set.of(targetStates); | ||
final long begin = time.milliseconds(); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -173,6 +173,7 @@ public class StreamsConfig extends AbstractConfig { | |
|
||
public static final long MAX_TASK_IDLE_MS_DISABLED = -1; | ||
|
||
public static final Duration DEFAULT_INIT_TIMEOUT_MS = Duration.ofMillis(30000); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think we should add this here, as it would "leak" into public API. Not even sure if we need a variable for it at all? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am marking this down along with the other concerns surrounding the timeout |
||
// 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; | ||
|
@@ -885,6 +886,14 @@ public class StreamsConfig extends AbstractConfig { | |
ProducerConfig.TRANSACTIONAL_ID_CONFIG | ||
}; | ||
|
||
public static final String INTERNAL_TOPIC_SETUP_CONFIG = "internal.topics.setup"; | ||
public static final String INTERNAL_TOPIC_SETUP_AUTOMATIC = "automatic"; | ||
public static final String INTERNAL_TOPIC_SETUP_MANUAL = "manual"; | ||
private static final String INTERNAL_TOPIC_SETUP_DOC = | ||
"Configures how internal topics (e.g., repartition or changelog topics) should be created. " + | ||
"Set to 'automatic' to allow internal topics to be created during a rebalance (default). " + | ||
"Set to 'manual' to disable automatic creation. Users must call KafkaStreams#init() instead."; | ||
|
||
static { | ||
CONFIG = new ConfigDef() | ||
|
||
|
@@ -1012,6 +1021,12 @@ public class StreamsConfig extends AbstractConfig { | |
LogAndFailProcessingExceptionHandler.class.getName(), | ||
Importance.MEDIUM, | ||
PROCESSING_EXCEPTION_HANDLER_CLASS_DOC) | ||
.define(INTERNAL_TOPIC_SETUP_CONFIG, | ||
ConfigDef.Type.STRING, | ||
INTERNAL_TOPIC_SETUP_AUTOMATIC, | ||
ConfigDef.ValidString.in(INTERNAL_TOPIC_SETUP_AUTOMATIC, INTERNAL_TOPIC_SETUP_MANUAL), | ||
Importance.MEDIUM, | ||
INTERNAL_TOPIC_SETUP_DOC) | ||
.define(PROCESSING_GUARANTEE_CONFIG, | ||
Type.STRING, | ||
AT_LEAST_ONCE, | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,26 @@ | ||
/* | ||
* 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.errors; | ||
|
||
public class InternalTopicsAlreadySetupException extends StreamsException { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
public InternalTopicsAlreadySetupException(final String message) { | ||
super(message); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,26 @@ | ||
/* | ||
* 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.errors; | ||
|
||
public class MisconfiguredInternalTopicException extends StreamsException { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
public MisconfiguredInternalTopicException(final String message) { | ||
super(message); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,44 @@ | ||
/* | ||
* 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.errors; | ||
|
||
import java.util.List; | ||
|
||
public class MissingInternalTopicsException extends StreamsException { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
private final List<String> topics; | ||
|
||
/** | ||
* Constructs a new MissingInternalTopicsException. | ||
* | ||
* @param message The detail message | ||
* @param topics the list of missing internal topic names | ||
*/ | ||
public MissingInternalTopicsException(final String message, final List<String> topics) { | ||
super(message); | ||
this.topics = topics; | ||
} | ||
|
||
/** | ||
* Returns the list of missing internal topics that caused the exception. | ||
*/ | ||
public List<String> topics() { | ||
return topics; | ||
} | ||
} |
Uh oh!
There was an error while loading. Please reload this page.