Merge pull request #14827 from MaineC/bug-fix/10021-error-listeners

Adds exception objects to log messages.
This commit is contained in:
Isabel Drost-Fromm 2015-11-25 10:42:38 +01:00
commit 473b19400f
19 changed files with 78 additions and 35 deletions

View File

@ -51,9 +51,13 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
private static final Map<Class<? extends ElasticsearchException>, ElasticsearchExceptionHandle> CLASS_TO_ELASTICSEARCH_EXCEPTION_HANDLE;
private final Map<String, List<String>> headers = new HashMap<>();
/**
* Construct a <code>ElasticsearchException</code> with the specified cause exception.
*/
public ElasticsearchException(Throwable cause) {
super(cause);
}
/**
* Construct a <code>ElasticsearchException</code> with the specified detail message.
*

View File

@ -191,7 +191,7 @@ class JNANatives {
if (logger.isDebugEnabled()) {
logger.debug("unable to install syscall filter", t);
}
logger.warn("unable to install syscall filter: " + t.getMessage());
logger.warn("unable to install syscall filter: ", t);
}
}
}

View File

@ -231,7 +231,7 @@ public class MetaDataIndexUpgradeService extends AbstractComponent {
}
} catch (Exception ex) {
// Wrap the inner exception so we have the index name in the exception message
throw new IllegalStateException("unable to upgrade the mappings for the index [" + indexMetaData.getIndex() + "], reason: [" + ex.getMessage() + "]", ex);
throw new IllegalStateException("unable to upgrade the mappings for the index [" + indexMetaData.getIndex() + "]", ex);
}
}

View File

@ -643,7 +643,8 @@ public class Base64 {
try {
encoded = encodeBytes(source, 0, source.length, NO_OPTIONS);
} catch (java.io.IOException ex) {
assert false : ex.getMessage();
// not sure why this was an assertion before, running with assertions disabled would mean swallowing this exception
throw new IllegalStateException(ex);
} // end catch
assert encoded != null;
return encoded;
@ -705,7 +706,7 @@ public class Base64 {
try {
encoded = encodeBytes(source, off, len, NO_OPTIONS);
} catch (java.io.IOException ex) {
assert false : ex.getMessage();
throw new IllegalStateException(ex);
} // end catch
assert encoded != null;
return encoded;
@ -766,7 +767,7 @@ public class Base64 {
try {
encoded = encodeBytesToBytes(source, 0, source.length, Base64.NO_OPTIONS);
} catch (java.io.IOException ex) {
assert false : "IOExceptions only come from GZipping, which is turned off: " + ex.getMessage();
throw new IllegalStateException("IOExceptions only come from GZipping, which is turned off: ", ex);
}
return encoded;
}

View File

@ -331,6 +331,6 @@ public abstract class Multibinder<T> {
NullPointerException npe = new NullPointerException(name);
throw new ConfigurationException(singleton(
new Message(emptyList(), npe.toString(), npe)));
new Message(emptyList(), npe)));
}
}

View File

@ -58,6 +58,10 @@ public final class Message implements Serializable, Element {
this(Collections.singletonList(source), message, null);
}
public Message(Object source, Throwable cause) {
this(Collections.singletonList(source), null, cause);
}
public Message(String message) {
this(Collections.emptyList(), message, null);
}

View File

@ -525,7 +525,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
}
});
} else if (node.equals(nodes().masterNode())) {
handleMasterGone(node, "shut_down");
handleMasterGone(node, null, "shut_down");
}
}
@ -615,7 +615,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
});
}
private void handleMasterGone(final DiscoveryNode masterNode, final String reason) {
private void handleMasterGone(final DiscoveryNode masterNode, final Throwable cause, final String reason) {
if (lifecycleState() != Lifecycle.State.STARTED) {
// not started, ignore a master failure
return;
@ -625,7 +625,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
return;
}
logger.info("master_left [{}], reason [{}]", masterNode, reason);
logger.info("master_left [{}], reason [{}]", cause, masterNode, reason);
clusterService.submitStateUpdateTask("zen-disco-master_failed (" + masterNode + ")", Priority.IMMEDIATE, new ClusterStateUpdateTask() {
@ -1078,8 +1078,8 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
private class MasterNodeFailureListener implements MasterFaultDetection.Listener {
@Override
public void onMasterFailure(DiscoveryNode masterNode, String reason) {
handleMasterGone(masterNode, reason);
public void onMasterFailure(DiscoveryNode masterNode, Throwable cause, String reason) {
handleMasterGone(masterNode, cause, reason);
}
}

View File

@ -49,7 +49,7 @@ public class MasterFaultDetection extends FaultDetection {
public static interface Listener {
/** called when pinging the master failed, like a timeout, transport disconnects etc */
void onMasterFailure(DiscoveryNode masterNode, String reason);
void onMasterFailure(DiscoveryNode masterNode, Throwable cause, String reason);
}
@ -117,7 +117,7 @@ public class MasterFaultDetection extends FaultDetection {
transportService.connectToNode(masterNode);
} catch (final Exception e) {
// notify master failure (which stops also) and bail..
notifyMasterFailure(masterNode, "failed to perform initial connect [" + e.getMessage() + "]");
notifyMasterFailure(masterNode, e, "failed to perform initial connect ");
return;
}
if (masterPinger != null) {
@ -176,22 +176,22 @@ public class MasterFaultDetection extends FaultDetection {
threadPool.schedule(TimeValue.timeValueMillis(0), ThreadPool.Names.SAME, masterPinger);
} catch (Exception e) {
logger.trace("[master] [{}] transport disconnected (with verified connect)", masterNode);
notifyMasterFailure(masterNode, "transport disconnected (with verified connect)");
notifyMasterFailure(masterNode, null, "transport disconnected (with verified connect)");
}
} else {
logger.trace("[master] [{}] transport disconnected", node);
notifyMasterFailure(node, "transport disconnected");
notifyMasterFailure(node, null, "transport disconnected");
}
}
}
private void notifyMasterFailure(final DiscoveryNode masterNode, final String reason) {
private void notifyMasterFailure(final DiscoveryNode masterNode, final Throwable cause, final String reason) {
if (notifiedMasterFailure.compareAndSet(false, true)) {
threadPool.generic().execute(new Runnable() {
@Override
public void run() {
for (Listener listener : listeners) {
listener.onMasterFailure(masterNode, reason);
listener.onMasterFailure(masterNode, cause, reason);
}
}
});
@ -255,15 +255,15 @@ public class MasterFaultDetection extends FaultDetection {
return;
} else if (exp.getCause() instanceof NotMasterException) {
logger.debug("[master] pinging a master {} that is no longer a master", masterNode);
notifyMasterFailure(masterToPing, "no longer master");
notifyMasterFailure(masterToPing, exp, "no longer master");
return;
} else if (exp.getCause() instanceof ThisIsNotTheMasterYouAreLookingForException) {
logger.debug("[master] pinging a master {} that is not the master", masterNode);
notifyMasterFailure(masterToPing, "not master");
notifyMasterFailure(masterToPing, exp,"not master");
return;
} else if (exp.getCause() instanceof NodeDoesNotExistOnMasterException) {
logger.debug("[master] pinging a master {} but we do not exists on it, act as if its master failure", masterNode);
notifyMasterFailure(masterToPing, "do not exists on master, act as master failure");
notifyMasterFailure(masterToPing, exp,"do not exists on master, act as master failure");
return;
}
@ -272,7 +272,7 @@ public class MasterFaultDetection extends FaultDetection {
if (retryCount >= pingRetryCount) {
logger.debug("[master] failed to ping [{}], tried [{}] times, each with maximum [{}] timeout", masterNode, pingRetryCount, pingRetryTimeout);
// not good, failure
notifyMasterFailure(masterToPing, "failed to ping, tried [" + pingRetryCount + "] times, each with maximum [" + pingRetryTimeout + "] timeout");
notifyMasterFailure(masterToPing, null, "failed to ping, tried [" + pingRetryCount + "] times, each with maximum [" + pingRetryTimeout + "] timeout");
} else {
// resend the request, not reschedule, rely on send timeout
transportService.sendRequest(masterToPing, MASTER_PING_ACTION_NAME, request, options, this);

View File

@ -140,9 +140,9 @@ public class PublishClusterStateAction extends AbstractComponent {
throw t;
} catch (Throwable t) {
// try to fail committing, in cause it's still on going
if (sendingController.markAsFailed("unexpected error [" + t.getMessage() + "]")) {
if (sendingController.markAsFailed("unexpected error", t)) {
// signal the change should be rejected
throw new Discovery.FailedToCommitClusterStateException("unexpected error [{}]", t, t.getMessage());
throw new Discovery.FailedToCommitClusterStateException("unexpected error", t);
} else {
throw t;
}
@ -583,6 +583,21 @@ public class PublishClusterStateAction extends AbstractComponent {
return true;
}
/**
* tries marking the publishing as failed, if a decision wasn't made yet
*
* @return true if the publishing was failed and the cluster state is *not* committed
**/
synchronized private boolean markAsFailed(String details, Throwable reason) {
if (committedOrFailed()) {
return committed == false;
}
logger.trace("failed to commit version [{}]. {}", reason, clusterState.version(), details);
committed = false;
committedOrFailedLatch.countDown();
return true;
}
/**
* tries marking the publishing as failed, if a decision wasn't made yet
*

View File

@ -235,7 +235,7 @@ public class Analysis {
try (BufferedReader reader = FileSystemUtils.newBufferedReader(wordListFile.toUri().toURL(), StandardCharsets.UTF_8)) {
return loadWordList(reader, "#");
} catch (IOException ioe) {
String message = String.format(Locale.ROOT, "IOException while reading %s_path: %s", settingPrefix, ioe.getMessage());
String message = String.format(Locale.ROOT, "IOException while reading %s_path: %s", settingPrefix);
throw new IllegalArgumentException(message, ioe);
}
}
@ -282,7 +282,7 @@ public class Analysis {
try {
return FileSystemUtils.newBufferedReader(path.toUri().toURL(), StandardCharsets.UTF_8);
} catch (IOException ioe) {
String message = String.format(Locale.ROOT, "IOException while reading %s_path: %s", settingPrefix, ioe.getMessage());
String message = String.format(Locale.ROOT, "IOException while reading %s_path: %s", settingPrefix);
throw new IllegalArgumentException(message, ioe);
}
}

View File

@ -67,7 +67,7 @@ public class TranslogRecoveryPerformer {
numOps++;
}
} catch (Throwable t) {
throw new BatchOperationException(shardId, "failed to apply batch translog operation [" + t.getMessage() + "]", numOps, t);
throw new BatchOperationException(shardId, "failed to apply batch translog operation", numOps, t);
}
return numOps;
}

View File

@ -33,6 +33,11 @@ public class IndexShardSnapshotException extends ElasticsearchException {
this(shardId, msg, null);
}
public IndexShardSnapshotException(ShardId shardId, Throwable cause) {
super(cause);
setShard(shardId);
}
public IndexShardSnapshotException(ShardId shardId, String msg, Throwable cause) {
super(msg, cause);
setShard(shardId);

View File

@ -32,6 +32,10 @@ public class IndexShardSnapshotFailedException extends IndexShardSnapshotExcepti
super(shardId, msg);
}
public IndexShardSnapshotFailedException(ShardId shardId, Throwable cause) {
super(shardId, cause);
}
public IndexShardSnapshotFailedException(ShardId shardId, String msg, Throwable cause) {
super(shardId, msg, cause);
}

View File

@ -191,7 +191,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
if (e instanceof IndexShardSnapshotFailedException) {
throw (IndexShardSnapshotFailedException) e;
} else {
throw new IndexShardSnapshotFailedException(shardId, e.getMessage(), e);
throw new IndexShardSnapshotFailedException(shardId, e);
}
}
}
@ -373,7 +373,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
} catch (IOException e) {
// We cannot delete index file - this is fatal, we cannot continue, otherwise we might end up
// with references to non-existing files
throw new IndexShardSnapshotFailedException(shardId, "error deleting index files during cleanup, reason: " + e.getMessage(), e);
throw new IndexShardSnapshotFailedException(shardId, "error deleting index files during cleanup", e);
}
blobsToDelete = new ArrayList<>();

View File

@ -22,6 +22,7 @@ package org.elasticsearch.indices.recovery;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
@ -175,7 +176,7 @@ public class RecoveryStatus extends AbstractRefCounted {
listener.onRecoveryFailure(state(), e, sendShardFailure);
} finally {
try {
cancellableThreads.cancel("failed recovery [" + e.getMessage() + "]");
cancellableThreads.cancel("failed recovery [" + ExceptionsHelper.stackTrace(e) + "]");
} finally {
// release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now
decRef();

View File

@ -130,8 +130,17 @@ public class RecoveryTarget extends AbstractComponent implements IndexEventListe
}
protected void retryRecovery(final RecoveryStatus recoveryStatus, final Throwable reason, TimeValue retryAfter, final StartRecoveryRequest currentRequest) {
logger.trace("will retry recovery with id [{}] in [{}]", reason, recoveryStatus.recoveryId(), retryAfter);
retryRecovery(recoveryStatus, retryAfter, currentRequest);
}
protected void retryRecovery(final RecoveryStatus recoveryStatus, final String reason, TimeValue retryAfter, final StartRecoveryRequest currentRequest) {
logger.trace("will retrying recovery with id [{}] in [{}] (reason [{}])", recoveryStatus.recoveryId(), retryAfter, reason);
logger.trace("will retry recovery with id [{}] in [{}] (reason [{}])", recoveryStatus.recoveryId(), retryAfter, reason);
retryRecovery(recoveryStatus, retryAfter, currentRequest);
}
private void retryRecovery(final RecoveryStatus recoveryStatus, TimeValue retryAfter, final StartRecoveryRequest currentRequest) {
try {
recoveryStatus.resetRecovery();
} catch (Throwable e) {
@ -224,7 +233,7 @@ public class RecoveryTarget extends AbstractComponent implements IndexEventListe
}
if (cause instanceof DelayRecoveryException) {
retryRecovery(recoveryStatus, cause.getMessage(), recoverySettings.retryDelayStateSync(), request);
retryRecovery(recoveryStatus, cause, recoverySettings.retryDelayStateSync(), request);
return;
}

View File

@ -69,7 +69,7 @@ public class SharedFSRecoverySourceHandler extends RecoverySourceHandler {
// create a new IndexWriter
logger.info("recovery failed for primary shadow shard, failing shard");
// pass the failure as null, as we want to ensure the store is not marked as corrupted
shard.failShard("primary relocation failed on shared filesystem caused by: [" + t.getMessage() + "]", null);
shard.failShard("primary relocation failed on shared filesystem", t);
} else {
logger.info("recovery failed on shared filesystem", t);
}

View File

@ -484,7 +484,7 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
}
protected void traceResponseSent(long requestId, String action, Throwable t) {
tracerLog.trace("[{}][{}] sent error response (error: [{}])", requestId, action, t.getMessage());
tracerLog.trace("[{}][{}] sent error response", t, requestId, action);
}
@Override

View File

@ -193,7 +193,7 @@ public class ZenFaultDetectionTests extends ESTestCase {
masterFD.addListener(new MasterFaultDetection.Listener() {
@Override
public void onMasterFailure(DiscoveryNode masterNode, String reason) {
public void onMasterFailure(DiscoveryNode masterNode, Throwable cause, String reason) {
failureNode[0] = masterNode;
failureReason[0] = reason;
notified.countDown();
@ -211,4 +211,4 @@ public class ZenFaultDetectionTests extends ESTestCase {
assertThat(failureReason[0], matcher);
}
}
}