From f456095e9b54d3256dd1148c217156fb6826950b Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Fri, 10 Jul 2020 02:48:33 +0000 Subject: [PATCH 01/24] implemented an azure blob storage filesystem removed ignored test from this PR removed unused file fixed a typo small fixes small fixes small fixes small fixes for readability cleaning code small fixes ignore mockito config, move test file implemented an azure blob storage filesystem removed ignored test from this PR removed unused file fixed a typo small fixes small fixes small fixes small fixes for readability cleaning code small fixes ignore mockito config, move test file implemented an azure blob storage filesystem removed ignored test from this PR removed unused file fixed a typo small fixes small fixes small fixes small fixes for readability cleaning code small fixes ignore mockito config, move test file organized dependencies fixed a typo --- build.gradle | 1 + sdks/java/io/azure/build.gradle | 17 +- .../io/azure/blobstore/AzfsResourceId.java | 62 ++- .../blobstore/AzureBlobStoreFileSystem.java | 399 +++++++++++++++++- .../AzureBlobStoreFileSystemRegistrar.java | 10 +- .../AzureReadableSeekableByteChannel.java | 118 ++++++ .../DefaultBlobstoreClientBuilderFactory.java | 47 +++ .../sdk/io/azure/options/AzureOptions.java | 86 ++++ .../AzurePipelineOptionsRegistrar.java | 36 ++ .../BlobstoreClientBuilderFactory.java | 25 ++ .../io/azure/options/BlobstoreOptions.java | 41 ++ .../sdk/io/azure/options/package-info.java | 23 + .../azure/blobstore/AzfsResourceIdTest.java | 23 +- .../sdk/io/azure/blobstore/AzfsTestUtils.java | 42 ++ .../AzureBlobStoreFileSystemTest.java | 353 ++++++++++++++++ sdks/java/io/azure/src/test/resources/in.txt | 23 + .../org.mockito.plugins.MockMaker | 1 + 17 files changed, 1261 insertions(+), 46 deletions(-) create mode 100644 sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureReadableSeekableByteChannel.java create mode 100644 sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java create mode 100644 sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java create mode 100644 sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzurePipelineOptionsRegistrar.java create mode 100644 sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreClientBuilderFactory.java create mode 100644 sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java create mode 100644 sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/package-info.java create mode 100644 sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsTestUtils.java create mode 100644 sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java create mode 100644 sdks/java/io/azure/src/test/resources/in.txt create mode 100644 sdks/java/io/azure/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker diff --git a/build.gradle b/build.gradle index 44a944928a5e..e2dec896c6f1 100644 --- a/build.gradle +++ b/build.gradle @@ -118,6 +118,7 @@ rat { // Mockito extensions "sdks/java/io/amazon-web-services2/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker", + "sdks/java/io/azure/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker", "sdks/java/extensions/ml/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker", // JupyterLab extensions diff --git a/sdks/java/io/azure/build.gradle b/sdks/java/io/azure/build.gradle index 3860acad8712..502e7fa83cdd 100644 --- a/sdks/java/io/azure/build.gradle +++ b/sdks/java/io/azure/build.gradle @@ -16,12 +16,27 @@ * limitations under the License. */ -plugins { id 'org.apache.beam.module' } +plugins { + id 'org.apache.beam.module' +} + applyJavaNature( automaticModuleName: 'org.apache.beam.sdk.io.azure', enableChecker: false) +description = "Apache Beam :: SDKs :: Java :: IO :: Azure" +ext.summary = "IO library to read and write Azure services from Beam." + dependencies { + compile library.java.vendored_guava_26_0_jre compile project(path: ":sdks:java:core", configuration: "shadow") + compile "com.azure:azure-storage-blob:12.1.0" + compile "com.azure:azure-identity:1.0.8" + compile "com.microsoft.azure:azure-storage:8.0.0" + compile "commons-io:commons-io:2.6" + compile library.java.slf4j_api + testCompile project(path: ":sdks:java:core", configuration: "shadowTest") + testCompile library.java.mockito_core testCompile library.java.junit + testRuntimeOnly library.java.slf4j_jdk14 } diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java index 6bf4df9f2b73..39cfe8f70598 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java @@ -20,33 +20,39 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; +import java.util.Date; import java.util.Objects; import java.util.regex.Matcher; import java.util.regex.Pattern; -import javax.annotation.Nullable; import org.apache.beam.sdk.io.fs.ResolveOptions; import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Optional; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; +import org.checkerframework.checker.nullness.qual.Nullable; class AzfsResourceId implements ResourceId { static final String SCHEME = "azfs"; - private static final Pattern AZFS_URI = Pattern.compile("(?[^:]+)://(?[^/]+)/(?[^/]+)(?:/(?.*))?"); - /** Matches a glob containing a wildcard, capturing the portion before the first wildcard. */ private static final Pattern GLOB_PREFIX = Pattern.compile("(?[^\\[*?]*)[\\[*?].*"); private final String account; private final String container; private final String blob; - - private AzfsResourceId(String account, String container, @Nullable String blob) { + private final Long size; + private final Date lastModified; + + private AzfsResourceId( + String account, + String container, + @Nullable String blob, + @Nullable Long size, + @Nullable Date lastModified) { // We are assuming that every resource id is either a container or a blob in a container, not - // just an account. - // This is because we will not enable users to create Azure containers through beam at this - // time. + // just an account. This is because we will not enable users to create Azure containers through + // beam at this time. checkArgument(!Strings.isNullOrEmpty(container), "container"); checkArgument(!container.contains("/"), "container must not contain '/': [%s]", container); this.account = account; @@ -56,14 +62,16 @@ private AzfsResourceId(String account, String container, @Nullable String blob) } else { this.blob = blob; } + this.size = size; + this.lastModified = lastModified; } static AzfsResourceId fromComponents(String account, String container, String blob) { - return new AzfsResourceId(account, container, blob); + return new AzfsResourceId(account, container, blob, null, null); } static AzfsResourceId fromComponents(String account, String container) { - return new AzfsResourceId(account, container, null); + return new AzfsResourceId(account, container, null, null, null); } static AzfsResourceId fromUri(String uri) { @@ -93,6 +101,22 @@ public String getScheme() { return SCHEME; } + Optional getSize() { + return Optional.fromNullable(size); + } + + AzfsResourceId withSize(long size) { + return new AzfsResourceId(account, container, blob, size, lastModified); + } + + Optional getLastModified() { + return Optional.fromNullable(lastModified); + } + + AzfsResourceId withLastModified(Date lastModified) { + return new AzfsResourceId(account, container, blob, size, lastModified); + } + @Override public boolean isDirectory() { return (blob == null) || (blob.endsWith("/")); @@ -105,6 +129,17 @@ boolean isWildcard() { return GLOB_PREFIX.matcher(blob).matches(); } + String getBlobNonWildcardPrefix() { + Matcher m = GLOB_PREFIX.matcher(getBlob()); + checkArgument( + m.matches(), String.format("Glob expression: [%s] is not expandable.", getBlob())); + return m.group("PREFIX"); + } + + public boolean isContainer() { + return blob == null; + } + @Override public ResourceId getCurrentDirectory() { if (isDirectory()) { @@ -129,7 +164,6 @@ public String getFilename() { return blobWithoutTrailingSlash.substring(blobWithoutTrailingSlash.lastIndexOf('/') + 1); } - // TODO: ensure that this function lines up with what the filesystem match method expects @Override public String toString() { if (blob != null) { @@ -156,8 +190,6 @@ public int hashCode() { @Override public ResourceId resolve(String other, ResolveOptions resolveOptions) { checkState(isDirectory(), "Expected this resource to be a directory, but was [%s]", toString()); - // TODO: check if resolve options are an illegal name in any way, see: - // https://docs.microsoft.com/en-us/rest/api/storageservices/Naming-and-Referencing-Containers--Blobs--and-Metadata if (resolveOptions == ResolveOptions.StandardResolveOptions.RESOLVE_DIRECTORY) { if ("..".equals(other)) { @@ -167,11 +199,9 @@ public ResourceId resolve(String other, ResolveOptions resolveOptions) { int parentStopsAt = blob.substring(0, blob.length() - 1).lastIndexOf('/'); return fromComponents(account, container, blob.substring(0, parentStopsAt + 1)); } - if ("".equals(other)) { return this; } - if (!other.endsWith("/")) { other += "/"; } @@ -183,7 +213,6 @@ public ResourceId resolve(String other, ResolveOptions resolveOptions) { } return fromComponents(account, container, blob + other); } - if (resolveOptions == ResolveOptions.StandardResolveOptions.RESOLVE_FILE) { checkArgument( !other.endsWith("/"), "Cannot resolve a file with a directory path: [%s]", other); @@ -196,7 +225,6 @@ public ResourceId resolve(String other, ResolveOptions resolveOptions) { } return fromComponents(account, container, blob + other); } - throw new UnsupportedOperationException( String.format("Unexpected StandardResolveOptions [%s]", resolveOptions)); } diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java index 843da202e96c..6af5dabb02e8 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java @@ -17,67 +17,438 @@ */ package org.apache.beam.sdk.io.azure.blobstore; +import static java.nio.channels.Channels.newChannel; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.storage.blob.BlobClient; +import com.azure.storage.blob.BlobContainerClient; +import com.azure.storage.blob.BlobServiceClient; +import com.azure.storage.blob.BlobServiceClientBuilder; +import com.azure.storage.blob.models.BlobItem; +import com.azure.storage.blob.models.BlobProperties; +import com.azure.storage.blob.models.BlobStorageException; +import com.azure.storage.blob.models.ListBlobsOptions; +import com.microsoft.azure.storage.CloudStorageAccount; +import com.microsoft.azure.storage.SharedAccessAccountPolicy; +import java.io.FileNotFoundException; import java.io.IOException; +import java.io.OutputStream; import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; +import java.time.Duration; +import java.util.ArrayList; import java.util.Collection; +import java.util.Date; +import java.util.Iterator; import java.util.List; +import java.util.regex.Pattern; +import java.util.stream.Collectors; import org.apache.beam.sdk.io.FileSystem; +import org.apache.beam.sdk.io.azure.options.BlobstoreClientBuilderFactory; +import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; import org.apache.beam.sdk.io.fs.CreateOptions; import org.apache.beam.sdk.io.fs.MatchResult; +import org.apache.beam.sdk.util.InstanceBuilder; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Suppliers; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class AzureBlobStoreFileSystem extends FileSystem { + private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class); + + private static final ImmutableSet NON_READ_SEEK_EFFICIENT_ENCODINGS = + ImmutableSet.of("gzip"); + + private Supplier client; + private final BlobstoreOptions options; + + AzureBlobStoreFileSystem(BlobstoreOptions options) { + this.options = checkNotNull(options, "options"); + + BlobServiceClientBuilder builder = + InstanceBuilder.ofType(BlobstoreClientBuilderFactory.class) + .fromClass(options.getBlobstoreClientFactoryClass()) + .build() + .createBuilder(options); + + // The Supplier is to make sure we don't call .build() unless we are actually using Azure. + client = Suppliers.memoize(builder::buildClient); + } + + @VisibleForTesting + void setClient(BlobServiceClient client) { + this.client = Suppliers.ofInstance(client); + } + + @VisibleForTesting + BlobServiceClient getClient() { + return client.get(); + } + @Override protected String getScheme() { - return "azfs"; + return AzfsResourceId.SCHEME; } @Override - protected List match(List specs) throws IOException { - // TODO - return null; + protected List match(List specs) { + List paths = + specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList()); + List globs = new ArrayList<>(); + List nonGlobs = new ArrayList<>(); + List isGlobBooleans = new ArrayList<>(); + + for (AzfsResourceId path : paths) { + if (path.isWildcard()) { + globs.add(path); + isGlobBooleans.add(true); + } else { + nonGlobs.add(path); + isGlobBooleans.add(false); + } + } + + Iterator globMatches = matchGlobPaths(globs).iterator(); + Iterator nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator(); + + ImmutableList.Builder matchResults = ImmutableList.builder(); + for (Boolean isGlob : isGlobBooleans) { + if (isGlob) { + checkState(globMatches.hasNext(), "Expect globMatches has next."); + matchResults.add(globMatches.next()); + } else { + checkState(nonGlobMatches.hasNext(), "Expect nonGlobMatches has next."); + matchResults.add(nonGlobMatches.next()); + } + } + checkState(!globMatches.hasNext(), "Expect no more elements in globMatches."); + checkState(!nonGlobMatches.hasNext(), "Expect no more elements in nonGlobMatches."); + + return matchResults.build(); + } + + /** + * Expands glob expressions to regular expressions. + * + * @param globExp the glob expression to expand + * @return a string with the regular expression this glob expands to + */ + @VisibleForTesting + static String wildcardToRegexp(String globExp) { + StringBuilder dst = new StringBuilder(); + char[] src = globExp.replace("**/*", "**").toCharArray(); + int i = 0; + while (i < src.length) { + char c = src[i++]; + switch (c) { + case '*': + // One char lookahead for ** + if (i < src.length && src[i] == '*') { + dst.append(".*"); + ++i; + } else { + dst.append("[^/]*"); + } + break; + case '?': + dst.append("[^/]"); + break; + case '.': + case '+': + case '{': + case '}': + case '(': + case ')': + case '|': + case '^': + case '$': + // These need to be escaped in regular expressions + dst.append('\\').append(c); + break; + case '\\': + i = doubleSlashes(dst, src, i); + break; + default: + dst.append(c); + break; + } + } + return dst.toString(); + } + + private static int doubleSlashes(StringBuilder dst, char[] src, int i) { + // Emit the next character without special interpretation + dst.append("\\\\"); + if ((i - 1) != src.length) { + dst.append(src[i]); + i++; + } else { + // A backslash at the very end is treated like an escaped backslash + dst.append('\\'); + } + return i; + } + + private List matchGlobPaths(List globs) { + return FluentIterable.from(globs).transform(this::expand).toList(); + } + + /** Expands a pattern into {@link MatchResult}. */ + @VisibleForTesting + MatchResult expand(AzfsResourceId azfsPattern) { + + checkArgument(azfsPattern.isWildcard(), "is Wildcard"); + String blobPrefix = azfsPattern.getBlobNonWildcardPrefix(); + Pattern wildcardAsRegexp = Pattern.compile(wildcardToRegexp(azfsPattern.getBlob())); + + LOG.debug( + "matching files in container {}, prefix {} against pattern {}", + azfsPattern.getContainer(), + blobPrefix, + wildcardAsRegexp.toString()); + + ListBlobsOptions listOptions = new ListBlobsOptions().setPrefix(blobPrefix); + Duration timeout = Duration.ZERO.plusMinutes(1); + + String account = azfsPattern.getAccount(); + String container = azfsPattern.getContainer(); + BlobContainerClient blobContainerClient = client.get().getBlobContainerClient(container); + PagedIterable blobs = blobContainerClient.listBlobs(listOptions, timeout); + List results = new ArrayList<>(); + + blobs.forEach( + blob -> { + String name = blob.getName(); + if (wildcardAsRegexp.matcher(name).matches() && !name.endsWith("/")) { + LOG.debug("Matched object: {}", name); + + BlobProperties properties = blobContainerClient.getBlobClient(name).getProperties(); + AzfsResourceId rid = + AzfsResourceId.fromComponents(account, container, name) + .withSize(properties.getBlobSize()) + .withLastModified(Date.from(properties.getLastModified().toInstant())); + + results.add(toMetadata(rid, properties.getContentEncoding())); + } + }); + + return MatchResult.create(MatchResult.Status.OK, results); + } + + private MatchResult.Metadata toMetadata(AzfsResourceId path, String contentEncoding) { + + checkArgument(path.getSize().isPresent(), "path has size"); + boolean isReadSeekEfficient = !NON_READ_SEEK_EFFICIENT_ENCODINGS.contains(contentEncoding); + + return MatchResult.Metadata.builder() + .setIsReadSeekEfficient(isReadSeekEfficient) + .setResourceId(path) + .setSizeBytes(path.getSize().get()) + .setLastModifiedMillis(path.getLastModified().transform(Date::getTime).or(0L)) + .build(); + } + + /** + * Returns {@link MatchResult MatchResults} for the given {@link AzfsResourceId paths}. + * + *

The number of returned {@link MatchResult MatchResults} equals to the number of given {@link + * AzfsResourceId paths}. Each {@link MatchResult} contains one {@link MatchResult.Metadata}. + */ + @VisibleForTesting + private Iterable matchNonGlobPaths(List paths) { + ImmutableList.Builder toReturn = ImmutableList.builder(); + for (AzfsResourceId path : paths) { + toReturn.add(toMatchResult(path)); + } + return toReturn.build(); + } + + private MatchResult toMatchResult(AzfsResourceId path) { + BlobClient blobClient = + client.get().getBlobContainerClient(path.getContainer()).getBlobClient(path.getBlob()); + BlobProperties blobProperties; + + try { + blobProperties = blobClient.getProperties(); + } catch (BlobStorageException e) { + if (e.getStatusCode() == 404) { + return MatchResult.create(MatchResult.Status.NOT_FOUND, new FileNotFoundException()); + } + return MatchResult.create(MatchResult.Status.ERROR, new IOException(e)); + } + + return MatchResult.create( + MatchResult.Status.OK, + ImmutableList.of( + toMetadata( + path.withSize(blobProperties.getBlobSize()) + .withLastModified(Date.from(blobProperties.getLastModified().toInstant())), + blobProperties.getContentEncoding()))); } @Override protected WritableByteChannel create(AzfsResourceId resourceId, CreateOptions createOptions) throws IOException { - // TODO - return null; + BlobContainerClient blobContainerClient = + client.get().getBlobContainerClient(resourceId.getContainer()); + if (!blobContainerClient.exists()) { + throw new UnsupportedOperationException("create does not create containers."); + } + + BlobClient blobClient = blobContainerClient.getBlobClient(resourceId.getBlob()); + // The getBlobOutputStream method overwrites existing blobs, + // so throw an error in this case to prevent data loss + if (blobClient.exists()) { + throw new IOException("This filename is already in use."); + } + + OutputStream outputStream; + try { + outputStream = blobClient.getBlockBlobClient().getBlobOutputStream(); + } catch (BlobStorageException e) { + throw (IOException) e.getCause(); + } + return newChannel(outputStream); } @Override protected ReadableByteChannel open(AzfsResourceId resourceId) throws IOException { - // TODO - return null; + BlobClient blobClient = + client + .get() + .getBlobContainerClient(resourceId.getContainer()) + .getBlobClient(resourceId.getBlob()); + if (!blobClient.exists()) { + throw new FileNotFoundException("The requested file doesn't exist."); + } + return new AzureReadableSeekableByteChannel(blobClient); } @Override protected void copy(List srcPaths, List destPaths) throws IOException { - // TODO + checkArgument( + srcPaths.size() == destPaths.size(), + "sizes of source paths and destination paths do not match"); + + Iterator sourcePathsIterator = srcPaths.iterator(); + Iterator destinationPathsIterator = destPaths.iterator(); + while (sourcePathsIterator.hasNext()) { + final AzfsResourceId sourcePath = sourcePathsIterator.next(); + final AzfsResourceId destinationPath = destinationPathsIterator.next(); + copy(sourcePath, destinationPath); + } } @VisibleForTesting void copy(AzfsResourceId sourcePath, AzfsResourceId destinationPath) throws IOException { - // TODO + checkArgument( + sourcePath.getBlob() != null && destinationPath.getBlob() != null, + "This method is intended to copy file-like resources, not directories."); + + // get source blob client + BlobClient srcBlobClient = + client + .get() + .getBlobContainerClient(sourcePath.getContainer()) + .getBlobClient(sourcePath.getBlob()); + if (!srcBlobClient.exists()) { + throw new FileNotFoundException("The copy source does not exist."); + } + + // get destination blob client + BlobContainerClient destBlobContainerClient = + client.get().getBlobContainerClient(destinationPath.getContainer()); + if (!destBlobContainerClient.exists()) { + client.get().createBlobContainer(destinationPath.getContainer()); + } + BlobClient destBlobClient = destBlobContainerClient.getBlobClient(destinationPath.getBlob()); + + destBlobClient.copyFromUrl(srcBlobClient.getBlobUrl() + generateSasToken()); + } + + @VisibleForTesting + String generateSasToken() throws IOException { + SharedAccessAccountPolicy sharedAccessAccountPolicy = new SharedAccessAccountPolicy(); + long date = new Date().getTime(); + long expiryDate = new Date(date + 8640000).getTime(); + + sharedAccessAccountPolicy.setPermissionsFromString("racwdlup"); + sharedAccessAccountPolicy.setSharedAccessStartTime(new Date(date)); + sharedAccessAccountPolicy.setSharedAccessExpiryTime(new Date(expiryDate)); + sharedAccessAccountPolicy.setResourceTypeFromString( + "co"); // container, object, add s for service + sharedAccessAccountPolicy.setServiceFromString("b"); // blob, add "fqt" for file, queue, table + + String storageConnectionString = options.getAzureConnectionString(); + try { + CloudStorageAccount storageAccount = CloudStorageAccount.parse(storageConnectionString); + return "?" + storageAccount.generateSharedAccessSignature(sharedAccessAccountPolicy); + } catch (Exception e) { + throw (IOException) e.getCause(); + } } @Override protected void rename(List srcResourceIds, List destResourceIds) throws IOException { - // TODO + copy(srcResourceIds, destResourceIds); + delete(srcResourceIds); } + // This method with delete a virtual folder or a blob @Override protected void delete(Collection resourceIds) throws IOException { - // TODO + for (AzfsResourceId resourceId : resourceIds) { + if (resourceId.getBlob() == null) { + throw new IOException("delete does not delete containers."); + } + + BlobContainerClient container = + client.get().getBlobContainerClient(resourceId.getContainer()); + + // deleting a blob that is not a directory + if (!resourceId.isDirectory()) { + BlobClient blob = container.getBlobClient(resourceId.getBlob()); + if (!blob.exists()) { + throw new FileNotFoundException("The resource to delete does not exist."); + } + blob.delete(); + } + + // deleting a directory (not a container) + else { + PagedIterable blobsInDirectory = + container.listBlobsByHierarchy(resourceId.getBlob()); + for (BlobItem blob : blobsInDirectory) { + String blobName = blob.getName(); + container.getBlobClient(blobName).delete(); + } + } + } } @Override protected AzfsResourceId matchNewResource(String singleResourceSpec, boolean isDirectory) { - // TODO - return null; + if (isDirectory) { + if (!singleResourceSpec.endsWith("/")) { + singleResourceSpec += "/"; + } + } else { + checkArgument( + !singleResourceSpec.endsWith("/"), + "Expected a file path, but [%s] ends with '/'. This is unsupported in AzfsFileSystem.", + singleResourceSpec); + } + return AzfsResourceId.fromUri(singleResourceSpec); } } diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemRegistrar.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemRegistrar.java index 33a705461baa..51e8e781d9ae 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemRegistrar.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemRegistrar.java @@ -19,17 +19,23 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; +import com.google.auto.service.AutoService; import javax.annotation.Nonnull; +import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.io.FileSystem; import org.apache.beam.sdk.io.FileSystemRegistrar; +import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +/** {@link AutoService} registrar for the {@link AzureBlobStoreFileSystem}. */ +@AutoService(FileSystemRegistrar.class) +@Experimental(Experimental.Kind.FILESYSTEM) public class AzureBlobStoreFileSystemRegistrar implements FileSystemRegistrar { + @Override public Iterable> fromOptions(@Nonnull PipelineOptions options) { checkNotNull(options, "Expect the runner have called FileSystems.setDefaultPipelineOptions()."); - // TODO - return ImmutableList.of(); + return ImmutableList.of(new AzureBlobStoreFileSystem(options.as(BlobstoreOptions.class))); } } diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureReadableSeekableByteChannel.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureReadableSeekableByteChannel.java new file mode 100644 index 000000000000..e88cbf96aeee --- /dev/null +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureReadableSeekableByteChannel.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.blobstore; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import com.azure.storage.blob.BlobClient; +import com.azure.storage.blob.specialized.BlobInputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; +import java.nio.channels.SeekableByteChannel; + +class AzureReadableSeekableByteChannel implements SeekableByteChannel { + + private final BlobInputStream inputStream; + private boolean closed; + private final long contentLength; + private long position = 0; + + public AzureReadableSeekableByteChannel(BlobClient blobClient) { + inputStream = blobClient.openInputStream(); + contentLength = blobClient.getProperties().getBlobSize(); + closed = false; + } + + @Override + public int read(ByteBuffer dst) throws IOException { + if (closed) { + throw new ClosedChannelException(); + } + if (!dst.hasRemaining()) { + return 0; + } + + int read = 0; + if (dst.hasArray()) { + // Stores up to dst.remaining() bytes into dst.array() starting at dst.position(). + // But dst can have an offset with its backing array, hence the + dst.arrayOffset(). + read = inputStream.read(dst.array(), dst.position() + dst.arrayOffset(), dst.remaining()); + } else { + byte[] myarray = new byte[dst.remaining()]; + read = inputStream.read(myarray, 0, myarray.length); + dst.put(myarray); + } + + if (read > 0) { + position += read; + } + return read; + } + + @Override + public int write(ByteBuffer src) { + throw new UnsupportedOperationException(); + } + + @Override + public long position() throws IOException { + if (closed) { + throw new ClosedChannelException(); + } + return position; + } + + @Override + public SeekableByteChannel position(long newPosition) throws IOException { + if (closed) { + throw new ClosedChannelException(); + } + checkArgument(newPosition >= 0, "newPosition too low"); + checkArgument(newPosition < contentLength, "new position too high"); + + Long bytesToSkip = newPosition - position; + Long n = inputStream.skip(bytesToSkip); + position += n; + return this; + } + + @Override + public long size() throws IOException { + if (closed) { + throw new ClosedChannelException(); + } + return contentLength; + } + + @Override + public SeekableByteChannel truncate(long size) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isOpen() { + return !closed; + } + + @Override + public void close() throws IOException { + closed = true; + inputStream.close(); + } +} diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java new file mode 100644 index 000000000000..c8d2c1ae5e4d --- /dev/null +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.blobstore; + +import com.azure.storage.blob.BlobServiceClientBuilder; +import org.apache.beam.sdk.io.azure.options.BlobstoreClientBuilderFactory; +import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; + +/** Construct BlobServiceClientBuilder with default values of Azure client properties. */ +public class DefaultBlobstoreClientBuilderFactory implements BlobstoreClientBuilderFactory { + + // TODO: add any other options that should be passed to BlobServiceClientBuilder + + @Override + public BlobServiceClientBuilder createBuilder(BlobstoreOptions blobstoreOptions) { + BlobServiceClientBuilder builder = new BlobServiceClientBuilder(); + + if (blobstoreOptions.getClientConfiguration() != null) { + builder = builder.configuration(blobstoreOptions.getClientConfiguration()); + } + + if (blobstoreOptions.getAzureConnectionString() != null) { + builder.connectionString(blobstoreOptions.getAzureConnectionString()); + } + + if (!Strings.isNullOrEmpty(blobstoreOptions.getAzureServiceEndpoint())) { + builder = builder.endpoint(blobstoreOptions.getAzureServiceEndpoint()); + } + return builder; + } +} diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java new file mode 100644 index 000000000000..c758cb4a4c78 --- /dev/null +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.options; + +import com.azure.core.credential.TokenCredential; +import com.azure.core.util.Configuration; +import com.azure.identity.DefaultAzureCredentialBuilder; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.DefaultValueFactory; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; + +public interface AzureOptions extends PipelineOptions { + + // TODO: Add any other azure options that users should be able to configure + // TODO: Confirm that Azure options are in this file, Blobstore options in BlobstoreOptions + + /** The Azure service endpoint used by the Azure client. */ + @Description("Azure service endpoint used by the Azure client") + String getAzureServiceEndpoint(); + + void setAzureServiceEndpoint(String value); + + /** + * The credential instance that should be used to authenticate against Azure services. The option + * value must contain a "@type" field and an Azure credentials provider class as the field value. + */ + @Description( + "The credential instance that should be used to authenticate " + + "against Azure services. The option value must contain \"@type\" field " + + "and an Azure credentials provider class name as the field value.") + @Default.InstanceFactory(AzureUserCredentialsFactory.class) + TokenCredential getAzureCredentialsProvider(); + + void setAzureCredentialsProvider(TokenCredential value); + + /** Attempts to load Azure credentials. */ + class AzureUserCredentialsFactory implements DefaultValueFactory { + + @Override + public TokenCredential create(PipelineOptions options) { + return new DefaultAzureCredentialBuilder().build(); + } + } + + /** The client configuration instance that should be used to configure Azure service clients. */ + @Description( + "The client configuration instance that should be used to configure Azure service clients") + @Default.InstanceFactory(ConfigurationFactory.class) + Configuration getClientConfiguration(); + + void setClientConfiguration(Configuration configuration); + + /** The client configuration instance that should be used to configure Azure service clients. */ + @Description( + "The client configuration instance that should be used to configure Azure http client configuration parameters." + + "Mentioned parameters are the available parameters that can be set. Set only those that need custom changes.") + @Default.InstanceFactory(ConfigurationFactory.class) + Configuration getAzureHttpConfiguration(); + + void setAzureHttpConfiguration(Configuration configuration); + + /** Default Azure client configuration. */ + class ConfigurationFactory implements DefaultValueFactory { + + @Override + public Configuration create(PipelineOptions options) { + return new Configuration(); + } + } +} diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzurePipelineOptionsRegistrar.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzurePipelineOptionsRegistrar.java new file mode 100644 index 000000000000..681342efb560 --- /dev/null +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzurePipelineOptionsRegistrar.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.options; + +import com.google.auto.service.AutoService; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsRegistrar; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; + +/** A registrar containing the default Azure options. */ +@AutoService(PipelineOptionsRegistrar.class) +public class AzurePipelineOptionsRegistrar implements PipelineOptionsRegistrar { + + @Override + public Iterable> getPipelineOptions() { + return ImmutableList.>builder() + .add(AzureOptions.class) + .add(BlobstoreOptions.class) + .build(); + } +} diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreClientBuilderFactory.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreClientBuilderFactory.java new file mode 100644 index 000000000000..ef6f15da919d --- /dev/null +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreClientBuilderFactory.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.options; + +import com.azure.storage.blob.BlobServiceClientBuilder; + +/** Construct BlobServiceClientBuilder from Azure pipeline options. */ +public interface BlobstoreClientBuilderFactory { + BlobServiceClientBuilder createBuilder(BlobstoreOptions blobstoreOptions); +} diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java new file mode 100644 index 000000000000..9529485918b3 --- /dev/null +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.options; + +import org.apache.beam.sdk.io.azure.blobstore.DefaultBlobstoreClientBuilderFactory; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; + +public interface BlobstoreOptions extends AzureOptions { + + // TODO: Add any other blobstore options that users should be able to configure + // TODO: Confirm that Blobstore options are in this file, Azure options in AzureOptions + + @Description( + "Factory class that should be created and used to create a builder of Azure Blobstore client." + + "Override the default value if you need a Azure client with custom properties.") + @Default.Class(DefaultBlobstoreClientBuilderFactory.class) + Class getBlobstoreClientFactoryClass(); + + void setBlobstoreClientFactoryClass( + Class blobstoreClientFactoryClass); + + String getAzureConnectionString(); + + void setAzureConnectionString(String connectionString); +} diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/package-info.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/package-info.java new file mode 100644 index 000000000000..bc7e0efb572a --- /dev/null +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ +/** Defines IO connectors for Microsoft Azure Blobstore. */ +@Experimental(Kind.FILESYSTEM) +package org.apache.beam.sdk.io.azure.options; + +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceIdTest.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceIdTest.java index ee281878d8be..83fb0cb396e2 100644 --- a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceIdTest.java +++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceIdTest.java @@ -27,8 +27,12 @@ import java.util.Arrays; import java.util.Collection; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; import org.apache.beam.sdk.io.fs.ResolveOptions; import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.io.fs.ResourceIdTester; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.runners.Enclosed; @@ -39,10 +43,8 @@ @RunWith(Enclosed.class) public class AzfsResourceIdTest { - @RunWith(Parameterized.class) public static class ResolveTest { - @Parameterized.Parameter(0) public String baseUri; @@ -112,7 +114,6 @@ public void testResolve() { @RunWith(JUnit4.class) public static class NonParameterizedTests { - @Rule public ExpectedException thrown = ExpectedException.none(); @Test @@ -136,7 +137,6 @@ public void testResolveInvalidInputs() { public void testResolveInvalidNotDirectory() { ResourceId tmpDir = AzfsResourceId.fromUri("azfs://account/my_container/").resolve("tmp dir", RESOLVE_FILE); - thrown.expect(IllegalStateException.class); thrown.expectMessage( "Expected this resource to be a directory, but was [azfs://account/my_container/tmp dir]"); @@ -162,19 +162,14 @@ public void testEquals() { AzfsResourceId a = AzfsResourceId.fromComponents("account", "container", "a/b/c"); AzfsResourceId b = AzfsResourceId.fromComponents("account", "container", "a/b/c"); assertEquals(a, b); - b = AzfsResourceId.fromComponents(a.getAccount(), a.getContainer(), "a/b/c/"); assertNotEquals(a, b); - b = AzfsResourceId.fromComponents(a.getAccount(), a.getContainer(), "x/y/z"); assertNotEquals(a, b); - b = AzfsResourceId.fromComponents(a.getAccount(), "other-container", a.getBlob()); assertNotEquals(a, b); - b = AzfsResourceId.fromComponents("other-account", a.getContainer(), a.getBlob()); assertNotEquals(a, b); - assertEquals( AzfsResourceId.fromUri("azfs://account/container"), AzfsResourceId.fromUri("azfs://account/container/")); @@ -285,11 +280,9 @@ public void testAzfsResourceIdToString() { String filename = "azfs://account/container/dir/file.txt"; AzfsResourceId path = AzfsResourceId.fromUri(filename); assertEquals(filename, path.toString()); - filename = "azfs://account/container/blob/"; path = AzfsResourceId.fromUri(filename); assertEquals(filename, path.toString()); - filename = "azfs://account/container/"; path = AzfsResourceId.fromUri(filename); assertEquals(filename, path.toString()); @@ -315,6 +308,12 @@ public void testIsWildcard() { assertFalse(AzfsResourceId.fromComponents("account", "container").isWildcard()); } - // TODO: Consider adding a ResourceIdTester.runResourceIdBattery() test + @Test + public void testResourceIdTester() { + BlobstoreOptions options = PipelineOptionsFactory.create().as(BlobstoreOptions.class); + FileSystems.setDefaultPipelineOptions(options); + ResourceIdTester.runResourceIdBattery( + AzfsResourceId.fromUri("azfs://account/container/blob/")); + } } } diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsTestUtils.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsTestUtils.java new file mode 100644 index 000000000000..1d4d2f972c43 --- /dev/null +++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsTestUtils.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.blobstore; + +import com.azure.storage.blob.BlobServiceClient; +import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.mockito.Mockito; + +class AzfsTestUtils { + static BlobstoreOptions azfsOptions() { + BlobstoreOptions options = PipelineOptionsFactory.as(BlobstoreOptions.class); + options.setAzureConnectionString(System.getenv("AZURE_STORAGE_CONNECTION_STRING")); + return options; + } + + static AzureBlobStoreFileSystem buildMockedAzureFileSystem(BlobstoreOptions options) { + return buildMockedAzureFileSystem(options, Mockito.mock(BlobServiceClient.class)); + } + + static AzureBlobStoreFileSystem buildMockedAzureFileSystem( + BlobstoreOptions options, BlobServiceClient client) { + AzureBlobStoreFileSystem azureFileSystem = new AzureBlobStoreFileSystem(options); + azureFileSystem.setClient(client); + return azureFileSystem; + } +} diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java new file mode 100644 index 000000000000..e69725d05a0c --- /dev/null +++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java @@ -0,0 +1,353 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.blobstore; + +import static java.util.UUID.randomUUID; +import static org.apache.beam.sdk.io.fs.CreateOptions.StandardCreateOptions.builder; +import static org.hamcrest.Matchers.contains; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import com.azure.storage.blob.BlobContainerClient; +import com.azure.storage.blob.BlobServiceClient; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; +import org.apache.beam.sdk.io.fs.MatchResult; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.commons.io.FileUtils; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +@SuppressWarnings("CannotMockFinalClass") // Mockito 2 and above can mock final classes +public class AzureBlobStoreFileSystemTest { + + private AzureBlobStoreFileSystem azureBlobStoreFileSystem; + + @Before + public void beforeClass() { + BlobstoreOptions options = PipelineOptionsFactory.as(BlobstoreOptions.class); + options.setAzureConnectionString(System.getenv("AZURE_STORAGE_CONNECTION_STRING")); + azureBlobStoreFileSystem = new AzureBlobStoreFileSystem(options); + + // TODO: Switch to mocks for BlobContainerClient, BlobClient, etc. + // azureBlobStoreFileSystem = buildMockedAzureFileSystem(azfsOptions()); + } + + @Test + public void testGetScheme() { + assertEquals("azfs", azureBlobStoreFileSystem.getScheme()); + } + + @Test + public void testGlobTranslation() { + assertEquals("foo", AzureBlobStoreFileSystem.wildcardToRegexp("foo")); + assertEquals("fo[^/]*o", AzureBlobStoreFileSystem.wildcardToRegexp("fo*o")); + assertEquals("f[^/]*o\\.[^/]", AzureBlobStoreFileSystem.wildcardToRegexp("f*o.?")); + assertEquals("foo-[0-9][^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo-[0-9]*")); + assertEquals("foo-[0-9].*", AzureBlobStoreFileSystem.wildcardToRegexp("foo-[0-9]**")); + assertEquals(".*foo", AzureBlobStoreFileSystem.wildcardToRegexp("**/*foo")); + assertEquals(".*foo", AzureBlobStoreFileSystem.wildcardToRegexp("**foo")); + assertEquals("foo/[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*")); + assertEquals("foo[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo*")); + assertEquals("foo/[^/]*/[^/]*/[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*/*/*")); + assertEquals("foo/[^/]*/.*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*/**")); + assertEquals("foo.*baz", AzureBlobStoreFileSystem.wildcardToRegexp("foo**baz")); + } + + @Test + public void testDelete() throws IOException { + String containerName = "test-container" + randomUUID(); + String blobName1 = "blob" + randomUUID(); + String blobName2 = "dir1/anotherBlob" + randomUUID(); + + // Create files to delete + BlobContainerClient blobContainerClient = + azureBlobStoreFileSystem.getClient().createBlobContainer(containerName); + assertTrue(blobContainerClient.exists()); + blobContainerClient.getBlobClient(blobName1).uploadFromFile("src/test/resources/in.txt"); + blobContainerClient.getBlobClient(blobName2).uploadFromFile("src/test/resources/in.txt"); + assertTrue(blobContainerClient.getBlobClient(blobName1).exists()); + assertTrue(blobContainerClient.getBlobClient(blobName2).exists()); + + // Delete the files + Collection toDelete = new ArrayList<>(); + String account = blobContainerClient.getAccountName(); + // delete blob + toDelete.add(AzfsResourceId.fromComponents(account, containerName, blobName1)); + // delete directory + toDelete.add(AzfsResourceId.fromComponents(account, containerName, "dir1/")); + azureBlobStoreFileSystem.delete(toDelete); + + // Ensure exception is thrown, clean up + assertFalse(blobContainerClient.getBlobClient(blobName1).exists()); + assertFalse(blobContainerClient.getBlobClient(blobName2).exists()); + assertThrows(FileNotFoundException.class, () -> azureBlobStoreFileSystem.delete(toDelete)); + blobContainerClient.delete(); + } + + @Test + public void testCopy() throws IOException { + BlobServiceClient blobServiceClient = azureBlobStoreFileSystem.getClient(); + String account = blobServiceClient.getAccountName(); + + List src = new ArrayList<>(); + List dest = new ArrayList<>(); + String srcContainer = "source-container" + randomUUID(); + String destContainer = "dest-container" + randomUUID(); + + // Create source file + BlobContainerClient srcContainerClient = blobServiceClient.createBlobContainer(srcContainer); + srcContainerClient.getBlobClient("src-blob").uploadFromFile("src/test/resources/in.txt"); + + // Copy source file to destination + src.add(AzfsResourceId.fromComponents(account, srcContainer, "src-blob")); + dest.add(AzfsResourceId.fromComponents(account, destContainer, "dest-blob")); + azureBlobStoreFileSystem.copy(src, dest); + + // Confirm the destination container was created + BlobContainerClient destContainerClient = + blobServiceClient.getBlobContainerClient(destContainer); + assertTrue(destContainerClient.getBlobClient("dest-blob").exists()); + + // Confirm that the source and destination files are the same + srcContainerClient.getBlobClient("src-blob").downloadToFile("./src/test/resources/blob1"); + destContainerClient.getBlobClient("dest-blob").downloadToFile("./src/test/resources/blob2"); + File file1 = new File("./src/test/resources/blob1"); + File file2 = new File("./src/test/resources/blob2"); + assertTrue("The files differ!", FileUtils.contentEquals(file1, file2)); + + // Clean up + assertTrue(file1.delete()); + assertTrue(file2.delete()); + blobServiceClient.deleteBlobContainer(srcContainer); + blobServiceClient.deleteBlobContainer(destContainer); + } + + @Test + public void testWriteAndRead() throws IOException { + BlobServiceClient client = azureBlobStoreFileSystem.getClient(); + String containerName = "test-container" + randomUUID(); + client.createBlobContainer(containerName); + + byte[] writtenArray = new byte[] {0}; + ByteBuffer bb = ByteBuffer.allocate(writtenArray.length); + bb.put(writtenArray); + + // Create an object and write data to it + AzfsResourceId path = + AzfsResourceId.fromUri( + "azfs://" + client.getAccountName() + "/" + containerName + "/foo/bar.txt"); + WritableByteChannel writableByteChannel = + azureBlobStoreFileSystem.create(path, builder().setMimeType("application/text").build()); + writableByteChannel.write(bb); + writableByteChannel.close(); + + // Read the same object + ByteBuffer bb2 = ByteBuffer.allocate(writtenArray.length); + ReadableByteChannel open = azureBlobStoreFileSystem.open(path); + open.read(bb2); + + // Compare the content with the one that was written + byte[] readArray = bb2.array(); + assertArrayEquals(readArray, writtenArray); + open.close(); + + // Clean up + client.getBlobContainerClient(containerName).delete(); + } + + @Test + public void testGlobExpansion() throws IOException { + String container = "test-container" + randomUUID(); + BlobContainerClient blobContainerClient = + azureBlobStoreFileSystem.getClient().createBlobContainer(container); + + // Create files + List blobNames = new ArrayList<>(); + blobNames.add("testdirectory/file1name"); + blobNames.add("testdirectory/file2name"); + blobNames.add("testdirectory/file3name"); + blobNames.add("testdirectory/otherfile"); + blobNames.add("testotherdirectory/file4name"); + for (String blob : blobNames) { + blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); + } + + // Test patterns + { + AzfsResourceId pattern = + AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file*"); + List expectedFiles = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name"); + + assertThat( + expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); + } + + { + AzfsResourceId pattern = + AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file[1-3]*"); + List expectedFiles = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name"); + + assertThat( + expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); + } + + { + AzfsResourceId pattern = + AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file?name"); + List expectedFiles = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name"); + + assertThat( + expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); + } + + { + AzfsResourceId pattern = + AzfsResourceId.fromUri("azfs://account/" + container + "/test*ectory/fi*name"); + List expectedFiles = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name", + "azfs://account/" + container + "/testotherdirectory/file4name"); + + assertThat( + expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); + } + + // Clean up + blobContainerClient.delete(); + } + + private List toFilenames(MatchResult matchResult) throws IOException { + return FluentIterable.from(matchResult.metadata()) + .transform(metadata -> (metadata.resourceId()).toString()) + .toList(); + } + + @Test + public void testMatch() throws Exception { + String container = "test-container" + randomUUID(); + BlobContainerClient blobContainerClient = + azureBlobStoreFileSystem.getClient().createBlobContainer(container); + + // Create files + List blobNames = new ArrayList<>(); + blobNames.add("testdirectory/file1name"); + blobNames.add("testdirectory/file2name"); + blobNames.add("testdirectory/file3name"); + blobNames.add("testdirectory/file4name"); + blobNames.add("testdirectory/otherfile"); + blobNames.add("testotherdirectory/anotherfile"); + for (String blob : blobNames) { + blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); + } + + List specs = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file[1-3]*", + "azfs://account/" + container + "/testdirectory/non-exist-file", + "azfs://account/" + container + "/testdirectory/otherfile"); + + List matchResults = azureBlobStoreFileSystem.match(specs); + + // Confirm that match results are as expected + assertEquals(3, matchResults.size()); + assertEquals(MatchResult.Status.OK, matchResults.get(0).status()); + assertThat( + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name"), + contains(toFilenames(matchResults.get(0)).toArray())); + assertEquals(MatchResult.Status.NOT_FOUND, matchResults.get(1).status()); + assertEquals(MatchResult.Status.OK, matchResults.get(2).status()); + assertThat( + ImmutableList.of("azfs://account/" + container + "/testdirectory/otherfile"), + contains(toFilenames(matchResults.get(2)).toArray())); + + blobContainerClient.delete(); + } + + @Test + public void testMatchNonGlobs() throws Exception { + String container = "test-container" + randomUUID(); + BlobContainerClient blobContainerClient = + azureBlobStoreFileSystem.getClient().createBlobContainer(container); + + List blobNames = new ArrayList<>(); + blobNames.add("testdirectory/file1name"); + blobNames.add("testdirectory/dir2name/"); + blobNames.add("testdirectory/file4name"); + // TODO: Also test match results where MatchResult.STATUS != OK (see gcs and s3 tests) + + for (String blob : blobNames) { + blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); + } + + List specs = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/dir2name/", + "azfs://account/" + container + "/testdirectory/file4name"); + + List matchResults = azureBlobStoreFileSystem.match(specs); + + assertEquals(3, matchResults.size()); + assertThat( + ImmutableList.of("azfs://account/" + container + "/testdirectory/file1name"), + contains(toFilenames(matchResults.get(0)).toArray())); + assertThat( + ImmutableList.of("azfs://account/" + container + "/testdirectory/dir2name/"), + contains(toFilenames(matchResults.get(1)).toArray())); + assertThat( + ImmutableList.of("azfs://account/" + container + "/testdirectory/file4name"), + contains(toFilenames(matchResults.get(2)).toArray())); + + blobContainerClient.delete(); + } +} diff --git a/sdks/java/io/azure/src/test/resources/in.txt b/sdks/java/io/azure/src/test/resources/in.txt new file mode 100644 index 000000000000..5f959b1e4e51 --- /dev/null +++ b/sdks/java/io/azure/src/test/resources/in.txt @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +This is a text file. +It will be uploaded to a blob. +It is used for testing and stuff like that. +It has a license header so that apache won't be upset. +Hello to anyone who's still reading at this point :). diff --git a/sdks/java/io/azure/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/sdks/java/io/azure/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 000000000000..1f0955d450f0 --- /dev/null +++ b/sdks/java/io/azure/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1 @@ +mock-maker-inline From 137b77741eb30678c027efd29562118de41760f6 Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Mon, 17 Aug 2020 18:28:51 +0000 Subject: [PATCH 02/24] added options to azure filesystem --- .../blobstore/AzureBlobStoreFileSystem.java | 32 ++++- .../DefaultBlobstoreClientBuilderFactory.java | 54 ++++++-- .../sdk/io/azure/options/AzureOptions.java | 64 +--------- .../io/azure/options/BlobstoreOptions.java | 116 +++++++++++++++++- 4 files changed, 185 insertions(+), 81 deletions(-) diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java index 6af5dabb02e8..7df6be65eeaa 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java @@ -53,6 +53,7 @@ import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.util.InstanceBuilder; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Suppliers; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable; @@ -322,11 +323,12 @@ protected WritableByteChannel create(AzfsResourceId resourceId, CreateOptions cr @Override protected ReadableByteChannel open(AzfsResourceId resourceId) throws IOException { - BlobClient blobClient = - client - .get() - .getBlobContainerClient(resourceId.getContainer()) - .getBlobClient(resourceId.getBlob()); + BlobContainerClient containerClient = + client.get().getBlobContainerClient(resourceId.getContainer()); + if (!containerClient.exists()) { + throw new FileNotFoundException("The requested file doesn't exist."); + } + BlobClient blobClient = containerClient.getBlobClient(resourceId.getBlob()); if (!blobClient.exists()) { throw new FileNotFoundException("The requested file doesn't exist."); } @@ -378,6 +380,10 @@ void copy(AzfsResourceId sourcePath, AzfsResourceId destinationPath) throws IOEx @VisibleForTesting String generateSasToken() throws IOException { + if (!Strings.isNullOrEmpty(options.getSasToken())) { + return options.getSasToken(); + } + SharedAccessAccountPolicy sharedAccessAccountPolicy = new SharedAccessAccountPolicy(); long date = new Date().getTime(); long expiryDate = new Date(date + 8640000).getTime(); @@ -389,7 +395,21 @@ String generateSasToken() throws IOException { "co"); // container, object, add s for service sharedAccessAccountPolicy.setServiceFromString("b"); // blob, add "fqt" for file, queue, table - String storageConnectionString = options.getAzureConnectionString(); + String storageConnectionString; + if (!Strings.isNullOrEmpty(options.getAzureConnectionString())) { + storageConnectionString = options.getAzureConnectionString(); + } else if (!Strings.isNullOrEmpty(options.getAccessKey())) { + storageConnectionString = + "DefaultEndpointsProtocol=https;AccountName=" + + client.get().getAccountName() + + ";AccountKey=" + + options.getAccessKey() + + ";EndpointSuffix=core.windows.net"; + } else { + throw new IOException( + "Copying blobs requires that a SAS token, connection string, or account key be provided."); + } + try { CloudStorageAccount storageAccount = CloudStorageAccount.parse(storageConnectionString); return "?" + storageAccount.generateSharedAccessSignature(sharedAccessAccountPolicy); diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java index c8d2c1ae5e4d..894caff5b122 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java @@ -18,30 +18,66 @@ package org.apache.beam.sdk.io.azure.blobstore; import com.azure.storage.blob.BlobServiceClientBuilder; +import com.azure.storage.common.StorageSharedKeyCredential; import org.apache.beam.sdk.io.azure.options.BlobstoreClientBuilderFactory; import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; -/** Construct BlobServiceClientBuilder with default values of Azure client properties. */ +/** Construct BlobServiceClientBuilder with given values of Azure client properties. */ public class DefaultBlobstoreClientBuilderFactory implements BlobstoreClientBuilderFactory { - // TODO: add any other options that should be passed to BlobServiceClientBuilder - @Override public BlobServiceClientBuilder createBuilder(BlobstoreOptions blobstoreOptions) { BlobServiceClientBuilder builder = new BlobServiceClientBuilder(); - if (blobstoreOptions.getClientConfiguration() != null) { - builder = builder.configuration(blobstoreOptions.getClientConfiguration()); + if (!Strings.isNullOrEmpty(blobstoreOptions.getAzureConnectionString())) { + builder = builder.connectionString(blobstoreOptions.getAzureConnectionString()); + } + + if (blobstoreOptions.getSharedKeyCredential() != null) { + builder = builder.credential(blobstoreOptions.getSharedKeyCredential()); + } + + if (blobstoreOptions.getTokenCredential() != null) { + builder = builder.credential(blobstoreOptions.getTokenCredential()); + } + + if (!Strings.isNullOrEmpty(blobstoreOptions.getSasToken())) { + builder = builder.sasToken(blobstoreOptions.getSasToken()); + } + + if (!Strings.isNullOrEmpty(blobstoreOptions.getAccountName()) + && !Strings.isNullOrEmpty(blobstoreOptions.getAccessKey())) { + StorageSharedKeyCredential credential = + new StorageSharedKeyCredential( + blobstoreOptions.getAccountName(), blobstoreOptions.getAccessKey()); + builder = builder.credential(credential); } - if (blobstoreOptions.getAzureConnectionString() != null) { - builder.connectionString(blobstoreOptions.getAzureConnectionString()); + if (!Strings.isNullOrEmpty(blobstoreOptions.getBlobServiceEndpoint())) { + builder = builder.endpoint(blobstoreOptions.getBlobServiceEndpoint()); } - if (!Strings.isNullOrEmpty(blobstoreOptions.getAzureServiceEndpoint())) { - builder = builder.endpoint(blobstoreOptions.getAzureServiceEndpoint()); + if (blobstoreOptions.getCustomerProvidedKey() != null) { + builder = builder.customerProvidedKey(blobstoreOptions.getCustomerProvidedKey()); } + + if (blobstoreOptions.getEnvironmentConfiguration() != null) { + builder = builder.configuration(blobstoreOptions.getEnvironmentConfiguration()); + } + + if (blobstoreOptions.getPipelinePolicy() != null) { + builder = builder.addPolicy(blobstoreOptions.getPipelinePolicy()); + } + + if (blobstoreOptions.getHttpClient() != null) { + builder = builder.httpClient(blobstoreOptions.getHttpClient()); + } + + if (blobstoreOptions.getHttpPipeline() != null) { + builder = builder.pipeline(blobstoreOptions.getHttpPipeline()); + } + return builder; } } diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java index c758cb4a4c78..bc5cf87079cd 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java @@ -17,70 +17,8 @@ */ package org.apache.beam.sdk.io.azure.options; -import com.azure.core.credential.TokenCredential; -import com.azure.core.util.Configuration; -import com.azure.identity.DefaultAzureCredentialBuilder; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.DefaultValueFactory; -import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; public interface AzureOptions extends PipelineOptions { - - // TODO: Add any other azure options that users should be able to configure - // TODO: Confirm that Azure options are in this file, Blobstore options in BlobstoreOptions - - /** The Azure service endpoint used by the Azure client. */ - @Description("Azure service endpoint used by the Azure client") - String getAzureServiceEndpoint(); - - void setAzureServiceEndpoint(String value); - - /** - * The credential instance that should be used to authenticate against Azure services. The option - * value must contain a "@type" field and an Azure credentials provider class as the field value. - */ - @Description( - "The credential instance that should be used to authenticate " - + "against Azure services. The option value must contain \"@type\" field " - + "and an Azure credentials provider class name as the field value.") - @Default.InstanceFactory(AzureUserCredentialsFactory.class) - TokenCredential getAzureCredentialsProvider(); - - void setAzureCredentialsProvider(TokenCredential value); - - /** Attempts to load Azure credentials. */ - class AzureUserCredentialsFactory implements DefaultValueFactory { - - @Override - public TokenCredential create(PipelineOptions options) { - return new DefaultAzureCredentialBuilder().build(); - } - } - - /** The client configuration instance that should be used to configure Azure service clients. */ - @Description( - "The client configuration instance that should be used to configure Azure service clients") - @Default.InstanceFactory(ConfigurationFactory.class) - Configuration getClientConfiguration(); - - void setClientConfiguration(Configuration configuration); - - /** The client configuration instance that should be used to configure Azure service clients. */ - @Description( - "The client configuration instance that should be used to configure Azure http client configuration parameters." - + "Mentioned parameters are the available parameters that can be set. Set only those that need custom changes.") - @Default.InstanceFactory(ConfigurationFactory.class) - Configuration getAzureHttpConfiguration(); - - void setAzureHttpConfiguration(Configuration configuration); - - /** Default Azure client configuration. */ - class ConfigurationFactory implements DefaultValueFactory { - - @Override - public Configuration create(PipelineOptions options) { - return new Configuration(); - } - } + // TODO: Add any general Azure options that are not specific to Blob storage } diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java index 9529485918b3..2a47ba75c9de 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java @@ -17,14 +17,25 @@ */ package org.apache.beam.sdk.io.azure.options; +import com.azure.core.credential.TokenCredential; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpPipeline; +import com.azure.core.http.policy.HttpPipelinePolicy; +import com.azure.core.util.Configuration; +import com.azure.identity.DefaultAzureCredentialBuilder; +import com.azure.storage.blob.models.CustomerProvidedKey; +import com.azure.storage.common.StorageSharedKeyCredential; import org.apache.beam.sdk.io.azure.blobstore.DefaultBlobstoreClientBuilderFactory; import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.checkerframework.checker.nullness.qual.Nullable; -public interface BlobstoreOptions extends AzureOptions { +// TODO: Tag each option with @Default or @Nullable - // TODO: Add any other blobstore options that users should be able to configure - // TODO: Confirm that Blobstore options are in this file, Azure options in AzureOptions +/** Options used to configure Microsoft Azure Blob Storage. */ +public interface BlobstoreOptions extends AzureOptions { @Description( "Factory class that should be created and used to create a builder of Azure Blobstore client." @@ -35,7 +46,106 @@ public interface BlobstoreOptions extends AzureOptions { void setBlobstoreClientFactoryClass( Class blobstoreClientFactoryClass); + @Description("Adds a pipeline policy to apply on each request sent to the blob service client.") + @Nullable + HttpPipelinePolicy getPipelinePolicy(); + + void setPipelinePolicy(HttpPipelinePolicy pipelinePolicy); + + /** The client configuration instance that should be used to configure Azure service clients. */ + @Description( + "The configuration instance used to retrieve environment configuration values " + + "when building an Azure Blobstore client. Set only those that need custom changes.") + @Default.InstanceFactory(BlobstoreOptions.ConfigurationFactory.class) + @Nullable + Configuration getEnvironmentConfiguration(); + + void setEnvironmentConfiguration(Configuration configuration); + + /** Default Azure client configuration. */ + class ConfigurationFactory implements DefaultValueFactory { + + @Override + public Configuration create(PipelineOptions options) { + return new Configuration(); + } + } + + @Description("Sets the connection string to connect to the Azure Blobstore client.") String getAzureConnectionString(); void setAzureConnectionString(String connectionString); + + @Description("Sets a StorageSharedKeyCredential used to authorize requests sent to the service.") + StorageSharedKeyCredential getSharedKeyCredential(); + + void setSharedKeyCredential(StorageSharedKeyCredential sharedKeyCredential); + + @Description("Sets a TokenCredential used to authorize requests sent to the service.") + TokenCredential getTokenCredential(); + + void setTokenCredential(TokenCredential tokenCredential); + + @Description("Sets the SAS token used to authorize requests sent to the service.") + String getSasToken(); + + void setSasToken(String sasToken); + + @Description("Blobstore account name") + String getAccountName(); + + void setAccountName(String name); + + @Description("Azure Blobstore access key") + String getAccessKey(); + + void setAccessKey(String key); + + @Description( + "Sets the customer provided key that is used to encrypt blob contents on the server.") + CustomerProvidedKey getCustomerProvidedKey(); + + void setCustomerProvidedKey(CustomerProvidedKey customerProvidedKey); + + /** The Azure Blobstore service endpoint used by the Blob service client. */ + @Description("Sets the blob service endpoint, additionally parses it for information (SAS token)") + @Nullable + String getBlobServiceEndpoint(); + + void setBlobServiceEndpoint(String endpoint); + + @Description( + "Sets the HttpClient to use for sending a receiving requests to and from the service.") + @Nullable + HttpClient getHttpClient(); + + void setHttpClient(HttpClient httpClient); + + @Description("Sets the HttpPipeline to use for the service client.") + @Nullable + HttpPipeline getHttpPipeline(); + + void setHttpPipeline(HttpPipeline httpPipeline); + + /** + * The credential instance that should be used to authenticate against Azure services. The option + * value must contain a "@type" field and an Azure credentials provider class as the field value. + */ + @Description( + "The credential instance that should be used to authenticate " + + "against Azure services. The option value must contain \"@type\" field " + + "and an Azure credentials provider class name as the field value.") + @Default.InstanceFactory(AzureUserCredentialsFactory.class) + TokenCredential getAzureCredentialsProvider(); + + void setAzureCredentialsProvider(TokenCredential value); + + /** Attempts to load Azure credentials. */ + class AzureUserCredentialsFactory implements DefaultValueFactory { + + @Override + public TokenCredential create(PipelineOptions options) { + return new DefaultAzureCredentialBuilder().build(); + } + } } From 58f7f028d109a5110c9258c681e07f2ec8e62ddb Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Mon, 17 Aug 2020 19:02:02 +0000 Subject: [PATCH 03/24] added experimental annotation --- .../java/org/apache/beam/sdk/io/azure/options/AzureOptions.java | 2 ++ .../org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java | 2 ++ 2 files changed, 4 insertions(+) diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java index bc5cf87079cd..63c5c4be3902 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java @@ -17,8 +17,10 @@ */ package org.apache.beam.sdk.io.azure.options; +import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.PipelineOptions; +@Experimental public interface AzureOptions extends PipelineOptions { // TODO: Add any general Azure options that are not specific to Blob storage } diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java index 2a47ba75c9de..6c5108dd50d0 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java @@ -25,6 +25,7 @@ import com.azure.identity.DefaultAzureCredentialBuilder; import com.azure.storage.blob.models.CustomerProvidedKey; import com.azure.storage.common.StorageSharedKeyCredential; +import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.io.azure.blobstore.DefaultBlobstoreClientBuilderFactory; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; @@ -34,6 +35,7 @@ // TODO: Tag each option with @Default or @Nullable +@Experimental /** Options used to configure Microsoft Azure Blob Storage. */ public interface BlobstoreOptions extends AzureOptions { From 1bbc9479faa2240743ad37050e63f9547e124b30 Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Mon, 17 Aug 2020 22:55:24 +0000 Subject: [PATCH 04/24] adding mocks to azure filesystem tests --- sdks/java/io/azure/build.gradle | 2 ++ .../sdk/io/azure/blobstore/AzfsTestUtils.java | 1 - .../AzureBlobStoreFileSystemTest.java | 34 ++++++++++++++++--- 3 files changed, 32 insertions(+), 5 deletions(-) diff --git a/sdks/java/io/azure/build.gradle b/sdks/java/io/azure/build.gradle index 502e7fa83cdd..6a26bd797cef 100644 --- a/sdks/java/io/azure/build.gradle +++ b/sdks/java/io/azure/build.gradle @@ -27,6 +27,8 @@ applyJavaNature( description = "Apache Beam :: SDKs :: Java :: IO :: Azure" ext.summary = "IO library to read and write Azure services from Beam." +repositories { jcenter() } + dependencies { compile library.java.vendored_guava_26_0_jre compile project(path: ":sdks:java:core", configuration: "shadow") diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsTestUtils.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsTestUtils.java index 1d4d2f972c43..8b123b4eb018 100644 --- a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsTestUtils.java +++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsTestUtils.java @@ -25,7 +25,6 @@ class AzfsTestUtils { static BlobstoreOptions azfsOptions() { BlobstoreOptions options = PipelineOptionsFactory.as(BlobstoreOptions.class); - options.setAzureConnectionString(System.getenv("AZURE_STORAGE_CONNECTION_STRING")); return options; } diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java index e69725d05a0c..d93ca2a630f2 100644 --- a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java +++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java @@ -26,7 +26,10 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.*; +import static org.mockito.Mockito.when; +import com.azure.storage.blob.BlobClient; import com.azure.storage.blob.BlobContainerClient; import com.azure.storage.blob.BlobServiceClient; import java.io.File; @@ -48,21 +51,44 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import org.mockito.Mockito; @RunWith(JUnit4.class) @SuppressWarnings("CannotMockFinalClass") // Mockito 2 and above can mock final classes public class AzureBlobStoreFileSystemTest { - private AzureBlobStoreFileSystem azureBlobStoreFileSystem; + private static AzureBlobStoreFileSystem azureBlobStoreFileSystem; @Before public void beforeClass() { BlobstoreOptions options = PipelineOptionsFactory.as(BlobstoreOptions.class); - options.setAzureConnectionString(System.getenv("AZURE_STORAGE_CONNECTION_STRING")); + BlobServiceClient mockedServiceClient = Mockito.mock(BlobServiceClient.class); + BlobContainerClient mockedContainerClient = Mockito.mock(BlobContainerClient.class); + BlobClient mockedBlobClient = Mockito.mock(BlobClient.class); + azureBlobStoreFileSystem = new AzureBlobStoreFileSystem(options); + azureBlobStoreFileSystem.setClient(mockedServiceClient); + + boolean[] created = {false}; + + when(azureBlobStoreFileSystem.getClient().createBlobContainer(anyString())) + .thenAnswer( + (invocation) -> { + created[0] = true; + return mockedContainerClient; + }); + + when(mockedContainerClient.exists()).thenReturn(created[0]); + when(azureBlobStoreFileSystem.getClient().getBlobContainerClient(anyString())) + .thenReturn(mockedContainerClient); + when(mockedContainerClient.getBlobClient(anyString())).thenReturn(mockedBlobClient); + } - // TODO: Switch to mocks for BlobContainerClient, BlobClient, etc. - // azureBlobStoreFileSystem = buildMockedAzureFileSystem(azfsOptions()); + @Test + public void test() { + BlobServiceClient client = azureBlobStoreFileSystem.getClient(); + BlobContainerClient blobContainerClient = client.createBlobContainer("container"); + assertTrue(blobContainerClient.exists()); } @Test From 4799c220ed088e72e394296b3e6303b038b2cbd1 Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Mon, 17 Aug 2020 23:33:00 +0000 Subject: [PATCH 05/24] adding mocks to azure filesystem test --- .../io/azure/blobstore/AzureBlobStoreFileSystemTest.java | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java index d93ca2a630f2..6b6bc4d33b91 100644 --- a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java +++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java @@ -78,19 +78,12 @@ public void beforeClass() { return mockedContainerClient; }); - when(mockedContainerClient.exists()).thenReturn(created[0]); + when(mockedContainerClient.exists()).thenAnswer((invocation) -> created[0]); when(azureBlobStoreFileSystem.getClient().getBlobContainerClient(anyString())) .thenReturn(mockedContainerClient); when(mockedContainerClient.getBlobClient(anyString())).thenReturn(mockedBlobClient); } - @Test - public void test() { - BlobServiceClient client = azureBlobStoreFileSystem.getClient(); - BlobContainerClient blobContainerClient = client.createBlobContainer("container"); - assertTrue(blobContainerClient.exists()); - } - @Test public void testGetScheme() { assertEquals("azfs", azureBlobStoreFileSystem.getScheme()); From 5fb8001b8a1aac042f8a3046dc9342611b9a0484 Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Mon, 17 Aug 2020 23:27:54 +0000 Subject: [PATCH 06/24] resolving various reviewer comments applied spotless to fix formatting --- .../io/azure/blobstore/AzfsResourceId.java | 4 +- .../blobstore/AzureBlobStoreFileSystem.java | 26 ++++++++---- .../sdk/io/azure/options/AzureOptions.java | 4 +- .../io/azure/options/BlobstoreOptions.java | 4 +- .../azure/blobstore/AzfsResourceIdTest.java | 2 +- .../sdk/io/azure/blobstore/AzfsTestUtils.java | 42 ------------------- 6 files changed, 26 insertions(+), 56 deletions(-) delete mode 100644 sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsTestUtils.java diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java index 39cfe8f70598..7905264955b3 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java @@ -101,8 +101,8 @@ public String getScheme() { return SCHEME; } - Optional getSize() { - return Optional.fromNullable(size); + Long getSize() { + return size; } AzfsResourceId withSize(long size) { diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java index 6af5dabb02e8..8702c84d1ed9 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java @@ -68,6 +68,8 @@ class AzureBlobStoreFileSystem extends FileSystem { private static final ImmutableSet NON_READ_SEEK_EFFICIENT_ENCODINGS = ImmutableSet.of("gzip"); + private static final int expiryTime = 86400000; + private Supplier client; private final BlobstoreOptions options; @@ -130,8 +132,12 @@ protected List match(List specs) { matchResults.add(nonGlobMatches.next()); } } - checkState(!globMatches.hasNext(), "Expect no more elements in globMatches."); - checkState(!nonGlobMatches.hasNext(), "Expect no more elements in nonGlobMatches."); + checkState( + !globMatches.hasNext(), + "Internal error encountered in AzureBlobStoreFileSystem: expected no more elements in globMatches."); + checkState( + !nonGlobMatches.hasNext(), + "Internal error encountered in AzureBlobStoreFileSystem: expected no more elements in nonGlobMatches."); return matchResults.build(); } @@ -206,7 +212,7 @@ private List matchGlobPaths(List globs) { @VisibleForTesting MatchResult expand(AzfsResourceId azfsPattern) { - checkArgument(azfsPattern.isWildcard(), "is Wildcard"); + checkArgument(azfsPattern.isWildcard(), "The resource id should be a wildcard."); String blobPrefix = azfsPattern.getBlobNonWildcardPrefix(); Pattern wildcardAsRegexp = Pattern.compile(wildcardToRegexp(azfsPattern.getBlob())); @@ -217,7 +223,7 @@ MatchResult expand(AzfsResourceId azfsPattern) { wildcardAsRegexp.toString()); ListBlobsOptions listOptions = new ListBlobsOptions().setPrefix(blobPrefix); - Duration timeout = Duration.ZERO.plusMinutes(1); + Duration timeout = Duration.ofMinutes(1); String account = azfsPattern.getAccount(); String container = azfsPattern.getContainer(); @@ -229,7 +235,7 @@ MatchResult expand(AzfsResourceId azfsPattern) { blob -> { String name = blob.getName(); if (wildcardAsRegexp.matcher(name).matches() && !name.endsWith("/")) { - LOG.debug("Matched object: {}", name); + LOG.debug("Matched object: azfs://{}/{}/{}", account, container, name); BlobProperties properties = blobContainerClient.getBlobClient(name).getProperties(); AzfsResourceId rid = @@ -246,13 +252,13 @@ MatchResult expand(AzfsResourceId azfsPattern) { private MatchResult.Metadata toMetadata(AzfsResourceId path, String contentEncoding) { - checkArgument(path.getSize().isPresent(), "path has size"); + checkArgument(path.getSize() != null, "The resource id should have a size."); boolean isReadSeekEfficient = !NON_READ_SEEK_EFFICIENT_ENCODINGS.contains(contentEncoding); return MatchResult.Metadata.builder() .setIsReadSeekEfficient(isReadSeekEfficient) .setResourceId(path) - .setSizeBytes(path.getSize().get()) + .setSizeBytes(path.getSize()) .setLastModifiedMillis(path.getLastModified().transform(Date::getTime).or(0L)) .build(); } @@ -301,7 +307,8 @@ protected WritableByteChannel create(AzfsResourceId resourceId, CreateOptions cr BlobContainerClient blobContainerClient = client.get().getBlobContainerClient(resourceId.getContainer()); if (!blobContainerClient.exists()) { - throw new UnsupportedOperationException("create does not create containers."); + throw new FileNotFoundException( + "This container does not exist. Creating containers is not supported."); } BlobClient blobClient = blobContainerClient.getBlobClient(resourceId.getBlob()); @@ -370,6 +377,7 @@ void copy(AzfsResourceId sourcePath, AzfsResourceId destinationPath) throws IOEx client.get().getBlobContainerClient(destinationPath.getContainer()); if (!destBlobContainerClient.exists()) { client.get().createBlobContainer(destinationPath.getContainer()); + LOG.info("Created a container called {}", destinationPath.getContainer()); } BlobClient destBlobClient = destBlobContainerClient.getBlobClient(destinationPath.getBlob()); @@ -380,7 +388,7 @@ void copy(AzfsResourceId sourcePath, AzfsResourceId destinationPath) throws IOEx String generateSasToken() throws IOException { SharedAccessAccountPolicy sharedAccessAccountPolicy = new SharedAccessAccountPolicy(); long date = new Date().getTime(); - long expiryDate = new Date(date + 8640000).getTime(); + long expiryDate = new Date(date + expiryTime).getTime(); sharedAccessAccountPolicy.setPermissionsFromString("racwdlup"); sharedAccessAccountPolicy.setSharedAccessStartTime(new Date(date)); diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java index c758cb4a4c78..c1b5e29c97a1 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java @@ -20,15 +20,17 @@ import com.azure.core.credential.TokenCredential; import com.azure.core.util.Configuration; import com.azure.identity.DefaultAzureCredentialBuilder; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; +@Experimental(Kind.FILESYSTEM) public interface AzureOptions extends PipelineOptions { // TODO: Add any other azure options that users should be able to configure - // TODO: Confirm that Azure options are in this file, Blobstore options in BlobstoreOptions /** The Azure service endpoint used by the Azure client. */ @Description("Azure service endpoint used by the Azure client") diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java index 9529485918b3..182b01932a07 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java @@ -17,14 +17,16 @@ */ package org.apache.beam.sdk.io.azure.options; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.io.azure.blobstore.DefaultBlobstoreClientBuilderFactory; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; +@Experimental(Kind.FILESYSTEM) public interface BlobstoreOptions extends AzureOptions { // TODO: Add any other blobstore options that users should be able to configure - // TODO: Confirm that Blobstore options are in this file, Azure options in AzureOptions @Description( "Factory class that should be created and used to create a builder of Azure Blobstore client." diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceIdTest.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceIdTest.java index 83fb0cb396e2..6a1379ba05eb 100644 --- a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceIdTest.java +++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceIdTest.java @@ -144,7 +144,7 @@ public void testResolveInvalidNotDirectory() { } @Test - public void testS3ResolveWithFileBase() { + public void testResolveWithFileBase() { ResourceId resourceId = AzfsResourceId.fromUri("azfs://account/container/path/to/file"); thrown.expect(IllegalStateException.class); resourceId.resolve("child-path", RESOLVE_DIRECTORY); // resource is not a directory diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsTestUtils.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsTestUtils.java deleted file mode 100644 index 1d4d2f972c43..000000000000 --- a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsTestUtils.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.blobstore; - -import com.azure.storage.blob.BlobServiceClient; -import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.mockito.Mockito; - -class AzfsTestUtils { - static BlobstoreOptions azfsOptions() { - BlobstoreOptions options = PipelineOptionsFactory.as(BlobstoreOptions.class); - options.setAzureConnectionString(System.getenv("AZURE_STORAGE_CONNECTION_STRING")); - return options; - } - - static AzureBlobStoreFileSystem buildMockedAzureFileSystem(BlobstoreOptions options) { - return buildMockedAzureFileSystem(options, Mockito.mock(BlobServiceClient.class)); - } - - static AzureBlobStoreFileSystem buildMockedAzureFileSystem( - BlobstoreOptions options, BlobServiceClient client) { - AzureBlobStoreFileSystem azureFileSystem = new AzureBlobStoreFileSystem(options); - azureFileSystem.setClient(client); - return azureFileSystem; - } -} From 8b4fd2a8080336d65913e4cdde1fdda5fe91f8e6 Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Tue, 18 Aug 2020 01:27:25 +0000 Subject: [PATCH 07/24] adding mocks to filesystem test --- .../AzureBlobStoreFileSystemTest.java | 42 +++++++++++++++---- 1 file changed, 33 insertions(+), 9 deletions(-) diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java index 6b6bc4d33b91..38bf3b49a09c 100644 --- a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java +++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java @@ -26,9 +26,12 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; -import static org.mockito.ArgumentMatchers.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.when; +import com.azure.core.http.rest.PagedIterable; import com.azure.storage.blob.BlobClient; import com.azure.storage.blob.BlobContainerClient; import com.azure.storage.blob.BlobServiceClient; @@ -38,10 +41,19 @@ import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; +import java.time.Duration; +import java.time.OffsetDateTime; import java.util.ArrayList; import java.util.Collection; import java.util.List; + +import com.azure.storage.blob.models.BlobItem; +import com.azure.storage.blob.models.BlobProperties; +import com.azure.storage.blob.models.ListBlobsOptions; +import com.azure.storage.blob.specialized.BlobOutputStream; +import com.azure.storage.blob.specialized.BlockBlobClient; import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; +import org.apache.beam.sdk.io.fs.CreateOptions; import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable; @@ -65,23 +77,35 @@ public void beforeClass() { BlobServiceClient mockedServiceClient = Mockito.mock(BlobServiceClient.class); BlobContainerClient mockedContainerClient = Mockito.mock(BlobContainerClient.class); BlobClient mockedBlobClient = Mockito.mock(BlobClient.class); + BlockBlobClient mockedBlockBlob = Mockito.mock(BlockBlobClient.class); + BlobProperties mockedProperties = Mockito.mock(BlobProperties.class); + PagedIterable mockedPagedIterable = Mockito.mock(PagedIterable.class); + BlobOutputStream mockedOutputStream = Mockito.mock(BlobOutputStream.class); azureBlobStoreFileSystem = new AzureBlobStoreFileSystem(options); azureBlobStoreFileSystem.setClient(mockedServiceClient); - boolean[] created = {false}; + boolean[] containerCreated = {false}; + when(mockedServiceClient.createBlobContainer(anyString())) + .thenAnswer((invocation) -> { containerCreated[0] = true; return mockedContainerClient; }); + when(mockedContainerClient.exists()).thenAnswer((invocation) -> containerCreated[0]); - when(azureBlobStoreFileSystem.getClient().createBlobContainer(anyString())) - .thenAnswer( - (invocation) -> { - created[0] = true; - return mockedContainerClient; - }); + boolean[] blobCreated = {false}; + doAnswer(invocation -> { + blobCreated[0] = true; return null; + }).when(mockedBlobClient).uploadFromFile(anyString()); + when(mockedBlobClient.exists()).thenAnswer((invocation) -> blobCreated[0]); - when(mockedContainerClient.exists()).thenAnswer((invocation) -> created[0]); when(azureBlobStoreFileSystem.getClient().getBlobContainerClient(anyString())) .thenReturn(mockedContainerClient); when(mockedContainerClient.getBlobClient(anyString())).thenReturn(mockedBlobClient); + when(mockedBlobClient.getBlockBlobClient()).thenReturn(mockedBlockBlob); + when(mockedBlobClient.getProperties()).thenReturn(mockedProperties); + when(mockedProperties.getBlobSize()).thenReturn(Long.valueOf(1)); + when(mockedProperties.getLastModified()).thenReturn(OffsetDateTime.now()); + when(mockedContainerClient.listBlobs(any(ListBlobsOptions.class), any(Duration.class))).thenReturn(mockedPagedIterable); + when(mockedContainerClient.listBlobsByHierarchy(any(String.class))).thenReturn(mockedPagedIterable); + when(mockedBlockBlob.getBlobOutputStream()).thenReturn(mockedOutputStream); } @Test From 8f073fd9c1b350a767ed66cfb62c6c56afd53efb Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Tue, 18 Aug 2020 01:30:19 +0000 Subject: [PATCH 08/24] adding mocks to filesystem test --- .../AzureBlobStoreFileSystemTest.java | 34 ++++++++++++------- 1 file changed, 21 insertions(+), 13 deletions(-) diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java index 38bf3b49a09c..cfc89a40edc9 100644 --- a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java +++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java @@ -35,6 +35,11 @@ import com.azure.storage.blob.BlobClient; import com.azure.storage.blob.BlobContainerClient; import com.azure.storage.blob.BlobServiceClient; +import com.azure.storage.blob.models.BlobItem; +import com.azure.storage.blob.models.BlobProperties; +import com.azure.storage.blob.models.ListBlobsOptions; +import com.azure.storage.blob.specialized.BlobOutputStream; +import com.azure.storage.blob.specialized.BlockBlobClient; import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -46,14 +51,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; - -import com.azure.storage.blob.models.BlobItem; -import com.azure.storage.blob.models.BlobProperties; -import com.azure.storage.blob.models.ListBlobsOptions; -import com.azure.storage.blob.specialized.BlobOutputStream; -import com.azure.storage.blob.specialized.BlockBlobClient; import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; -import org.apache.beam.sdk.io.fs.CreateOptions; import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable; @@ -87,13 +85,21 @@ public void beforeClass() { boolean[] containerCreated = {false}; when(mockedServiceClient.createBlobContainer(anyString())) - .thenAnswer((invocation) -> { containerCreated[0] = true; return mockedContainerClient; }); + .thenAnswer( + (invocation) -> { + containerCreated[0] = true; + return mockedContainerClient; + }); when(mockedContainerClient.exists()).thenAnswer((invocation) -> containerCreated[0]); boolean[] blobCreated = {false}; - doAnswer(invocation -> { - blobCreated[0] = true; return null; - }).when(mockedBlobClient).uploadFromFile(anyString()); + doAnswer( + invocation -> { + blobCreated[0] = true; + return null; + }) + .when(mockedBlobClient) + .uploadFromFile(anyString()); when(mockedBlobClient.exists()).thenAnswer((invocation) -> blobCreated[0]); when(azureBlobStoreFileSystem.getClient().getBlobContainerClient(anyString())) @@ -103,8 +109,10 @@ public void beforeClass() { when(mockedBlobClient.getProperties()).thenReturn(mockedProperties); when(mockedProperties.getBlobSize()).thenReturn(Long.valueOf(1)); when(mockedProperties.getLastModified()).thenReturn(OffsetDateTime.now()); - when(mockedContainerClient.listBlobs(any(ListBlobsOptions.class), any(Duration.class))).thenReturn(mockedPagedIterable); - when(mockedContainerClient.listBlobsByHierarchy(any(String.class))).thenReturn(mockedPagedIterable); + when(mockedContainerClient.listBlobs(any(ListBlobsOptions.class), any(Duration.class))) + .thenReturn(mockedPagedIterable); + when(mockedContainerClient.listBlobsByHierarchy(any(String.class))) + .thenReturn(mockedPagedIterable); when(mockedBlockBlob.getBlobOutputStream()).thenReturn(mockedOutputStream); } From 6f30dc6a8be69026b1a39debdceb95023cbacfbf Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Wed, 19 Aug 2020 02:29:39 +0000 Subject: [PATCH 09/24] working on options and mocks --- .../blobstore/AzureBlobStoreFileSystem.java | 9 +- .../sdk/io/azure/options/AzureOptions.java | 26 -- .../AzurePipelineOptionsRegistrar.java | 1 - .../io/azure/options/BlobstoreOptions.java | 2 +- .../blobstore/AzureBlobStoreFileSystemIT.java | 353 ++++++++++++++++++ .../AzureBlobStoreFileSystemTest.java | 136 +++---- 6 files changed, 413 insertions(+), 114 deletions(-) delete mode 100644 sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java create mode 100644 sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java index 7df6be65eeaa..9e22137afdec 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java @@ -449,10 +449,11 @@ protected void delete(Collection resourceIds) throws IOException else { PagedIterable blobsInDirectory = container.listBlobsByHierarchy(resourceId.getBlob()); - for (BlobItem blob : blobsInDirectory) { - String blobName = blob.getName(); - container.getBlobClient(blobName).delete(); - } + blobsInDirectory.forEach( + blob -> { + String blobName = blob.getName(); + container.getBlobClient(blobName).delete(); + }); } } } diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java deleted file mode 100644 index 63c5c4be3902..000000000000 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureOptions.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.options; - -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.options.PipelineOptions; - -@Experimental -public interface AzureOptions extends PipelineOptions { - // TODO: Add any general Azure options that are not specific to Blob storage -} diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzurePipelineOptionsRegistrar.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzurePipelineOptionsRegistrar.java index 681342efb560..353792d52da0 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzurePipelineOptionsRegistrar.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzurePipelineOptionsRegistrar.java @@ -29,7 +29,6 @@ public class AzurePipelineOptionsRegistrar implements PipelineOptionsRegistrar { @Override public Iterable> getPipelineOptions() { return ImmutableList.>builder() - .add(AzureOptions.class) .add(BlobstoreOptions.class) .build(); } diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java index 6c5108dd50d0..b929302d2434 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java @@ -37,7 +37,7 @@ @Experimental /** Options used to configure Microsoft Azure Blob Storage. */ -public interface BlobstoreOptions extends AzureOptions { +public interface BlobstoreOptions extends PipelineOptions { @Description( "Factory class that should be created and used to create a builder of Azure Blobstore client." diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java new file mode 100644 index 000000000000..3ee5c47e5dab --- /dev/null +++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java @@ -0,0 +1,353 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.blobstore; + +import static java.util.UUID.randomUUID; +import static org.apache.beam.sdk.io.fs.CreateOptions.StandardCreateOptions.builder; +import static org.hamcrest.Matchers.contains; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import com.azure.storage.blob.BlobContainerClient; +import com.azure.storage.blob.BlobServiceClient; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; +import org.apache.beam.sdk.io.fs.MatchResult; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.commons.io.FileUtils; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class AzureBlobStoreFileSystemIT { + + private static AzureBlobStoreFileSystem azureBlobStoreFileSystem; + + @Before + public void beforeClass() { + BlobstoreOptions options = PipelineOptionsFactory.as(BlobstoreOptions.class); + // TODO: Change next line to Beam's Azure connection string + String connectionString = System.getenv("AZURE_STORAGE_CONNECTION_STRING"); + options.setAzureConnectionString(connectionString); + azureBlobStoreFileSystem = new AzureBlobStoreFileSystem(options); + } + + // TODO: Write integration tests that build pipelines, etc. + + @Test + public void testGetScheme() { + assertEquals("azfs", azureBlobStoreFileSystem.getScheme()); + } + + @Test + public void testGlobTranslation() { + assertEquals("foo", AzureBlobStoreFileSystem.wildcardToRegexp("foo")); + assertEquals("fo[^/]*o", AzureBlobStoreFileSystem.wildcardToRegexp("fo*o")); + assertEquals("f[^/]*o\\.[^/]", AzureBlobStoreFileSystem.wildcardToRegexp("f*o.?")); + assertEquals("foo-[0-9][^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo-[0-9]*")); + assertEquals("foo-[0-9].*", AzureBlobStoreFileSystem.wildcardToRegexp("foo-[0-9]**")); + assertEquals(".*foo", AzureBlobStoreFileSystem.wildcardToRegexp("**/*foo")); + assertEquals(".*foo", AzureBlobStoreFileSystem.wildcardToRegexp("**foo")); + assertEquals("foo/[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*")); + assertEquals("foo[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo*")); + assertEquals("foo/[^/]*/[^/]*/[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*/*/*")); + assertEquals("foo/[^/]*/.*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*/**")); + assertEquals("foo.*baz", AzureBlobStoreFileSystem.wildcardToRegexp("foo**baz")); + } + + @Test + public void testDelete() throws IOException { + String containerName = "test-container" + randomUUID(); + String blobName1 = "blob" + randomUUID(); + String blobName2 = "dir1/anotherBlob" + randomUUID(); + + // Create files to delete + BlobContainerClient blobContainerClient = + azureBlobStoreFileSystem.getClient().createBlobContainer(containerName); + assertTrue(blobContainerClient.exists()); + blobContainerClient.getBlobClient(blobName1).uploadFromFile("src/test/resources/in.txt"); + blobContainerClient.getBlobClient(blobName2).uploadFromFile("src/test/resources/in.txt"); + assertTrue(blobContainerClient.getBlobClient(blobName1).exists()); + assertTrue(blobContainerClient.getBlobClient(blobName2).exists()); + + // Delete the files + Collection toDelete = new ArrayList<>(); + String account = blobContainerClient.getAccountName(); + // delete blob + toDelete.add(AzfsResourceId.fromComponents(account, containerName, blobName1)); + // delete directory + toDelete.add(AzfsResourceId.fromComponents(account, containerName, "dir1/")); + azureBlobStoreFileSystem.delete(toDelete); + + // Ensure exception is thrown, clean up + assertFalse(blobContainerClient.getBlobClient(blobName1).exists()); + assertFalse(blobContainerClient.getBlobClient(blobName2).exists()); + assertThrows(FileNotFoundException.class, () -> azureBlobStoreFileSystem.delete(toDelete)); + blobContainerClient.delete(); + } + + @Test + public void testCopy() throws IOException { + BlobServiceClient blobServiceClient = azureBlobStoreFileSystem.getClient(); + String account = blobServiceClient.getAccountName(); + + List src = new ArrayList<>(); + List dest = new ArrayList<>(); + String srcContainer = "source-container" + randomUUID(); + String destContainer = "dest-container" + randomUUID(); + + // Create source file + BlobContainerClient srcContainerClient = blobServiceClient.createBlobContainer(srcContainer); + srcContainerClient.getBlobClient("src-blob").uploadFromFile("src/test/resources/in.txt"); + + // Copy source file to destination + src.add(AzfsResourceId.fromComponents(account, srcContainer, "src-blob")); + dest.add(AzfsResourceId.fromComponents(account, destContainer, "dest-blob")); + azureBlobStoreFileSystem.copy(src, dest); + + // Confirm the destination container was created + BlobContainerClient destContainerClient = + blobServiceClient.getBlobContainerClient(destContainer); + assertTrue(destContainerClient.getBlobClient("dest-blob").exists()); + + // Confirm that the source and destination files are the same + srcContainerClient.getBlobClient("src-blob").downloadToFile("./src/test/resources/blob1"); + destContainerClient.getBlobClient("dest-blob").downloadToFile("./src/test/resources/blob2"); + File file1 = new File("./src/test/resources/blob1"); + File file2 = new File("./src/test/resources/blob2"); + assertTrue("The files differ!", FileUtils.contentEquals(file1, file2)); + + // Clean up + assertTrue(file1.delete()); + assertTrue(file2.delete()); + blobServiceClient.deleteBlobContainer(srcContainer); + blobServiceClient.deleteBlobContainer(destContainer); + } + + @Test + public void testWriteAndRead() throws IOException { + BlobServiceClient client = azureBlobStoreFileSystem.getClient(); + String containerName = "test-container" + randomUUID(); + client.createBlobContainer(containerName); + + byte[] writtenArray = new byte[] {0}; + ByteBuffer bb = ByteBuffer.allocate(writtenArray.length); + bb.put(writtenArray); + + // Create an object and write data to it + AzfsResourceId path = + AzfsResourceId.fromUri( + "azfs://" + client.getAccountName() + "/" + containerName + "/foo/bar.txt"); + WritableByteChannel writableByteChannel = + azureBlobStoreFileSystem.create(path, builder().setMimeType("application/text").build()); + writableByteChannel.write(bb); + writableByteChannel.close(); + + // Read the same object + ByteBuffer bb2 = ByteBuffer.allocate(writtenArray.length); + ReadableByteChannel open = azureBlobStoreFileSystem.open(path); + open.read(bb2); + + // Compare the content with the one that was written + byte[] readArray = bb2.array(); + assertArrayEquals(readArray, writtenArray); + open.close(); + + // Clean up + client.getBlobContainerClient(containerName).delete(); + } + + @Test + public void testGlobExpansion() throws IOException { + String container = "test-container" + randomUUID(); + BlobContainerClient blobContainerClient = + azureBlobStoreFileSystem.getClient().createBlobContainer(container); + + // Create files + List blobNames = new ArrayList<>(); + blobNames.add("testdirectory/file1name"); + blobNames.add("testdirectory/file2name"); + blobNames.add("testdirectory/file3name"); + blobNames.add("testdirectory/otherfile"); + blobNames.add("testotherdirectory/file4name"); + for (String blob : blobNames) { + blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); + } + + // Test patterns + { + AzfsResourceId pattern = + AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file*"); + List expectedFiles = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name"); + + assertThat( + expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); + } + + { + AzfsResourceId pattern = + AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file[1-3]*"); + List expectedFiles = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name"); + + assertThat( + expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); + } + + { + AzfsResourceId pattern = + AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file?name"); + List expectedFiles = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name"); + + assertThat( + expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); + } + + { + AzfsResourceId pattern = + AzfsResourceId.fromUri("azfs://account/" + container + "/test*ectory/fi*name"); + List expectedFiles = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name", + "azfs://account/" + container + "/testotherdirectory/file4name"); + + assertThat( + expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); + } + + // Clean up + blobContainerClient.delete(); + } + + private List toFilenames(MatchResult matchResult) throws IOException { + return FluentIterable.from(matchResult.metadata()) + .transform(metadata -> (metadata.resourceId()).toString()) + .toList(); + } + + @Test + public void testMatch() throws Exception { + String container = "test-container" + randomUUID(); + BlobContainerClient blobContainerClient = + azureBlobStoreFileSystem.getClient().createBlobContainer(container); + + // Create files + List blobNames = new ArrayList<>(); + blobNames.add("testdirectory/file1name"); + blobNames.add("testdirectory/file2name"); + blobNames.add("testdirectory/file3name"); + blobNames.add("testdirectory/file4name"); + blobNames.add("testdirectory/otherfile"); + blobNames.add("testotherdirectory/anotherfile"); + for (String blob : blobNames) { + blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); + } + + List specs = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file[1-3]*", + "azfs://account/" + container + "/testdirectory/non-exist-file", + "azfs://account/" + container + "/testdirectory/otherfile"); + + List matchResults = azureBlobStoreFileSystem.match(specs); + + // Confirm that match results are as expected + assertEquals(3, matchResults.size()); + assertEquals(MatchResult.Status.OK, matchResults.get(0).status()); + assertThat( + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name"), + contains(toFilenames(matchResults.get(0)).toArray())); + assertEquals(MatchResult.Status.NOT_FOUND, matchResults.get(1).status()); + assertEquals(MatchResult.Status.OK, matchResults.get(2).status()); + assertThat( + ImmutableList.of("azfs://account/" + container + "/testdirectory/otherfile"), + contains(toFilenames(matchResults.get(2)).toArray())); + + blobContainerClient.delete(); + } + + @Test + public void testMatchNonGlobs() throws Exception { + String container = "test-container" + randomUUID(); + BlobContainerClient blobContainerClient = + azureBlobStoreFileSystem.getClient().createBlobContainer(container); + + List blobNames = new ArrayList<>(); + blobNames.add("testdirectory/file1name"); + blobNames.add("testdirectory/dir2name/"); + blobNames.add("testdirectory/file4name"); + // TODO: Also test match results where MatchResult.STATUS != OK (see gcs and s3 tests) + + for (String blob : blobNames) { + blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); + } + + List specs = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/dir2name/", + "azfs://account/" + container + "/testdirectory/file4name"); + + List matchResults = azureBlobStoreFileSystem.match(specs); + + assertEquals(3, matchResults.size()); + assertThat( + ImmutableList.of("azfs://account/" + container + "/testdirectory/file1name"), + contains(toFilenames(matchResults.get(0)).toArray())); + assertThat( + ImmutableList.of("azfs://account/" + container + "/testdirectory/dir2name/"), + contains(toFilenames(matchResults.get(1)).toArray())); + assertThat( + ImmutableList.of("azfs://account/" + container + "/testdirectory/file4name"), + contains(toFilenames(matchResults.get(2)).toArray())); + + blobContainerClient.delete(); + } +} diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java index cfc89a40edc9..1d59cf1f7126 100644 --- a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java +++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java @@ -22,14 +22,10 @@ import static org.hamcrest.Matchers.contains; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; import com.azure.core.http.rest.PagedIterable; import com.azure.storage.blob.BlobClient; @@ -38,10 +34,9 @@ import com.azure.storage.blob.models.BlobItem; import com.azure.storage.blob.models.BlobProperties; import com.azure.storage.blob.models.ListBlobsOptions; +import com.azure.storage.blob.specialized.BlobInputStream; import com.azure.storage.blob.specialized.BlobOutputStream; import com.azure.storage.blob.specialized.BlockBlobClient; -import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; @@ -49,14 +44,13 @@ import java.time.Duration; import java.time.OffsetDateTime; import java.util.ArrayList; -import java.util.Collection; +import java.util.Arrays; import java.util.List; import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; -import org.apache.commons.io.FileUtils; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -68,19 +62,21 @@ public class AzureBlobStoreFileSystemTest { private static AzureBlobStoreFileSystem azureBlobStoreFileSystem; + BlobstoreOptions options = PipelineOptionsFactory.as(BlobstoreOptions.class); + BlobstoreOptions spyOptions = Mockito.spy(options); + BlobServiceClient mockedServiceClient = Mockito.mock(BlobServiceClient.class); + BlobContainerClient mockedContainerClient = Mockito.mock(BlobContainerClient.class); + BlobClient mockedBlobClient = Mockito.mock(BlobClient.class); + BlockBlobClient mockedBlockBlob = Mockito.mock(BlockBlobClient.class); + BlobProperties mockedProperties = Mockito.mock(BlobProperties.class); + PagedIterable mockedPagedIterable = Mockito.mock(PagedIterable.class); + BlobOutputStream mockedOutputStream = Mockito.mock(BlobOutputStream.class); + BlobItem mockedBlobItem = Mockito.mock(BlobItem.class); + BlobInputStream mockedInputStream = Mockito.mock(BlobInputStream.class); @Before public void beforeClass() { - BlobstoreOptions options = PipelineOptionsFactory.as(BlobstoreOptions.class); - BlobServiceClient mockedServiceClient = Mockito.mock(BlobServiceClient.class); - BlobContainerClient mockedContainerClient = Mockito.mock(BlobContainerClient.class); - BlobClient mockedBlobClient = Mockito.mock(BlobClient.class); - BlockBlobClient mockedBlockBlob = Mockito.mock(BlockBlobClient.class); - BlobProperties mockedProperties = Mockito.mock(BlobProperties.class); - PagedIterable mockedPagedIterable = Mockito.mock(PagedIterable.class); - BlobOutputStream mockedOutputStream = Mockito.mock(BlobOutputStream.class); - - azureBlobStoreFileSystem = new AzureBlobStoreFileSystem(options); + azureBlobStoreFileSystem = new AzureBlobStoreFileSystem(spyOptions); azureBlobStoreFileSystem.setClient(mockedServiceClient); boolean[] containerCreated = {false}; @@ -91,7 +87,6 @@ public void beforeClass() { return mockedContainerClient; }); when(mockedContainerClient.exists()).thenAnswer((invocation) -> containerCreated[0]); - boolean[] blobCreated = {false}; doAnswer( invocation -> { @@ -101,7 +96,6 @@ public void beforeClass() { .when(mockedBlobClient) .uploadFromFile(anyString()); when(mockedBlobClient.exists()).thenAnswer((invocation) -> blobCreated[0]); - when(azureBlobStoreFileSystem.getClient().getBlobContainerClient(anyString())) .thenReturn(mockedContainerClient); when(mockedContainerClient.getBlobClient(anyString())).thenReturn(mockedBlobClient); @@ -113,7 +107,15 @@ public void beforeClass() { .thenReturn(mockedPagedIterable); when(mockedContainerClient.listBlobsByHierarchy(any(String.class))) .thenReturn(mockedPagedIterable); - when(mockedBlockBlob.getBlobOutputStream()).thenReturn(mockedOutputStream); + when(mockedBlockBlob.getBlobOutputStream()) + .thenAnswer( + (i) -> { + blobCreated[0] = true; + return mockedOutputStream; + }); + when(mockedBlobItem.getName()).thenReturn("name"); + when(spyOptions.getSasToken()).thenReturn("sas-token"); + when(mockedBlobClient.openInputStream()).thenReturn(mockedInputStream); } @Test @@ -138,76 +140,46 @@ public void testGlobTranslation() { } @Test - public void testDelete() throws IOException { - String containerName = "test-container" + randomUUID(); - String blobName1 = "blob" + randomUUID(); - String blobName2 = "dir1/anotherBlob" + randomUUID(); - - // Create files to delete - BlobContainerClient blobContainerClient = - azureBlobStoreFileSystem.getClient().createBlobContainer(containerName); - assertTrue(blobContainerClient.exists()); - blobContainerClient.getBlobClient(blobName1).uploadFromFile("src/test/resources/in.txt"); - blobContainerClient.getBlobClient(blobName2).uploadFromFile("src/test/resources/in.txt"); - assertTrue(blobContainerClient.getBlobClient(blobName1).exists()); - assertTrue(blobContainerClient.getBlobClient(blobName2).exists()); - - // Delete the files - Collection toDelete = new ArrayList<>(); - String account = blobContainerClient.getAccountName(); - // delete blob - toDelete.add(AzfsResourceId.fromComponents(account, containerName, blobName1)); - // delete directory - toDelete.add(AzfsResourceId.fromComponents(account, containerName, "dir1/")); - azureBlobStoreFileSystem.delete(toDelete); - - // Ensure exception is thrown, clean up - assertFalse(blobContainerClient.getBlobClient(blobName1).exists()); - assertFalse(blobContainerClient.getBlobClient(blobName2).exists()); - assertThrows(FileNotFoundException.class, () -> azureBlobStoreFileSystem.delete(toDelete)); - blobContainerClient.delete(); + @SuppressWarnings("CheckReturnValue") + public void testCopy() throws IOException { + List src = + new ArrayList<>( + Arrays.asList(AzfsResourceId.fromComponents("account", "container", "from"))); + List dest = + new ArrayList<>(Arrays.asList(AzfsResourceId.fromComponents("account", "container", "to"))); + when(mockedBlobClient.exists()).thenReturn(true); + azureBlobStoreFileSystem.copy(src, dest); + verify(mockedBlobClient, times(1)).copyFromUrl(any(String.class)); } @Test - public void testCopy() throws IOException { - BlobServiceClient blobServiceClient = azureBlobStoreFileSystem.getClient(); - String account = blobServiceClient.getAccountName(); - - List src = new ArrayList<>(); - List dest = new ArrayList<>(); - String srcContainer = "source-container" + randomUUID(); - String destContainer = "dest-container" + randomUUID(); - - // Create source file - BlobContainerClient srcContainerClient = blobServiceClient.createBlobContainer(srcContainer); - srcContainerClient.getBlobClient("src-blob").uploadFromFile("src/test/resources/in.txt"); + public void testWriteAndRead() throws IOException { + azureBlobStoreFileSystem.getClient().createBlobContainer("testcontainer"); - // Copy source file to destination - src.add(AzfsResourceId.fromComponents(account, srcContainer, "src-blob")); - dest.add(AzfsResourceId.fromComponents(account, destContainer, "dest-blob")); - azureBlobStoreFileSystem.copy(src, dest); + byte[] writtenArray = new byte[] {0}; + ByteBuffer bb = ByteBuffer.allocate(writtenArray.length); + bb.put(writtenArray); - // Confirm the destination container was created - BlobContainerClient destContainerClient = - blobServiceClient.getBlobContainerClient(destContainer); - assertTrue(destContainerClient.getBlobClient("dest-blob").exists()); + // First create an object and write data to it + AzfsResourceId path = AzfsResourceId.fromUri("azfs://account/testcontainer/foo/bar.txt"); + WritableByteChannel writableByteChannel = + azureBlobStoreFileSystem.create(path, builder().setMimeType("application/text").build()); + writableByteChannel.write(bb); + writableByteChannel.close(); - // Confirm that the source and destination files are the same - srcContainerClient.getBlobClient("src-blob").downloadToFile("./src/test/resources/blob1"); - destContainerClient.getBlobClient("dest-blob").downloadToFile("./src/test/resources/blob2"); - File file1 = new File("./src/test/resources/blob1"); - File file2 = new File("./src/test/resources/blob2"); - assertTrue("The files differ!", FileUtils.contentEquals(file1, file2)); + // Now read the same object + ByteBuffer bb2 = ByteBuffer.allocate(writtenArray.length); + ReadableByteChannel open = azureBlobStoreFileSystem.open(path); + open.read(bb2); - // Clean up - assertTrue(file1.delete()); - assertTrue(file2.delete()); - blobServiceClient.deleteBlobContainer(srcContainer); - blobServiceClient.deleteBlobContainer(destContainer); + // And compare the content with the one that was written + byte[] readArray = bb2.array(); + assertArrayEquals(readArray, writtenArray); + open.close(); } @Test - public void testWriteAndRead() throws IOException { + public void testWriteAndRead2() throws IOException { BlobServiceClient client = azureBlobStoreFileSystem.getClient(); String containerName = "test-container" + randomUUID(); client.createBlobContainer(containerName); From 26690cb6941af77369d1533a6424313d33000abf Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Wed, 19 Aug 2020 03:00:22 +0000 Subject: [PATCH 10/24] applied spotless --- .../apache/beam/sdk/io/azure/options/BlobstoreOptions.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java index e8f1c90deb7c..1060177c0494 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.io.azure.options; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.annotations.Experimental.Kind; import com.azure.core.credential.TokenCredential; import com.azure.core.http.HttpClient; import com.azure.core.http.HttpPipeline; @@ -27,6 +25,8 @@ import com.azure.identity.DefaultAzureCredentialBuilder; import com.azure.storage.blob.models.CustomerProvidedKey; import com.azure.storage.common.StorageSharedKeyCredential; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.io.azure.blobstore.DefaultBlobstoreClientBuilderFactory; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; From 4cf0f99721238f74803609cdd93d36c3903c11e4 Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Tue, 18 Aug 2020 23:21:01 -0400 Subject: [PATCH 11/24] Update sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java Co-authored-by: Pablo --- .../beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java index 55bfac39058f..10f2ee449c53 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java @@ -69,7 +69,7 @@ class AzureBlobStoreFileSystem extends FileSystem { private static final ImmutableSet NON_READ_SEEK_EFFICIENT_ENCODINGS = ImmutableSet.of("gzip"); - private static final int expiryTime = 86400000; + private static final int DEFAULT_EXPIRY_TIME = 86400000; private Supplier client; private final BlobstoreOptions options; From c2d91af03b59baa1e2865bc3d68f68daf17abec1 Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Wed, 19 Aug 2020 03:36:47 +0000 Subject: [PATCH 12/24] incorporating reviewer feeback --- .../io/azure/blobstore/AzureBlobStoreFileSystem.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java index 10f2ee449c53..1097f0ab6fa0 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java @@ -69,7 +69,12 @@ class AzureBlobStoreFileSystem extends FileSystem { private static final ImmutableSet NON_READ_SEEK_EFFICIENT_ENCODINGS = ImmutableSet.of("gzip"); + // By default generateSasToken returns an SAS token that is valid for one day (86,400,000 millis) private static final int DEFAULT_EXPIRY_TIME = 86400000; + // By default, generateSasToken returns an SAS token that grants all permission + private static final String DEFAULT_PERMISSIONS = "racwdlup"; + // By default generateSasToken returns an SAS token that grants access to containers and objects + private static final String DEFAULT_RESOURCE_TYPES = "co"; private Supplier client; private final BlobstoreOptions options; @@ -387,6 +392,7 @@ void copy(AzfsResourceId sourcePath, AzfsResourceId destinationPath) throws IOEx } @VisibleForTesting + // Generate an SAS Token if the user did not provide one through pipeline options String generateSasToken() throws IOException { if (!Strings.isNullOrEmpty(options.getSasToken())) { return options.getSasToken(); @@ -394,13 +400,13 @@ String generateSasToken() throws IOException { SharedAccessAccountPolicy sharedAccessAccountPolicy = new SharedAccessAccountPolicy(); long date = new Date().getTime(); - long expiryDate = new Date(date + expiryTime).getTime(); + long expiryDate = new Date(date + DEFAULT_EXPIRY_TIME).getTime(); - sharedAccessAccountPolicy.setPermissionsFromString("racwdlup"); + sharedAccessAccountPolicy.setPermissionsFromString(DEFAULT_PERMISSIONS); sharedAccessAccountPolicy.setSharedAccessStartTime(new Date(date)); sharedAccessAccountPolicy.setSharedAccessExpiryTime(new Date(expiryDate)); sharedAccessAccountPolicy.setResourceTypeFromString( - "co"); // container, object, add s for service + DEFAULT_RESOURCE_TYPES); // container, object, add s for service sharedAccessAccountPolicy.setServiceFromString("b"); // blob, add "fqt" for file, queue, table String storageConnectionString; From 4d14586568f2df76318bae05a55fdf73d3bc4d20 Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Wed, 19 Aug 2020 04:36:20 +0000 Subject: [PATCH 13/24] fixed a checkstyle issue, annotated tests in progress with @Ignore --- .../io/azure/blobstore/AzureBlobStoreFileSystemTest.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java index c6157ade70c4..e853e9f57127 100644 --- a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java +++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java @@ -25,7 +25,10 @@ import static org.junit.Assert.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import com.azure.core.http.rest.PagedIterable; import com.azure.storage.blob.BlobClient; @@ -52,6 +55,7 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -179,6 +183,7 @@ public void testWriteAndRead() throws IOException { } @Test + @Ignore public void testGlobExpansion() throws IOException { // TODO: Write this test with mocks - see GcsFileSystemTest String container = "test-container" + randomUUID(); @@ -261,6 +266,7 @@ private List toFilenames(MatchResult matchResult) throws IOException { } @Test + @Ignore public void testMatch() throws Exception { // TODO: Write this test with mocks - see GcsFileSystemTest String container = "test-container" + randomUUID(); From f97873dc95e43804cce90a0af45a8d6dd80e7c5d Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Wed, 19 Aug 2020 15:04:43 +0000 Subject: [PATCH 14/24] adding javadoc --- .../io/azure/blobstore/AzureBlobStoreFileSystem.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java index 1097f0ab6fa0..8076201c40fe 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java @@ -62,6 +62,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** {@link FileSystem} implementation for Azure Blob Storage. */ class AzureBlobStoreFileSystem extends FileSystem { private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class); @@ -75,6 +76,8 @@ class AzureBlobStoreFileSystem extends FileSystem { private static final String DEFAULT_PERMISSIONS = "racwdlup"; // By default generateSasToken returns an SAS token that grants access to containers and objects private static final String DEFAULT_RESOURCE_TYPES = "co"; + // This filesystem is for blob storage (not file, queue, or table) + private static final String DEFAULT_SERVICES = "b"; private Supplier client; private final BlobstoreOptions options; @@ -392,7 +395,7 @@ void copy(AzfsResourceId sourcePath, AzfsResourceId destinationPath) throws IOEx } @VisibleForTesting - // Generate an SAS Token if the user did not provide one through pipeline options + /** Generate an SAS Token if the user did not provide one through pipeline options */ String generateSasToken() throws IOException { if (!Strings.isNullOrEmpty(options.getSasToken())) { return options.getSasToken(); @@ -405,9 +408,8 @@ String generateSasToken() throws IOException { sharedAccessAccountPolicy.setPermissionsFromString(DEFAULT_PERMISSIONS); sharedAccessAccountPolicy.setSharedAccessStartTime(new Date(date)); sharedAccessAccountPolicy.setSharedAccessExpiryTime(new Date(expiryDate)); - sharedAccessAccountPolicy.setResourceTypeFromString( - DEFAULT_RESOURCE_TYPES); // container, object, add s for service - sharedAccessAccountPolicy.setServiceFromString("b"); // blob, add "fqt" for file, queue, table + sharedAccessAccountPolicy.setResourceTypeFromString(DEFAULT_RESOURCE_TYPES); + sharedAccessAccountPolicy.setServiceFromString(DEFAULT_SERVICES); String storageConnectionString; if (!Strings.isNullOrEmpty(options.getAzureConnectionString())) { @@ -439,7 +441,7 @@ protected void rename(List srcResourceIds, List delete(srcResourceIds); } - // This method with delete a virtual folder or a blob + /** This method will delete a virtual folder or a blob, not a container */ @Override protected void delete(Collection resourceIds) throws IOException { for (AzfsResourceId resourceId : resourceIds) { From 9aea1b2a4846e156ceeeb880d798fd1896d14045 Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Wed, 19 Aug 2020 15:54:53 +0000 Subject: [PATCH 15/24] fixing a bug in AzureReadableSeekableByteChannel read --- .../blobstore/AzureBlobStoreFileSystem.java | 3 ++- .../AzureReadableSeekableByteChannel.java | 26 +++++++++++++++---- 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java index 8076201c40fe..0f3000e9b5d4 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java @@ -347,6 +347,7 @@ protected ReadableByteChannel open(AzfsResourceId resourceId) throws IOException if (!blobClient.exists()) { throw new FileNotFoundException("The requested file doesn't exist."); } + LOG.info("Creating a ReadableByteChannel for {}", resourceId); return new AzureReadableSeekableByteChannel(blobClient); } @@ -441,7 +442,7 @@ protected void rename(List srcResourceIds, List delete(srcResourceIds); } - /** This method will delete a virtual folder or a blob, not a container */ + /** This method will delete a virtual folder or a blob, not a container. */ @Override protected void delete(Collection resourceIds) throws IOException { for (AzfsResourceId resourceId : resourceIds) { diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureReadableSeekableByteChannel.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureReadableSeekableByteChannel.java index e88cbf96aeee..e0b4594f1017 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureReadableSeekableByteChannel.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureReadableSeekableByteChannel.java @@ -25,17 +25,23 @@ import java.nio.ByteBuffer; import java.nio.channels.ClosedChannelException; import java.nio.channels.SeekableByteChannel; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class AzureReadableSeekableByteChannel implements SeekableByteChannel { + private static final Logger LOG = LoggerFactory.getLogger(AzureReadableSeekableByteChannel.class); private final BlobInputStream inputStream; private boolean closed; - private final long contentLength; + private final Long contentLength; private long position = 0; public AzureReadableSeekableByteChannel(BlobClient blobClient) { inputStream = blobClient.openInputStream(); contentLength = blobClient.getProperties().getBlobSize(); + inputStream.mark(contentLength.intValue()); closed = false; } @@ -44,23 +50,22 @@ public int read(ByteBuffer dst) throws IOException { if (closed) { throw new ClosedChannelException(); } - if (!dst.hasRemaining()) { - return 0; - } int read = 0; if (dst.hasArray()) { // Stores up to dst.remaining() bytes into dst.array() starting at dst.position(). // But dst can have an offset with its backing array, hence the + dst.arrayOffset(). read = inputStream.read(dst.array(), dst.position() + dst.arrayOffset(), dst.remaining()); + LOG.info("PArray: " + StandardCharsets.UTF_8.decode(dst).toString()); } else { byte[] myarray = new byte[dst.remaining()]; read = inputStream.read(myarray, 0, myarray.length); dst.put(myarray); + LOG.info("Array: " + Arrays.toString(myarray)); } if (read > 0) { - position += read; + dst.position(dst.position() + read); } return read; } @@ -87,6 +92,17 @@ public SeekableByteChannel position(long newPosition) throws IOException { checkArgument(newPosition < contentLength, "new position too high"); Long bytesToSkip = newPosition - position; + LOG.info( + "Blob length: {}. Current position: {}. New position: {}. Skipping {} bytes.", + contentLength, + position, + newPosition, + bytesToSkip); + if (bytesToSkip < 0) { + inputStream.reset(); + bytesToSkip = newPosition; + LOG.info("As bytes to skip is negative. resetting. Skipping {}", bytesToSkip); + } Long n = inputStream.skip(bytesToSkip); position += n; return this; From aab24bfc50fd379a2a013ebe0ef52ef047756a48 Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Wed, 19 Aug 2020 16:27:11 +0000 Subject: [PATCH 16/24] removing logger from azure bytechannel --- .../azure/blobstore/AzureBlobStoreFileSystem.java | 8 ++++++-- .../AzureReadableSeekableByteChannel.java | 14 -------------- 2 files changed, 6 insertions(+), 16 deletions(-) diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java index 0f3000e9b5d4..2306a21ccd04 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java @@ -134,10 +134,14 @@ protected List match(List specs) { ImmutableList.Builder matchResults = ImmutableList.builder(); for (Boolean isGlob : isGlobBooleans) { if (isGlob) { - checkState(globMatches.hasNext(), "Expect globMatches has next."); + checkState( + globMatches.hasNext(), + "Internal error encountered in AzureBlobStoreFileSystem: expected more elements in globMatches."); matchResults.add(globMatches.next()); } else { - checkState(nonGlobMatches.hasNext(), "Expect nonGlobMatches has next."); + checkState( + nonGlobMatches.hasNext(), + "Internal error encountered in AzureBlobStoreFileSystem: expected more elements in nonGlobMatches."); matchResults.add(nonGlobMatches.next()); } } diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureReadableSeekableByteChannel.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureReadableSeekableByteChannel.java index e0b4594f1017..458198de8080 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureReadableSeekableByteChannel.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureReadableSeekableByteChannel.java @@ -25,14 +25,9 @@ import java.nio.ByteBuffer; import java.nio.channels.ClosedChannelException; import java.nio.channels.SeekableByteChannel; -import java.nio.charset.StandardCharsets; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; class AzureReadableSeekableByteChannel implements SeekableByteChannel { - private static final Logger LOG = LoggerFactory.getLogger(AzureReadableSeekableByteChannel.class); private final BlobInputStream inputStream; private boolean closed; private final Long contentLength; @@ -56,12 +51,10 @@ public int read(ByteBuffer dst) throws IOException { // Stores up to dst.remaining() bytes into dst.array() starting at dst.position(). // But dst can have an offset with its backing array, hence the + dst.arrayOffset(). read = inputStream.read(dst.array(), dst.position() + dst.arrayOffset(), dst.remaining()); - LOG.info("PArray: " + StandardCharsets.UTF_8.decode(dst).toString()); } else { byte[] myarray = new byte[dst.remaining()]; read = inputStream.read(myarray, 0, myarray.length); dst.put(myarray); - LOG.info("Array: " + Arrays.toString(myarray)); } if (read > 0) { @@ -92,16 +85,9 @@ public SeekableByteChannel position(long newPosition) throws IOException { checkArgument(newPosition < contentLength, "new position too high"); Long bytesToSkip = newPosition - position; - LOG.info( - "Blob length: {}. Current position: {}. New position: {}. Skipping {} bytes.", - contentLength, - position, - newPosition, - bytesToSkip); if (bytesToSkip < 0) { inputStream.reset(); bytesToSkip = newPosition; - LOG.info("As bytes to skip is negative. resetting. Skipping {}", bytesToSkip); } Long n = inputStream.skip(bytesToSkip); position += n; From e2f7eb278887104294ba087ea49a73d7b6987ac5 Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Wed, 19 Aug 2020 17:16:39 +0000 Subject: [PATCH 17/24] removed non-serializable option --- .../apache/beam/sdk/io/azure/options/BlobstoreOptions.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java index 1060177c0494..a0ee8abdbfee 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java @@ -56,14 +56,14 @@ void setBlobstoreClientFactoryClass( void setPipelinePolicy(HttpPipelinePolicy pipelinePolicy); /** The client configuration instance that should be used to configure Azure service clients. */ - @Description( + /* @Description( "The configuration instance used to retrieve environment configuration values " + "when building an Azure Blobstore client. Set only those that need custom changes.") @Default.InstanceFactory(BlobstoreOptions.ConfigurationFactory.class) @Nullable Configuration getEnvironmentConfiguration(); - void setEnvironmentConfiguration(Configuration configuration); + void setEnvironmentConfiguration(Configuration configuration); */ /** Default Azure client configuration. */ class ConfigurationFactory implements DefaultValueFactory { From 4bc1857b7fab3c8383e4d80d7284333a7165857d Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Wed, 19 Aug 2020 17:27:10 +0000 Subject: [PATCH 18/24] removed non-serializable configuration option --- .../DefaultBlobstoreClientBuilderFactory.java | 4 ---- .../io/azure/options/BlobstoreOptions.java | 20 ------------------- 2 files changed, 24 deletions(-) diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java index 894caff5b122..d8b44e298385 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java @@ -62,10 +62,6 @@ public BlobServiceClientBuilder createBuilder(BlobstoreOptions blobstoreOptions) builder = builder.customerProvidedKey(blobstoreOptions.getCustomerProvidedKey()); } - if (blobstoreOptions.getEnvironmentConfiguration() != null) { - builder = builder.configuration(blobstoreOptions.getEnvironmentConfiguration()); - } - if (blobstoreOptions.getPipelinePolicy() != null) { builder = builder.addPolicy(blobstoreOptions.getPipelinePolicy()); } diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java index a0ee8abdbfee..108cb0c31d37 100644 --- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java +++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java @@ -21,7 +21,6 @@ import com.azure.core.http.HttpClient; import com.azure.core.http.HttpPipeline; import com.azure.core.http.policy.HttpPipelinePolicy; -import com.azure.core.util.Configuration; import com.azure.identity.DefaultAzureCredentialBuilder; import com.azure.storage.blob.models.CustomerProvidedKey; import com.azure.storage.common.StorageSharedKeyCredential; @@ -55,25 +54,6 @@ void setBlobstoreClientFactoryClass( void setPipelinePolicy(HttpPipelinePolicy pipelinePolicy); - /** The client configuration instance that should be used to configure Azure service clients. */ - /* @Description( - "The configuration instance used to retrieve environment configuration values " - + "when building an Azure Blobstore client. Set only those that need custom changes.") - @Default.InstanceFactory(BlobstoreOptions.ConfigurationFactory.class) - @Nullable - Configuration getEnvironmentConfiguration(); - - void setEnvironmentConfiguration(Configuration configuration); */ - - /** Default Azure client configuration. */ - class ConfigurationFactory implements DefaultValueFactory { - - @Override - public Configuration create(PipelineOptions options) { - return new Configuration(); - } - } - @Description("Sets the connection string to connect to the Azure Blobstore client.") String getAzureConnectionString(); From fb1467dfdeca12d7f66c83ec4c1ae8ccdf5cbc8a Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Thu, 20 Aug 2020 03:30:50 +0000 Subject: [PATCH 19/24] removed integration tests - they will be in a followup PR --- .../blobstore/AzureBlobStoreFileSystemIT.java | 353 ------------------ 1 file changed, 353 deletions(-) delete mode 100644 sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java deleted file mode 100644 index 3ee5c47e5dab..000000000000 --- a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java +++ /dev/null @@ -1,353 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.blobstore; - -import static java.util.UUID.randomUUID; -import static org.apache.beam.sdk.io.fs.CreateOptions.StandardCreateOptions.builder; -import static org.hamcrest.Matchers.contains; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; - -import com.azure.storage.blob.BlobContainerClient; -import com.azure.storage.blob.BlobServiceClient; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.ReadableByteChannel; -import java.nio.channels.WritableByteChannel; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; -import org.apache.beam.sdk.io.fs.MatchResult; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; -import org.apache.commons.io.FileUtils; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -@RunWith(JUnit4.class) -public class AzureBlobStoreFileSystemIT { - - private static AzureBlobStoreFileSystem azureBlobStoreFileSystem; - - @Before - public void beforeClass() { - BlobstoreOptions options = PipelineOptionsFactory.as(BlobstoreOptions.class); - // TODO: Change next line to Beam's Azure connection string - String connectionString = System.getenv("AZURE_STORAGE_CONNECTION_STRING"); - options.setAzureConnectionString(connectionString); - azureBlobStoreFileSystem = new AzureBlobStoreFileSystem(options); - } - - // TODO: Write integration tests that build pipelines, etc. - - @Test - public void testGetScheme() { - assertEquals("azfs", azureBlobStoreFileSystem.getScheme()); - } - - @Test - public void testGlobTranslation() { - assertEquals("foo", AzureBlobStoreFileSystem.wildcardToRegexp("foo")); - assertEquals("fo[^/]*o", AzureBlobStoreFileSystem.wildcardToRegexp("fo*o")); - assertEquals("f[^/]*o\\.[^/]", AzureBlobStoreFileSystem.wildcardToRegexp("f*o.?")); - assertEquals("foo-[0-9][^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo-[0-9]*")); - assertEquals("foo-[0-9].*", AzureBlobStoreFileSystem.wildcardToRegexp("foo-[0-9]**")); - assertEquals(".*foo", AzureBlobStoreFileSystem.wildcardToRegexp("**/*foo")); - assertEquals(".*foo", AzureBlobStoreFileSystem.wildcardToRegexp("**foo")); - assertEquals("foo/[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*")); - assertEquals("foo[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo*")); - assertEquals("foo/[^/]*/[^/]*/[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*/*/*")); - assertEquals("foo/[^/]*/.*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*/**")); - assertEquals("foo.*baz", AzureBlobStoreFileSystem.wildcardToRegexp("foo**baz")); - } - - @Test - public void testDelete() throws IOException { - String containerName = "test-container" + randomUUID(); - String blobName1 = "blob" + randomUUID(); - String blobName2 = "dir1/anotherBlob" + randomUUID(); - - // Create files to delete - BlobContainerClient blobContainerClient = - azureBlobStoreFileSystem.getClient().createBlobContainer(containerName); - assertTrue(blobContainerClient.exists()); - blobContainerClient.getBlobClient(blobName1).uploadFromFile("src/test/resources/in.txt"); - blobContainerClient.getBlobClient(blobName2).uploadFromFile("src/test/resources/in.txt"); - assertTrue(blobContainerClient.getBlobClient(blobName1).exists()); - assertTrue(blobContainerClient.getBlobClient(blobName2).exists()); - - // Delete the files - Collection toDelete = new ArrayList<>(); - String account = blobContainerClient.getAccountName(); - // delete blob - toDelete.add(AzfsResourceId.fromComponents(account, containerName, blobName1)); - // delete directory - toDelete.add(AzfsResourceId.fromComponents(account, containerName, "dir1/")); - azureBlobStoreFileSystem.delete(toDelete); - - // Ensure exception is thrown, clean up - assertFalse(blobContainerClient.getBlobClient(blobName1).exists()); - assertFalse(blobContainerClient.getBlobClient(blobName2).exists()); - assertThrows(FileNotFoundException.class, () -> azureBlobStoreFileSystem.delete(toDelete)); - blobContainerClient.delete(); - } - - @Test - public void testCopy() throws IOException { - BlobServiceClient blobServiceClient = azureBlobStoreFileSystem.getClient(); - String account = blobServiceClient.getAccountName(); - - List src = new ArrayList<>(); - List dest = new ArrayList<>(); - String srcContainer = "source-container" + randomUUID(); - String destContainer = "dest-container" + randomUUID(); - - // Create source file - BlobContainerClient srcContainerClient = blobServiceClient.createBlobContainer(srcContainer); - srcContainerClient.getBlobClient("src-blob").uploadFromFile("src/test/resources/in.txt"); - - // Copy source file to destination - src.add(AzfsResourceId.fromComponents(account, srcContainer, "src-blob")); - dest.add(AzfsResourceId.fromComponents(account, destContainer, "dest-blob")); - azureBlobStoreFileSystem.copy(src, dest); - - // Confirm the destination container was created - BlobContainerClient destContainerClient = - blobServiceClient.getBlobContainerClient(destContainer); - assertTrue(destContainerClient.getBlobClient("dest-blob").exists()); - - // Confirm that the source and destination files are the same - srcContainerClient.getBlobClient("src-blob").downloadToFile("./src/test/resources/blob1"); - destContainerClient.getBlobClient("dest-blob").downloadToFile("./src/test/resources/blob2"); - File file1 = new File("./src/test/resources/blob1"); - File file2 = new File("./src/test/resources/blob2"); - assertTrue("The files differ!", FileUtils.contentEquals(file1, file2)); - - // Clean up - assertTrue(file1.delete()); - assertTrue(file2.delete()); - blobServiceClient.deleteBlobContainer(srcContainer); - blobServiceClient.deleteBlobContainer(destContainer); - } - - @Test - public void testWriteAndRead() throws IOException { - BlobServiceClient client = azureBlobStoreFileSystem.getClient(); - String containerName = "test-container" + randomUUID(); - client.createBlobContainer(containerName); - - byte[] writtenArray = new byte[] {0}; - ByteBuffer bb = ByteBuffer.allocate(writtenArray.length); - bb.put(writtenArray); - - // Create an object and write data to it - AzfsResourceId path = - AzfsResourceId.fromUri( - "azfs://" + client.getAccountName() + "/" + containerName + "/foo/bar.txt"); - WritableByteChannel writableByteChannel = - azureBlobStoreFileSystem.create(path, builder().setMimeType("application/text").build()); - writableByteChannel.write(bb); - writableByteChannel.close(); - - // Read the same object - ByteBuffer bb2 = ByteBuffer.allocate(writtenArray.length); - ReadableByteChannel open = azureBlobStoreFileSystem.open(path); - open.read(bb2); - - // Compare the content with the one that was written - byte[] readArray = bb2.array(); - assertArrayEquals(readArray, writtenArray); - open.close(); - - // Clean up - client.getBlobContainerClient(containerName).delete(); - } - - @Test - public void testGlobExpansion() throws IOException { - String container = "test-container" + randomUUID(); - BlobContainerClient blobContainerClient = - azureBlobStoreFileSystem.getClient().createBlobContainer(container); - - // Create files - List blobNames = new ArrayList<>(); - blobNames.add("testdirectory/file1name"); - blobNames.add("testdirectory/file2name"); - blobNames.add("testdirectory/file3name"); - blobNames.add("testdirectory/otherfile"); - blobNames.add("testotherdirectory/file4name"); - for (String blob : blobNames) { - blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); - } - - // Test patterns - { - AzfsResourceId pattern = - AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file*"); - List expectedFiles = - ImmutableList.of( - "azfs://account/" + container + "/testdirectory/file1name", - "azfs://account/" + container + "/testdirectory/file2name", - "azfs://account/" + container + "/testdirectory/file3name"); - - assertThat( - expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); - } - - { - AzfsResourceId pattern = - AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file[1-3]*"); - List expectedFiles = - ImmutableList.of( - "azfs://account/" + container + "/testdirectory/file1name", - "azfs://account/" + container + "/testdirectory/file2name", - "azfs://account/" + container + "/testdirectory/file3name"); - - assertThat( - expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); - } - - { - AzfsResourceId pattern = - AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file?name"); - List expectedFiles = - ImmutableList.of( - "azfs://account/" + container + "/testdirectory/file1name", - "azfs://account/" + container + "/testdirectory/file2name", - "azfs://account/" + container + "/testdirectory/file3name"); - - assertThat( - expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); - } - - { - AzfsResourceId pattern = - AzfsResourceId.fromUri("azfs://account/" + container + "/test*ectory/fi*name"); - List expectedFiles = - ImmutableList.of( - "azfs://account/" + container + "/testdirectory/file1name", - "azfs://account/" + container + "/testdirectory/file2name", - "azfs://account/" + container + "/testdirectory/file3name", - "azfs://account/" + container + "/testotherdirectory/file4name"); - - assertThat( - expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); - } - - // Clean up - blobContainerClient.delete(); - } - - private List toFilenames(MatchResult matchResult) throws IOException { - return FluentIterable.from(matchResult.metadata()) - .transform(metadata -> (metadata.resourceId()).toString()) - .toList(); - } - - @Test - public void testMatch() throws Exception { - String container = "test-container" + randomUUID(); - BlobContainerClient blobContainerClient = - azureBlobStoreFileSystem.getClient().createBlobContainer(container); - - // Create files - List blobNames = new ArrayList<>(); - blobNames.add("testdirectory/file1name"); - blobNames.add("testdirectory/file2name"); - blobNames.add("testdirectory/file3name"); - blobNames.add("testdirectory/file4name"); - blobNames.add("testdirectory/otherfile"); - blobNames.add("testotherdirectory/anotherfile"); - for (String blob : blobNames) { - blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); - } - - List specs = - ImmutableList.of( - "azfs://account/" + container + "/testdirectory/file[1-3]*", - "azfs://account/" + container + "/testdirectory/non-exist-file", - "azfs://account/" + container + "/testdirectory/otherfile"); - - List matchResults = azureBlobStoreFileSystem.match(specs); - - // Confirm that match results are as expected - assertEquals(3, matchResults.size()); - assertEquals(MatchResult.Status.OK, matchResults.get(0).status()); - assertThat( - ImmutableList.of( - "azfs://account/" + container + "/testdirectory/file1name", - "azfs://account/" + container + "/testdirectory/file2name", - "azfs://account/" + container + "/testdirectory/file3name"), - contains(toFilenames(matchResults.get(0)).toArray())); - assertEquals(MatchResult.Status.NOT_FOUND, matchResults.get(1).status()); - assertEquals(MatchResult.Status.OK, matchResults.get(2).status()); - assertThat( - ImmutableList.of("azfs://account/" + container + "/testdirectory/otherfile"), - contains(toFilenames(matchResults.get(2)).toArray())); - - blobContainerClient.delete(); - } - - @Test - public void testMatchNonGlobs() throws Exception { - String container = "test-container" + randomUUID(); - BlobContainerClient blobContainerClient = - azureBlobStoreFileSystem.getClient().createBlobContainer(container); - - List blobNames = new ArrayList<>(); - blobNames.add("testdirectory/file1name"); - blobNames.add("testdirectory/dir2name/"); - blobNames.add("testdirectory/file4name"); - // TODO: Also test match results where MatchResult.STATUS != OK (see gcs and s3 tests) - - for (String blob : blobNames) { - blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); - } - - List specs = - ImmutableList.of( - "azfs://account/" + container + "/testdirectory/file1name", - "azfs://account/" + container + "/testdirectory/dir2name/", - "azfs://account/" + container + "/testdirectory/file4name"); - - List matchResults = azureBlobStoreFileSystem.match(specs); - - assertEquals(3, matchResults.size()); - assertThat( - ImmutableList.of("azfs://account/" + container + "/testdirectory/file1name"), - contains(toFilenames(matchResults.get(0)).toArray())); - assertThat( - ImmutableList.of("azfs://account/" + container + "/testdirectory/dir2name/"), - contains(toFilenames(matchResults.get(1)).toArray())); - assertThat( - ImmutableList.of("azfs://account/" + container + "/testdirectory/file4name"), - contains(toFilenames(matchResults.get(2)).toArray())); - - blobContainerClient.delete(); - } -} From 8daeb4da94d2d2ace3000183cb0f90befcadc692 Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Thu, 20 Aug 2020 04:24:52 +0000 Subject: [PATCH 20/24] working on integration tests for azure filesystem --- .../blobstore/AzureBlobStoreFileSystemIT.java | 353 ++++++++++++++++++ 1 file changed, 353 insertions(+) create mode 100644 sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java new file mode 100644 index 000000000000..3ee5c47e5dab --- /dev/null +++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java @@ -0,0 +1,353 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.blobstore; + +import static java.util.UUID.randomUUID; +import static org.apache.beam.sdk.io.fs.CreateOptions.StandardCreateOptions.builder; +import static org.hamcrest.Matchers.contains; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import com.azure.storage.blob.BlobContainerClient; +import com.azure.storage.blob.BlobServiceClient; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; +import org.apache.beam.sdk.io.fs.MatchResult; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.commons.io.FileUtils; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class AzureBlobStoreFileSystemIT { + + private static AzureBlobStoreFileSystem azureBlobStoreFileSystem; + + @Before + public void beforeClass() { + BlobstoreOptions options = PipelineOptionsFactory.as(BlobstoreOptions.class); + // TODO: Change next line to Beam's Azure connection string + String connectionString = System.getenv("AZURE_STORAGE_CONNECTION_STRING"); + options.setAzureConnectionString(connectionString); + azureBlobStoreFileSystem = new AzureBlobStoreFileSystem(options); + } + + // TODO: Write integration tests that build pipelines, etc. + + @Test + public void testGetScheme() { + assertEquals("azfs", azureBlobStoreFileSystem.getScheme()); + } + + @Test + public void testGlobTranslation() { + assertEquals("foo", AzureBlobStoreFileSystem.wildcardToRegexp("foo")); + assertEquals("fo[^/]*o", AzureBlobStoreFileSystem.wildcardToRegexp("fo*o")); + assertEquals("f[^/]*o\\.[^/]", AzureBlobStoreFileSystem.wildcardToRegexp("f*o.?")); + assertEquals("foo-[0-9][^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo-[0-9]*")); + assertEquals("foo-[0-9].*", AzureBlobStoreFileSystem.wildcardToRegexp("foo-[0-9]**")); + assertEquals(".*foo", AzureBlobStoreFileSystem.wildcardToRegexp("**/*foo")); + assertEquals(".*foo", AzureBlobStoreFileSystem.wildcardToRegexp("**foo")); + assertEquals("foo/[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*")); + assertEquals("foo[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo*")); + assertEquals("foo/[^/]*/[^/]*/[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*/*/*")); + assertEquals("foo/[^/]*/.*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*/**")); + assertEquals("foo.*baz", AzureBlobStoreFileSystem.wildcardToRegexp("foo**baz")); + } + + @Test + public void testDelete() throws IOException { + String containerName = "test-container" + randomUUID(); + String blobName1 = "blob" + randomUUID(); + String blobName2 = "dir1/anotherBlob" + randomUUID(); + + // Create files to delete + BlobContainerClient blobContainerClient = + azureBlobStoreFileSystem.getClient().createBlobContainer(containerName); + assertTrue(blobContainerClient.exists()); + blobContainerClient.getBlobClient(blobName1).uploadFromFile("src/test/resources/in.txt"); + blobContainerClient.getBlobClient(blobName2).uploadFromFile("src/test/resources/in.txt"); + assertTrue(blobContainerClient.getBlobClient(blobName1).exists()); + assertTrue(blobContainerClient.getBlobClient(blobName2).exists()); + + // Delete the files + Collection toDelete = new ArrayList<>(); + String account = blobContainerClient.getAccountName(); + // delete blob + toDelete.add(AzfsResourceId.fromComponents(account, containerName, blobName1)); + // delete directory + toDelete.add(AzfsResourceId.fromComponents(account, containerName, "dir1/")); + azureBlobStoreFileSystem.delete(toDelete); + + // Ensure exception is thrown, clean up + assertFalse(blobContainerClient.getBlobClient(blobName1).exists()); + assertFalse(blobContainerClient.getBlobClient(blobName2).exists()); + assertThrows(FileNotFoundException.class, () -> azureBlobStoreFileSystem.delete(toDelete)); + blobContainerClient.delete(); + } + + @Test + public void testCopy() throws IOException { + BlobServiceClient blobServiceClient = azureBlobStoreFileSystem.getClient(); + String account = blobServiceClient.getAccountName(); + + List src = new ArrayList<>(); + List dest = new ArrayList<>(); + String srcContainer = "source-container" + randomUUID(); + String destContainer = "dest-container" + randomUUID(); + + // Create source file + BlobContainerClient srcContainerClient = blobServiceClient.createBlobContainer(srcContainer); + srcContainerClient.getBlobClient("src-blob").uploadFromFile("src/test/resources/in.txt"); + + // Copy source file to destination + src.add(AzfsResourceId.fromComponents(account, srcContainer, "src-blob")); + dest.add(AzfsResourceId.fromComponents(account, destContainer, "dest-blob")); + azureBlobStoreFileSystem.copy(src, dest); + + // Confirm the destination container was created + BlobContainerClient destContainerClient = + blobServiceClient.getBlobContainerClient(destContainer); + assertTrue(destContainerClient.getBlobClient("dest-blob").exists()); + + // Confirm that the source and destination files are the same + srcContainerClient.getBlobClient("src-blob").downloadToFile("./src/test/resources/blob1"); + destContainerClient.getBlobClient("dest-blob").downloadToFile("./src/test/resources/blob2"); + File file1 = new File("./src/test/resources/blob1"); + File file2 = new File("./src/test/resources/blob2"); + assertTrue("The files differ!", FileUtils.contentEquals(file1, file2)); + + // Clean up + assertTrue(file1.delete()); + assertTrue(file2.delete()); + blobServiceClient.deleteBlobContainer(srcContainer); + blobServiceClient.deleteBlobContainer(destContainer); + } + + @Test + public void testWriteAndRead() throws IOException { + BlobServiceClient client = azureBlobStoreFileSystem.getClient(); + String containerName = "test-container" + randomUUID(); + client.createBlobContainer(containerName); + + byte[] writtenArray = new byte[] {0}; + ByteBuffer bb = ByteBuffer.allocate(writtenArray.length); + bb.put(writtenArray); + + // Create an object and write data to it + AzfsResourceId path = + AzfsResourceId.fromUri( + "azfs://" + client.getAccountName() + "/" + containerName + "/foo/bar.txt"); + WritableByteChannel writableByteChannel = + azureBlobStoreFileSystem.create(path, builder().setMimeType("application/text").build()); + writableByteChannel.write(bb); + writableByteChannel.close(); + + // Read the same object + ByteBuffer bb2 = ByteBuffer.allocate(writtenArray.length); + ReadableByteChannel open = azureBlobStoreFileSystem.open(path); + open.read(bb2); + + // Compare the content with the one that was written + byte[] readArray = bb2.array(); + assertArrayEquals(readArray, writtenArray); + open.close(); + + // Clean up + client.getBlobContainerClient(containerName).delete(); + } + + @Test + public void testGlobExpansion() throws IOException { + String container = "test-container" + randomUUID(); + BlobContainerClient blobContainerClient = + azureBlobStoreFileSystem.getClient().createBlobContainer(container); + + // Create files + List blobNames = new ArrayList<>(); + blobNames.add("testdirectory/file1name"); + blobNames.add("testdirectory/file2name"); + blobNames.add("testdirectory/file3name"); + blobNames.add("testdirectory/otherfile"); + blobNames.add("testotherdirectory/file4name"); + for (String blob : blobNames) { + blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); + } + + // Test patterns + { + AzfsResourceId pattern = + AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file*"); + List expectedFiles = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name"); + + assertThat( + expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); + } + + { + AzfsResourceId pattern = + AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file[1-3]*"); + List expectedFiles = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name"); + + assertThat( + expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); + } + + { + AzfsResourceId pattern = + AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file?name"); + List expectedFiles = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name"); + + assertThat( + expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); + } + + { + AzfsResourceId pattern = + AzfsResourceId.fromUri("azfs://account/" + container + "/test*ectory/fi*name"); + List expectedFiles = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name", + "azfs://account/" + container + "/testotherdirectory/file4name"); + + assertThat( + expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); + } + + // Clean up + blobContainerClient.delete(); + } + + private List toFilenames(MatchResult matchResult) throws IOException { + return FluentIterable.from(matchResult.metadata()) + .transform(metadata -> (metadata.resourceId()).toString()) + .toList(); + } + + @Test + public void testMatch() throws Exception { + String container = "test-container" + randomUUID(); + BlobContainerClient blobContainerClient = + azureBlobStoreFileSystem.getClient().createBlobContainer(container); + + // Create files + List blobNames = new ArrayList<>(); + blobNames.add("testdirectory/file1name"); + blobNames.add("testdirectory/file2name"); + blobNames.add("testdirectory/file3name"); + blobNames.add("testdirectory/file4name"); + blobNames.add("testdirectory/otherfile"); + blobNames.add("testotherdirectory/anotherfile"); + for (String blob : blobNames) { + blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); + } + + List specs = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file[1-3]*", + "azfs://account/" + container + "/testdirectory/non-exist-file", + "azfs://account/" + container + "/testdirectory/otherfile"); + + List matchResults = azureBlobStoreFileSystem.match(specs); + + // Confirm that match results are as expected + assertEquals(3, matchResults.size()); + assertEquals(MatchResult.Status.OK, matchResults.get(0).status()); + assertThat( + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/file2name", + "azfs://account/" + container + "/testdirectory/file3name"), + contains(toFilenames(matchResults.get(0)).toArray())); + assertEquals(MatchResult.Status.NOT_FOUND, matchResults.get(1).status()); + assertEquals(MatchResult.Status.OK, matchResults.get(2).status()); + assertThat( + ImmutableList.of("azfs://account/" + container + "/testdirectory/otherfile"), + contains(toFilenames(matchResults.get(2)).toArray())); + + blobContainerClient.delete(); + } + + @Test + public void testMatchNonGlobs() throws Exception { + String container = "test-container" + randomUUID(); + BlobContainerClient blobContainerClient = + azureBlobStoreFileSystem.getClient().createBlobContainer(container); + + List blobNames = new ArrayList<>(); + blobNames.add("testdirectory/file1name"); + blobNames.add("testdirectory/dir2name/"); + blobNames.add("testdirectory/file4name"); + // TODO: Also test match results where MatchResult.STATUS != OK (see gcs and s3 tests) + + for (String blob : blobNames) { + blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); + } + + List specs = + ImmutableList.of( + "azfs://account/" + container + "/testdirectory/file1name", + "azfs://account/" + container + "/testdirectory/dir2name/", + "azfs://account/" + container + "/testdirectory/file4name"); + + List matchResults = azureBlobStoreFileSystem.match(specs); + + assertEquals(3, matchResults.size()); + assertThat( + ImmutableList.of("azfs://account/" + container + "/testdirectory/file1name"), + contains(toFilenames(matchResults.get(0)).toArray())); + assertThat( + ImmutableList.of("azfs://account/" + container + "/testdirectory/dir2name/"), + contains(toFilenames(matchResults.get(1)).toArray())); + assertThat( + ImmutableList.of("azfs://account/" + container + "/testdirectory/file4name"), + contains(toFilenames(matchResults.get(2)).toArray())); + + blobContainerClient.delete(); + } +} From b228a326287ff70bc29cbfd37e70b7bc2b1fde1e Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Thu, 20 Aug 2020 21:48:24 +0000 Subject: [PATCH 21/24] adding integration tests --- sdks/java/io/azure/build.gradle | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/sdks/java/io/azure/build.gradle b/sdks/java/io/azure/build.gradle index 6a26bd797cef..833c23f45707 100644 --- a/sdks/java/io/azure/build.gradle +++ b/sdks/java/io/azure/build.gradle @@ -1,3 +1,5 @@ +import groovy.json.JsonOutput + /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -42,3 +44,30 @@ dependencies { testCompile library.java.junit testRuntimeOnly library.java.slf4j_jdk14 } + +/** + * These are integration tests with the real Azure service and the DirectRunner. + */ +task integrationTest(type: Test, dependsOn: processTestResources) { + group = "Verification" + def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' + def gcpTempRoot = project.findProperty('gcpTempRoot') ?: 'gs://temp-storage-for-end-to-end-tests' + systemProperty "beamTestPipelineOptions", JsonOutput.toJson([ + "--runner=DirectRunner", + "--project=${gcpProject}", + "--tempRoot=${gcpTempRoot}", + "--inputFile=azfs://ettasaccount/test-container/test-blob" + ]) + + // Disable Gradle cache: these ITs interact with live service that should always be considered "out of date" + outputs.upToDateWhen { false } + + include '**/WordCountIT.class' + maxParallelForks 4 + classpath = sourceSets.test.runtimeClasspath + testClassesDirs = sourceSets.test.output.classesDirs + + useJUnit { + excludeCategories "org.apache.beam.sdk.testing.UsesKms" + } +} From 91fff24c496beff95d961060370a44c0d600783c Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Thu, 20 Aug 2020 21:52:14 +0000 Subject: [PATCH 22/24] organizing imports --- sdks/java/io/azure/build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/azure/build.gradle b/sdks/java/io/azure/build.gradle index 833c23f45707..2930e63c379e 100644 --- a/sdks/java/io/azure/build.gradle +++ b/sdks/java/io/azure/build.gradle @@ -1,5 +1,3 @@ -import groovy.json.JsonOutput - /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -18,6 +16,8 @@ import groovy.json.JsonOutput * limitations under the License. */ +import groovy.json.JsonOutput + plugins { id 'org.apache.beam.module' } From 8d75641c85f033a383941e1b303ff53e4af73c59 Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Thu, 20 Aug 2020 21:53:13 +0000 Subject: [PATCH 23/24] removed unused tests --- .../blobstore/AzureBlobStoreFileSystemIT.java | 353 ------------------ 1 file changed, 353 deletions(-) delete mode 100644 sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java deleted file mode 100644 index 3ee5c47e5dab..000000000000 --- a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemIT.java +++ /dev/null @@ -1,353 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.blobstore; - -import static java.util.UUID.randomUUID; -import static org.apache.beam.sdk.io.fs.CreateOptions.StandardCreateOptions.builder; -import static org.hamcrest.Matchers.contains; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; - -import com.azure.storage.blob.BlobContainerClient; -import com.azure.storage.blob.BlobServiceClient; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.ReadableByteChannel; -import java.nio.channels.WritableByteChannel; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; -import org.apache.beam.sdk.io.fs.MatchResult; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; -import org.apache.commons.io.FileUtils; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -@RunWith(JUnit4.class) -public class AzureBlobStoreFileSystemIT { - - private static AzureBlobStoreFileSystem azureBlobStoreFileSystem; - - @Before - public void beforeClass() { - BlobstoreOptions options = PipelineOptionsFactory.as(BlobstoreOptions.class); - // TODO: Change next line to Beam's Azure connection string - String connectionString = System.getenv("AZURE_STORAGE_CONNECTION_STRING"); - options.setAzureConnectionString(connectionString); - azureBlobStoreFileSystem = new AzureBlobStoreFileSystem(options); - } - - // TODO: Write integration tests that build pipelines, etc. - - @Test - public void testGetScheme() { - assertEquals("azfs", azureBlobStoreFileSystem.getScheme()); - } - - @Test - public void testGlobTranslation() { - assertEquals("foo", AzureBlobStoreFileSystem.wildcardToRegexp("foo")); - assertEquals("fo[^/]*o", AzureBlobStoreFileSystem.wildcardToRegexp("fo*o")); - assertEquals("f[^/]*o\\.[^/]", AzureBlobStoreFileSystem.wildcardToRegexp("f*o.?")); - assertEquals("foo-[0-9][^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo-[0-9]*")); - assertEquals("foo-[0-9].*", AzureBlobStoreFileSystem.wildcardToRegexp("foo-[0-9]**")); - assertEquals(".*foo", AzureBlobStoreFileSystem.wildcardToRegexp("**/*foo")); - assertEquals(".*foo", AzureBlobStoreFileSystem.wildcardToRegexp("**foo")); - assertEquals("foo/[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*")); - assertEquals("foo[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo*")); - assertEquals("foo/[^/]*/[^/]*/[^/]*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*/*/*")); - assertEquals("foo/[^/]*/.*", AzureBlobStoreFileSystem.wildcardToRegexp("foo/*/**")); - assertEquals("foo.*baz", AzureBlobStoreFileSystem.wildcardToRegexp("foo**baz")); - } - - @Test - public void testDelete() throws IOException { - String containerName = "test-container" + randomUUID(); - String blobName1 = "blob" + randomUUID(); - String blobName2 = "dir1/anotherBlob" + randomUUID(); - - // Create files to delete - BlobContainerClient blobContainerClient = - azureBlobStoreFileSystem.getClient().createBlobContainer(containerName); - assertTrue(blobContainerClient.exists()); - blobContainerClient.getBlobClient(blobName1).uploadFromFile("src/test/resources/in.txt"); - blobContainerClient.getBlobClient(blobName2).uploadFromFile("src/test/resources/in.txt"); - assertTrue(blobContainerClient.getBlobClient(blobName1).exists()); - assertTrue(blobContainerClient.getBlobClient(blobName2).exists()); - - // Delete the files - Collection toDelete = new ArrayList<>(); - String account = blobContainerClient.getAccountName(); - // delete blob - toDelete.add(AzfsResourceId.fromComponents(account, containerName, blobName1)); - // delete directory - toDelete.add(AzfsResourceId.fromComponents(account, containerName, "dir1/")); - azureBlobStoreFileSystem.delete(toDelete); - - // Ensure exception is thrown, clean up - assertFalse(blobContainerClient.getBlobClient(blobName1).exists()); - assertFalse(blobContainerClient.getBlobClient(blobName2).exists()); - assertThrows(FileNotFoundException.class, () -> azureBlobStoreFileSystem.delete(toDelete)); - blobContainerClient.delete(); - } - - @Test - public void testCopy() throws IOException { - BlobServiceClient blobServiceClient = azureBlobStoreFileSystem.getClient(); - String account = blobServiceClient.getAccountName(); - - List src = new ArrayList<>(); - List dest = new ArrayList<>(); - String srcContainer = "source-container" + randomUUID(); - String destContainer = "dest-container" + randomUUID(); - - // Create source file - BlobContainerClient srcContainerClient = blobServiceClient.createBlobContainer(srcContainer); - srcContainerClient.getBlobClient("src-blob").uploadFromFile("src/test/resources/in.txt"); - - // Copy source file to destination - src.add(AzfsResourceId.fromComponents(account, srcContainer, "src-blob")); - dest.add(AzfsResourceId.fromComponents(account, destContainer, "dest-blob")); - azureBlobStoreFileSystem.copy(src, dest); - - // Confirm the destination container was created - BlobContainerClient destContainerClient = - blobServiceClient.getBlobContainerClient(destContainer); - assertTrue(destContainerClient.getBlobClient("dest-blob").exists()); - - // Confirm that the source and destination files are the same - srcContainerClient.getBlobClient("src-blob").downloadToFile("./src/test/resources/blob1"); - destContainerClient.getBlobClient("dest-blob").downloadToFile("./src/test/resources/blob2"); - File file1 = new File("./src/test/resources/blob1"); - File file2 = new File("./src/test/resources/blob2"); - assertTrue("The files differ!", FileUtils.contentEquals(file1, file2)); - - // Clean up - assertTrue(file1.delete()); - assertTrue(file2.delete()); - blobServiceClient.deleteBlobContainer(srcContainer); - blobServiceClient.deleteBlobContainer(destContainer); - } - - @Test - public void testWriteAndRead() throws IOException { - BlobServiceClient client = azureBlobStoreFileSystem.getClient(); - String containerName = "test-container" + randomUUID(); - client.createBlobContainer(containerName); - - byte[] writtenArray = new byte[] {0}; - ByteBuffer bb = ByteBuffer.allocate(writtenArray.length); - bb.put(writtenArray); - - // Create an object and write data to it - AzfsResourceId path = - AzfsResourceId.fromUri( - "azfs://" + client.getAccountName() + "/" + containerName + "/foo/bar.txt"); - WritableByteChannel writableByteChannel = - azureBlobStoreFileSystem.create(path, builder().setMimeType("application/text").build()); - writableByteChannel.write(bb); - writableByteChannel.close(); - - // Read the same object - ByteBuffer bb2 = ByteBuffer.allocate(writtenArray.length); - ReadableByteChannel open = azureBlobStoreFileSystem.open(path); - open.read(bb2); - - // Compare the content with the one that was written - byte[] readArray = bb2.array(); - assertArrayEquals(readArray, writtenArray); - open.close(); - - // Clean up - client.getBlobContainerClient(containerName).delete(); - } - - @Test - public void testGlobExpansion() throws IOException { - String container = "test-container" + randomUUID(); - BlobContainerClient blobContainerClient = - azureBlobStoreFileSystem.getClient().createBlobContainer(container); - - // Create files - List blobNames = new ArrayList<>(); - blobNames.add("testdirectory/file1name"); - blobNames.add("testdirectory/file2name"); - blobNames.add("testdirectory/file3name"); - blobNames.add("testdirectory/otherfile"); - blobNames.add("testotherdirectory/file4name"); - for (String blob : blobNames) { - blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); - } - - // Test patterns - { - AzfsResourceId pattern = - AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file*"); - List expectedFiles = - ImmutableList.of( - "azfs://account/" + container + "/testdirectory/file1name", - "azfs://account/" + container + "/testdirectory/file2name", - "azfs://account/" + container + "/testdirectory/file3name"); - - assertThat( - expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); - } - - { - AzfsResourceId pattern = - AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file[1-3]*"); - List expectedFiles = - ImmutableList.of( - "azfs://account/" + container + "/testdirectory/file1name", - "azfs://account/" + container + "/testdirectory/file2name", - "azfs://account/" + container + "/testdirectory/file3name"); - - assertThat( - expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); - } - - { - AzfsResourceId pattern = - AzfsResourceId.fromUri("azfs://account/" + container + "/testdirectory/file?name"); - List expectedFiles = - ImmutableList.of( - "azfs://account/" + container + "/testdirectory/file1name", - "azfs://account/" + container + "/testdirectory/file2name", - "azfs://account/" + container + "/testdirectory/file3name"); - - assertThat( - expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); - } - - { - AzfsResourceId pattern = - AzfsResourceId.fromUri("azfs://account/" + container + "/test*ectory/fi*name"); - List expectedFiles = - ImmutableList.of( - "azfs://account/" + container + "/testdirectory/file1name", - "azfs://account/" + container + "/testdirectory/file2name", - "azfs://account/" + container + "/testdirectory/file3name", - "azfs://account/" + container + "/testotherdirectory/file4name"); - - assertThat( - expectedFiles, contains(toFilenames(azureBlobStoreFileSystem.expand(pattern)).toArray())); - } - - // Clean up - blobContainerClient.delete(); - } - - private List toFilenames(MatchResult matchResult) throws IOException { - return FluentIterable.from(matchResult.metadata()) - .transform(metadata -> (metadata.resourceId()).toString()) - .toList(); - } - - @Test - public void testMatch() throws Exception { - String container = "test-container" + randomUUID(); - BlobContainerClient blobContainerClient = - azureBlobStoreFileSystem.getClient().createBlobContainer(container); - - // Create files - List blobNames = new ArrayList<>(); - blobNames.add("testdirectory/file1name"); - blobNames.add("testdirectory/file2name"); - blobNames.add("testdirectory/file3name"); - blobNames.add("testdirectory/file4name"); - blobNames.add("testdirectory/otherfile"); - blobNames.add("testotherdirectory/anotherfile"); - for (String blob : blobNames) { - blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); - } - - List specs = - ImmutableList.of( - "azfs://account/" + container + "/testdirectory/file[1-3]*", - "azfs://account/" + container + "/testdirectory/non-exist-file", - "azfs://account/" + container + "/testdirectory/otherfile"); - - List matchResults = azureBlobStoreFileSystem.match(specs); - - // Confirm that match results are as expected - assertEquals(3, matchResults.size()); - assertEquals(MatchResult.Status.OK, matchResults.get(0).status()); - assertThat( - ImmutableList.of( - "azfs://account/" + container + "/testdirectory/file1name", - "azfs://account/" + container + "/testdirectory/file2name", - "azfs://account/" + container + "/testdirectory/file3name"), - contains(toFilenames(matchResults.get(0)).toArray())); - assertEquals(MatchResult.Status.NOT_FOUND, matchResults.get(1).status()); - assertEquals(MatchResult.Status.OK, matchResults.get(2).status()); - assertThat( - ImmutableList.of("azfs://account/" + container + "/testdirectory/otherfile"), - contains(toFilenames(matchResults.get(2)).toArray())); - - blobContainerClient.delete(); - } - - @Test - public void testMatchNonGlobs() throws Exception { - String container = "test-container" + randomUUID(); - BlobContainerClient blobContainerClient = - azureBlobStoreFileSystem.getClient().createBlobContainer(container); - - List blobNames = new ArrayList<>(); - blobNames.add("testdirectory/file1name"); - blobNames.add("testdirectory/dir2name/"); - blobNames.add("testdirectory/file4name"); - // TODO: Also test match results where MatchResult.STATUS != OK (see gcs and s3 tests) - - for (String blob : blobNames) { - blobContainerClient.getBlobClient(blob).uploadFromFile("src/test/resources/in.txt"); - } - - List specs = - ImmutableList.of( - "azfs://account/" + container + "/testdirectory/file1name", - "azfs://account/" + container + "/testdirectory/dir2name/", - "azfs://account/" + container + "/testdirectory/file4name"); - - List matchResults = azureBlobStoreFileSystem.match(specs); - - assertEquals(3, matchResults.size()); - assertThat( - ImmutableList.of("azfs://account/" + container + "/testdirectory/file1name"), - contains(toFilenames(matchResults.get(0)).toArray())); - assertThat( - ImmutableList.of("azfs://account/" + container + "/testdirectory/dir2name/"), - contains(toFilenames(matchResults.get(1)).toArray())); - assertThat( - ImmutableList.of("azfs://account/" + container + "/testdirectory/file4name"), - contains(toFilenames(matchResults.get(2)).toArray())); - - blobContainerClient.delete(); - } -} From 59d5c67a5e72855aa45783e0dd1226f9dddf5204 Mon Sep 17 00:00:00 2001 From: Etta Rapp Date: Fri, 21 Aug 2020 00:35:34 +0000 Subject: [PATCH 24/24] adding azure wordcount test --- .../org/apache/beam/examples/WordCount.java | 2 +- sdks/java/io/azure/build.gradle | 10 ++- .../io/azure/blobstore/AzureWordCountIT.java | 79 +++++++++++++++++++ 3 files changed, 87 insertions(+), 4 deletions(-) create mode 100644 sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureWordCountIT.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java index 90b8dc3f7761..268b793820d7 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java @@ -172,7 +172,7 @@ public interface WordCountOptions extends PipelineOptions { void setOutput(String value); } - static void runWordCount(WordCountOptions options) { + public static void runWordCount(WordCountOptions options) { Pipeline p = Pipeline.create(options); // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the diff --git a/sdks/java/io/azure/build.gradle b/sdks/java/io/azure/build.gradle index 2930e63c379e..7847057ffe0d 100644 --- a/sdks/java/io/azure/build.gradle +++ b/sdks/java/io/azure/build.gradle @@ -31,6 +31,8 @@ ext.summary = "IO library to read and write Azure services from Beam." repositories { jcenter() } +evaluationDependsOn(":examples:java") + dependencies { compile library.java.vendored_guava_26_0_jre compile project(path: ":sdks:java:core", configuration: "shadow") @@ -40,6 +42,8 @@ dependencies { compile "commons-io:commons-io:2.6" compile library.java.slf4j_api testCompile project(path: ":sdks:java:core", configuration: "shadowTest") + testCompile project(path: ":sdks:java:testing:test-utils", configuration: "testRuntime") + testCompile project(path: ":examples:java", configuration: "testRuntime") testCompile library.java.mockito_core testCompile library.java.junit testRuntimeOnly library.java.slf4j_jdk14 @@ -51,21 +55,21 @@ dependencies { task integrationTest(type: Test, dependsOn: processTestResources) { group = "Verification" def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' - def gcpTempRoot = project.findProperty('gcpTempRoot') ?: 'gs://temp-storage-for-end-to-end-tests' + def gcpTempRoot = project.findProperty('System.getProperty("user.dir")') ?: 'gs://temp-storage-for-end-to-end-tests' systemProperty "beamTestPipelineOptions", JsonOutput.toJson([ "--runner=DirectRunner", "--project=${gcpProject}", "--tempRoot=${gcpTempRoot}", - "--inputFile=azfs://ettasaccount/test-container/test-blob" ]) // Disable Gradle cache: these ITs interact with live service that should always be considered "out of date" outputs.upToDateWhen { false } - include '**/WordCountIT.class' + include '**/AzureWordCountIT.class' maxParallelForks 4 classpath = sourceSets.test.runtimeClasspath testClassesDirs = sourceSets.test.output.classesDirs + testClassesDirs += files(project(":examples:java").sourceSets.test.output.classesDirs) useJUnit { excludeCategories "org.apache.beam.sdk.testing.UsesKms" diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureWordCountIT.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureWordCountIT.java new file mode 100644 index 000000000000..192599c7d55b --- /dev/null +++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureWordCountIT.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.beam.sdk.io.azure.blobstore; + +import static org.apache.beam.sdk.testing.FileChecksumMatcher.fileContentsHaveChecksum; +import static org.hamcrest.MatcherAssert.assertThat; + +import java.util.Date; +import org.apache.beam.examples.WordCount; +import org.apache.beam.examples.WordCount.WordCountOptions; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.azure.options.BlobstoreOptions; +import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestPipelineOptions; +import org.apache.beam.sdk.util.NumberedShardedFile; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** End-to-end tests of WordCount. */ +@RunWith(JUnit4.class) +public class AzureWordCountIT { + private static final String DEFAULT_INPUT = "azfs://ettasaccount/test-container/test-blob"; + // this comes from running with the input file sdks/java/io/azure/test/resources/in.txt + private static final String DEFAULT_OUTPUT_CHECKSUM = "9725c097a55d3d3d7cc17936b0839c26dbb4991a"; + + /** + * Options for the WordCount Integration Test. + * + *

Define expected output file checksum to verify WordCount pipeline result with customized + * input. + */ + public interface AzureWordCountITOptions + extends TestPipelineOptions, WordCountOptions, BlobstoreOptions {} + + @BeforeClass + public static void setUp() { + PipelineOptionsFactory.register(TestPipelineOptions.class); + } + + @Test + public void testE2EWordCount() throws Exception { + AzureWordCountITOptions options = + TestPipeline.testingPipelineOptions().as(AzureWordCountITOptions.class); + options.setAzureConnectionString(System.getenv("AZURE_STORAGE_CONNECTION_STRING")); + + options.setInputFile(DEFAULT_INPUT); + options.setOutput( + FileSystems.matchNewResource(options.getTempRoot(), true) + .resolve( + String.format("WordCountIT-%tF-%