Merge pull request #14827 from MaineC/bug-fix/10021-error-listeners
Adds exception objects to log messages.
This commit is contained in:
commit
473b19400f
|
@ -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.
|
||||
*
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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)));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
*
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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<>();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue