forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
KAFKA-17871: avoid blocking the herder thread when producer flushing …
…hangs The call to `backingStore.get()` (called by connector task threads through `OffsetStorageReaderImpl.offsets()`) can block for long time waiting for data flush to complete (`KafkaProducer.flush()`). This change moves that call outside the synchronized clause that holds `offsetReadFutures`, so that if `backingStore.get()` hangs then it does not keep `offsetReadFutures` locked. The access to `closed` flag (`closed.get()`) is kept inside the synchronize clause to avoid race condition with `close()`. This is important because `OffsetStorageReaderImpl.close()` needs to lock `offsetReadFutures` as well in order to cancel the futures. Since the herder thread calls `OffsetStorageReaderImpl.close()` when attempting to stops a task, before this change this was resulting in the herder thread hanging indefinetely waiting for `backingStore.get()` to complete.
- Loading branch information
1 parent
8181c8e
commit 7d1945f
Showing
2 changed files
with
143 additions
and
1 deletion.
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
138 changes: 138 additions & 0 deletions
138
connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageReaderTest.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,138 @@ | ||
/* | ||
* 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.connect.storage; | ||
|
||
import static org.mockito.ArgumentMatchers.any; | ||
import static org.mockito.Mockito.doAnswer; | ||
import static org.mockito.Mockito.mock; | ||
import static org.mockito.Mockito.times; | ||
import static org.mockito.Mockito.verify; | ||
|
||
import java.util.Collections; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.Executors; | ||
|
||
import org.easymock.Mock; | ||
import org.junit.Test; | ||
import org.junit.runner.RunWith; | ||
import org.mockito.invocation.InvocationOnMock; | ||
import org.mockito.junit.MockitoJUnitRunner; | ||
import org.mockito.stubbing.Answer; | ||
|
||
@RunWith(MockitoJUnitRunner.StrictStubs.class) | ||
public class OffsetStorageReaderTest { | ||
|
||
@Mock | ||
private Converter taskKeyConverter; | ||
|
||
@Mock | ||
private Converter taskValueConverter; | ||
|
||
@Test(timeout = 60 * 1000) | ||
public void testClosingOffsetReaderWhenOffsetStoreHangs() throws Exception { | ||
ExecutorService executor = Executors.newSingleThreadExecutor(); | ||
OffsetBackingStore offsetBackingStore = mock(OffsetBackingStore.class); | ||
|
||
OffsetStorageReaderImpl offsetStorageReaderImpl = new OffsetStorageReaderImpl( | ||
offsetBackingStore, "namespace", taskKeyConverter, taskValueConverter); | ||
|
||
doAnswer(invocation -> { | ||
// Sleep for a long time to simulate a hanging offset store | ||
Thread.sleep(9999 * 1000); | ||
throw new RuntimeException("Should never get here"); | ||
}).when(offsetBackingStore).get(any()); | ||
|
||
// Connector task thread hanging | ||
executor.submit(() -> { | ||
// Does call offsetBackingStore.get() and hangs | ||
offsetStorageReaderImpl.offsets(Collections.emptyList()); | ||
}); | ||
Thread.sleep(3000); | ||
|
||
verify(offsetBackingStore, times(1)).get(any()); | ||
|
||
// The herder thread should not block when trying to close `offsetStorageReaderImpl` | ||
// and complete before test timeout | ||
offsetStorageReaderImpl.close(); | ||
} | ||
|
||
@Test(timeout = 60 * 1000) | ||
public void testClosingOffsetReaderWhenOffsetStoreHangsAndHasIncompleteFutures() throws Exception { | ||
// Test similar to `testClosingOffsetReaderWhenOffsetStoreHangs` above, but in this case | ||
// `OffsetStorageReaderImpl.offsetReadFutures` contains a future when `offsetStorageReaderImpl.close()` is called. | ||
|
||
ExecutorService executor = Executors.newFixedThreadPool(2); | ||
OffsetBackingStore offsetBackingStore = mock(OffsetBackingStore.class); | ||
CompletableFuture<?> hangingFuture = mock(CompletableFuture.class); | ||
|
||
OffsetStorageReaderImpl offsetStorageReaderImpl = new OffsetStorageReaderImpl( | ||
offsetBackingStore, "namespace", taskKeyConverter, taskValueConverter); | ||
|
||
// Mock hanging future | ||
doAnswer(invocation -> { | ||
Thread.sleep(9999 * 1000); | ||
throw new RuntimeException("Should never get here"); | ||
}).when(hangingFuture).get(); | ||
|
||
// Mock `offsetBackingStore.get()` | ||
doAnswer(new Answer<Object>() { | ||
int callCount = 0; | ||
|
||
@Override | ||
public Object answer(InvocationOnMock invocation) throws Throwable { | ||
if (callCount == 0) { | ||
callCount += 1; | ||
// First connector task | ||
return hangingFuture; | ||
} else { | ||
// Second connector task | ||
Thread.sleep(9999 * 1000); | ||
throw new RuntimeException("Should never get here"); | ||
} | ||
} | ||
} | ||
).when(offsetBackingStore).get(any()); | ||
|
||
|
||
// Connector task thread calls `offsets()` --> hangs on `hangingFuture.get()` | ||
// --> the future is added to `offsetStorageReaderImpl.offsetReadFutures` and never removed | ||
executor.submit(() -> { | ||
offsetStorageReaderImpl.offsets(Collections.emptyList()); | ||
}); | ||
Thread.sleep(3000); | ||
|
||
verify(offsetBackingStore, times(1)).get(any()); | ||
verify(hangingFuture, times(1)).get(); | ||
|
||
// Another connector task thread calls `offsets()` --> hangs on offsetBackingStore.get() | ||
// --> the future is never added to `offsetStorageReaderImpl.offsetReadFutures` | ||
executor.submit(() -> { | ||
offsetStorageReaderImpl.offsets(Collections.emptyList()); | ||
}); | ||
Thread.sleep(3000); | ||
|
||
verify(offsetBackingStore, times(2)).get(any()); | ||
|
||
// The herder thread should not block when trying to close `offsetStorageReaderImpl` and should complete | ||
// before the test timeout | ||
offsetStorageReaderImpl.close(); | ||
|
||
// The hanging future should be cancelled by `close()` | ||
verify(hangingFuture, times(1)).cancel(true); | ||
} | ||
} |