You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/05/25 13:13:49 UTC

[GitHub] [kafka] cadonna commented on a change in pull request #8716: KAFKA-6145: KIP-441: Fix assignor config passthough

cadonna commented on a change in pull request #8716:
URL: https://github.com/apache/kafka/pull/8716#discussion_r429927490



##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfigurationTest.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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 static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsString;
+import static org.junit.Assert.assertThrows;
+
+public class AssignorConfigurationTest {

Review comment:
       Yes, please!

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java
##########
@@ -359,6 +359,10 @@ private AssignmentConfigs(final StreamsConfig configs) {
                           final Integer maxWarmupReplicas,
                           final Integer numStandbyReplicas,
                           final Long probingRebalanceIntervalMs) {
+            if (maxWarmupReplicas < 1) {
+                throw new IllegalArgumentException("must configure at least one warmup replica");
+            }
+

Review comment:
       IMO, we should check the limits for all configs. However, I am not sure if we should check `probingRebalanceIntervalMs` to be `>= 60 * 1000L` (as we do in `StreamsConfig`) or just `>= 0. 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java
##########
@@ -241,16 +239,29 @@ private static void assignStatelessActiveTasks(final TreeMap<UUID, ClientState>
         final Map<TaskId, SortedSet<UUID>> taskToCaughtUpClients = new HashMap<>();
 
         for (final TaskId task : statefulTasks) {
-
+            final TreeSet<UUID> caughtUpClients = new TreeSet<>();
             for (final Map.Entry<UUID, ClientState> clientEntry : clientStates.entrySet()) {
                 final UUID client = clientEntry.getKey();
                 final long taskLag = clientEntry.getValue().lagFor(task);
-                if (taskLag == Task.LATEST_OFFSET || taskLag <= acceptableRecoveryLag) {
-                    taskToCaughtUpClients.computeIfAbsent(task, ignored -> new TreeSet<>()).add(client);
+                if (active(taskLag) || unbounded(acceptableRecoveryLag) || acceptable(acceptableRecoveryLag, taskLag)) {

Review comment:
       Nice catch indeed! I agree with @ableegoldman about the renaming. I am in favour of `activeRunning` or `activeAndRunning`.

##########
File path: streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
##########
@@ -1148,6 +1148,9 @@ private void verifyMaxInFlightRequestPerConnection(final Object maxInFlightReque
         consumerProps.put(REPLICATION_FACTOR_CONFIG, getInt(REPLICATION_FACTOR_CONFIG));
         consumerProps.put(APPLICATION_SERVER_CONFIG, getString(APPLICATION_SERVER_CONFIG));
         consumerProps.put(NUM_STANDBY_REPLICAS_CONFIG, getInt(NUM_STANDBY_REPLICAS_CONFIG));
+        consumerProps.put(ACCEPTABLE_RECOVERY_LAG_CONFIG, getLong(ACCEPTABLE_RECOVERY_LAG_CONFIG));
+        consumerProps.put(MAX_WARMUP_REPLICAS_CONFIG, getInt(MAX_WARMUP_REPLICAS_CONFIG));
+        consumerProps.put(PROBING_REBALANCE_INTERVAL_MS_CONFIG, getLong(PROBING_REBALANCE_INTERVAL_MS_CONFIG));

Review comment:
       req: Please add verifications to `StreamsConfigTest#consumerConfigMustContainStreamPartitionAssignorConfig()` 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org