-
Notifications
You must be signed in to change notification settings - Fork 14.2k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
KAFKA-15900, KAFKA-18310: fix flaky test testOutdatedCoordinatorAssig…
…nment and AbstractCoordinatorTest Signed-off-by: PoAn Yang <payang@apache.org>
- Loading branch information
1 parent
900d81b
commit 53d3094
Showing
7 changed files
with
177 additions
and
81 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
64 changes: 64 additions & 0 deletions
64
...ts/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatThread.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,64 @@ | ||
/* | ||
* 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.clients.consumer.internals; | ||
|
||
import org.apache.kafka.common.utils.KafkaThread; | ||
|
||
import java.util.concurrent.atomic.AtomicBoolean; | ||
import java.util.concurrent.atomic.AtomicReference; | ||
|
||
public abstract class AbstractHeartbeatThread extends KafkaThread implements AutoCloseable { | ||
private final AtomicBoolean enabled = new AtomicBoolean(false); | ||
private final AtomicBoolean closed = new AtomicBoolean(false); | ||
private final AtomicReference<RuntimeException> failed = new AtomicReference<>(null); | ||
|
||
public AbstractHeartbeatThread(String name, boolean daemon) { | ||
super(name, daemon); | ||
} | ||
|
||
public void enable() { | ||
enabled.set(true); | ||
} | ||
|
||
public void disable() { | ||
enabled.set(false); | ||
} | ||
|
||
public boolean isEnabled() { | ||
return enabled.get(); | ||
} | ||
|
||
public void setFailed(RuntimeException e) { | ||
failed.set(e); | ||
} | ||
|
||
public boolean hasFailed() { | ||
return failed.get() != null; | ||
} | ||
|
||
public RuntimeException failureCause() { | ||
return failed.get(); | ||
} | ||
|
||
public void close() { | ||
closed.set(true); | ||
} | ||
|
||
public boolean isClosed() { | ||
return closed.get(); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.