Merge pull request #13796 from s1monw/remove_reflection_hacks_from_es_exception

Remove reflection hacks from ElasticsearchException
This commit is contained in:
Jason Tedor 2015-09-25 15:02:48 -04:00
commit 65577816de
2 changed files with 351 additions and 182 deletions

View File

@ -30,9 +30,8 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.util.*;
import java.util.stream.Collectors;
/**
* A base class for all elasticsearch exceptions.
@ -48,8 +47,8 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
private static final String RESOURCE_HEADER_TYPE_KEY = "es.resource.type";
private static final String RESOURCE_HEADER_ID_KEY = "es.resource.id";
private static final Constructor<? extends ElasticsearchException>[] ID_TO_SUPPLIER;
private static final Map<Class<? extends ElasticsearchException>, Integer> CLASS_TO_ID;
private static final Map<Integer, FunctionThatThrowsIOException<StreamInput, ? extends ElasticsearchException>> ID_TO_SUPPLIER;
private static final Map<Class<? extends ElasticsearchException>, ElasticsearchExceptionHandle> CLASS_TO_ELASTICSEARCH_EXCEPTION_HANDLE;
private final Map<String, List<String>> headers = new HashMap<>();
/**
@ -232,33 +231,29 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
}
public static ElasticsearchException readException(StreamInput input, int id) throws IOException {
Constructor<? extends ElasticsearchException> elasticsearchException = ID_TO_SUPPLIER[id];
FunctionThatThrowsIOException<StreamInput, ? extends ElasticsearchException> elasticsearchException = ID_TO_SUPPLIER.get(id);
if (elasticsearchException == null) {
throw new IllegalStateException("unknown exception for id: " + id);
}
try {
return elasticsearchException.newInstance(input);
} catch (InstantiationException|IllegalAccessException|InvocationTargetException e) {
throw new IOException("failed to read exception for id [" + id + "]", e);
}
return elasticsearchException.apply(input);
}
/**
* Retruns <code>true</code> iff the given class is a registered for an exception to be read.
*/
public static boolean isRegistered(Class<? extends Throwable> exception) {
return CLASS_TO_ID.containsKey(exception);
return CLASS_TO_ELASTICSEARCH_EXCEPTION_HANDLE.containsKey(exception);
}
static Set<Class<? extends ElasticsearchException>> getRegisteredKeys() { // for testing
return CLASS_TO_ID.keySet();
return CLASS_TO_ELASTICSEARCH_EXCEPTION_HANDLE.keySet();
}
/**
* Returns the serialization id the given exception.
*/
public static int getId(Class<? extends ElasticsearchException> exception) {
return CLASS_TO_ID.get(exception).intValue();
return CLASS_TO_ELASTICSEARCH_EXCEPTION_HANDLE.get(exception).id;
}
@Override
@ -458,179 +453,171 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
return throwable;
}
static {
// each exception gets an ordinal assigned that must never change. While the exception name can
enum ElasticsearchExceptionHandle {
// each exception gets an assigned id that must never change. While the exception name can
// change due to refactorings etc. like renaming we have to keep the ordinal <--> class mapping
// to deserialize the exception coming from another node or from an corruption marker on
// a corrupted index.
// NOTE: ONLY APPEND TO THE END and NEVER REMOVE EXCEPTIONS IN MINOR VERSIONS
final Map<Class<? extends ElasticsearchException>, Integer> exceptions = new HashMap<>();
exceptions.put(org.elasticsearch.index.snapshots.IndexShardSnapshotFailedException.class, exceptions.size());
exceptions.put(org.elasticsearch.search.dfs.DfsPhaseExecutionException.class, exceptions.size());
exceptions.put(org.elasticsearch.common.util.CancellableThreads.ExecutionCancelledException.class, exceptions.size());
exceptions.put(org.elasticsearch.discovery.MasterNotDiscoveredException.class, exceptions.size());
exceptions.put(org.elasticsearch.ElasticsearchSecurityException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.snapshots.IndexShardRestoreException.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.IndexClosedException.class, exceptions.size());
exceptions.put(org.elasticsearch.http.BindHttpException.class, exceptions.size());
exceptions.put(org.elasticsearch.action.search.ReduceSearchPhaseException.class, exceptions.size());
exceptions.put(org.elasticsearch.node.NodeClosedException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.SnapshotFailedEngineException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.shard.ShardNotFoundException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.ConnectTransportException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.NotSerializableTransportException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.ResponseHandlerFailureTransportException.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.IndexCreationException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.IndexNotFoundException.class, exceptions.size());
exceptions.put(org.elasticsearch.cluster.routing.IllegalShardRoutingStateException.class, exceptions.size());
exceptions.put(org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException.class, exceptions.size());
exceptions.put(org.elasticsearch.ResourceNotFoundException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.ActionTransportException.class, exceptions.size());
exceptions.put(org.elasticsearch.ElasticsearchGenerationException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.CreateFailedEngineException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.shard.IndexShardStartedException.class, exceptions.size());
exceptions.put(org.elasticsearch.search.SearchContextMissingException.class, exceptions.size());
exceptions.put(org.elasticsearch.script.ScriptException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.shard.TranslogRecoveryPerformer.BatchOperationException.class, exceptions.size());
exceptions.put(org.elasticsearch.snapshots.SnapshotCreationException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.DeleteFailedEngineException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.DocumentMissingException.class, exceptions.size());
exceptions.put(org.elasticsearch.snapshots.SnapshotException.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.InvalidAliasNameException.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.InvalidIndexNameException.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.IndexPrimaryShardNotAllocatedException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.TransportException.class, exceptions.size());
exceptions.put(org.elasticsearch.ElasticsearchParseException.class, exceptions.size());
exceptions.put(org.elasticsearch.search.SearchException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.mapper.MapperException.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.InvalidTypeNameException.class, exceptions.size());
exceptions.put(org.elasticsearch.snapshots.SnapshotRestoreException.class, exceptions.size());
exceptions.put(org.elasticsearch.common.ParsingException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.shard.IndexShardClosedException.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.recovery.RecoverFilesRecoveryException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.translog.TruncatedTranslogException.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.recovery.RecoveryFailedException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.shard.IndexShardRelocatedException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.NodeShouldNotConnectException.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.IndexTemplateAlreadyExistsException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.translog.TranslogCorruptedException.class, exceptions.size());
exceptions.put(org.elasticsearch.cluster.block.ClusterBlockException.class, exceptions.size());
exceptions.put(org.elasticsearch.search.fetch.FetchPhaseExecutionException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.IndexShardAlreadyExistsException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.VersionConflictEngineException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.EngineException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.DocumentAlreadyExistsException.class, exceptions.size());
exceptions.put(org.elasticsearch.action.NoSuchNodeException.class, exceptions.size());
exceptions.put(org.elasticsearch.common.settings.SettingsException.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.IndexTemplateMissingException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.SendRequestTransportException.class, exceptions.size());
exceptions.put(org.elasticsearch.common.util.concurrent.EsRejectedExecutionException.class, exceptions.size());
exceptions.put(org.elasticsearch.common.lucene.Lucene.EarlyTerminationException.class, exceptions.size());
exceptions.put(org.elasticsearch.cluster.routing.RoutingValidationException.class, exceptions.size());
exceptions.put(org.elasticsearch.common.io.stream.NotSerializableExceptionWrapper.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.AliasFilterParsingException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.DeleteByQueryFailedEngineException.class, exceptions.size());
exceptions.put(org.elasticsearch.gateway.GatewayException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.shard.IndexShardNotRecoveringException.class, exceptions.size());
exceptions.put(org.elasticsearch.http.HttpException.class, exceptions.size());
exceptions.put(org.elasticsearch.ElasticsearchException.class, exceptions.size());
exceptions.put(org.elasticsearch.snapshots.SnapshotMissingException.class, exceptions.size());
exceptions.put(org.elasticsearch.action.PrimaryMissingActionException.class, exceptions.size());
exceptions.put(org.elasticsearch.action.FailedNodeException.class, exceptions.size());
exceptions.put(org.elasticsearch.search.SearchParseException.class, exceptions.size());
exceptions.put(org.elasticsearch.snapshots.ConcurrentSnapshotExecutionException.class, exceptions.size());
exceptions.put(org.elasticsearch.common.blobstore.BlobStoreException.class, exceptions.size());
exceptions.put(org.elasticsearch.cluster.IncompatibleClusterStateVersionException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.RecoveryEngineException.class, exceptions.size());
exceptions.put(org.elasticsearch.common.util.concurrent.UncategorizedExecutionException.class, exceptions.size());
exceptions.put(org.elasticsearch.action.TimestampParsingException.class, exceptions.size());
exceptions.put(org.elasticsearch.action.RoutingMissingException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.IndexFailedEngineException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.snapshots.IndexShardRestoreFailedException.class, exceptions.size());
exceptions.put(org.elasticsearch.repositories.RepositoryException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.ReceiveTimeoutTransportException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.NodeDisconnectedException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.AlreadyExpiredException.class, exceptions.size());
exceptions.put(org.elasticsearch.search.aggregations.AggregationExecutionException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.mapper.MergeMappingException.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.InvalidIndexTemplateException.class, exceptions.size());
exceptions.put(org.elasticsearch.percolator.PercolateException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.RefreshFailedEngineException.class, exceptions.size());
exceptions.put(org.elasticsearch.search.aggregations.AggregationInitializationException.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.recovery.DelayRecoveryException.class, exceptions.size());
exceptions.put(org.elasticsearch.search.warmer.IndexWarmerMissingException.class, exceptions.size());
exceptions.put(org.elasticsearch.client.transport.NoNodeAvailableException.class, exceptions.size());
exceptions.put(org.elasticsearch.script.groovy.GroovyScriptCompilationException.class, exceptions.size());
exceptions.put(org.elasticsearch.snapshots.InvalidSnapshotNameException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.shard.IllegalIndexShardStateException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.snapshots.IndexShardSnapshotException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.shard.IndexShardNotStartedException.class, exceptions.size());
exceptions.put(org.elasticsearch.action.search.SearchPhaseExecutionException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.ActionNotFoundTransportException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.TransportSerializationException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.RemoteTransportException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.EngineCreationFailureException.class, exceptions.size());
exceptions.put(org.elasticsearch.cluster.routing.RoutingException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.shard.IndexShardRecoveryException.class, exceptions.size());
exceptions.put(org.elasticsearch.repositories.RepositoryMissingException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.percolator.PercolatorException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.DocumentSourceMissingException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.FlushNotAllowedEngineException.class, exceptions.size());
exceptions.put(org.elasticsearch.common.settings.NoClassSettingsException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.BindTransportException.class, exceptions.size());
exceptions.put(org.elasticsearch.rest.action.admin.indices.alias.delete.AliasesNotFoundException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.shard.IndexShardRecoveringException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.translog.TranslogException.class, exceptions.size());
exceptions.put(org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException.class, exceptions.size());
exceptions.put(org.elasticsearch.action.support.replication.TransportReplicationAction.RetryOnPrimaryException.class, exceptions.size());
exceptions.put(org.elasticsearch.ElasticsearchTimeoutException.class, exceptions.size());
exceptions.put(org.elasticsearch.search.query.QueryPhaseExecutionException.class, exceptions.size());
exceptions.put(org.elasticsearch.repositories.RepositoryVerificationException.class, exceptions.size());
exceptions.put(org.elasticsearch.search.aggregations.InvalidAggregationPathException.class, exceptions.size());
exceptions.put(org.elasticsearch.script.groovy.GroovyScriptExecutionException.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.IndexAlreadyExistsException.class, exceptions.size());
exceptions.put(org.elasticsearch.script.Script.ScriptParseException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.netty.SizeHeaderFrameDecoder.HttpOnTransportException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.mapper.MapperParsingException.class, exceptions.size());
exceptions.put(org.elasticsearch.search.SearchContextException.class, exceptions.size());
exceptions.put(org.elasticsearch.search.builder.SearchSourceBuilderException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.EngineClosedException.class, exceptions.size());
exceptions.put(org.elasticsearch.action.NoShardAvailableActionException.class, exceptions.size());
exceptions.put(org.elasticsearch.action.UnavailableShardsException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.engine.FlushFailedEngineException.class, exceptions.size());
exceptions.put(org.elasticsearch.common.breaker.CircuitBreakingException.class, exceptions.size());
exceptions.put(org.elasticsearch.transport.NodeNotConnectedException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.mapper.StrictDynamicMappingException.class, exceptions.size());
exceptions.put(org.elasticsearch.action.support.replication.TransportReplicationAction.RetryOnReplicaException.class, exceptions.size());
exceptions.put(org.elasticsearch.indices.TypeMissingException.class, exceptions.size());
// added in 3.x
exceptions.put(org.elasticsearch.discovery.Discovery.FailedToCommitClusterStateException.class, exceptions.size());
exceptions.put(org.elasticsearch.index.query.QueryShardException.class, exceptions.size());
// NOTE: ONLY APPEND TO THE END and NEVER REMOVE EXCEPTIONS IN MINOR VERSIONS
Constructor<? extends ElasticsearchException>[] idToSupplier = new Constructor[exceptions.size()];
for (Map.Entry<Class<? extends ElasticsearchException>, Integer> e : exceptions.entrySet()) {
try {
Constructor<? extends ElasticsearchException> constructor = e.getKey().getDeclaredConstructor(StreamInput.class);
if (constructor == null) {
throw new IllegalStateException(e.getKey().getName() + " has not StreamInput ctor");
}
assert e.getValue().intValue() >= 0;
if (idToSupplier[e.getValue().intValue()] != null) {
throw new IllegalStateException("ordinal [" + e.getValue().intValue() +"] is used more than once");
}
idToSupplier[e.getValue().intValue()] = constructor;
} catch (NoSuchMethodException t) {
throw new RuntimeException("failed to register [" + e.getKey().getName() + "] exception must have a public StreamInput ctor", t);
}
}
for (int i = 0; i < idToSupplier.length; i++) {
if (idToSupplier[i] == null) {
throw new IllegalStateException("missing exception for ordinal [" + i + "]");
// these exceptions can be ordered and removed, but (repeating) the ids must never change
// to remove an exception, remove the enum value below, and mark the id as null in ExceptionSerializationTests.testIds.ids
INDEX_SHARD_SNAPSHOT_FAILED_EXCEPTION(org.elasticsearch.index.snapshots.IndexShardSnapshotFailedException.class, org.elasticsearch.index.snapshots.IndexShardSnapshotFailedException::new, 0),
DFS_PHASE_EXECUTION_EXCEPTION(org.elasticsearch.search.dfs.DfsPhaseExecutionException.class, org.elasticsearch.search.dfs.DfsPhaseExecutionException::new, 1),
EXECUTION_CANCELLED_EXCEPTION(org.elasticsearch.common.util.CancellableThreads.ExecutionCancelledException.class, org.elasticsearch.common.util.CancellableThreads.ExecutionCancelledException::new, 2),
MASTER_NOT_DISCOVERED_EXCEPTION(org.elasticsearch.discovery.MasterNotDiscoveredException.class, org.elasticsearch.discovery.MasterNotDiscoveredException::new, 3),
ELASTICSEARCH_SECURITY_EXCEPTION(org.elasticsearch.ElasticsearchSecurityException.class, org.elasticsearch.ElasticsearchSecurityException::new, 4),
INDEX_SHARD_RESTORE_EXCEPTION(org.elasticsearch.index.snapshots.IndexShardRestoreException.class, org.elasticsearch.index.snapshots.IndexShardRestoreException::new, 5),
INDEX_CLOSED_EXCEPTION(org.elasticsearch.indices.IndexClosedException.class, org.elasticsearch.indices.IndexClosedException::new, 6),
BIND_HTTP_EXCEPTION(org.elasticsearch.http.BindHttpException.class, org.elasticsearch.http.BindHttpException::new, 7),
REDUCE_SEARCH_PHASE_EXCEPTION(org.elasticsearch.action.search.ReduceSearchPhaseException.class, org.elasticsearch.action.search.ReduceSearchPhaseException::new, 8),
NODE_CLOSED_EXCEPTION(org.elasticsearch.node.NodeClosedException.class, org.elasticsearch.node.NodeClosedException::new, 9),
SNAPSHOT_FAILED_ENGINE_EXCEPTION(org.elasticsearch.index.engine.SnapshotFailedEngineException.class, org.elasticsearch.index.engine.SnapshotFailedEngineException::new, 10),
SHARD_NOT_FOUND_EXCEPTION(org.elasticsearch.index.shard.ShardNotFoundException.class, org.elasticsearch.index.shard.ShardNotFoundException::new, 11),
CONNECT_TRANSPORT_EXCEPTION(org.elasticsearch.transport.ConnectTransportException.class, org.elasticsearch.transport.ConnectTransportException::new, 12),
NOT_SERIALIZABLE_TRANSPORT_EXCEPTION(org.elasticsearch.transport.NotSerializableTransportException.class, org.elasticsearch.transport.NotSerializableTransportException::new, 13),
RESPONSE_HANDLER_FAILURE_TRANSPORT_EXCEPTION(org.elasticsearch.transport.ResponseHandlerFailureTransportException.class, org.elasticsearch.transport.ResponseHandlerFailureTransportException::new, 14),
INDEX_CREATION_EXCEPTION(org.elasticsearch.indices.IndexCreationException.class, org.elasticsearch.indices.IndexCreationException::new, 15),
INDEX_NOT_FOUND_EXCEPTION(org.elasticsearch.index.IndexNotFoundException.class, org.elasticsearch.index.IndexNotFoundException::new, 16),
ILLEGAL_SHARD_ROUTING_STATE_EXCEPTION(org.elasticsearch.cluster.routing.IllegalShardRoutingStateException.class, org.elasticsearch.cluster.routing.IllegalShardRoutingStateException::new, 17),
BROADCAST_SHARD_OPERATION_FAILED_EXCEPTION(org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException.class, org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException::new, 18),
RESOURCE_NOT_FOUND_EXCEPTION(org.elasticsearch.ResourceNotFoundException.class, org.elasticsearch.ResourceNotFoundException::new, 19),
ACTION_TRANSPORT_EXCEPTION(org.elasticsearch.transport.ActionTransportException.class, org.elasticsearch.transport.ActionTransportException::new, 20),
ELASTICSEARCH_GENERATION_EXCEPTION(org.elasticsearch.ElasticsearchGenerationException.class, org.elasticsearch.ElasticsearchGenerationException::new, 21),
CREATE_FAILED_ENGINE_EXCEPTION(org.elasticsearch.index.engine.CreateFailedEngineException.class, org.elasticsearch.index.engine.CreateFailedEngineException::new, 22),
INDEX_SHARD_STARTED_EXCEPTION(org.elasticsearch.index.shard.IndexShardStartedException.class, org.elasticsearch.index.shard.IndexShardStartedException::new, 23),
SEARCH_CONTEXT_MISSING_EXCEPTION(org.elasticsearch.search.SearchContextMissingException.class, org.elasticsearch.search.SearchContextMissingException::new, 24),
SCRIPT_EXCEPTION(org.elasticsearch.script.ScriptException.class, org.elasticsearch.script.ScriptException::new, 25),
BATCH_OPERATION_EXCEPTION(org.elasticsearch.index.shard.TranslogRecoveryPerformer.BatchOperationException.class, org.elasticsearch.index.shard.TranslogRecoveryPerformer.BatchOperationException::new, 26),
SNAPSHOT_CREATION_EXCEPTION(org.elasticsearch.snapshots.SnapshotCreationException.class, org.elasticsearch.snapshots.SnapshotCreationException::new, 27),
DELETE_FAILED_ENGINE_EXCEPTION(org.elasticsearch.index.engine.DeleteFailedEngineException.class, org.elasticsearch.index.engine.DeleteFailedEngineException::new, 28),
DOCUMENT_MISSING_EXCEPTION(org.elasticsearch.index.engine.DocumentMissingException.class, org.elasticsearch.index.engine.DocumentMissingException::new, 29),
SNAPSHOT_EXCEPTION(org.elasticsearch.snapshots.SnapshotException.class, org.elasticsearch.snapshots.SnapshotException::new, 30),
INVALID_ALIAS_NAME_EXCEPTION(org.elasticsearch.indices.InvalidAliasNameException.class, org.elasticsearch.indices.InvalidAliasNameException::new, 31),
INVALID_INDEX_NAME_EXCEPTION(org.elasticsearch.indices.InvalidIndexNameException.class, org.elasticsearch.indices.InvalidIndexNameException::new, 32),
INDEX_PRIMARY_SHARD_NOT_ALLOCATED_EXCEPTION(org.elasticsearch.indices.IndexPrimaryShardNotAllocatedException.class, org.elasticsearch.indices.IndexPrimaryShardNotAllocatedException::new, 33),
TRANSPORT_EXCEPTION(org.elasticsearch.transport.TransportException.class, org.elasticsearch.transport.TransportException::new, 34),
ELASTICSEARCH_PARSE_EXCEPTION(org.elasticsearch.ElasticsearchParseException.class, org.elasticsearch.ElasticsearchParseException::new, 35),
SEARCH_EXCEPTION(org.elasticsearch.search.SearchException.class, org.elasticsearch.search.SearchException::new, 36),
MAPPER_EXCEPTION(org.elasticsearch.index.mapper.MapperException.class, org.elasticsearch.index.mapper.MapperException::new, 37),
INVALID_TYPE_NAME_EXCEPTION(org.elasticsearch.indices.InvalidTypeNameException.class, org.elasticsearch.indices.InvalidTypeNameException::new, 38),
SNAPSHOT_RESTORE_EXCEPTION(org.elasticsearch.snapshots.SnapshotRestoreException.class, org.elasticsearch.snapshots.SnapshotRestoreException::new, 39),
PARSING_EXCEPTION(org.elasticsearch.common.ParsingException.class, org.elasticsearch.common.ParsingException::new, 40),
INDEX_SHARD_CLOSED_EXCEPTION(org.elasticsearch.index.shard.IndexShardClosedException.class, org.elasticsearch.index.shard.IndexShardClosedException::new, 41),
RECOVER_FILES_RECOVERY_EXCEPTION(org.elasticsearch.indices.recovery.RecoverFilesRecoveryException.class, org.elasticsearch.indices.recovery.RecoverFilesRecoveryException::new, 42),
TRUNCATED_TRANSLOG_EXCEPTION(org.elasticsearch.index.translog.TruncatedTranslogException.class, org.elasticsearch.index.translog.TruncatedTranslogException::new, 43),
RECOVERY_FAILED_EXCEPTION(org.elasticsearch.indices.recovery.RecoveryFailedException.class, org.elasticsearch.indices.recovery.RecoveryFailedException::new, 44),
INDEX_SHARD_RELOCATED_EXCEPTION(org.elasticsearch.index.shard.IndexShardRelocatedException.class, org.elasticsearch.index.shard.IndexShardRelocatedException::new, 45),
NODE_SHOULD_NOT_CONNECT_EXCEPTION(org.elasticsearch.transport.NodeShouldNotConnectException.class, org.elasticsearch.transport.NodeShouldNotConnectException::new, 46),
INDEX_TEMPLATE_ALREADY_EXISTS_EXCEPTION(org.elasticsearch.indices.IndexTemplateAlreadyExistsException.class, org.elasticsearch.indices.IndexTemplateAlreadyExistsException::new, 47),
TRANSLOG_CORRUPTED_EXCEPTION(org.elasticsearch.index.translog.TranslogCorruptedException.class, org.elasticsearch.index.translog.TranslogCorruptedException::new, 48),
CLUSTER_BLOCK_EXCEPTION(org.elasticsearch.cluster.block.ClusterBlockException.class, org.elasticsearch.cluster.block.ClusterBlockException::new, 49),
FETCH_PHASE_EXECUTION_EXCEPTION(org.elasticsearch.search.fetch.FetchPhaseExecutionException.class, org.elasticsearch.search.fetch.FetchPhaseExecutionException::new, 50),
INDEX_SHARD_ALREADY_EXISTS_EXCEPTION(org.elasticsearch.index.IndexShardAlreadyExistsException.class, org.elasticsearch.index.IndexShardAlreadyExistsException::new, 51),
VERSION_CONFLICT_ENGINE_EXCEPTION(org.elasticsearch.index.engine.VersionConflictEngineException.class, org.elasticsearch.index.engine.VersionConflictEngineException::new, 52),
ENGINE_EXCEPTION(org.elasticsearch.index.engine.EngineException.class, org.elasticsearch.index.engine.EngineException::new, 53),
DOCUMENT_ALREADY_EXISTS_EXCEPTION(org.elasticsearch.index.engine.DocumentAlreadyExistsException.class, org.elasticsearch.index.engine.DocumentAlreadyExistsException::new, 54),
NO_SUCH_NODE_EXCEPTION(org.elasticsearch.action.NoSuchNodeException.class, org.elasticsearch.action.NoSuchNodeException::new, 55),
SETTINGS_EXCEPTION(org.elasticsearch.common.settings.SettingsException.class, org.elasticsearch.common.settings.SettingsException::new, 56),
INDEX_TEMPLATE_MISSING_EXCEPTION(org.elasticsearch.indices.IndexTemplateMissingException.class, org.elasticsearch.indices.IndexTemplateMissingException::new, 57),
SEND_REQUEST_TRANSPORT_EXCEPTION(org.elasticsearch.transport.SendRequestTransportException.class, org.elasticsearch.transport.SendRequestTransportException::new, 58),
ES_REJECTED_EXECUTION_EXCEPTION(org.elasticsearch.common.util.concurrent.EsRejectedExecutionException.class, org.elasticsearch.common.util.concurrent.EsRejectedExecutionException::new, 59),
EARLY_TERMINATION_EXCEPTION(org.elasticsearch.common.lucene.Lucene.EarlyTerminationException.class, org.elasticsearch.common.lucene.Lucene.EarlyTerminationException::new, 60),
ROUTING_VALIDATION_EXCEPTION(org.elasticsearch.cluster.routing.RoutingValidationException.class, org.elasticsearch.cluster.routing.RoutingValidationException::new, 61),
NOT_SERIALIZABLE_EXCEPTION_WRAPPER(org.elasticsearch.common.io.stream.NotSerializableExceptionWrapper.class, org.elasticsearch.common.io.stream.NotSerializableExceptionWrapper::new, 62),
ALIAS_FILTER_PARSING_EXCEPTION(org.elasticsearch.indices.AliasFilterParsingException.class, org.elasticsearch.indices.AliasFilterParsingException::new, 63),
DELETE_BY_QUERY_FAILED_ENGINE_EXCEPTION(org.elasticsearch.index.engine.DeleteByQueryFailedEngineException.class, org.elasticsearch.index.engine.DeleteByQueryFailedEngineException::new, 64),
GATEWAY_EXCEPTION(org.elasticsearch.gateway.GatewayException.class, org.elasticsearch.gateway.GatewayException::new, 65),
INDEX_SHARD_NOT_RECOVERING_EXCEPTION(org.elasticsearch.index.shard.IndexShardNotRecoveringException.class, org.elasticsearch.index.shard.IndexShardNotRecoveringException::new, 66),
HTTP_EXCEPTION(org.elasticsearch.http.HttpException.class, org.elasticsearch.http.HttpException::new, 67),
ELASTICSEARCH_EXCEPTION(org.elasticsearch.ElasticsearchException.class, org.elasticsearch.ElasticsearchException::new, 68),
SNAPSHOT_MISSING_EXCEPTION(org.elasticsearch.snapshots.SnapshotMissingException.class, org.elasticsearch.snapshots.SnapshotMissingException::new, 69),
PRIMARY_MISSING_ACTION_EXCEPTION(org.elasticsearch.action.PrimaryMissingActionException.class, org.elasticsearch.action.PrimaryMissingActionException::new, 70),
FAILED_NODE_EXCEPTION(org.elasticsearch.action.FailedNodeException.class, org.elasticsearch.action.FailedNodeException::new, 71),
SEARCH_PARSE_EXCEPTION(org.elasticsearch.search.SearchParseException.class, org.elasticsearch.search.SearchParseException::new, 72),
CONCURRENT_SNAPSHOT_EXECUTION_EXCEPTION(org.elasticsearch.snapshots.ConcurrentSnapshotExecutionException.class, org.elasticsearch.snapshots.ConcurrentSnapshotExecutionException::new, 73),
BLOB_STORE_EXCEPTION(org.elasticsearch.common.blobstore.BlobStoreException.class, org.elasticsearch.common.blobstore.BlobStoreException::new, 74),
INCOMPATIBLE_CLUSTER_STATE_VERSION_EXCEPTION(org.elasticsearch.cluster.IncompatibleClusterStateVersionException.class, org.elasticsearch.cluster.IncompatibleClusterStateVersionException::new, 75),
RECOVERY_ENGINE_EXCEPTION(org.elasticsearch.index.engine.RecoveryEngineException.class, org.elasticsearch.index.engine.RecoveryEngineException::new, 76),
UNCATEGORIZED_EXECUTION_EXCEPTION(org.elasticsearch.common.util.concurrent.UncategorizedExecutionException.class, org.elasticsearch.common.util.concurrent.UncategorizedExecutionException::new, 77),
TIMESTAMP_PARSING_EXCEPTION(org.elasticsearch.action.TimestampParsingException.class, org.elasticsearch.action.TimestampParsingException::new, 78),
ROUTING_MISSING_EXCEPTION(org.elasticsearch.action.RoutingMissingException.class, org.elasticsearch.action.RoutingMissingException::new, 79),
INDEX_FAILED_ENGINE_EXCEPTION(org.elasticsearch.index.engine.IndexFailedEngineException.class, org.elasticsearch.index.engine.IndexFailedEngineException::new, 80),
INDEX_SHARD_RESTORE_FAILED_EXCEPTION(org.elasticsearch.index.snapshots.IndexShardRestoreFailedException.class, org.elasticsearch.index.snapshots.IndexShardRestoreFailedException::new, 81),
REPOSITORY_EXCEPTION(org.elasticsearch.repositories.RepositoryException.class, org.elasticsearch.repositories.RepositoryException::new, 82),
RECEIVE_TIMEOUT_TRANSPORT_EXCEPTION(org.elasticsearch.transport.ReceiveTimeoutTransportException.class, org.elasticsearch.transport.ReceiveTimeoutTransportException::new, 83),
NODE_DISCONNECTED_EXCEPTION(org.elasticsearch.transport.NodeDisconnectedException.class, org.elasticsearch.transport.NodeDisconnectedException::new, 84),
ALREADY_EXPIRED_EXCEPTION(org.elasticsearch.index.AlreadyExpiredException.class, org.elasticsearch.index.AlreadyExpiredException::new, 85),
AGGREGATION_EXECUTION_EXCEPTION(org.elasticsearch.search.aggregations.AggregationExecutionException.class, org.elasticsearch.search.aggregations.AggregationExecutionException::new, 86),
MERGE_MAPPING_EXCEPTION(org.elasticsearch.index.mapper.MergeMappingException.class, org.elasticsearch.index.mapper.MergeMappingException::new, 87),
INVALID_INDEX_TEMPLATE_EXCEPTION(org.elasticsearch.indices.InvalidIndexTemplateException.class, org.elasticsearch.indices.InvalidIndexTemplateException::new, 88),
PERCOLATE_EXCEPTION(org.elasticsearch.percolator.PercolateException.class, org.elasticsearch.percolator.PercolateException::new, 89),
REFRESH_FAILED_ENGINE_EXCEPTION(org.elasticsearch.index.engine.RefreshFailedEngineException.class, org.elasticsearch.index.engine.RefreshFailedEngineException::new, 90),
AGGREGATION_INITIALIZATION_EXCEPTION(org.elasticsearch.search.aggregations.AggregationInitializationException.class, org.elasticsearch.search.aggregations.AggregationInitializationException::new, 91),
DELAY_RECOVERY_EXCEPTION(org.elasticsearch.indices.recovery.DelayRecoveryException.class, org.elasticsearch.indices.recovery.DelayRecoveryException::new, 92),
INDEX_WARMER_MISSING_EXCEPTION(org.elasticsearch.search.warmer.IndexWarmerMissingException.class, org.elasticsearch.search.warmer.IndexWarmerMissingException::new, 93),
NO_NODE_AVAILABLE_EXCEPTION(org.elasticsearch.client.transport.NoNodeAvailableException.class, org.elasticsearch.client.transport.NoNodeAvailableException::new, 94),
GROOVY_SCRIPT_COMPILATION_EXCEPTION(org.elasticsearch.script.groovy.GroovyScriptCompilationException.class, org.elasticsearch.script.groovy.GroovyScriptCompilationException::new, 95),
INVALID_SNAPSHOT_NAME_EXCEPTION(org.elasticsearch.snapshots.InvalidSnapshotNameException.class, org.elasticsearch.snapshots.InvalidSnapshotNameException::new, 96),
ILLEGAL_INDEX_SHARD_STATE_EXCEPTION(org.elasticsearch.index.shard.IllegalIndexShardStateException.class, org.elasticsearch.index.shard.IllegalIndexShardStateException::new, 97),
INDEX_SHARD_SNAPSHOT_EXCEPTION(org.elasticsearch.index.snapshots.IndexShardSnapshotException.class, org.elasticsearch.index.snapshots.IndexShardSnapshotException::new, 98),
INDEX_SHARD_NOT_STARTED_EXCEPTION(org.elasticsearch.index.shard.IndexShardNotStartedException.class, org.elasticsearch.index.shard.IndexShardNotStartedException::new, 99),
SEARCH_PHASE_EXECUTION_EXCEPTION(org.elasticsearch.action.search.SearchPhaseExecutionException.class, org.elasticsearch.action.search.SearchPhaseExecutionException::new, 100),
ACTION_NOT_FOUND_TRANSPORT_EXCEPTION(org.elasticsearch.transport.ActionNotFoundTransportException.class, org.elasticsearch.transport.ActionNotFoundTransportException::new, 101),
TRANSPORT_SERIALIZATION_EXCEPTION(org.elasticsearch.transport.TransportSerializationException.class, org.elasticsearch.transport.TransportSerializationException::new, 102),
REMOTE_TRANSPORT_EXCEPTION(org.elasticsearch.transport.RemoteTransportException.class, org.elasticsearch.transport.RemoteTransportException::new, 103),
ENGINE_CREATION_FAILURE_EXCEPTION(org.elasticsearch.index.engine.EngineCreationFailureException.class, org.elasticsearch.index.engine.EngineCreationFailureException::new, 104),
ROUTING_EXCEPTION(org.elasticsearch.cluster.routing.RoutingException.class, org.elasticsearch.cluster.routing.RoutingException::new, 105),
INDEX_SHARD_RECOVERY_EXCEPTION(org.elasticsearch.index.shard.IndexShardRecoveryException.class, org.elasticsearch.index.shard.IndexShardRecoveryException::new, 106),
REPOSITORY_MISSING_EXCEPTION(org.elasticsearch.repositories.RepositoryMissingException.class, org.elasticsearch.repositories.RepositoryMissingException::new, 107),
PERCOLATOR_EXCEPTION(org.elasticsearch.index.percolator.PercolatorException.class, org.elasticsearch.index.percolator.PercolatorException::new, 108),
DOCUMENT_SOURCE_MISSING_EXCEPTION(org.elasticsearch.index.engine.DocumentSourceMissingException.class, org.elasticsearch.index.engine.DocumentSourceMissingException::new, 109),
FLUSH_NOT_ALLOWED_ENGINE_EXCEPTION(org.elasticsearch.index.engine.FlushNotAllowedEngineException.class, org.elasticsearch.index.engine.FlushNotAllowedEngineException::new, 110),
NO_CLASS_SETTINGS_EXCEPTION(org.elasticsearch.common.settings.NoClassSettingsException.class, org.elasticsearch.common.settings.NoClassSettingsException::new, 111),
BIND_TRANSPORT_EXCEPTION(org.elasticsearch.transport.BindTransportException.class, org.elasticsearch.transport.BindTransportException::new, 112),
ALIASES_NOT_FOUND_EXCEPTION(org.elasticsearch.rest.action.admin.indices.alias.delete.AliasesNotFoundException.class, org.elasticsearch.rest.action.admin.indices.alias.delete.AliasesNotFoundException::new, 113),
INDEX_SHARD_RECOVERING_EXCEPTION(org.elasticsearch.index.shard.IndexShardRecoveringException.class, org.elasticsearch.index.shard.IndexShardRecoveringException::new, 114),
TRANSLOG_EXCEPTION(org.elasticsearch.index.translog.TranslogException.class, org.elasticsearch.index.translog.TranslogException::new, 115),
PROCESS_CLUSTER_EVENT_TIMEOUT_EXCEPTION(org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException.class, org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException::new, 116),
RETRY_ON_PRIMARY_EXCEPTION(org.elasticsearch.action.support.replication.TransportReplicationAction.RetryOnPrimaryException.class, org.elasticsearch.action.support.replication.TransportReplicationAction.RetryOnPrimaryException::new, 117),
ELASTICSEARCH_TIMEOUT_EXCEPTION(org.elasticsearch.ElasticsearchTimeoutException.class, org.elasticsearch.ElasticsearchTimeoutException::new, 118),
QUERY_PHASE_EXECUTION_EXCEPTION(org.elasticsearch.search.query.QueryPhaseExecutionException.class, org.elasticsearch.search.query.QueryPhaseExecutionException::new, 119),
REPOSITORY_VERIFICATION_EXCEPTION(org.elasticsearch.repositories.RepositoryVerificationException.class, org.elasticsearch.repositories.RepositoryVerificationException::new, 120),
INVALID_AGGREGATION_PATH_EXCEPTION(org.elasticsearch.search.aggregations.InvalidAggregationPathException.class, org.elasticsearch.search.aggregations.InvalidAggregationPathException::new, 121),
GROOVY_SCRIPT_EXECUTION_EXCEPTION(org.elasticsearch.script.groovy.GroovyScriptExecutionException.class, org.elasticsearch.script.groovy.GroovyScriptExecutionException::new, 122),
INDEX_ALREADY_EXISTS_EXCEPTION(org.elasticsearch.indices.IndexAlreadyExistsException.class, org.elasticsearch.indices.IndexAlreadyExistsException::new, 123),
SCRIPT_PARSE_EXCEPTION(org.elasticsearch.script.Script.ScriptParseException.class, org.elasticsearch.script.Script.ScriptParseException::new, 124),
HTTP_ON_TRANSPORT_EXCEPTION(org.elasticsearch.transport.netty.SizeHeaderFrameDecoder.HttpOnTransportException.class, org.elasticsearch.transport.netty.SizeHeaderFrameDecoder.HttpOnTransportException::new, 125),
MAPPER_PARSING_EXCEPTION(org.elasticsearch.index.mapper.MapperParsingException.class, org.elasticsearch.index.mapper.MapperParsingException::new, 126),
SEARCH_CONTEXT_EXCEPTION(org.elasticsearch.search.SearchContextException.class, org.elasticsearch.search.SearchContextException::new, 127),
SEARCH_SOURCE_BUILDER_EXCEPTION(org.elasticsearch.search.builder.SearchSourceBuilderException.class, org.elasticsearch.search.builder.SearchSourceBuilderException::new, 128),
ENGINE_CLOSED_EXCEPTION(org.elasticsearch.index.engine.EngineClosedException.class, org.elasticsearch.index.engine.EngineClosedException::new, 129),
NO_SHARD_AVAILABLE_ACTION_EXCEPTION(org.elasticsearch.action.NoShardAvailableActionException.class, org.elasticsearch.action.NoShardAvailableActionException::new, 130),
UNAVAILABLE_SHARDS_EXCEPTION(org.elasticsearch.action.UnavailableShardsException.class, org.elasticsearch.action.UnavailableShardsException::new, 131),
FLUSH_FAILED_ENGINE_EXCEPTION(org.elasticsearch.index.engine.FlushFailedEngineException.class, org.elasticsearch.index.engine.FlushFailedEngineException::new, 132),
CIRCUIT_BREAKING_EXCEPTION(org.elasticsearch.common.breaker.CircuitBreakingException.class, org.elasticsearch.common.breaker.CircuitBreakingException::new, 133),
NODE_NOT_CONNECTED_EXCEPTION(org.elasticsearch.transport.NodeNotConnectedException.class, org.elasticsearch.transport.NodeNotConnectedException::new, 134),
STRICT_DYNAMIC_MAPPING_EXCEPTION(org.elasticsearch.index.mapper.StrictDynamicMappingException.class, org.elasticsearch.index.mapper.StrictDynamicMappingException::new, 135),
RETRY_ON_REPLICA_EXCEPTION(org.elasticsearch.action.support.replication.TransportReplicationAction.RetryOnReplicaException.class, org.elasticsearch.action.support.replication.TransportReplicationAction.RetryOnReplicaException::new, 136),
TYPE_MISSING_EXCEPTION(org.elasticsearch.indices.TypeMissingException.class, org.elasticsearch.indices.TypeMissingException::new, 137),
FAILED_TO_COMMIT_CLUSTER_STATE_EXCEPTION(org.elasticsearch.discovery.Discovery.FailedToCommitClusterStateException.class, org.elasticsearch.discovery.Discovery.FailedToCommitClusterStateException::new, 138),
QUERY_SHARD_EXCEPTION(org.elasticsearch.index.query.QueryShardException.class, org.elasticsearch.index.query.QueryShardException::new, 139);
final Class<? extends ElasticsearchException> exceptionClass;
final FunctionThatThrowsIOException<StreamInput, ? extends ElasticsearchException> constructor;
final int id;
ElasticsearchExceptionHandle(Class<? extends ElasticsearchException> exceptionClass, FunctionThatThrowsIOException<StreamInput, ? extends ElasticsearchException> constructor, int id) {
this.exceptionClass = exceptionClass;
this.constructor = constructor;
this.id = id;
}
}
ID_TO_SUPPLIER = idToSupplier;
CLASS_TO_ID = Collections.unmodifiableMap(exceptions);
static {
final Map<Class<? extends ElasticsearchException>, ElasticsearchExceptionHandle> exceptions = Arrays.stream(ElasticsearchExceptionHandle.values()).collect(Collectors.toMap(e -> e.exceptionClass, e -> e));
final Map<Integer, FunctionThatThrowsIOException<StreamInput, ? extends ElasticsearchException>> idToSupplier = Arrays.stream(ElasticsearchExceptionHandle.values()).collect(Collectors.toMap(e -> e.id, e -> e.constructor));
ID_TO_SUPPLIER = Collections.unmodifiableMap(idToSupplier);
CLASS_TO_ELASTICSEARCH_EXCEPTION_HANDLE = Collections.unmodifiableMap(exceptions);
}
public String getIndex() {
@ -702,4 +689,8 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
ElasticsearchException.toXContent(builder, params, t);
builder.endObject();
}
interface FunctionThatThrowsIOException<T, R> {
R apply(T t) throws IOException;
}
}

View File

@ -91,9 +91,13 @@ import java.nio.file.FileVisitor;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.attribute.BasicFileAttributes;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
public class ExceptionSerializationTests extends ESTestCase {
public void testExceptionRegistration()
@ -637,4 +641,178 @@ public class ExceptionSerializationTests extends ESTestCase {
InterruptedException ex = serialize(orig);
assertEquals(orig.getMessage(), ex.getMessage());
}
public void testThatIdsArePositive() {
for (ElasticsearchException.ElasticsearchExceptionHandle handle : ElasticsearchException.ElasticsearchExceptionHandle.values()) {
assertThat("negative id", handle.id, greaterThanOrEqualTo(0));
}
}
public void testThatIdsAreUnique() {
Set<Integer> ids = new HashSet<>();
for (ElasticsearchException.ElasticsearchExceptionHandle handle : ElasticsearchException.ElasticsearchExceptionHandle.values()) {
assertTrue("duplicate id", ids.add(handle.id));
}
}
public void testIds() {
Map<Integer, Class<? extends ElasticsearchException>> ids = new HashMap<>();
ids.put(0, org.elasticsearch.index.snapshots.IndexShardSnapshotFailedException.class);
ids.put(1, org.elasticsearch.search.dfs.DfsPhaseExecutionException.class);
ids.put(2, org.elasticsearch.common.util.CancellableThreads.ExecutionCancelledException.class);
ids.put(3, org.elasticsearch.discovery.MasterNotDiscoveredException.class);
ids.put(4, org.elasticsearch.ElasticsearchSecurityException.class);
ids.put(5, org.elasticsearch.index.snapshots.IndexShardRestoreException.class);
ids.put(6, org.elasticsearch.indices.IndexClosedException.class);
ids.put(7, org.elasticsearch.http.BindHttpException.class);
ids.put(8, org.elasticsearch.action.search.ReduceSearchPhaseException.class);
ids.put(9, org.elasticsearch.node.NodeClosedException.class);
ids.put(10, org.elasticsearch.index.engine.SnapshotFailedEngineException.class);
ids.put(11, org.elasticsearch.index.shard.ShardNotFoundException.class);
ids.put(12, org.elasticsearch.transport.ConnectTransportException.class);
ids.put(13, org.elasticsearch.transport.NotSerializableTransportException.class);
ids.put(14, org.elasticsearch.transport.ResponseHandlerFailureTransportException.class);
ids.put(15, org.elasticsearch.indices.IndexCreationException.class);
ids.put(16, org.elasticsearch.index.IndexNotFoundException.class);
ids.put(17, org.elasticsearch.cluster.routing.IllegalShardRoutingStateException.class);
ids.put(18, org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException.class);
ids.put(19, org.elasticsearch.ResourceNotFoundException.class);
ids.put(20, org.elasticsearch.transport.ActionTransportException.class);
ids.put(21, org.elasticsearch.ElasticsearchGenerationException.class);
ids.put(22, org.elasticsearch.index.engine.CreateFailedEngineException.class);
ids.put(23, org.elasticsearch.index.shard.IndexShardStartedException.class);
ids.put(24, org.elasticsearch.search.SearchContextMissingException.class);
ids.put(25, org.elasticsearch.script.ScriptException.class);
ids.put(26, org.elasticsearch.index.shard.TranslogRecoveryPerformer.BatchOperationException.class);
ids.put(27, org.elasticsearch.snapshots.SnapshotCreationException.class);
ids.put(28, org.elasticsearch.index.engine.DeleteFailedEngineException.class);
ids.put(29, org.elasticsearch.index.engine.DocumentMissingException.class);
ids.put(30, org.elasticsearch.snapshots.SnapshotException.class);
ids.put(31, org.elasticsearch.indices.InvalidAliasNameException.class);
ids.put(32, org.elasticsearch.indices.InvalidIndexNameException.class);
ids.put(33, org.elasticsearch.indices.IndexPrimaryShardNotAllocatedException.class);
ids.put(34, org.elasticsearch.transport.TransportException.class);
ids.put(35, org.elasticsearch.ElasticsearchParseException.class);
ids.put(36, org.elasticsearch.search.SearchException.class);
ids.put(37, org.elasticsearch.index.mapper.MapperException.class);
ids.put(38, org.elasticsearch.indices.InvalidTypeNameException.class);
ids.put(39, org.elasticsearch.snapshots.SnapshotRestoreException.class);
ids.put(40, org.elasticsearch.common.ParsingException.class);
ids.put(41, org.elasticsearch.index.shard.IndexShardClosedException.class);
ids.put(42, org.elasticsearch.indices.recovery.RecoverFilesRecoveryException.class);
ids.put(43, org.elasticsearch.index.translog.TruncatedTranslogException.class);
ids.put(44, org.elasticsearch.indices.recovery.RecoveryFailedException.class);
ids.put(45, org.elasticsearch.index.shard.IndexShardRelocatedException.class);
ids.put(46, org.elasticsearch.transport.NodeShouldNotConnectException.class);
ids.put(47, org.elasticsearch.indices.IndexTemplateAlreadyExistsException.class);
ids.put(48, org.elasticsearch.index.translog.TranslogCorruptedException.class);
ids.put(49, org.elasticsearch.cluster.block.ClusterBlockException.class);
ids.put(50, org.elasticsearch.search.fetch.FetchPhaseExecutionException.class);
ids.put(51, org.elasticsearch.index.IndexShardAlreadyExistsException.class);
ids.put(52, org.elasticsearch.index.engine.VersionConflictEngineException.class);
ids.put(53, org.elasticsearch.index.engine.EngineException.class);
ids.put(54, org.elasticsearch.index.engine.DocumentAlreadyExistsException.class);
ids.put(55, org.elasticsearch.action.NoSuchNodeException.class);
ids.put(56, org.elasticsearch.common.settings.SettingsException.class);
ids.put(57, org.elasticsearch.indices.IndexTemplateMissingException.class);
ids.put(58, org.elasticsearch.transport.SendRequestTransportException.class);
ids.put(59, org.elasticsearch.common.util.concurrent.EsRejectedExecutionException.class);
ids.put(60, org.elasticsearch.common.lucene.Lucene.EarlyTerminationException.class);
ids.put(61, org.elasticsearch.cluster.routing.RoutingValidationException.class);
ids.put(62, org.elasticsearch.common.io.stream.NotSerializableExceptionWrapper.class);
ids.put(63, org.elasticsearch.indices.AliasFilterParsingException.class);
ids.put(64, org.elasticsearch.index.engine.DeleteByQueryFailedEngineException.class);
ids.put(65, org.elasticsearch.gateway.GatewayException.class);
ids.put(66, org.elasticsearch.index.shard.IndexShardNotRecoveringException.class);
ids.put(67, org.elasticsearch.http.HttpException.class);
ids.put(68, org.elasticsearch.ElasticsearchException.class);
ids.put(69, org.elasticsearch.snapshots.SnapshotMissingException.class);
ids.put(70, org.elasticsearch.action.PrimaryMissingActionException.class);
ids.put(71, org.elasticsearch.action.FailedNodeException.class);
ids.put(72, org.elasticsearch.search.SearchParseException.class);
ids.put(73, org.elasticsearch.snapshots.ConcurrentSnapshotExecutionException.class);
ids.put(74, org.elasticsearch.common.blobstore.BlobStoreException.class);
ids.put(75, org.elasticsearch.cluster.IncompatibleClusterStateVersionException.class);
ids.put(76, org.elasticsearch.index.engine.RecoveryEngineException.class);
ids.put(77, org.elasticsearch.common.util.concurrent.UncategorizedExecutionException.class);
ids.put(78, org.elasticsearch.action.TimestampParsingException.class);
ids.put(79, org.elasticsearch.action.RoutingMissingException.class);
ids.put(80, org.elasticsearch.index.engine.IndexFailedEngineException.class);
ids.put(81, org.elasticsearch.index.snapshots.IndexShardRestoreFailedException.class);
ids.put(82, org.elasticsearch.repositories.RepositoryException.class);
ids.put(83, org.elasticsearch.transport.ReceiveTimeoutTransportException.class);
ids.put(84, org.elasticsearch.transport.NodeDisconnectedException.class);
ids.put(85, org.elasticsearch.index.AlreadyExpiredException.class);
ids.put(86, org.elasticsearch.search.aggregations.AggregationExecutionException.class);
ids.put(87, org.elasticsearch.index.mapper.MergeMappingException.class);
ids.put(88, org.elasticsearch.indices.InvalidIndexTemplateException.class);
ids.put(89, org.elasticsearch.percolator.PercolateException.class);
ids.put(90, org.elasticsearch.index.engine.RefreshFailedEngineException.class);
ids.put(91, org.elasticsearch.search.aggregations.AggregationInitializationException.class);
ids.put(92, org.elasticsearch.indices.recovery.DelayRecoveryException.class);
ids.put(93, org.elasticsearch.search.warmer.IndexWarmerMissingException.class);
ids.put(94, org.elasticsearch.client.transport.NoNodeAvailableException.class);
ids.put(95, org.elasticsearch.script.groovy.GroovyScriptCompilationException.class);
ids.put(96, org.elasticsearch.snapshots.InvalidSnapshotNameException.class);
ids.put(97, org.elasticsearch.index.shard.IllegalIndexShardStateException.class);
ids.put(98, org.elasticsearch.index.snapshots.IndexShardSnapshotException.class);
ids.put(99, org.elasticsearch.index.shard.IndexShardNotStartedException.class);
ids.put(100, org.elasticsearch.action.search.SearchPhaseExecutionException.class);
ids.put(101, org.elasticsearch.transport.ActionNotFoundTransportException.class);
ids.put(102, org.elasticsearch.transport.TransportSerializationException.class);
ids.put(103, org.elasticsearch.transport.RemoteTransportException.class);
ids.put(104, org.elasticsearch.index.engine.EngineCreationFailureException.class);
ids.put(105, org.elasticsearch.cluster.routing.RoutingException.class);
ids.put(106, org.elasticsearch.index.shard.IndexShardRecoveryException.class);
ids.put(107, org.elasticsearch.repositories.RepositoryMissingException.class);
ids.put(108, org.elasticsearch.index.percolator.PercolatorException.class);
ids.put(109, org.elasticsearch.index.engine.DocumentSourceMissingException.class);
ids.put(110, org.elasticsearch.index.engine.FlushNotAllowedEngineException.class);
ids.put(111, org.elasticsearch.common.settings.NoClassSettingsException.class);
ids.put(112, org.elasticsearch.transport.BindTransportException.class);
ids.put(113, org.elasticsearch.rest.action.admin.indices.alias.delete.AliasesNotFoundException.class);
ids.put(114, org.elasticsearch.index.shard.IndexShardRecoveringException.class);
ids.put(115, org.elasticsearch.index.translog.TranslogException.class);
ids.put(116, org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException.class);
ids.put(117, org.elasticsearch.action.support.replication.TransportReplicationAction.RetryOnPrimaryException.class);
ids.put(118, org.elasticsearch.ElasticsearchTimeoutException.class);
ids.put(119, org.elasticsearch.search.query.QueryPhaseExecutionException.class);
ids.put(120, org.elasticsearch.repositories.RepositoryVerificationException.class);
ids.put(121, org.elasticsearch.search.aggregations.InvalidAggregationPathException.class);
ids.put(122, org.elasticsearch.script.groovy.GroovyScriptExecutionException.class);
ids.put(123, org.elasticsearch.indices.IndexAlreadyExistsException.class);
ids.put(124, org.elasticsearch.script.Script.ScriptParseException.class);
ids.put(125, org.elasticsearch.transport.netty.SizeHeaderFrameDecoder.HttpOnTransportException.class);
ids.put(126, org.elasticsearch.index.mapper.MapperParsingException.class);
ids.put(127, org.elasticsearch.search.SearchContextException.class);
ids.put(128, org.elasticsearch.search.builder.SearchSourceBuilderException.class);
ids.put(129, org.elasticsearch.index.engine.EngineClosedException.class);
ids.put(130, org.elasticsearch.action.NoShardAvailableActionException.class);
ids.put(131, org.elasticsearch.action.UnavailableShardsException.class);
ids.put(132, org.elasticsearch.index.engine.FlushFailedEngineException.class);
ids.put(133, org.elasticsearch.common.breaker.CircuitBreakingException.class);
ids.put(134, org.elasticsearch.transport.NodeNotConnectedException.class);
ids.put(135, org.elasticsearch.index.mapper.StrictDynamicMappingException.class);
ids.put(136, org.elasticsearch.action.support.replication.TransportReplicationAction.RetryOnReplicaException.class);
ids.put(137, org.elasticsearch.indices.TypeMissingException.class);
ids.put(138, org.elasticsearch.discovery.Discovery.FailedToCommitClusterStateException.class);
ids.put(139, org.elasticsearch.index.query.QueryShardException.class);
Map<Class<? extends ElasticsearchException>, Integer> reverse = new HashMap<>();
for (Map.Entry<Integer, Class<? extends ElasticsearchException>> entry : ids.entrySet()) {
if (entry.getValue() != null) {
reverse.put(entry.getValue(), entry.getKey());
}
}
for (ElasticsearchException.ElasticsearchExceptionHandle handle : ElasticsearchException.ElasticsearchExceptionHandle.values()) {
assertEquals((int)reverse.get(handle.exceptionClass), handle.id);
}
for (Map.Entry<Integer, Class<? extends ElasticsearchException>> entry : ids.entrySet()) {
if (entry.getValue() != null) {
assertEquals((int) entry.getKey(), ElasticsearchException.getId(entry.getValue()));
}
}
}
}