Browse Source

Merge branch 'pr/remove-azure-container-auto-creation'

# Conflicts:
#	docs/reference/migration/migrate_6_0/plugins.asciidoc
David Pilato 8 years ago
parent
commit
858333246d

+ 2 - 1
docs/plugins/repository-azure.asciidoc

@@ -102,7 +102,8 @@ The Azure repository supports following settings:
 
 
 `container`::
 `container`::
 
 
-    Container name. Defaults to `elasticsearch-snapshots`
+    Container name. You must create the azure container before creating the repository.
+    Defaults to `elasticsearch-snapshots`.
 
 
 `base_path`::
 `base_path`::
 
 

+ 5 - 0
docs/reference/migration/migrate_6_0/plugins.asciidoc

@@ -20,3 +20,8 @@ to securely store the credentials.
 region inside the repository settings. Instead, specify the full endpoint if a custom
 region inside the repository settings. Instead, specify the full endpoint if a custom
 s3 location is needed, or rely on the default behavior which automatically locates
 s3 location is needed, or rely on the default behavior which automatically locates
 the region of the configured bucket.
 the region of the configured bucket.
+
+==== Azure Repository plugin
+
+* The container an azure repository is configured with will no longer be created automatically.
+It must exist before the azure repository is created.

+ 0 - 10
plugins/repository-azure/src/main/java/org/elasticsearch/cloud/azure/blobstore/AzureBlobStore.java

@@ -100,16 +100,6 @@ public class AzureBlobStore extends AbstractComponent implements BlobStore {
         return this.client.doesContainerExist(this.accountName, this.locMode, container);
         return this.client.doesContainerExist(this.accountName, this.locMode, container);
     }
     }
 
 
-    public void removeContainer(String container) throws URISyntaxException, StorageException
-    {
-        this.client.removeContainer(this.accountName, this.locMode, container);
-    }
-
-    public void createContainer(String container) throws URISyntaxException, StorageException
-    {
-        this.client.createContainer(this.accountName, this.locMode, container);
-    }
-
     public void deleteFiles(String container, String path) throws URISyntaxException, StorageException
     public void deleteFiles(String container, String path) throws URISyntaxException, StorageException
     {
     {
         this.client.deleteFiles(this.accountName, this.locMode, container, path);
         this.client.deleteFiles(this.accountName, this.locMode, container, path);

+ 12 - 36
plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java

@@ -19,12 +19,6 @@
 
 
 package org.elasticsearch.repositories.azure;
 package org.elasticsearch.repositories.azure;
 
 
-import java.io.IOException;
-import java.net.URISyntaxException;
-import java.util.List;
-import java.util.Locale;
-import java.util.function.Function;
-
 import com.microsoft.azure.storage.LocationMode;
 import com.microsoft.azure.storage.LocationMode;
 import com.microsoft.azure.storage.StorageException;
 import com.microsoft.azure.storage.StorageException;
 import org.elasticsearch.cloud.azure.blobstore.AzureBlobStore;
 import org.elasticsearch.cloud.azure.blobstore.AzureBlobStore;
@@ -32,19 +26,23 @@ import org.elasticsearch.cloud.azure.storage.AzureStorageService;
 import org.elasticsearch.cloud.azure.storage.AzureStorageService.Storage;
 import org.elasticsearch.cloud.azure.storage.AzureStorageService.Storage;
 import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.cluster.metadata.RepositoryMetaData;
 import org.elasticsearch.cluster.metadata.RepositoryMetaData;
-import org.elasticsearch.common.xcontent.NamedXContentRegistry;
-import org.elasticsearch.repositories.IndexId;
-import org.elasticsearch.snapshots.SnapshotId;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.blobstore.BlobPath;
 import org.elasticsearch.common.blobstore.BlobPath;
 import org.elasticsearch.common.blobstore.BlobStore;
 import org.elasticsearch.common.blobstore.BlobStore;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting.Property;
 import org.elasticsearch.common.settings.Setting.Property;
 import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.Environment;
-import org.elasticsearch.repositories.RepositoryVerificationException;
+import org.elasticsearch.repositories.IndexId;
 import org.elasticsearch.repositories.blobstore.BlobStoreRepository;
 import org.elasticsearch.repositories.blobstore.BlobStoreRepository;
-import org.elasticsearch.snapshots.SnapshotCreationException;
+import org.elasticsearch.snapshots.SnapshotId;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Locale;
+import java.util.function.Function;
 
 
 import static org.elasticsearch.cloud.azure.storage.AzureStorageService.MAX_CHUNK_SIZE;
 import static org.elasticsearch.cloud.azure.storage.AzureStorageService.MAX_CHUNK_SIZE;
 import static org.elasticsearch.cloud.azure.storage.AzureStorageService.MIN_CHUNK_SIZE;
 import static org.elasticsearch.cloud.azure.storage.AzureStorageService.MIN_CHUNK_SIZE;
@@ -154,32 +152,10 @@ public class AzureRepository extends BlobStoreRepository {
 
 
     @Override
     @Override
     public void initializeSnapshot(SnapshotId snapshotId, List<IndexId> indices, MetaData clusterMetadata) {
     public void initializeSnapshot(SnapshotId snapshotId, List<IndexId> indices, MetaData clusterMetadata) {
-        try {
-            if (!blobStore.doesContainerExist(blobStore.container())) {
-                logger.debug("container [{}] does not exist. Creating...", blobStore.container());
-                blobStore.createContainer(blobStore.container());
-            }
-            super.initializeSnapshot(snapshotId, indices, clusterMetadata);
-        } catch (StorageException | URISyntaxException e) {
-            logger.warn("can not initialize container [{}]: [{}]", blobStore.container(), e.getMessage());
-            throw new SnapshotCreationException(getMetadata().name(), snapshotId, e);
-        }
-    }
-
-    @Override
-    public String startVerification() {
-        if (readonly == false) {
-            try {
-                if (!blobStore.doesContainerExist(blobStore.container())) {
-                    logger.debug("container [{}] does not exist. Creating...", blobStore.container());
-                    blobStore.createContainer(blobStore.container());
-                }
-            } catch (StorageException | URISyntaxException e) {
-                logger.warn("can not initialize container [{}]: [{}]", blobStore.container(), e.getMessage());
-                throw new RepositoryVerificationException(getMetadata().name(), "can not initialize container " + blobStore.container(), e);
-            }
+        if (blobStore.doesContainerExist(blobStore.container()) == false) {
+            throw new IllegalArgumentException("The bucket [" + blobStore.container() + "] does not exist. Please create it before " +
+                " creating an azure snapshot repository backed by it.");
         }
         }
-        return super.startVerification();
     }
     }
 
 
     @Override
     @Override

+ 0 - 113
plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureRepositoryServiceIntegTestCase.java

@@ -1,113 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.elasticsearch.cloud.azure;
-
-import com.microsoft.azure.storage.LocationMode;
-import com.microsoft.azure.storage.StorageException;
-
-import org.elasticsearch.cloud.azure.storage.AzureStorageService;
-import org.elasticsearch.cloud.azure.storage.AzureStorageService.Storage;
-import org.elasticsearch.cloud.azure.storage.AzureStorageServiceMock;
-import org.elasticsearch.cluster.metadata.IndexMetaData;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.plugin.repository.azure.AzureRepositoryPlugin;
-import org.elasticsearch.plugins.Plugin;
-import org.elasticsearch.repositories.RepositoryMissingException;
-import org.elasticsearch.test.store.MockFSDirectoryService;
-import org.elasticsearch.test.store.MockFSIndexStore;
-import org.junit.After;
-import org.junit.Before;
-
-import java.net.URISyntaxException;
-import java.util.Arrays;
-import java.util.Collection;
-
-public abstract class AbstractAzureRepositoryServiceIntegTestCase extends AbstractAzureIntegTestCase {
-
-    private static final AzureStorageService storageService = new AzureStorageServiceMock();
-
-    public static class TestPlugin extends AzureRepositoryPlugin {
-        @Override
-        protected AzureStorageService createStorageService(Settings settings) {
-            return storageService;
-        }
-    }
-
-    protected String basePath;
-
-    public AbstractAzureRepositoryServiceIntegTestCase(String basePath) {
-        this.basePath = basePath;
-    }
-
-    /**
-     * Deletes repositories, supports wildcard notation.
-     */
-    public static void wipeRepositories(String... repositories) {
-        // if nothing is provided, delete all
-        if (repositories.length == 0) {
-            repositories = new String[]{"*"};
-        }
-        for (String repository : repositories) {
-            try {
-                client().admin().cluster().prepareDeleteRepository(repository).get();
-            } catch (RepositoryMissingException ex) {
-                // ignore
-            }
-        }
-    }
-
-    @Override
-    protected Settings nodeSettings(int nodeOrdinal) {
-        Settings.Builder builder = Settings.builder()
-                .put(Storage.CONTAINER_SETTING.getKey(), "snapshots");
-        return builder.build();
-    }
-
-    @Override
-    protected Collection<Class<? extends Plugin>> nodePlugins() {
-        return Arrays.asList(TestPlugin.class, MockFSIndexStore.TestPlugin.class);
-    }
-
-    @Override
-    public Settings indexSettings() {
-        // During restore we frequently restore index to exactly the same state it was before, that might cause the same
-        // checksum file to be written twice during restore operation
-        return Settings.builder().put(super.indexSettings())
-                .put(MockFSDirectoryService.RANDOM_PREVENT_DOUBLE_WRITE_SETTING.getKey(), false)
-                .put(MockFSDirectoryService.RANDOM_NO_DELETE_OPEN_FILE_SETTING.getKey(), false)
-                .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
-                .build();
-    }
-
-    @Before @After
-    public final void wipe() throws StorageException, URISyntaxException {
-        wipeRepositories();
-        cleanRepositoryFiles(basePath);
-    }
-
-    /**
-     * Purge the test container
-     */
-    public void cleanRepositoryFiles(String path) throws StorageException, URISyntaxException {
-        String container = internalCluster().getInstance(Settings.class).get("repositories.azure.container");
-        logger.info("--> remove blobs in container [{}]", container);
-        storageService.deleteFiles(null, LocationMode.PRIMARY_ONLY, container, path);
-    }
-}

+ 0 - 56
plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobStoreTests.java

@@ -1,56 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.elasticsearch.repositories.azure;
-
-import com.microsoft.azure.storage.StorageException;
-import org.elasticsearch.cloud.azure.blobstore.AzureBlobStore;
-import org.elasticsearch.cloud.azure.storage.AzureStorageService;
-import org.elasticsearch.cloud.azure.storage.AzureStorageServiceImpl;
-import org.elasticsearch.cluster.metadata.RepositoryMetaData;
-import org.elasticsearch.common.blobstore.BlobStore;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.repositories.ESBlobStoreTestCase;
-import org.elasticsearch.test.ESIntegTestCase;
-
-import java.io.IOException;
-import java.net.URISyntaxException;
-
-import static org.elasticsearch.cloud.azure.AzureTestUtils.readSettingsFromFile;
-
-/**
- * You must specify {@code -Dtests.thirdparty=true -Dtests.config=/path/to/elasticsearch.yml}
- * in order to run these tests.
- */
-@ESIntegTestCase.ThirdParty
-public class AzureBlobStoreTests extends ESBlobStoreTestCase {
-    @Override
-    protected BlobStore newBlobStore() throws IOException {
-        try {
-            Settings settings = readSettingsFromFile();
-            RepositoryMetaData metadata = new RepositoryMetaData("ittest", "azure", Settings.EMPTY);
-            AzureStorageService storageService = new AzureStorageServiceImpl(settings);
-            AzureBlobStore blobStore = new AzureBlobStore(metadata, settings, storageService);
-            blobStore.createContainer(blobStore.container());
-            return blobStore;
-        } catch (URISyntaxException | StorageException e) {
-            throw new IOException(e);
-        }
-    }
-}

+ 0 - 133
plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureSnapshotRestoreServiceIntegTests.java

@@ -1,133 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.elasticsearch.repositories.azure;
-
-
-import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryResponse;
-import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
-import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
-import org.elasticsearch.client.Client;
-import org.elasticsearch.cloud.azure.AbstractAzureRepositoryServiceIntegTestCase;
-import org.elasticsearch.cluster.ClusterState;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.unit.ByteSizeUnit;
-import org.elasticsearch.snapshots.SnapshotInfo;
-import org.elasticsearch.snapshots.SnapshotShardFailure;
-import org.elasticsearch.snapshots.SnapshotState;
-import org.elasticsearch.test.ESIntegTestCase;
-
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.is;
-
-@ESIntegTestCase.ClusterScope(
-        scope = ESIntegTestCase.Scope.SUITE,
-        supportsDedicatedMasters = false,
-        numDataNodes = 1,
-        numClientNodes = 0,
-        transportClientRatio = 0.0)
-public class AzureSnapshotRestoreServiceIntegTests extends AbstractAzureRepositoryServiceIntegTestCase {
-    public AzureSnapshotRestoreServiceIntegTests() {
-        super("/snapshot-test/repo-" + randomInt());
-    }
-
-    public void testSimpleWorkflow() {
-        Client client = client();
-        logger.info("-->  creating azure repository with path [{}]", basePath);
-        PutRepositoryResponse putRepositoryResponse = client.admin().cluster().preparePutRepository("test-repo")
-                .setType("azure").setSettings(Settings.builder()
-                        .put("base_path", basePath)
-                        .put("chunk_size", randomIntBetween(1000, 10000), ByteSizeUnit.BYTES)
-                ).get();
-        assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true));
-
-        createIndex("test-idx-1", "test-idx-2", "test-idx-3");
-        ensureGreen();
-
-        logger.info("--> indexing some data");
-        for (int i = 0; i < 100; i++) {
-            index("test-idx-1", "doc", Integer.toString(i), "foo", "bar" + i);
-            index("test-idx-2", "doc", Integer.toString(i), "foo", "baz" + i);
-            index("test-idx-3", "doc", Integer.toString(i), "foo", "baz" + i);
-        }
-        refresh();
-        assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L));
-        assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(100L));
-        assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().totalHits(), equalTo(100L));
-
-        logger.info("--> snapshot");
-        CreateSnapshotResponse createSnapshotResponse = client.admin().cluster()
-                .prepareCreateSnapshot("test-repo", "test-snap")
-                .setWaitForCompletion(true)
-                .setIndices("test-idx-*", "-test-idx-3")
-                .get();
-
-        final SnapshotInfo snapshotInfo = createSnapshotResponse.getSnapshotInfo();
-        if (snapshotInfo.shardFailures() != null) {
-            for (SnapshotShardFailure shardFailure : snapshotInfo.shardFailures()) {
-                logger.warn("shard failure during snapshot: {}", shardFailure::toString);
-            }
-        }
-        assertThat(snapshotInfo.state(), is(SnapshotState.SUCCESS));
-        assertEquals(snapshotInfo.failedShards(), 0);
-
-        logger.info("--> delete some data");
-        for (int i = 0; i < 50; i++) {
-            client.prepareDelete("test-idx-1", "doc", Integer.toString(i)).get();
-        }
-        for (int i = 50; i < 100; i++) {
-            client.prepareDelete("test-idx-2", "doc", Integer.toString(i)).get();
-        }
-        for (int i = 0; i < 100; i += 2) {
-            client.prepareDelete("test-idx-3", "doc", Integer.toString(i)).get();
-        }
-        refresh();
-        assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(50L));
-        assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(50L));
-        assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().totalHits(), equalTo(50L));
-
-        logger.info("--> close indices");
-        client.admin().indices().prepareClose("test-idx-1", "test-idx-2").get();
-
-        logger.info("--> restore all indices from the snapshot");
-        RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap")
-            .setWaitForCompletion(true).get();
-        assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
-
-        ensureGreen();
-        assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L));
-        assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(100L));
-        assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().totalHits(), equalTo(50L));
-
-        // Test restore after index deletion
-        logger.info("--> delete indices");
-        cluster().wipeIndices("test-idx-1", "test-idx-2");
-        logger.info("--> restore one index after deletion");
-        restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true)
-            .setIndices("test-idx-*", "-test-idx-2").get();
-        assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
-        ensureGreen();
-        assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L));
-        ClusterState clusterState = client.admin().cluster().prepareState().get().getState();
-        assertThat(clusterState.getMetaData().hasIndex("test-idx-1"), equalTo(true));
-        assertThat(clusterState.getMetaData().hasIndex("test-idx-2"), equalTo(false));
-    }
-
-}