Replace IndexAlreadyExistsException with ResourceAlreadyExistsException (#21494)

This commit is contained in:
Dimitris Athanasiou 2016-11-17 14:30:21 +00:00 committed by GitHub
parent b08a2e1f31
commit a75320f89b
16 changed files with 43 additions and 40 deletions

View File

@ -653,8 +653,9 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
org.elasticsearch.repositories.RepositoryVerificationException::new, 120),
INVALID_AGGREGATION_PATH_EXCEPTION(org.elasticsearch.search.aggregations.InvalidAggregationPathException.class,
org.elasticsearch.search.aggregations.InvalidAggregationPathException::new, 121),
INDEX_ALREADY_EXISTS_EXCEPTION(org.elasticsearch.indices.IndexAlreadyExistsException.class,
org.elasticsearch.indices.IndexAlreadyExistsException::new, 123),
// 123 used to be IndexAlreadyExistsException and was renamed
RESOURCE_ALREADY_EXISTS_EXCEPTION(ResourceAlreadyExistsException.class,
ResourceAlreadyExistsException::new, 123),
// 124 used to be Script.ScriptParseException
HTTP_ON_TRANSPORT_EXCEPTION(TcpTransport.HttpOnTransportException.class,
TcpTransport.HttpOnTransportException::new, 125),
@ -694,6 +695,7 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
TASK_CANCELLED_EXCEPTION(org.elasticsearch.tasks.TaskCancelledException.class,
org.elasticsearch.tasks.TaskCancelledException::new, 146);
final Class<? extends ElasticsearchException> exceptionClass;
final FunctionThatThrowsIOException<StreamInput, ? extends ElasticsearchException> constructor;
final int id;

View File

@ -17,27 +17,26 @@
* under the License.
*/
package org.elasticsearch.indices;
package org.elasticsearch;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.index.Index;
import org.elasticsearch.rest.RestStatus;
import java.io.IOException;
public class IndexAlreadyExistsException extends ElasticsearchException {
public class ResourceAlreadyExistsException extends ElasticsearchException {
public IndexAlreadyExistsException(Index index) {
this(index, "index " + index.toString() + " already exists");
}
public IndexAlreadyExistsException(Index index, String message) {
super(message);
public ResourceAlreadyExistsException(Index index) {
this("index {} already exists", index.toString());
setIndex(index);
}
public IndexAlreadyExistsException(StreamInput in) throws IOException{
public ResourceAlreadyExistsException(String msg, Object... args) {
super(msg, args);
}
public ResourceAlreadyExistsException(StreamInput in) throws IOException{
super(in);
}

View File

@ -44,7 +44,6 @@ import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.DocsStats;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;

View File

@ -48,7 +48,7 @@ import org.elasticsearch.common.util.concurrent.AtomicArray;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.ResourceAlreadyExistsException;
import org.elasticsearch.indices.IndexClosedException;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
@ -142,7 +142,7 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
@Override
public void onFailure(Exception e) {
if (!(ExceptionsHelper.unwrapCause(e) instanceof IndexAlreadyExistsException)) {
if (!(ExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException)) {
// fail all requests involving this index, if create didnt work
for (int i = 0; i < bulkRequest.requests.size(); i++) {
DocWriteRequest request = bulkRequest.requests.get(i);

View File

@ -39,7 +39,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.ResourceAlreadyExistsException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
@ -81,7 +81,7 @@ public class TransportDeleteAction extends TransportWriteAction<DeleteRequest, D
@Override
public void onFailure(Exception e) {
if (ExceptionsHelper.unwrapCause(e) instanceof IndexAlreadyExistsException) {
if (ExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException) {
// we have the index, do it
innerExecute(task, request, listener);
} else {

View File

@ -44,7 +44,7 @@ import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.index.mapper.SourceToParse;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.ResourceAlreadyExistsException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
@ -101,7 +101,7 @@ public class TransportIndexAction extends TransportWriteAction<IndexRequest, Ind
@Override
public void onFailure(Exception e) {
if (ExceptionsHelper.unwrapCause(e) instanceof IndexAlreadyExistsException) {
if (ExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException) {
// we have the index, do it
try {
innerExecute(task, request, listener);

View File

@ -54,7 +54,7 @@ import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.VersionConflictEngineException;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.ResourceAlreadyExistsException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
@ -127,7 +127,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
@Override
public void onFailure(Exception e) {
if (unwrapCause(e) instanceof IndexAlreadyExistsException) {
if (unwrapCause(e) instanceof ResourceAlreadyExistsException) {
// we have the index, do it
try {
innerExecute(request, listener);

View File

@ -21,11 +21,11 @@ package org.elasticsearch.cluster.metadata;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ResourceAlreadyExistsException;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.alias.Alias;
@ -68,7 +68,6 @@ import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.indices.IndexCreationException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.InvalidIndexNameException;
@ -137,10 +136,10 @@ public class MetaDataCreateIndexService extends AbstractComponent {
throw new InvalidIndexNameException(index, "must be lowercase");
}
if (state.routingTable().hasIndex(index)) {
throw new IndexAlreadyExistsException(state.routingTable().index(index).getIndex());
throw new ResourceAlreadyExistsException(state.routingTable().index(index).getIndex());
}
if (state.metaData().hasIndex(index)) {
throw new IndexAlreadyExistsException(state.metaData().index(index).getIndex());
throw new ResourceAlreadyExistsException(state.metaData().index(index).getIndex());
}
if (state.metaData().hasAlias(index)) {
throw new InvalidIndexNameException(index, "already exists as alias");
@ -453,7 +452,7 @@ public class MetaDataCreateIndexService extends AbstractComponent {
@Override
public void onFailure(String source, Exception e) {
if (e instanceof IndexAlreadyExistsException) {
if (e instanceof ResourceAlreadyExistsException) {
logger.trace((Supplier<?>) () -> new ParameterizedMessage("[{}] failed to create", request.index()), e);
} else {
logger.debug((Supplier<?>) () -> new ParameterizedMessage("[{}] failed to create", request.index()), e);
@ -520,7 +519,7 @@ public class MetaDataCreateIndexService extends AbstractComponent {
Set<String> targetIndexMappingsTypes, String targetIndexName,
Settings targetIndexSettings) {
if (state.metaData().hasIndex(targetIndexName)) {
throw new IndexAlreadyExistsException(state.metaData().index(targetIndexName).getIndex());
throw new ResourceAlreadyExistsException(state.metaData().index(targetIndexName).getIndex());
}
final IndexMetaData sourceMetaData = state.metaData().index(sourceIndex);
if (sourceMetaData == null) {

View File

@ -28,6 +28,7 @@ import org.apache.lucene.util.CollectionUtil;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ResourceAlreadyExistsException;
import org.elasticsearch.action.admin.indices.stats.CommonStats;
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags.Flag;
@ -373,7 +374,7 @@ public class IndicesService extends AbstractLifecycleComponent
* Creates a new {@link IndexService} for the given metadata.
* @param indexMetaData the index metadata to create the index for
* @param builtInListeners a list of built-in lifecycle {@link IndexEventListener} that should should be used along side with the per-index listeners
* @throws IndexAlreadyExistsException if the index already exists.
* @throws ResourceAlreadyExistsException if the index already exists.
*/
@Override
public synchronized IndexService createIndex(IndexMetaData indexMetaData, List<IndexEventListener> builtInListeners, Consumer<ShardId> globalCheckpointSyncer) throws IOException {
@ -383,7 +384,7 @@ public class IndicesService extends AbstractLifecycleComponent
}
final Index index = indexMetaData.getIndex();
if (hasIndex(index)) {
throw new IndexAlreadyExistsException(index);
throw new ResourceAlreadyExistsException(index);
}
List<IndexEventListener> finalListeners = new ArrayList<>(builtInListeners);
final IndexEventListener onStoreClose = new IndexEventListener() {

View File

@ -23,6 +23,7 @@ import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.apache.lucene.store.LockObtainFailedException;
import org.elasticsearch.ResourceAlreadyExistsException;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateListener;
@ -59,7 +60,6 @@ import org.elasticsearch.index.shard.IndexShardRelocatedException;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardNotFoundException;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.flush.SyncedFlushService;
import org.elasticsearch.indices.recovery.PeerRecoverySourceService;
@ -767,7 +767,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
* @param builtInIndexListener a list of built-in lifecycle {@link IndexEventListener} that should should be used along side with
* the per-index listeners
* @param globalCheckpointSyncer the global checkpoint syncer
* @throws IndexAlreadyExistsException if the index already exists.
* @throws ResourceAlreadyExistsException if the index already exists.
*/
U createIndex(IndexMetaData indexMetaData,
List<IndexEventListener> builtInIndexListener,

View File

@ -42,7 +42,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.ResourceAlreadyExistsException;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
@ -93,7 +93,7 @@ public class TaskResultsService extends AbstractComponent {
@Override
public void onFailure(Exception e) {
if (ExceptionsHelper.unwrapCause(e) instanceof IndexAlreadyExistsException) {
if (ExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException) {
// we have the index, do it
try {
doStoreResult(taskResult, listener);

View File

@ -78,6 +78,9 @@ public class ESExceptionTests extends ESTestCase {
exception = new RemoteTransportException("test", new ResourceNotFoundException("test"));
assertThat(exception.status(), equalTo(RestStatus.NOT_FOUND));
exception = new RemoteTransportException("test", new ResourceAlreadyExistsException("test"));
assertThat(exception.status(), equalTo(RestStatus.BAD_REQUEST));
exception = new RemoteTransportException("test", new IllegalArgumentException("foobar"));
assertThat(exception.status(), equalTo(RestStatus.BAD_REQUEST));

View File

@ -745,7 +745,7 @@ public class ExceptionSerializationTests extends ESTestCase {
ids.put(120, org.elasticsearch.repositories.RepositoryVerificationException.class);
ids.put(121, org.elasticsearch.search.aggregations.InvalidAggregationPathException.class);
ids.put(122, null);
ids.put(123, org.elasticsearch.indices.IndexAlreadyExistsException.class);
ids.put(123, org.elasticsearch.ResourceAlreadyExistsException.class);
ids.put(124, null);
ids.put(125, TcpTransport.HttpOnTransportException.class);
ids.put(126, org.elasticsearch.index.mapper.MapperParsingException.class);

View File

@ -25,7 +25,7 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.ResourceAlreadyExistsException;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.InternalSettingsPlugin;
@ -171,7 +171,7 @@ public class RolloverIT extends ESIntegTestCase {
try {
client().admin().indices().prepareRolloverIndex("test_alias").get();
fail("expected failure due to existing rollover index");
} catch (IndexAlreadyExistsException e) {
} catch (ResourceAlreadyExistsException e) {
assertThat(e.getIndex().getName(), equalTo("test_index-000001"));
}
}

View File

@ -35,7 +35,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDe
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.ResourceAlreadyExistsException;
import org.elasticsearch.indices.InvalidIndexNameException;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.gateway.TestGatewayAllocator;
@ -78,7 +78,7 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase {
Settings.builder().put("index.blocks.write", true).build());
assertEquals("index [source] already exists",
expectThrows(IndexAlreadyExistsException.class, () ->
expectThrows(ResourceAlreadyExistsException.class, () ->
MetaDataCreateIndexService.validateShrinkIndex(state, "target", Collections.emptySet(), "source", Settings.EMPTY)
).getMessage());

View File

@ -124,7 +124,7 @@
index: logs-000002
- do:
catch: /index_already_exists_exception/
catch: /(index|resource)_already_exists_exception/
indices.rollover:
dry_run: true
alias: "logs_search"
@ -135,7 +135,7 @@
# also do it without dry_run
- do:
catch: /index_already_exists_exception/
catch: /(index|resource)_already_exists_exception/
indices.rollover:
dry_run: false
alias: "logs_search"