Skip to content

HBASE-29397 Deadlock in BufferedMuratorOverAsyncBufferedMutator #7107

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

Merged
merged 1 commit into from
Jun 17, 2025
Merged
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 @@ -135,16 +135,23 @@ public void mutate(List<? extends Mutation> mutations) throws IOException {
long heapSize = mutation.heapSize();
bufferedSize.addAndGet(heapSize);
addListener(fs.get(i), (r, e) -> {
synchronized (this) {
futures.remove(toComplete);
bufferedSize.addAndGet(-heapSize);
if (e != null) {
errors.add(Pair.newPair(mutation, e));
toComplete.completeExceptionally(e);
} else {
toComplete.complete(r);
}
bufferedSize.addAndGet(-heapSize);
if (e != null) {
errors.add(Pair.newPair(mutation, e));
toComplete.completeExceptionally(e);
} else {
toComplete.complete(r);
}
// Only remove future after completing, and add the error to errors field before completing,
// this is used as a guard in internalFlush method, which is used to make sure that
// 1. If the future is already completed and removed from futures, the error should have
// already been in errors field, so in internalFlush method, even if we do not wait on the
// future, we should still get this error in errors field at the end of the internalFlush
// method
// 2. If we get this future in the internalFlush method for waiting, then after the future
// complete, we should get this error in the errors field at the end of the internalFlush
// method
futures.remove(toComplete);
});
}
synchronized (this) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,14 @@
package org.apache.hadoop.hbase.client;

import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;

import java.io.IOException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.hadoop.hbase.HBaseClassTestRule;
Expand All @@ -28,12 +34,16 @@
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;

import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;

@Category({ MediumTests.class, ClientTests.class })
public class TestBufferedMutator {

Expand All @@ -49,40 +59,85 @@ public class TestBufferedMutator {

private static byte[] CQ = Bytes.toBytes("cq");

private static int COUNT = 1024;

private static byte[] VALUE = new byte[1024];

@BeforeClass
public static void setUp() throws Exception {
TEST_UTIL.startMiniCluster(1);
TEST_UTIL.createTable(TABLE_NAME, CF);
ThreadLocalRandom.current().nextBytes(VALUE);
}

@AfterClass
public static void tearDown() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}

@Before
public void setUpBeforeTest() throws IOException {
TEST_UTIL.createTable(TABLE_NAME, CF);
}

@After
public void tearDownAfterTest() throws IOException {
TEST_UTIL.deleteTable(TABLE_NAME);
}

@Test
public void test() throws Exception {
int count = 1024;
try (BufferedMutator mutator = TEST_UTIL.getConnection()
.getBufferedMutator(new BufferedMutatorParams(TABLE_NAME).writeBufferSize(64 * 1024))) {
mutator.mutate(IntStream.range(0, COUNT / 2)
mutator.mutate(IntStream.range(0, count / 2)
.mapToObj(i -> new Put(Bytes.toBytes(i)).addColumn(CF, CQ, VALUE))
.collect(Collectors.toList()));
mutator.flush();
mutator.mutate(IntStream.range(COUNT / 2, COUNT)
mutator.mutate(IntStream.range(count / 2, count)
.mapToObj(i -> new Put(Bytes.toBytes(i)).addColumn(CF, CQ, VALUE))
.collect(Collectors.toList()));
mutator.close();
verifyData();
verifyData(count);
}
}

@Test
public void testMultiThread() throws Exception {
ExecutorService executor =
Executors.newFixedThreadPool(16, new ThreadFactoryBuilder().setDaemon(true).build());
// use a greater count and less write buffer size to trigger auto flush when mutate
int count = 16384;
try (BufferedMutator mutator = TEST_UTIL.getConnection()
.getBufferedMutator(new BufferedMutatorParams(TABLE_NAME).writeBufferSize(4 * 1024))) {
IntStream.range(0, count / 2)
.mapToObj(i -> new Put(Bytes.toBytes(i)).addColumn(CF, CQ, VALUE))
.forEach(put -> executor.execute(() -> {
try {
mutator.mutate(put);
} catch (IOException e) {
fail("failed to mutate: " + e.getMessage());
}
}));
mutator.flush();
IntStream.range(count / 2, count)
.mapToObj(i -> new Put(Bytes.toBytes(i)).addColumn(CF, CQ, VALUE))
.forEach(put -> executor.execute(() -> {
try {
mutator.mutate(put);
} catch (IOException e) {
fail("failed to mutate: " + e.getMessage());
}
}));
executor.shutdown();
assertTrue(executor.awaitTermination(15, TimeUnit.SECONDS));
mutator.close();
} finally {
executor.shutdownNow();
}
verifyData(count);
}

private void verifyData() throws IOException {
private void verifyData(int count) throws IOException {
try (Table table = TEST_UTIL.getConnection().getTable(TABLE_NAME)) {
for (int i = 0; i < COUNT; i++) {
for (int i = 0; i < count; i++) {
Result r = table.get(new Get(Bytes.toBytes(i)));
assertArrayEquals(VALUE, ((Result) r).getValue(CF, CQ));
}
Expand Down