Skip to content

Commit

Permalink
Use fixture to test repository-azure plugin (#29347)
Browse files Browse the repository at this point in the history
This commit adds a new fixture that emulates an
Azure Storage service in order to improve the
existing integration tests. This is very similar
to what has been made for Google Cloud Storage
in #28788 and for Amazon S3 in #29296, and it
would have helped a lot to catch bugs like #22534.
  • Loading branch information
tlrx authored Apr 6, 2018
1 parent 85f5382 commit 26fc8ad
Show file tree
Hide file tree
Showing 8 changed files with 806 additions and 136 deletions.
30 changes: 25 additions & 5 deletions plugins/repository-azure/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
import org.elasticsearch.gradle.test.AntFixture

esplugin {
description 'The Azure Repository plugin adds support for Azure storage repositories.'
Expand All @@ -42,9 +43,28 @@ thirdPartyAudit.excludes = [
'org.slf4j.LoggerFactory',
]

integTestCluster {
keystoreSetting 'azure.client.default.account', 'cloudazureresource'
keystoreSetting 'azure.client.default.key', 'abcdefgh'
keystoreSetting 'azure.client.secondary.account', 'cloudazureresource'
keystoreSetting 'azure.client.secondary.key', 'abcdefgh'
forbiddenApisTest {
// we are using jdk-internal instead of jdk-non-portable to allow for com.sun.net.httpserver.* usage
bundledSignatures -= 'jdk-non-portable'
bundledSignatures += 'jdk-internal'
}

/** A task to start the fixture which emulates an Azure Storage service **/
task azureStorageFixture(type: AntFixture) {
dependsOn compileTestJava
env 'CLASSPATH', "${ -> project.sourceSets.test.runtimeClasspath.asPath }"
executable = new File(project.runtimeJavaHome, 'bin/java')
args 'org.elasticsearch.repositories.azure.AzureStorageFixture', baseDir, 'container_test'
}

integTestCluster {
dependsOn azureStorageFixture

keystoreSetting 'azure.client.integration_test.account', "azure_integration_test_account"
/* The key is "azure_integration_test_key" encoded using base64 */
keystoreSetting 'azure.client.integration_test.key', "YXp1cmVfaW50ZWdyYXRpb25fdGVzdF9rZXk="
// Use a closure on the string to delay evaluation until tests are executed. The endpoint_suffix is used
// in a hacky way to change the protocol and endpoint. We must fix that.
setting 'azure.client.integration_test.endpoint_suffix',
"ignored;DefaultEndpointsProtocol=http;BlobEndpoint=http://${ -> azureStorageFixture.addressAndPort }"
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* 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.cluster.metadata.RepositoryMetaData;
import org.elasticsearch.common.blobstore.BlobStore;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.repositories.ESBlobStoreTestCase;

import java.io.IOException;
import java.net.URISyntaxException;

public class AzureBlobStoreTests extends ESBlobStoreTestCase {

@Override
protected BlobStore newBlobStore() throws IOException {
try {
RepositoryMetaData repositoryMetaData = new RepositoryMetaData("azure", "ittest", Settings.EMPTY);
AzureStorageServiceMock client = new AzureStorageServiceMock();
return new AzureBlobStore(repositoryMetaData, Settings.EMPTY, client);
} catch (URISyntaxException | StorageException e) {
throw new IOException(e);
}
}
}

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,6 @@ private AzureRepository azureRepository(Settings settings) throws StorageExcepti
TestEnvironment.newEnvironment(internalSettings), NamedXContentRegistry.EMPTY, null);
}


public void testReadonlyDefault() throws StorageException, IOException, URISyntaxException {
assertThat(azureRepository(Settings.EMPTY).isReadOnly(), is(false));
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,136 @@
/*
* 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.sun.net.httpserver.HttpExchange;
import com.sun.net.httpserver.HttpHandler;
import com.sun.net.httpserver.HttpServer;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.io.Streams;
import org.elasticsearch.mocksocket.MockHttpServer;

import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.net.Inet6Address;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.SocketAddress;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.nio.file.StandardCopyOption;
import java.util.List;
import java.util.Map;

import static java.util.Collections.singleton;
import static java.util.Collections.singletonList;

/**
* {@link AzureStorageFixture} is a fixture that emulates an Azure Storage service.
* <p>
* It starts an asynchronous socket server that binds to a random local port. The server parses
* HTTP requests and uses a {@link AzureStorageTestServer} to handle them before returning
* them to the client as HTTP responses.
*/
public class AzureStorageFixture {

public static void main(String[] args) throws Exception {
if (args == null || args.length != 2) {
throw new IllegalArgumentException("AzureStorageFixture <working directory> <container>");
}

final InetSocketAddress socketAddress = new InetSocketAddress(InetAddress.getLoopbackAddress(), 0);
final HttpServer httpServer = MockHttpServer.createHttp(socketAddress, 0);

try {
final Path workingDirectory = workingDir(args[0]);
/// Writes the PID of the current Java process in a `pid` file located in the working directory
writeFile(workingDirectory, "pid", ManagementFactory.getRuntimeMXBean().getName().split("@")[0]);

final String addressAndPort = addressToString(httpServer.getAddress());
// Writes the address and port of the http server in a `ports` file located in the working directory
writeFile(workingDirectory, "ports", addressAndPort);

// Emulates Azure
final String storageUrl = "http://" + addressAndPort;
final AzureStorageTestServer testServer = new AzureStorageTestServer(storageUrl);
testServer.createContainer(args[1]);

httpServer.createContext("/", new ResponseHandler(testServer));
httpServer.start();

// Wait to be killed
Thread.sleep(Long.MAX_VALUE);

} finally {
httpServer.stop(0);
}
}

@SuppressForbidden(reason = "Paths#get is fine - we don't have environment here")
private static Path workingDir(final String dir) {
return Paths.get(dir);
}

private static void writeFile(final Path dir, final String fileName, final String content) throws IOException {
final Path tempPidFile = Files.createTempFile(dir, null, null);
Files.write(tempPidFile, singleton(content));
Files.move(tempPidFile, dir.resolve(fileName), StandardCopyOption.ATOMIC_MOVE);
}

private static String addressToString(final SocketAddress address) {
final InetSocketAddress inetSocketAddress = (InetSocketAddress) address;
if (inetSocketAddress.getAddress() instanceof Inet6Address) {
return "[" + inetSocketAddress.getHostString() + "]:" + inetSocketAddress.getPort();
} else {
return inetSocketAddress.getHostString() + ":" + inetSocketAddress.getPort();
}
}

static class ResponseHandler implements HttpHandler {

private final AzureStorageTestServer server;

private ResponseHandler(final AzureStorageTestServer server) {
this.server = server;
}

@Override
public void handle(HttpExchange exchange) throws IOException {
String method = exchange.getRequestMethod();
String path = server.getEndpoint() + exchange.getRequestURI().getRawPath();
String query = exchange.getRequestURI().getRawQuery();
Map<String, List<String>> headers = exchange.getRequestHeaders();
ByteArrayOutputStream out = new ByteArrayOutputStream();
Streams.copy(exchange.getRequestBody(), out);

final AzureStorageTestServer.Response response = server.handle(method, path, query, headers, out.toByteArray());

Map<String, List<String>> responseHeaders = exchange.getResponseHeaders();
responseHeaders.put("Content-Type", singletonList(response.contentType));
response.headers.forEach((k, v) -> responseHeaders.put(k, singletonList(v)));
exchange.sendResponseHeaders(response.status.getStatus(), response.body.length);
if (response.body.length > 0) {
exchange.getResponseBody().write(response.body);
}
exchange.close();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,8 @@ public void createContainer(String account, LocationMode mode, String container)

@Override
public void deleteFiles(String account, LocationMode mode, String container, String path) {
final Map<String, BlobMetaData> blobs = listBlobsByPrefix(account, mode, container, path, null);
blobs.keySet().forEach(key -> deleteBlob(account, mode, container, key));
}

@Override
Expand Down
Loading

0 comments on commit 26fc8ad

Please sign in to comment.