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 name. Defaults to `elasticsearch-snapshots`
+    Container name. You must create the azure container before creating the repository.
+    Defaults to `elasticsearch-snapshots`.
 
 `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
 s3 location is needed, or rely on the default behavior which automatically locates
 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);
     }
 
-    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
     {
         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;
 
-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.StorageException;
 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.cluster.metadata.MetaData;
 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.blobstore.BlobPath;
 import org.elasticsearch.common.blobstore.BlobStore;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting.Property;
 import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
-import org.elasticsearch.repositories.RepositoryVerificationException;
+import org.elasticsearch.repositories.IndexId;
 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.MIN_CHUNK_SIZE;
@@ -154,32 +152,10 @@ public class AzureRepository extends BlobStoreRepository {
 
     @Override
     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

+ 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));
-    }
-
-}