HBASE-22316 Record the stack trace for current thread in FutureUtils.get

This commit is contained in:
Duo Zhang 2019-05-30 09:59:23 +08:00 committed by Apache9
parent 70fb30cd56
commit 1be93fcaec
2 changed files with 122 additions and 8 deletions

View File

@ -32,8 +32,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
/**
* Helper class for processing futures.
*/
@ -122,6 +120,41 @@ public final class FutureUtils {
return error;
}
// This method is used to record the stack trace that calling the FutureUtils.get method. As in
// async client, the retry will be done in the retry timer thread, so the exception we get from
// the CompletableFuture will have a stack trace starting from the root of the retry timer. If we
// just throw this exception out when calling future.get(by unwrapping the ExecutionException),
// the upper layer even can not know where is the exception thrown...
// See HBASE-22316.
private static void setStackTrace(Throwable error) {
StackTraceElement[] localStackTrace = Thread.currentThread().getStackTrace();
StackTraceElement[] originalStackTrace = error.getStackTrace();
StackTraceElement[] newStackTrace =
new StackTraceElement[localStackTrace.length + originalStackTrace.length + 1];
System.arraycopy(localStackTrace, 0, newStackTrace, 0, localStackTrace.length);
newStackTrace[localStackTrace.length] =
new StackTraceElement("--------Future", "get--------", null, -1);
System.arraycopy(originalStackTrace, 0, newStackTrace, localStackTrace.length + 1,
originalStackTrace.length);
error.setStackTrace(newStackTrace);
}
private static IOException rethrow(ExecutionException error) throws IOException {
Throwable cause = error.getCause();
if (cause instanceof IOException) {
setStackTrace(cause);
throw (IOException) cause;
} else if (cause instanceof RuntimeException) {
setStackTrace(cause);
throw (RuntimeException) cause;
} else if (cause instanceof Error) {
setStackTrace(cause);
throw (Error) cause;
} else {
throw new IOException(cause);
}
}
/**
* A helper class for getting the result of a Future, and convert the error to an
* {@link IOException}.
@ -132,9 +165,7 @@ public final class FutureUtils {
} catch (InterruptedException e) {
throw (IOException) new InterruptedIOException().initCause(e);
} catch (ExecutionException e) {
Throwable cause = e.getCause();
Throwables.propagateIfPossible(cause, IOException.class);
throw new IOException(cause);
throw rethrow(e);
}
}
@ -148,9 +179,7 @@ public final class FutureUtils {
} catch (InterruptedException e) {
throw (IOException) new InterruptedIOException().initCause(e);
} catch (ExecutionException e) {
Throwable cause = e.getCause();
Throwables.propagateIfPossible(cause, IOException.class);
throw new IOException(cause);
throw rethrow(e);
} catch (TimeoutException e) {
throw new TimeoutIOException(e);
}

View File

@ -0,0 +1,85 @@
/**
* 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.hadoop.hbase.util;
import static org.hamcrest.CoreMatchers.startsWith;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.stream.Stream;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.After;
import org.junit.Before;
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({ MiscTests.class, SmallTests.class })
public class TestFutureUtils {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestFutureUtils.class);
private ExecutorService executor;
@Before
public void setUp() {
executor = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder().setDaemon(true).build());
}
@After
public void tearDown() {
executor.shutdownNow();
}
@Test
public void testRecordStackTrace() throws IOException {
CompletableFuture<Void> future = new CompletableFuture<>();
executor.execute(() -> future.completeExceptionally(new HBaseIOException("Inject error!")));
try {
FutureUtils.get(future);
fail("The future should have been completed exceptionally");
} catch (HBaseIOException e) {
assertEquals("Inject error!", e.getMessage());
StackTraceElement[] elements = e.getStackTrace();
assertThat(elements[0].toString(), startsWith("java.lang.Thread.getStackTrace"));
assertThat(elements[1].toString(),
startsWith("org.apache.hadoop.hbase.util.FutureUtils.setStackTrace"));
assertThat(elements[2].toString(),
startsWith("org.apache.hadoop.hbase.util.FutureUtils.rethrow"));
assertThat(elements[3].toString(),
startsWith("org.apache.hadoop.hbase.util.FutureUtils.get"));
assertThat(elements[4].toString(),
startsWith("org.apache.hadoop.hbase.util.TestFutureUtils.testRecordStackTrace"));
assertTrue(Stream.of(elements)
.anyMatch(element -> element.toString().contains("--------Future.get--------")));
}
}
}