forked from apache/kafka
/
AssignorConfiguration.java
340 lines (306 loc) · 16.1 KB
/
AssignorConfiguration.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
/*
* 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.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.RebalanceProtocol;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.StreamsConfig.InternalConfig;
import org.apache.kafka.streams.internals.UpgradeFromValues;
import org.apache.kafka.streams.processor.internals.ClientUtils;
import org.apache.kafka.streams.processor.internals.InternalTopicManager;
import org.slf4j.Logger;
import java.util.List;
import java.util.Map;
import static org.apache.kafka.common.utils.Utils.getHost;
import static org.apache.kafka.common.utils.Utils.getPort;
import static org.apache.kafka.streams.StreamsConfig.InternalConfig.INTERNAL_TASK_ASSIGNOR_CLASS;
import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION;
public final class AssignorConfiguration {
private final String taskAssignorClass;
private final String logPrefix;
private final Logger log;
private final ReferenceContainer referenceContainer;
private final StreamsConfig streamsConfig;
private final Map<String, ?> internalConfigs;
public AssignorConfiguration(final Map<String, ?> configs) {
// NOTE: If you add a new config to pass through to here, be sure to test it in a real
// application. Since we filter out some configurations, we may have to explicitly copy
// them over when we construct the Consumer.
streamsConfig = new ClientUtils.QuietStreamsConfig(configs);
internalConfigs = configs;
// Setting the logger with the passed in client thread name
logPrefix = String.format("stream-thread [%s] ", streamsConfig.getString(CommonClientConfigs.CLIENT_ID_CONFIG));
final LogContext logContext = new LogContext(logPrefix);
log = logContext.logger(getClass());
{
final Object o = configs.get(InternalConfig.REFERENCE_CONTAINER_PARTITION_ASSIGNOR);
if (o == null) {
final KafkaException fatalException = new KafkaException("ReferenceContainer is not specified");
log.error(fatalException.getMessage(), fatalException);
throw fatalException;
}
if (!(o instanceof ReferenceContainer)) {
final KafkaException fatalException = new KafkaException(
String.format("%s is not an instance of %s", o.getClass().getName(), ReferenceContainer.class.getName())
);
log.error(fatalException.getMessage(), fatalException);
throw fatalException;
}
referenceContainer = (ReferenceContainer) o;
}
{
final String o = (String) configs.get(INTERNAL_TASK_ASSIGNOR_CLASS);
if (o == null) {
taskAssignorClass = HighAvailabilityTaskAssignor.class.getName();
} else {
taskAssignorClass = o;
}
}
}
public ReferenceContainer referenceContainer() {
return referenceContainer;
}
public RebalanceProtocol rebalanceProtocol() {
final String upgradeFrom = streamsConfig.getString(StreamsConfig.UPGRADE_FROM_CONFIG);
if (upgradeFrom != null) {
switch (UpgradeFromValues.getValueFromString(upgradeFrom)) {
case UPGRADE_FROM_0100:
case UPGRADE_FROM_0101:
case UPGRADE_FROM_0102:
case UPGRADE_FROM_0110:
case UPGRADE_FROM_10:
case UPGRADE_FROM_11:
case UPGRADE_FROM_20:
case UPGRADE_FROM_21:
case UPGRADE_FROM_22:
case UPGRADE_FROM_23:
// ATTENTION: The following log messages is used for verification in system test
// streams/streams_cooperative_rebalance_upgrade_test.py::StreamsCooperativeRebalanceUpgradeTest.test_upgrade_to_cooperative_rebalance
// If you change it, please do also change the system test accordingly and
// verify whether the test passes.
log.info("Eager rebalancing protocol is enabled now for upgrade from {}.x", upgradeFrom);
log.warn("The eager rebalancing protocol is deprecated and will stop being supported in a future release." +
" Please be prepared to remove the 'upgrade.from' config soon.");
return RebalanceProtocol.EAGER;
case UPGRADE_FROM_24:
case UPGRADE_FROM_25:
case UPGRADE_FROM_26:
case UPGRADE_FROM_27:
case UPGRADE_FROM_28:
case UPGRADE_FROM_30:
case UPGRADE_FROM_31:
case UPGRADE_FROM_32:
case UPGRADE_FROM_33:
case UPGRADE_FROM_34:
case UPGRADE_FROM_35:
// we need to add new version when new "upgrade.from" values become available
// This config is for explicitly sending FK response to a requested partition
// and should not affect the rebalance protocol
break;
default:
throw new IllegalArgumentException("Unknown configuration value for parameter 'upgrade.from': " + upgradeFrom);
}
}
// ATTENTION: The following log messages is used for verification in system test
// streams/streams_cooperative_rebalance_upgrade_test.py::StreamsCooperativeRebalanceUpgradeTest.test_upgrade_to_cooperative_rebalance
// If you change it, please do also change the system test accordingly and
// verify whether the test passes.
log.info("Cooperative rebalancing protocol is enabled now");
return RebalanceProtocol.COOPERATIVE;
}
public String logPrefix() {
return logPrefix;
}
public int configuredMetadataVersion(final int priorVersion) {
final String upgradeFrom = streamsConfig.getString(StreamsConfig.UPGRADE_FROM_CONFIG);
if (upgradeFrom != null) {
switch (UpgradeFromValues.getValueFromString(upgradeFrom)) {
case UPGRADE_FROM_0100:
log.info(
"Downgrading metadata version from {} to 1 for upgrade from 0.10.0.x.",
LATEST_SUPPORTED_VERSION
);
return 1;
case UPGRADE_FROM_0101:
case UPGRADE_FROM_0102:
case UPGRADE_FROM_0110:
case UPGRADE_FROM_10:
case UPGRADE_FROM_11:
log.info(
"Downgrading metadata version from {} to 2 for upgrade from {}.x.",
LATEST_SUPPORTED_VERSION,
upgradeFrom
);
return 2;
case UPGRADE_FROM_20:
case UPGRADE_FROM_21:
case UPGRADE_FROM_22:
case UPGRADE_FROM_23:
// These configs are for cooperative rebalancing and should not affect the metadata version
break;
case UPGRADE_FROM_24:
case UPGRADE_FROM_25:
case UPGRADE_FROM_26:
case UPGRADE_FROM_27:
case UPGRADE_FROM_28:
case UPGRADE_FROM_30:
case UPGRADE_FROM_31:
case UPGRADE_FROM_32:
case UPGRADE_FROM_33:
case UPGRADE_FROM_34:
case UPGRADE_FROM_35:
// we need to add new version when new "upgrade.from" values become available
// This config is for explicitly sending FK response to a requested partition
// and should not affect the metadata version
break;
default:
throw new IllegalArgumentException(
"Unknown configuration value for parameter 'upgrade.from': " + upgradeFrom
);
}
}
return priorVersion;
}
public String userEndPoint() {
final String configuredUserEndpoint = streamsConfig.getString(StreamsConfig.APPLICATION_SERVER_CONFIG);
if (configuredUserEndpoint != null && !configuredUserEndpoint.isEmpty()) {
try {
final String host = getHost(configuredUserEndpoint);
final Integer port = getPort(configuredUserEndpoint);
if (host == null || port == null) {
throw new ConfigException(
String.format(
"%s Config %s isn't in the correct format. Expected a host:port pair but received %s",
logPrefix, StreamsConfig.APPLICATION_SERVER_CONFIG, configuredUserEndpoint
)
);
}
} catch (final NumberFormatException nfe) {
throw new ConfigException(
String.format("%s Invalid port supplied in %s for config %s: %s",
logPrefix, configuredUserEndpoint, StreamsConfig.APPLICATION_SERVER_CONFIG, nfe)
);
}
return configuredUserEndpoint;
} else {
return null;
}
}
public InternalTopicManager internalTopicManager() {
return new InternalTopicManager(referenceContainer.time, referenceContainer.adminClient, streamsConfig);
}
public CopartitionedTopicsEnforcer copartitionedTopicsEnforcer() {
return new CopartitionedTopicsEnforcer(logPrefix);
}
public AssignmentConfigs assignmentConfigs() {
return new AssignmentConfigs(streamsConfig);
}
public TaskAssignor taskAssignor() {
try {
return Utils.newInstance(taskAssignorClass, TaskAssignor.class);
} catch (final ClassNotFoundException e) {
throw new IllegalArgumentException(
"Expected an instantiable class name for " + INTERNAL_TASK_ASSIGNOR_CLASS,
e
);
}
}
public AssignmentListener assignmentListener() {
final Object o = internalConfigs.get(InternalConfig.ASSIGNMENT_LISTENER);
if (o == null) {
return stable -> { };
}
if (!(o instanceof AssignmentListener)) {
final KafkaException fatalException = new KafkaException(
String.format("%s is not an instance of %s", o.getClass().getName(), AssignmentListener.class.getName())
);
log.error(fatalException.getMessage(), fatalException);
throw fatalException;
}
return (AssignmentListener) o;
}
public interface AssignmentListener {
void onAssignmentComplete(final boolean stable);
}
public static class AssignmentConfigs {
public final long acceptableRecoveryLag;
public final int maxWarmupReplicas;
public final int numStandbyReplicas;
public final long probingRebalanceIntervalMs;
public final List<String> rackAwareAssignmentTags;
public final Integer rackAwareAssignmentTrafficCost;
public final Integer rackAwareAssignmentNonOverlapCost;
public final String rackAwareAssignmentStrategy;
private AssignmentConfigs(final StreamsConfig configs) {
acceptableRecoveryLag = configs.getLong(StreamsConfig.ACCEPTABLE_RECOVERY_LAG_CONFIG);
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 = configs.getList(StreamsConfig.RACK_AWARE_ASSIGNMENT_TAGS_CONFIG);
rackAwareAssignmentTrafficCost = configs.getInt(StreamsConfig.RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG);
rackAwareAssignmentNonOverlapCost = configs.getInt(StreamsConfig.RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG);
rackAwareAssignmentStrategy = configs.getString(StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_CONFIG);
}
AssignmentConfigs(final Long acceptableRecoveryLag,
final Integer maxWarmupReplicas,
final Integer numStandbyReplicas,
final Long probingRebalanceIntervalMs,
final List<String> rackAwareAssignmentTags) {
this(acceptableRecoveryLag, maxWarmupReplicas, numStandbyReplicas, probingRebalanceIntervalMs, rackAwareAssignmentTags,
null, null, StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_NONE);
}
AssignmentConfigs(final Long acceptableRecoveryLag,
final Integer maxWarmupReplicas,
final Integer numStandbyReplicas,
final Long probingRebalanceIntervalMs,
final List<String> rackAwareAssignmentTags,
final Integer rackAwareAssignmentTrafficCost,
final Integer rackAwareAssignmentNonOverlapCost,
final String rackAwareAssignmentStrategy) {
this.acceptableRecoveryLag = validated(StreamsConfig.ACCEPTABLE_RECOVERY_LAG_CONFIG, acceptableRecoveryLag);
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 = validated(StreamsConfig.RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, rackAwareAssignmentTags);
this.rackAwareAssignmentTrafficCost = validated(StreamsConfig.RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG, rackAwareAssignmentTrafficCost);
this.rackAwareAssignmentNonOverlapCost = validated(StreamsConfig.RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG, rackAwareAssignmentNonOverlapCost);
this.rackAwareAssignmentStrategy = validated(StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_CONFIG, rackAwareAssignmentStrategy);
}
private static <T> T validated(final String configKey, final T value) {
final ConfigDef.Validator validator = StreamsConfig.configDef().configKeys().get(configKey).validator;
if (validator != null) {
validator.ensureValid(configKey, value);
}
return value;
}
@Override
public String toString() {
return "AssignmentConfigs{" +
"\n acceptableRecoveryLag=" + acceptableRecoveryLag +
"\n maxWarmupReplicas=" + maxWarmupReplicas +
"\n numStandbyReplicas=" + numStandbyReplicas +
"\n probingRebalanceIntervalMs=" + probingRebalanceIntervalMs +
"\n rackAwareAssignmentTags=" + rackAwareAssignmentTags +
"\n}";
}
}
}