Skip to content

Commit

Permalink
java: reduce scary exception spew and improve logging
Browse files Browse the repository at this point in the history
In real environments there are various types of failures that are
"expected". In particular, it's not unlikely for the client to
occasionally connect to a server which has gone down or which is not the
leader for a given tablet anymore. In that case, we should make sure
that the logging output is easy to read and doesn't include irrelevant
stack traces stemming from Netty internals.

This patch addresses several of those cases. In particular, it gets rid
of "unexpected exception" messages (and associated stacks) in the case
of "connection refused" errors. It also helpfully includes the
underlying error when invalidating the tablet cache for a location.

There's a relatively simple new unit test, and I also ran some
before/after comparisons using the "loadgen" sample from the
kudu-examples repo against a local cluster with one TS:

Case 1: Master down when we start the client:
============================================================

Before the patch
--------------
[New I/O worker apache#12] WARN org.apache.kudu.client.ConnectToCluster - Error receiving response from localhost:7051
org.apache.kudu.client.RecoverableException: connection closed
	at org.apache.kudu.client.Connection.channelClosed(Connection.java:254)
	at org.apache.kudu.shaded.org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:88)
	at org.apache.kudu.client.Connection.handleUpstream(Connection.java:238)
	at org.apache.kudu.shaded.org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
	at org.apache.kudu.shaded.org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791)
	at org.apache.kudu.shaded.org.jboss.netty.handler.timeout.ReadTimeoutHandler.channelClosed(ReadTimeoutHandler.java:176)
	at org.apache.kudu.shaded.org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:88)
	at org.apache.kudu.shaded.org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
	at org.apache.kudu.shaded.org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791)
	at org.apache.kudu.shaded.org.jboss.netty.handler.codec.oneone.OneToOneDecoder.handleUpstream(OneToOneDecoder.java:60)
	at org.apache.kudu.shaded.org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
	at org.apache.kudu.shaded.org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791)
	at org.apache.kudu.shaded.org.jboss.netty.handler.codec.frame.FrameDecoder.cleanup(FrameDecoder.java:493)
	at org.apache.kudu.shaded.org.jboss.netty.handler.codec.frame.FrameDecoder.channelClosed(FrameDecoder.java:371)
	at org.apache.kudu.shaded.org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:88)
	at org.apache.kudu.shaded.org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
	at org.apache.kudu.shaded.org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:559)
	at org.apache.kudu.shaded.org.jboss.netty.channel.Channels.fireChannelClosed(Channels.java:468)
	at org.apache.kudu.shaded.org.jboss.netty.channel.Channels$6.run(Channels.java:457)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.ChannelRunnableWrapper.run(ChannelRunnableWrapper.java:40)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.AbstractNioSelector.processTaskQueue(AbstractNioSelector.java:391)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:315)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:89)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178)
	at org.apache.kudu.shaded.org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
	at org.apache.kudu.shaded.org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
[New I/O boss apache#17] ERROR org.apache.kudu.client.Connection - [peer master-localhost:7051(localhost:7051)] unexpected exception from downstream on [id: 0xf9759051]
java.net.ConnectException: Connection refused: localhost/127.0.0.1:7051
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.NioClientBoss.connect(NioClientBoss.java:152)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.NioClientBoss.processSelectedKeys(NioClientBoss.java:105)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.NioClientBoss.process(NioClientBoss.java:79)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:337)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.NioClientBoss.run(NioClientBoss.java:42)
	at org.apache.kudu.shaded.org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
	at org.apache.kudu.shaded.org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
[New I/O worker apache#12] WARN org.apache.kudu.client.ConnectToCluster - Unable to find the leader master localhost:7051; will retry

... repeats with exponential backoff

After the patch
--------------
[New I/O boss apache#17] INFO org.apache.kudu.client.Connection - Failed to connect to peer master-localhost:7051(localhost:7051): Connection refused: localhost/127.0.0.1:7051
[New I/O worker apache#1] INFO org.apache.kudu.client.ConnectToCluster - Unable to connect to master localhost:7051: Connection refused: localhost/127.0.0.1:7051
[New I/O worker apache#1] WARN org.apache.kudu.client.ConnectToCluster - Unable to find the leader master localhost:7051; will retry

... repeats with exponential backoff

Case 2: Tserver is down
============================================================

Before the patch
----------------
[New I/O worker apache#11] INFO org.apache.kudu.client.AsyncKuduClient - Removing server fc9f6ab955cc47a8ab3653d0170305b0 from this tablet's cache 9da164fbcd93404ea8e38e9491eb3fa8
[New I/O boss apache#17] ERROR org.apache.kudu.client.Connection - [peer fc9f6ab955cc47a8ab3653d0170305b0(todd-laptop:7050)] unexpected exception from downstream on [id: 0x270e3a82]
java.net.ConnectException: Connection refused: todd-laptop/127.0.1.1:7050
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.NioClientBoss.connect(NioClientBoss.java:152)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.NioClientBoss.processSelectedKeys(NioClientBoss.java:105)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.NioClientBoss.process(NioClientBoss.java:79)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:337)
	at org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.NioClientBoss.run(NioClientBoss.java:42)
	at org.apache.kudu.shaded.org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
	at org.apache.kudu.shaded.org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)

... repeats with exponential backoff

After the patch
---------------
[New I/O boss apache#17] INFO org.apache.kudu.client.Connection - Failed to connect to peer fc9f6ab955cc47a8ab3653d0170305b0(todd-laptop:7050): Connection refused: todd-laptop/127.0.1.1:7050
[New I/O worker apache#15] INFO org.apache.kudu.client.AsyncKuduClient - Invalidating location fc9f6ab955cc47a8ab3653d0170305b0(todd-laptop:7050) for tablet 478ddbaafb5b494ea0de8e2d0ed45a00: Connection refused: todd-laptop/127.0.1.1:7050

... repeats with exponential backoff

Case 3: A tablet is bad
============================================================
In this case I manually delete a non-replicated tablet that is being written to
using 'kudu remote_replica delete'. This is unrecoverable since there is
no other replica.

Before the patch
-----------------
[New I/O worker apache#10] INFO org.apache.kudu.client.AsyncKuduClient - Removing server fc9f6ab955cc47a8ab3653d0170305b0 from this tablet's cache a633f4a0c14b4404a8ae1f825b378867

... repeats with exponential backoff

After the patch
----------------
[New I/O worker apache#9] INFO org.apache.kudu.client.AsyncKuduClient - Invalidating location 36622c46b25f4da9ada43f8591728053(todd-laptop:7050) for tablet c0b9488597e8447595ad0ee0a3378f95: Tablet not RUNNING: STOPPED

... repeats with exponential backoff

Change-Id: I4b8be871693fecc1ee46a4e238dd2ed8f0730d4b
Reviewed-on: http://gerrit.cloudera.org:8080/9644
Reviewed-by: Grant Henke <[email protected]>
Tested-by: Grant Henke <[email protected]>
  • Loading branch information
toddlipcon authored and granthenke committed Mar 15, 2018
1 parent 8204f43 commit ead7568
Show file tree
Hide file tree
Showing 5 changed files with 78 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1643,7 +1643,7 @@ Deferred<List<LocatedTablet>> locateTable(final KuduTable table,
* We're in the context of decode() meaning we need to either callback or retry later.
*/
<R> void handleTabletNotFound(final KuduRpc<R> rpc, KuduException ex, ServerInfo info) {
invalidateTabletCache(rpc.getTablet(), info);
invalidateTabletCache(rpc.getTablet(), info, ex.getMessage());
handleRetryableError(rpc, ex);
}

Expand Down Expand Up @@ -1732,9 +1732,11 @@ public void run(final Timeout timeout) {
* Remove the tablet server from the RemoteTablet's locations. Right now nothing is removing
* the tablet itself from the caches.
*/
private void invalidateTabletCache(RemoteTablet tablet, ServerInfo info) {
private void invalidateTabletCache(RemoteTablet tablet, ServerInfo info,
String errorMessage) {
final String uuid = info.getUuid();
LOG.info("Removing server {} from this tablet's cache {}", uuid, tablet.getTabletId());
LOG.info("Invalidating location {} for tablet {}: {}",
info, tablet.getTabletId(), errorMessage);
tablet.removeTabletClient(uuid);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -360,7 +360,7 @@ public ConnectToMasterErrCB(HostAndPort hostAndPort) {

@Override
public Void call(Exception e) throws Exception {
LOG.warn("Error receiving response from {}", hostAndPort, e);
LOG.info("Unable to connect to master {}: {}", hostAndPort, e.getMessage());
exceptionsReceived.add(e);
incrementCountAndCheckExhausted();
return null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.kudu.client;

import java.net.ConnectException;
import java.nio.channels.ClosedChannelException;
import java.util.ArrayList;
import java.util.HashMap;
Expand Down Expand Up @@ -168,6 +169,11 @@ public enum CredentialsPolicy {
@GuardedBy("lock")
private int nextCallId = 0;

@Nullable
@GuardedBy("lock")
/** The future for the connection attempt. Set only once connect() is called. */
private ChannelFuture connectFuture;

/**
* Create a new Connection object to the specified destination.
*
Expand Down Expand Up @@ -249,9 +255,21 @@ public void channelDisconnected(final ChannelHandlerContext ctx, final ChannelSt
/** {@inheritDoc} */
@Override
public void channelClosed(final ChannelHandlerContext ctx, final ChannelStateEvent e) {
String msg = "connection closed";
// Connection failures are reported as channelClosed() before exceptionCaught() is called.
// We can detect this case by looking at whether connectFuture has been marked complete
// and grabbing the exception from there.
lock.lock();
try {
if (connectFuture != null && connectFuture.getCause() != null) {
msg = connectFuture.getCause().getMessage();
}
} finally {
lock.unlock();
}
// No need to call super.channelClosed(ctx, e) -- there should be nobody in the upstream
// pipeline after Connection itself. So, just handle the close event ourselves.
cleanup(new RecoverableException(Status.NetworkError("connection closed")));
cleanup(new RecoverableException(Status.NetworkError(msg)));
}

/** {@inheritDoc} */
Expand Down Expand Up @@ -415,7 +433,11 @@ public void exceptionCaught(final ChannelHandlerContext ctx, final ExceptionEven
explicitlyDisconnected ? "%s disconnected from peer" : "%s lost connection to peer",
getLogPrefix());
error = new RecoverableException(Status.NetworkError(message), e);
LOG.info(message, e);
LOG.info(message);
} else if (e instanceof ConnectException) {
String message = "Failed to connect to peer " + serverInfo + ": " + e.getMessage();
error = new RecoverableException(Status.NetworkError(message), e);
LOG.info(message);
} else if (e instanceof SSLException && explicitlyDisconnected) {
// There's a race in Netty where, when we call Channel.close(), it tries
// to send a TLS 'shutdown' message and enters a shutdown state. If another
Expand Down Expand Up @@ -703,7 +725,7 @@ private void connect() {
Preconditions.checkState(lock.isHeldByCurrentThread());
Preconditions.checkState(state == State.NEW);
state = State.CONNECTING;
channel.connect(serverInfo.getResolvedAddress());
connectFuture = channel.connect(serverInfo.getResolvedAddress());
}

/** Enumeration to represent the internal state of the Connection object. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,16 +20,21 @@
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;

import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import org.apache.kudu.util.AssertHelpers.BooleanExpression;
import org.apache.kudu.util.CapturingLogAppender;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;

import com.google.common.net.HostAndPort;

public class TestClientFailoverSupport extends BaseKuduTest {
private CapturingLogAppender cla = new CapturingLogAppender();
private Closeable claAttach;

enum MasterFailureType {
RESTART,
Expand All @@ -42,12 +47,25 @@ public static void setUpBeforeClass() throws Exception {
BaseKuduTest.doSetup(3, NUM_TABLET_SERVERS);
}

@Before
public void attachToLog() {
claAttach = cla.attach();
}

@After
public void restartKilledMaster() throws IOException {
miniCluster.restartDeadMasters();
miniCluster.restartDeadTservers();
}

@After
public void checkLogs() throws IOException {
claAttach.close();
String log = cla.getAppendedText();
assertFalse("Log should not contain Netty internals",
log.contains("socket.nio.AbstractNioSelector"));
}

private void waitUntilRowCount(final KuduTable table, final int rowCount, long timeoutMs)
throws Exception {
assertEventuallyTrue(String.format("Read count should be %s", rowCount),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -906,6 +906,35 @@ public void testCloseShortlyAfterOpen() throws Exception {
exception_text.contains("lost connection to peer"));
}

/**
* Test that, if the masters are down when we attempt to connect, we don't end up
* logging any nonsensical stack traces including Netty internals.
*/
@Test(timeout = 100000)
public void testNoLogSpewOnConnectionRefused() throws Exception {
CapturingLogAppender cla = new CapturingLogAppender();
try (Closeable c = cla.attach()) {
miniCluster.killMasters();
try (KuduClient localClient = new KuduClient.KuduClientBuilder(masterAddresses).build()) {
// Force the client to connect to the masters.
localClient.exportAuthenticationCredentials();
} catch (NoLeaderFoundException e) {
assertTrue("Bad exception string: " + e.getMessage(),
e.getMessage().matches(".*Master config .+ has no leader. " +
"Exceptions received:.*Connection refused.*Connection refused" +
".*Connection refused.*"));
}
} finally {
miniCluster.restartDeadMasters();
}
// Ensure there is no log spew due to an unexpected lost connection.
String logText = cla.getAppendedText();
assertFalse("Should not claim to have lost a connection in the log",
logText.contains("lost connection to peer"));
assertFalse("Should not have netty spew in log",
logText.contains("socket.nio.AbstractNioSelector"));
}

@Test(timeout = 100000)
public void testCustomNioExecutor() throws Exception {
long startTime = System.nanoTime();
Expand Down

0 comments on commit ead7568

Please sign in to comment.