Browse Source

Replace IndexAlreadyExistsException with ResourceAlreadyExistsException (#21494)

Dimitris Athanasiou 9 years ago
parent
commit
a75320f89b

+ 4 - 2
core/src/main/java/org/elasticsearch/ElasticsearchException.java

@@ -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;

+ 8 - 9
core/src/main/java/org/elasticsearch/indices/IndexAlreadyExistsException.java → core/src/main/java/org/elasticsearch/ResourceAlreadyExistsException.java

@@ -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 ResourceAlreadyExistsException(Index index) {
+        this("index {} already exists", index.toString());
+        setIndex(index);
     }
 
-    public IndexAlreadyExistsException(Index index, String message) {
-        super(message);
-        setIndex(index);
+    public ResourceAlreadyExistsException(String msg, Object... args) {
+        super(msg, args);
     }
 
-    public IndexAlreadyExistsException(StreamInput in) throws IOException{
+    public ResourceAlreadyExistsException(StreamInput in) throws IOException{
         super(in);
     }
 

+ 0 - 1
core/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java

@@ -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;
 

+ 2 - 2
core/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java

@@ -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);

+ 2 - 2
core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java

@@ -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 {

+ 2 - 2
core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java

@@ -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);

+ 2 - 2
core/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java

@@ -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);

+ 5 - 6
core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java

@@ -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) {

+ 3 - 2
core/src/main/java/org/elasticsearch/indices/IndicesService.java

@@ -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() {

+ 2 - 2
core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java

@@ -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,

+ 2 - 2
core/src/main/java/org/elasticsearch/tasks/TaskResultsService.java

@@ -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);

+ 3 - 0
core/src/test/java/org/elasticsearch/ESExceptionTests.java

@@ -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));
 

+ 1 - 1
core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java

@@ -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);

+ 2 - 2
core/src/test/java/org/elasticsearch/action/admin/indices/rollover/RolloverIT.java

@@ -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"));
         }
     }

+ 2 - 2
core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java

@@ -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());
 

+ 2 - 2
rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/10_basic.yaml

@@ -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"