Skip to content
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

Cherrypick lock backoff retry to 3.8 #18653

Open
wants to merge 1 commit into
base: 3.8
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
*/
package org.apache.kafka.streams.processor.internals;

import org.apache.kafka.common.utils.ExponentialBackoff;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.StreamsConfig;
Expand Down Expand Up @@ -96,6 +97,12 @@ public StateDirectoryProcessFile() {

private final HashMap<TaskId, Thread> lockedTasksToOwner = new HashMap<>();

private final HashMap<TaskId, BackoffRecord> lockedTasksToBackoffRecord = new HashMap<>();

private static final long INTERVALS_MS = 10;
private static final long MAX_ATTEMPTS = 50;


private FileChannel stateDirLockChannel;
private FileLock stateDirLock;

Expand Down Expand Up @@ -347,27 +354,48 @@ synchronized boolean lock(final TaskId taskId) {
return true;
}

try {
Thread.sleep(updateOrCreateBackoffRecord(taskId));
} catch (final InterruptedException e) {
throw new RuntimeException(e);
}
Comment on lines +357 to +361
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Where does this come from?

final Thread lockOwner = lockedTasksToOwner.get(taskId);
if (lockOwner != null) {
if (lockOwner.equals(Thread.currentThread())) {
log.trace("{} Found cached state dir lock for task {}", logPrefix(), taskId);
// we already own the lock
lockedTasksToBackoffRecord.remove(taskId);
return true;
} else {
// another thread owns the lock
return false;
}
} else if (!stateDir.exists()) {
log.error("Tried to lock task directory for {} but the state directory does not exist", taskId);
lockedTasksToBackoffRecord.remove(taskId);
throw new IllegalStateException("The state directory has been deleted");
} else {
lockedTasksToOwner.put(taskId, Thread.currentThread());
// make sure the task directory actually exists, and create it if not
getOrCreateDirectoryForTask(taskId);
lockedTasksToBackoffRecord.remove(taskId);
return true;
}
}

/* Visible for testing*/
public long updateOrCreateBackoffRecord(final TaskId taskId) {
long sleepTime = 0;
if (lockedTasksToBackoffRecord.containsKey(taskId)) {
final BackoffRecord backoffRecord = lockedTasksToBackoffRecord.get(taskId);
sleepTime = backoffRecord.exponentialBackoff.backoff(backoffRecord.attempts);
backoffRecord.attempts++;
} else {
lockedTasksToBackoffRecord.put(taskId, new BackoffRecord());
}
return sleepTime;
}

/**
* Unlock the state directory for the given {@link TaskId}.
*/
Expand Down Expand Up @@ -680,4 +708,19 @@ public int hashCode() {
}
}

public static class BackoffRecord {
private static final long INITIAL_INTERVAL = 1;
private static final int MULTIPLIER = 2;
private static final long MAX_INTERVAL = 1000;
private static final double JITTER = 0.5;

private long attempts;
private ExponentialBackoff exponentialBackoff;

public BackoffRecord() {
this.attempts = 0;
this.exponentialBackoff = new ExponentialBackoff(INITIAL_INTERVAL, MULTIPLIER, MAX_INTERVAL, JITTER);
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -26,13 +26,13 @@
import java.util.List;
import java.util.UUID;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.LogCaptureAppender;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.errors.ProcessorStateException;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.StateDirectory.TaskDirectory;
import org.apache.kafka.common.utils.LogCaptureAppender;
import org.apache.kafka.streams.state.internals.OffsetCheckpoint;
import org.apache.kafka.test.TestUtils;

Expand All @@ -58,6 +58,9 @@
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;

import static java.util.Collections.emptyList;
import static java.util.Collections.singleton;
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;
Expand Down Expand Up @@ -444,6 +447,18 @@ public void shouldNotLockStateDirLockedByAnotherThread() throws Exception {
assertFalse(directory.lock(taskId));
}

@Test
public void shouldSleepIfStateDirLockedByAnotherThread() throws Exception {
final TaskId taskId = new TaskId(0, 0);
assertEquals(directory.updateOrCreateBackoffRecord(taskId), 0);
final Thread thread = new Thread(() -> directory.lock(taskId));
thread.start();
thread.join(30000);
assertEquals(directory.updateOrCreateBackoffRecord(taskId), 0);
assertFalse(directory.lock(taskId));
assertNotEquals(directory.updateOrCreateBackoffRecord(taskId), 0);
}

@Test
public void shouldNotUnLockStateDirLockedByAnotherThread() throws Exception {
final TaskId taskId = new TaskId(0, 0);
Expand All @@ -463,14 +478,14 @@ public void shouldNotUnLockStateDirLockedByAnotherThread() throws Exception {
thread.start();
lockLatch.await(5, TimeUnit.SECONDS);

assertNull("should not have had an exception on other thread", exceptionOnThread.get());
assertNull(exceptionOnThread.get(), "should not have had an exception on other thread");
directory.unlock(taskId);
assertFalse(directory.lock(taskId));

unlockLatch.countDown();
thread.join(30000);

assertNull("should not have had an exception on other thread", exceptionOnThread.get());
assertNull(exceptionOnThread.get(), "should not have had an exception on other thread");
assertTrue(directory.lock(taskId));
}

Expand Down