You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/08/14 04:45:08 UTC

[GitHub] [beam] ettirapp opened a new pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

ettirapp opened a new pull request #12581:
URL: https://github.com/apache/beam/pull/12581


   This PR implements a filesystem for Azure Blobstore.  It is mostly complete, with the exception of the following TODOs that I will resolve in a separate PR:
   
   1) Add more pipeline options to the filesystem, including adding more credential options to authenticate an azure blob service client.  These will be added in AzureOptions.java, BlobstoreOptions.java, and DefaultAzureClientBuilderFactory.java.  I also plan to write a file called AzureModule.java, similar to the AwsModule.java file in aws/options, to parse credentials passed as pipeline options.
   
   2) Create mocks for unit tests, to be used in AzureBlobStoreFileSystemTest.java.  The current tests in AzureBlobStoreFileSystemTest are integration tests, using an actual Azure client, that can be run by setting the AZURE_STORAGE_CONNECTION_STRING environment variable.
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Format the pull request title like `[BEAM-XXX] Fixes bug in ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   Post-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   Lang | SDK | Dataflow | Flink | Samza | Spark | Twister2
   --- | --- | --- | --- | --- | --- | ---
   Go | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/) | ---
   Java | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/badge/i
 con)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/)<br>[![Build Status](htt
 ps://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/)
   Python | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_
 Py_VR_Dataflow/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_P
 ostCommit_Python_VR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/) | ---
   XLang | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/) | ---
   
   Pre-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   --- |Java | Python | Go | Website
   --- | --- | --- | --- | ---
   Non-portable | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/b
 eam_PreCommit_Website_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/)
   Portable | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/) | --- | ---
   
   See [.test-infra/jenkins/README](https://github.com/apache/beam/blob/master/.test-infra/jenkins/README.md) for trigger phrase, status and link of all Jenkins jobs.
   
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   ![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r473344165



##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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) {

Review comment:
       As we discussed, there seems to be an error in this method, but we will resolve it in a follow-up PR.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674471650


   Run PythonDocker PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-673885143


   I'm not sure why commit 4971763 by kennknowles is in the PR - I must have accidentally included it while resolving an issue with my git branches.  The commit has no content, so it shouldn't make a difference.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-675838534


   Run Python2_PVR_Flink PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674230136


   Run PythonDocs PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r473195191



##########
File path: sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.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.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;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+// TODO: Tag each option with @Default or @Nullable
+
+@Experimental(Kind.FILESYSTEM)
+/** Options used to configure Microsoft Azure Blob Storage. */
+public interface BlobstoreOptions extends PipelineOptions {
+
+  @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<? extends BlobstoreClientBuilderFactory> getBlobstoreClientFactoryClass();
+
+  void setBlobstoreClientFactoryClass(
+      Class<? extends BlobstoreClientBuilderFactory> 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<Configuration> {
+
+    @Override
+    public Configuration create(PipelineOptions options) {
+      return new Configuration();
+    }
+  }
+

Review comment:
       Sure, I'll remove it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r471738659



##########
File path: sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java
##########
@@ -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;

Review comment:
       (it seems like it's not possible, so feel free to ignore the comment if that's correct)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r471739116



##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> 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}.
+   *
+   * <p>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<MatchResult> matchNonGlobPaths(List<AzfsResourceId> paths) {
+    ImmutableList.Builder<MatchResult> 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<AzfsResourceId> srcPaths, List<AzfsResourceId> destPaths)
       throws IOException {
-    // TODO
+    checkArgument(
+        srcPaths.size() == destPaths.size(),
+        "sizes of source paths and destination paths do not match");
+
+    Iterator<AzfsResourceId> sourcePathsIterator = srcPaths.iterator();
+    Iterator<AzfsResourceId> 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<AzfsResourceId> srcResourceIds, List<AzfsResourceId> destResourceIds)
       throws IOException {
-    // TODO
+    copy(srcResourceIds, destResourceIds);
+    delete(srcResourceIds);
   }
 
+  // This method with delete a virtual folder or a blob
   @Override
   protected void delete(Collection<AzfsResourceId> 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();
+      }
+

Review comment:
       (meaning the empty 427 line)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r472639381



##########
File path: sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java
##########
@@ -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:

Review comment:
       I decided that it makes most sense to check that a filename is valid in the `create` method of the AzureBlobStoreFileSystem, where a new file is created.  In the `create` method, the `getBlobOutputStream()` method will throw a BlobStorageException if a filename is invalid.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp removed a comment on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp removed a comment on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674227537






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r472673864



##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> 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);

Review comment:
       I looked into this and it seems that it depends on the `EmptyMatchTreatment` used in the `Filesystems::match` call.  See https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/EmptyMatchTreatment.java.  So in the case where there are no matches, I can either return `MatchResult.create(MatchResult.Status.OK, results)` or `MatchResult.create(MatchResult.Status.NOT_FOUND, new FileNotFoundException())`, and these two options will be treated in the same way at https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java#L159-L160.  So I can leave the code as is, unless the second option is more readable.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674227960


   Run JavaPortabilityApiJava11 PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp removed a comment on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp removed a comment on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674227105


   Run PythonDocker PreCommit
   Run Python2_PVR_Flink PreCommit
   Run Portable_Python PreCommit
   Run JavaPortabilityApiJava11 PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-673886130


   R: @boyuanzz 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
pabloem commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-675870865


   Run Python Precommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r471714312



##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> 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}.
+   *
+   * <p>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<MatchResult> matchNonGlobPaths(List<AzfsResourceId> paths) {
+    ImmutableList.Builder<MatchResult> 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.");

Review comment:
       The operation is unsupported because the user is trying to create a file in a container that doesn't exist, and I made a design decision that the create method does not recursively create containers, it only creates files within containers that already exist.  I can change this to a `FileNotFoundException` stating that the container doesn't exist if that is more intuitive.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r471825367



##########
File path: sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java
##########
@@ -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;

Review comment:
       It says in the javadoc for ResourceId.java that a resource id represents a file-like resource, so AzfsResourceId does not support resource id's without a container.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r473626270



##########
File path: 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 {

Review comment:
       I agree. We only need a configuration in a build.gradle file that runs WordCount from an azfs:// file.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674227733


   Run Python2_PVR_Flink PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp removed a comment on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp removed a comment on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674228584


   Run PythonDocs PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r472655207



##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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) {

Review comment:
       Sure, I filed a ticket at https://issues.apache.org/jira/browse/BEAM-10758.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp removed a comment on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp removed a comment on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-675838534


   Run Python2_PVR_Flink PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674227105


   Run PythonDocker PreCommit
   Run Python2_PVR_Flink PreCommit
   Run Portable_Python PreCommit
   Run JavaPortabilityApiJava11 PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674561559


   Run Java_Examples_Dataflow PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-676573357


   Run Python2_PVR_Flink PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r471651523



##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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) {

Review comment:
       I see that this code is duplicated in other filesystems. Would you be willing to move this to something like FileSystemUtils.java to live right next to FileSystem.java and remove the duplicated code from other places? (this can be done as part of a follow-up PR, but can you file a JIRA ticket to track it?)

##########
File path: sdks/java/io/azure/testingFiles/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.
+ */

Review comment:
       you can move this to `src/test/resources`

##########
File path: sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java
##########
@@ -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;

Review comment:
       is it possible to have `blob == null` and also `container == null` so that it's an account?

##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> 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);

Review comment:
       What happens if there are no matches whatsoever? I guess `results` would be empty and that's it?

##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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.");

Review comment:
       For line 133 and 134 Can you add something like "Internal error encountered in AzureBlobStoreFileSystem: Expect no more elements in ..." ?

##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> results = new ArrayList<>();
+
+    blobs.forEach(
+        blob -> {
+          String name = blob.getName();
+          if (wildcardAsRegexp.matcher(name).matches() && !name.endsWith("/")) {
+            LOG.debug("Matched object: {}", name);

Review comment:
       it's not that important, but `name` here contains only the blob name, right? Can you include the container and account in the log?

##########
File path: 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)

Review comment:
       this comment is more related to the options classes, but you can add this annotation to AzureOptions and BlobstoreOptions, so that we will be able to move options around. As discussed in person, when in doubt about where to put an option, err on the side of caution, and put them in BlobstoreOptions.

##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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) {

Review comment:
       Should this be `i-1`? or should it actually be `i+1`? I understand this code comes from elsewhere (thus it's a good idea to standardize and make it a utility) - can you add a unit test with a file pattern containing a backslash at the end to see what happens?

##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> 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}.
+   *
+   * <p>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<MatchResult> matchNonGlobPaths(List<AzfsResourceId> paths) {
+    ImmutableList.Builder<MatchResult> 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<AzfsResourceId> srcPaths, List<AzfsResourceId> destPaths)
       throws IOException {
-    // TODO
+    checkArgument(
+        srcPaths.size() == destPaths.size(),
+        "sizes of source paths and destination paths do not match");
+
+    Iterator<AzfsResourceId> sourcePathsIterator = srcPaths.iterator();
+    Iterator<AzfsResourceId> 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());

Review comment:
       Can you `LOG.info` the fact that a container is being created?

##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> 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}.
+   *
+   * <p>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<MatchResult> matchNonGlobPaths(List<AzfsResourceId> paths) {
+    ImmutableList.Builder<MatchResult> 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<AzfsResourceId> srcPaths, List<AzfsResourceId> destPaths)
       throws IOException {
-    // TODO
+    checkArgument(
+        srcPaths.size() == destPaths.size(),
+        "sizes of source paths and destination paths do not match");
+
+    Iterator<AzfsResourceId> sourcePathsIterator = srcPaths.iterator();
+    Iterator<AzfsResourceId> 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<AzfsResourceId> srcResourceIds, List<AzfsResourceId> destResourceIds)
       throws IOException {
-    // TODO
+    copy(srcResourceIds, destResourceIds);
+    delete(srcResourceIds);
   }
 
+  // This method with delete a virtual folder or a blob
   @Override
   protected void delete(Collection<AzfsResourceId> 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();
+      }
+

Review comment:
       remove this line

##########
File path: 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<TokenCredential> {
+
+    @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<Configuration> {
+
+    @Override
+    public Configuration create(PipelineOptions options) {
+      return new Configuration();
+    }
+  }

Review comment:
       These configuration-related variables seem to be non-serializable. I think we need to implement the special Json serializer before moving forward with adding these changes. We can remove these changes from this PR (and rely purely on a `String`-type connectionString) and add them in a follow-up PR, or get the Json serializer in this PR. What do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r472676925



##########
File path: 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

Review comment:
       I have decided to delete AzureOptions, as per our conversation.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
pabloem commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674191684


   @ettirapp you can add the RAT exception in build.gradle, like this: https://github.com/apache/beam/blob/master/build.gradle#L119


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r472678977



##########
File path: 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"));

Review comment:
       I had been running these tests as integration tests with my Azure account. Since Beam does not have an Azure account as of now, I will convert these tests to unit tests and I will move integration tests to a separate file.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674471666


   Run PythonDocs PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674230488






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r473160375



##########
File path: sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureReadableSeekableByteChannel.java
##########
@@ -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));

Review comment:
       Got it, will do.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp removed a comment on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp removed a comment on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674227960






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-676981830


   Run Python2_PVR_Flink PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp removed a comment on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp removed a comment on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-675848483


   Run Spotless PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674228584


   Run PythonDocs PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r471718807



##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> 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}.
+   *
+   * <p>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<MatchResult> matchNonGlobPaths(List<AzfsResourceId> paths) {
+    ImmutableList.Builder<MatchResult> 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<AzfsResourceId> srcPaths, List<AzfsResourceId> destPaths)
       throws IOException {
-    // TODO
+    checkArgument(
+        srcPaths.size() == destPaths.size(),
+        "sizes of source paths and destination paths do not match");
+
+    Iterator<AzfsResourceId> sourcePathsIterator = srcPaths.iterator();
+    Iterator<AzfsResourceId> 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();

Review comment:
       This is a typo, I meant 86400000. 86,400,000 millis = 1 day.  I can make it a final constant.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-673886084


   R: @pabloem 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp removed a comment on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp removed a comment on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674227330


   Run PythonDocker PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
pabloem commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-677229693


   Thanks @ettirapp @boyuanzz ! : D


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp removed a comment on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp removed a comment on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674561559


   Run Java_Examples_Dataflow PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r473164476



##########
File path: 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<TokenCredential> {
+
+    @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<Configuration> {
+
+    @Override
+    public Configuration create(PipelineOptions options) {
+      return new Configuration();
+    }
+  }

Review comment:
       I was planning to implement a module similar to AwsModule to serialize the non-serializable variables.  I can include it in this PR, I'll work on it today.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674471625


   Run Portable_Python PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674227330


   Run PythonDocker PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r471738659



##########
File path: sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java
##########
@@ -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;

Review comment:
       (it seems like it's not possible)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r473454273



##########
File path: sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
+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;
+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.BlobInputStream;
+import com.azure.storage.blob.specialized.BlobOutputStream;
+import com.azure.storage.blob.specialized.BlockBlobClient;
+import java.io.IOException;
+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.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.junit.Before;
+import org.junit.Ignore;
+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 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<BlobItem> 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() {
+    azureBlobStoreFileSystem = new AzureBlobStoreFileSystem(spyOptions);
+    azureBlobStoreFileSystem.setClient(mockedServiceClient);
+
+    boolean[] containerCreated = {false};
+    when(mockedServiceClient.createBlobContainer(anyString()))
+        .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());
+    when(mockedBlobClient.exists()).thenAnswer((invocation) -> blobCreated[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())
+        .thenAnswer(
+            (i) -> {
+              blobCreated[0] = true;
+              return mockedOutputStream;
+            });
+    when(mockedBlobItem.getName()).thenReturn("name");
+    when(spyOptions.getSasToken()).thenReturn("sas-token");
+    when(mockedBlobClient.openInputStream()).thenReturn(mockedInputStream);
+  }
+
+  @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
+  @SuppressWarnings("CheckReturnValue")
+  public void testCopy() throws IOException {
+    List<AzfsResourceId> src =
+        new ArrayList<>(
+            Arrays.asList(AzfsResourceId.fromComponents("account", "container", "from")));
+    List<AzfsResourceId> 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 testWriteAndRead() throws IOException {
+    azureBlobStoreFileSystem.getClient().createBlobContainer("testcontainer");
+
+    byte[] writtenArray = new byte[] {0};
+    ByteBuffer bb = ByteBuffer.allocate(writtenArray.length);
+    bb.put(writtenArray);
+
+    // 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();
+
+    // Now read the same object
+    ByteBuffer bb2 = ByteBuffer.allocate(writtenArray.length);
+    ReadableByteChannel open = azureBlobStoreFileSystem.open(path);
+    open.read(bb2);
+
+    // And compare the content with the one that was written
+    byte[] readArray = bb2.array();
+    assertArrayEquals(readArray, writtenArray);
+    open.close();
+  }
+
+  @Test
+  @Ignore
+  public void testGlobExpansion() throws IOException {
+    // TODO: Write this test with mocks - see GcsFileSystemTest

Review comment:
       Please file a JIRA issue to track this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] boyuanzz commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
boyuanzz commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r473563038



##########
File path: 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 {

Review comment:
       I don't think this test suite is necessary for IT since test cases here should be covered by the unit tests already. I would image the IT we want is a a pipeline reading from azure file system and writing to azure file system. @pabloem what do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674230226






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem merged pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
pabloem merged pull request #12581:
URL: https://github.com/apache/beam/pull/12581


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r473193248



##########
File path: sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.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.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;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+// TODO: Tag each option with @Default or @Nullable
+
+@Experimental(Kind.FILESYSTEM)
+/** Options used to configure Microsoft Azure Blob Storage. */
+public interface BlobstoreOptions extends PipelineOptions {
+
+  @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<? extends BlobstoreClientBuilderFactory> getBlobstoreClientFactoryClass();
+
+  void setBlobstoreClientFactoryClass(
+      Class<? extends BlobstoreClientBuilderFactory> 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<Configuration> {
+
+    @Override
+    public Configuration create(PipelineOptions options) {
+      return new Configuration();
+    }
+  }
+

Review comment:
       The Environment Configuration property is failing to serialize, and thus it's preventing me from running the pipeline on DirectRunner. We can work around this by writing a custom JSON serializer (like in S3). But maybe we can remove `environmentConfiguration` for now and add it on a later change?

##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> 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);

Review comment:
       perfect. thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r472644790



##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> 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}.
+   *
+   * <p>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<MatchResult> matchNonGlobPaths(List<AzfsResourceId> paths) {
+    ImmutableList.Builder<MatchResult> 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<AzfsResourceId> srcPaths, List<AzfsResourceId> destPaths)
       throws IOException {
-    // TODO
+    checkArgument(
+        srcPaths.size() == destPaths.size(),
+        "sizes of source paths and destination paths do not match");
+
+    Iterator<AzfsResourceId> sourcePathsIterator = srcPaths.iterator();
+    Iterator<AzfsResourceId> 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");

Review comment:
       Sure, done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-673886040


   RAT is failing because the file org.mockito.plugins.MockMaker doesn't have a license header.  I see that beam/sdks/java/extensions/ml also has an org.mockito.plugins.MockMaker file, and it doesn't have the license header either since it is a mockito configuration file.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r472482298



##########
File path: sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java
##########
@@ -68,6 +68,8 @@
   private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
       ImmutableSet.of("gzip");
 
+  private static final int expiryTime = 86400000;

Review comment:
       Constants usually have uppercase names, like so:
   
   ```suggestion
     private static final int DEFAULT_EXPIRY_TIME = 86400000;
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r473458851



##########
File path: sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
+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;
+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.BlobInputStream;
+import com.azure.storage.blob.specialized.BlobOutputStream;
+import com.azure.storage.blob.specialized.BlockBlobClient;
+import java.io.IOException;
+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.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.junit.Before;
+import org.junit.Ignore;
+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 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<BlobItem> 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() {
+    azureBlobStoreFileSystem = new AzureBlobStoreFileSystem(spyOptions);
+    azureBlobStoreFileSystem.setClient(mockedServiceClient);
+
+    boolean[] containerCreated = {false};
+    when(mockedServiceClient.createBlobContainer(anyString()))
+        .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());
+    when(mockedBlobClient.exists()).thenAnswer((invocation) -> blobCreated[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())
+        .thenAnswer(
+            (i) -> {
+              blobCreated[0] = true;
+              return mockedOutputStream;
+            });
+    when(mockedBlobItem.getName()).thenReturn("name");
+    when(spyOptions.getSasToken()).thenReturn("sas-token");
+    when(mockedBlobClient.openInputStream()).thenReturn(mockedInputStream);
+  }
+
+  @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
+  @SuppressWarnings("CheckReturnValue")
+  public void testCopy() throws IOException {
+    List<AzfsResourceId> src =
+        new ArrayList<>(
+            Arrays.asList(AzfsResourceId.fromComponents("account", "container", "from")));
+    List<AzfsResourceId> 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 testWriteAndRead() throws IOException {
+    azureBlobStoreFileSystem.getClient().createBlobContainer("testcontainer");
+
+    byte[] writtenArray = new byte[] {0};
+    ByteBuffer bb = ByteBuffer.allocate(writtenArray.length);
+    bb.put(writtenArray);
+
+    // 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();
+
+    // Now read the same object
+    ByteBuffer bb2 = ByteBuffer.allocate(writtenArray.length);
+    ReadableByteChannel open = azureBlobStoreFileSystem.open(path);
+    open.read(bb2);
+
+    // And compare the content with the one that was written
+    byte[] readArray = bb2.array();
+    assertArrayEquals(readArray, writtenArray);
+    open.close();
+  }
+
+  @Test
+  @Ignore
+  public void testGlobExpansion() throws IOException {
+    // TODO: Write this test with mocks - see GcsFileSystemTest

Review comment:
       Done: https://issues.apache.org/jira/browse/BEAM-10772




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r472678181



##########
File path: 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 {

Review comment:
       I deleted this file for now.  I was going to use it in my unit tests, but I decided to build mocks in AzureBlobStoreFileSystemTest for now, and I will move them to a TestUtils file if the filesystem test code becomes over-complicated as is.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r473564853



##########
File path: 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 {

Review comment:
       Yes, the integration tests are a work in progress.  I put these tests here in case I ended up needing any of them later, but I can move them out of this PR since they aren't being used now.  Thanks for reviewing everything!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674471645


   Run Python2_PVR_Flink PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r473155299



##########
File path: sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureReadableSeekableByteChannel.java
##########
@@ -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));

Review comment:
       I recommend you remove all the LOG.info lines from this class. Because there may be 100s or 1000s of AzureReadableSeekableByteChannel objects in a Beam pipeline, this logging may overwhelm the logging system




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-677014075


   Run Python PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-676554009


   Run Python PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674227537


   Run Portable_Python PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp removed a comment on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp removed a comment on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-673885143


   I'm not sure why commit 4971763 by kennknowles is in the PR - I must have accidentally included it while resolving an issue with my git branches.  The commit has no content, so it shouldn't make a difference.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-676526096


   Run Python2_PVR_Flink PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-675848483


   Run Spotless PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674230887


   Run PythonDocs PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] boyuanzz commented on a change in pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
boyuanzz commented on a change in pull request #12581:
URL: https://github.com/apache/beam/pull/12581#discussion_r471195136



##########
File path: sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java
##########
@@ -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:

Review comment:
       Is this resolved now?

##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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");

Review comment:
       Can you elaborate more on the error msg? Like `the pattern should be wildcard`?

##########
File path: sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java
##########
@@ -93,6 +101,22 @@ public String getScheme() {
     return SCHEME;
   }
 
+  Optional<Long> getSize() {

Review comment:
       I think you can return `Long` directly and the caller should check whether it's `null`.

##########
File path: 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<AzfsResourceId> {

Review comment:
       Please have a class-level javadoc to explain what the class is for.

##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> 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}.
+   *
+   * <p>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<MatchResult> matchNonGlobPaths(List<AzfsResourceId> paths) {
+    ImmutableList.Builder<MatchResult> 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<AzfsResourceId> srcPaths, List<AzfsResourceId> destPaths)
       throws IOException {
-    // TODO
+    checkArgument(
+        srcPaths.size() == destPaths.size(),
+        "sizes of source paths and destination paths do not match");
+
+    Iterator<AzfsResourceId> sourcePathsIterator = srcPaths.iterator();
+    Iterator<AzfsResourceId> 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<AzfsResourceId> srcResourceIds, List<AzfsResourceId> destResourceIds)
       throws IOException {
-    // TODO
+    copy(srcResourceIds, destResourceIds);
+    delete(srcResourceIds);
   }
 
+  // This method with delete a virtual folder or a blob

Review comment:
       Please make the comment as the javadoc.

##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> 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}.
+   *
+   * <p>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<MatchResult> matchNonGlobPaths(List<AzfsResourceId> paths) {
+    ImmutableList.Builder<MatchResult> 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<AzfsResourceId> srcPaths, List<AzfsResourceId> destPaths)
       throws IOException {
-    // TODO
+    checkArgument(
+        srcPaths.size() == destPaths.size(),
+        "sizes of source paths and destination paths do not match");
+
+    Iterator<AzfsResourceId> sourcePathsIterator = srcPaths.iterator();
+    Iterator<AzfsResourceId> 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();

Review comment:
       How do we pick up the number `8640000 ` here? If it should be a constant, we can make it as static final.

##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> 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");

Review comment:
       Please make the error message specific.

##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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);

Review comment:
       `Duration.ofMinutes(1)`

##########
File path: 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"));

Review comment:
       It assumes that the running env has `AZURE_STORAGE_CONNECTION_STRING`, which should be false for most of time. 

##########
File path: 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

Review comment:
       It seems like you have already separated the Azure options and Blob options.

##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> 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}.
+   *
+   * <p>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<MatchResult> matchNonGlobPaths(List<AzfsResourceId> paths) {
+    ImmutableList.Builder<MatchResult> 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<AzfsResourceId> srcPaths, List<AzfsResourceId> destPaths)
       throws IOException {
-    // TODO
+    checkArgument(
+        srcPaths.size() == destPaths.size(),
+        "sizes of source paths and destination paths do not match");
+
+    Iterator<AzfsResourceId> sourcePathsIterator = srcPaths.iterator();
+    Iterator<AzfsResourceId> 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");

Review comment:
       Can we make the constant string `racwdlup ` and `co ` as static final attribute of the class?

##########
File path: 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<AzfsResourceId> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
+
+  private static final ImmutableSet<String> NON_READ_SEEK_EFFICIENT_ENCODINGS =
+      ImmutableSet.of("gzip");
+
+  private Supplier<BlobServiceClient> 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<MatchResult> match(List<String> specs) throws IOException {
-    // TODO
-    return null;
+  protected List<MatchResult> match(List<String> specs) {
+    List<AzfsResourceId> paths =
+        specs.stream().map(AzfsResourceId::fromUri).collect(Collectors.toList());
+    List<AzfsResourceId> globs = new ArrayList<>();
+    List<AzfsResourceId> nonGlobs = new ArrayList<>();
+    List<Boolean> isGlobBooleans = new ArrayList<>();
+
+    for (AzfsResourceId path : paths) {
+      if (path.isWildcard()) {
+        globs.add(path);
+        isGlobBooleans.add(true);
+      } else {
+        nonGlobs.add(path);
+        isGlobBooleans.add(false);
+      }
+    }
+
+    Iterator<MatchResult> globMatches = matchGlobPaths(globs).iterator();
+    Iterator<MatchResult> nonGlobMatches = matchNonGlobPaths(nonGlobs).iterator();
+
+    ImmutableList.Builder<MatchResult> 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<MatchResult> matchGlobPaths(List<AzfsResourceId> 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<BlobItem> blobs = blobContainerClient.listBlobs(listOptions, timeout);
+    List<MatchResult.Metadata> 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}.
+   *
+   * <p>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<MatchResult> matchNonGlobPaths(List<AzfsResourceId> paths) {
+    ImmutableList.Builder<MatchResult> 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.");

Review comment:
       Why is it an `UnsupportedOperationException `? Under what kind of condition the `blobContainerClient.exists()` will be `false`?

##########
File path: 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 {

Review comment:
       Is it used anywhere?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] boyuanzz commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
boyuanzz commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-675040135


   And please fix the test name:
   https://github.com/apache/beam/blob/master/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceIdTest.java#L146-L151


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp commented on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp commented on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674471637


   Run Python PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ettirapp removed a comment on pull request #12581: [BEAM-10378] Add Azure Blob Storage Filesystem

Posted by GitBox <gi...@apache.org>.
ettirapp removed a comment on pull request #12581:
URL: https://github.com/apache/beam/pull/12581#issuecomment-674471625






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org