You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by lc...@apache.org on 2017/04/29 00:16:26 UTC

[1/2] beam git commit: [BEAM-2005] Swap to use Lists within MatchResult instead of arrays.

Repository: beam
Updated Branches:
  refs/heads/master 29b828235 -> b23610409


[BEAM-2005] Swap to use Lists within MatchResult instead of arrays.


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/a83524c2
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/a83524c2
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/a83524c2

Branch: refs/heads/master
Commit: a83524c2c092d62953851d7b2885338b71ddaced
Parents: 29b8282
Author: Luke Cwik <lc...@google.com>
Authored: Fri Apr 28 16:27:53 2017 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Fri Apr 28 17:15:52 2017 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/io/FileBasedSource.java |  6 +-
 .../org/apache/beam/sdk/io/FileSystems.java     | 11 ++--
 .../org/apache/beam/sdk/io/LocalFileSystem.java |  9 ++-
 .../org/apache/beam/sdk/io/fs/MatchResult.java  | 63 +++++++++-----------
 .../beam/sdk/util/NumberedShardedFile.java      |  5 +-
 .../apache/beam/sdk/io/LocalFileSystemTest.java |  3 +-
 .../beam/sdk/io/gcp/storage/GcsFileSystem.java  |  4 +-
 7 files changed, 45 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/a83524c2/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java
index 7f89c1c..2beb5e0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java
@@ -210,14 +210,14 @@ public abstract class FileBasedSource<T> extends OffsetBasedSource<T> {
           "Error matching the pattern or glob %s: status %s",
           pattern,
           result.status());
-      Metadata[] allMatches = result.metadata();
+      List<Metadata> allMatches = result.metadata();
       for (Metadata metadata : allMatches) {
         totalSize += metadata.sizeBytes();
       }
       LOG.info(
           "Filepattern {} matched {} files with total size {}",
           fileOrPatternSpec.get(),
-          allMatches.length,
+          allMatches.size(),
           totalSize);
       return totalSize;
     } else {
@@ -383,7 +383,7 @@ public abstract class FileBasedSource<T> extends OffsetBasedSource<T> {
   private static List<Metadata> expandFilePattern(String fileOrPatternSpec) throws IOException {
     MatchResult matches =
         Iterables.getOnlyElement(FileSystems.match(Collections.singletonList(fileOrPatternSpec)));
-    LOG.info("Matched {} files for pattern {}", matches.metadata().length, fileOrPatternSpec);
+    LOG.info("Matched {} files for pattern {}", matches.metadata().size(), fileOrPatternSpec);
     return ImmutableList.copyOf(matches.metadata());
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/a83524c2/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java
index c41ee48..e4f00ea 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java
@@ -36,7 +36,6 @@ import java.io.IOException;
 import java.nio.channels.ReadableByteChannel;
 import java.nio.channels.WritableByteChannel;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -122,16 +121,16 @@ public class FileSystems {
       throw new IOException(
           String.format("Error matching file spec %s: status %s", spec, matchResult.status()));
     }
-    Metadata[] metadata = matchResult.metadata();
-    if (metadata.length != 1) {
+    List<Metadata> metadata = matchResult.metadata();
+    if (metadata.size() != 1) {
       throw new IOException(
         String.format(
             "Expecting spec %s to match exactly one file, but matched %s: %s",
             spec,
-            metadata.length,
-            Arrays.toString(metadata)));
+            metadata.size(),
+            metadata));
     }
-    return metadata[0];
+    return metadata.get(0);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/beam/blob/a83524c2/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java
index 235b77d..b732bee 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java
@@ -39,6 +39,7 @@ import java.nio.file.PathMatcher;
 import java.nio.file.Paths;
 import java.nio.file.StandardCopyOption;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.regex.Matcher;
 import org.apache.beam.sdk.io.fs.CreateOptions;
@@ -55,8 +56,6 @@ class LocalFileSystem extends FileSystem<LocalResourceId> {
 
   private static final Logger LOG = LoggerFactory.getLogger(LocalFileSystem.class);
 
-  private static final Metadata[] EMPTY_METADATA = new Metadata[0];
-
   LocalFileSystem() {
   }
 
@@ -180,12 +179,12 @@ class LocalFileSystem extends FileSystem<LocalResourceId> {
     File file = Paths.get(spec).toFile();
 
     if (file.exists()) {
-      return MatchResult.create(Status.OK, new Metadata[]{toMetadata(file)});
+      return MatchResult.create(Status.OK, ImmutableList.of(toMetadata(file)));
     }
 
     File parent = file.getAbsoluteFile().getParentFile();
     if (!parent.exists()) {
-      return MatchResult.create(Status.NOT_FOUND, EMPTY_METADATA);
+      return MatchResult.create(Status.NOT_FOUND, Collections.<Metadata>emptyList());
     }
 
     // Method getAbsolutePath() on Windows platform may return something like
@@ -223,7 +222,7 @@ class LocalFileSystem extends FileSystem<LocalResourceId> {
           Status.NOT_FOUND,
           new FileNotFoundException(String.format("No files found for spec: %s.", spec)));
     } else {
-      return MatchResult.create(Status.OK, result.toArray(new Metadata[result.size()]));
+      return MatchResult.create(Status.OK, result);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/a83524c2/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MatchResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MatchResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MatchResult.java
index ab11d66..642c049 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MatchResult.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MatchResult.java
@@ -20,6 +20,7 @@ package org.apache.beam.sdk.io.fs;
 import com.google.auto.value.AutoValue;
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.List;
 
 /**
  * The result of {@link org.apache.beam.sdk.io.FileSystem#match}.
@@ -31,52 +32,44 @@ public abstract class MatchResult {
   /**
    * Returns a {@link MatchResult} given the {@link Status} and {@link Metadata}.
    */
-  public static MatchResult create(final Status status, final Metadata[] metadata) {
-    return new MatchResult() {
-      @Override
-      public Status status() {
-        return status;
-      }
-
-      @Override
-      public Metadata[] metadata() throws IOException {
-        return metadata;
-      }
-    };
+  public static MatchResult create(Status status, List<Metadata> metadata) {
+    return new AutoValue_MatchResult_Success(status, metadata);
+  }
+
+  @AutoValue
+  abstract static class Success extends MatchResult {
+    abstract List<Metadata> getMetadata();
+
+    @Override
+    public List<Metadata> metadata() throws IOException {
+      return getMetadata();
+    }
   }
 
   /**
    * Returns a {@link MatchResult} given the {@link Status} and {@link IOException}.
    */
   public static MatchResult create(final Status status, final IOException e) {
-    return new MatchResult() {
-      @Override
-      public Status status() {
-        return status;
-      }
-
-      @Override
-      public Metadata[] metadata() throws IOException {
-        throw e;
-      }
-    };
+    return new AutoValue_MatchResult_Failure(status, e);
+  }
+
+  @AutoValue
+  abstract static class Failure extends MatchResult {
+    abstract IOException getException();
+
+    @Override
+    public List<Metadata> metadata() throws IOException {
+      throw getException();
+    }
   }
 
   /**
    * Returns a {@link MatchResult} with {@link Status#UNKNOWN}.
    */
   public static MatchResult unknown() {
-    return new MatchResult() {
-      @Override
-      public Status status() {
-        return Status.UNKNOWN;
-      }
-
-      @Override
-      public Metadata[] metadata() throws IOException {
-        throw new IOException("MatchResult status is UNKNOWN, and metadata is not available.");
-      }
-    };
+    return new AutoValue_MatchResult_Failure(
+        Status.UNKNOWN,
+        new IOException("MatchResult status is UNKNOWN, and metadata is not available."));
   }
 
   /**
@@ -87,7 +80,7 @@ public abstract class MatchResult {
   /**
    * {@link Metadata} of matched files.
    */
-  public abstract Metadata[] metadata() throws IOException;
+  public abstract List<Metadata> metadata() throws IOException;
 
   /**
    * {@link Metadata} of a matched file.

http://git-wip-us.apache.org/repos/asf/beam/blob/a83524c2/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java
index e904382..786cdcb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java
@@ -37,7 +37,6 @@ import java.io.Reader;
 import java.nio.channels.Channels;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import java.util.regex.Matcher;
@@ -120,8 +119,8 @@ public class NumberedShardedFile implements ShardedFile {
     do {
       try {
         // Match inputPath which may contains glob
-        Collection<Metadata> files = Arrays.asList(Iterables.getOnlyElement(
-            FileSystems.match(ImmutableList.of(filePattern))).metadata());
+        Collection<Metadata> files = Iterables.getOnlyElement(
+            FileSystems.match(ImmutableList.of(filePattern))).metadata();
 
         LOG.debug("Found {} file(s) by matching the path: {}", files.size(), filePattern);
 

http://git-wip-us.apache.org/repos/asf/beam/blob/a83524c2/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemTest.java
index ac4fe61..048908f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemTest.java
@@ -40,7 +40,6 @@ import java.nio.channels.Channels;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Path;
 import java.nio.file.Paths;
-import java.util.Arrays;
 import java.util.List;
 import org.apache.beam.sdk.io.fs.CreateOptions.StandardCreateOptions;
 import org.apache.beam.sdk.io.fs.MatchResult;
@@ -357,7 +356,7 @@ public class LocalFileSystemTest {
           @Override
           public Iterable<Metadata> apply(MatchResult matchResult) {
             try {
-              return Arrays.asList(matchResult.metadata());
+              return matchResult.metadata();
             } catch (IOException e) {
               throw new RuntimeException(e);
             }

http://git-wip-us.apache.org/repos/asf/beam/blob/a83524c2/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystem.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystem.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystem.java
index ff71f3c..2663864 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystem.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystem.java
@@ -197,7 +197,7 @@ class GcsFileSystem extends FileSystem<GcsResourceId> {
       }
       pageToken = objects.getNextPageToken();
     } while (pageToken != null);
-    return MatchResult.create(Status.OK, results.toArray(new Metadata[results.size()]));
+    return MatchResult.create(Status.OK, results);
   }
 
   /**
@@ -226,7 +226,7 @@ class GcsFileSystem extends FileSystem<GcsResourceId> {
     } else {
       StorageObject object = objectOrException.storageObject();
       assert object != null; // fix a warning; guaranteed by StorageObjectOrIOException semantics.
-      return MatchResult.create(Status.OK, new Metadata[]{toMetadata(object)});
+      return MatchResult.create(Status.OK, ImmutableList.of(toMetadata(object)));
     }
   }
 


[2/2] beam git commit: [BEAM-2005] Swap to use Lists within MatchResult instead of arrays.

Posted by lc...@apache.org.
[BEAM-2005] Swap to use Lists within MatchResult instead of arrays.

This closes #2772


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/b2361040
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/b2361040
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/b2361040

Branch: refs/heads/master
Commit: b236104099a920b9eaad4b09304a94b11e358081
Parents: 29b8282 a83524c
Author: Luke Cwik <lc...@google.com>
Authored: Fri Apr 28 17:16:14 2017 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Fri Apr 28 17:16:14 2017 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/io/FileBasedSource.java |  6 +-
 .../org/apache/beam/sdk/io/FileSystems.java     | 11 ++--
 .../org/apache/beam/sdk/io/LocalFileSystem.java |  9 ++-
 .../org/apache/beam/sdk/io/fs/MatchResult.java  | 63 +++++++++-----------
 .../beam/sdk/util/NumberedShardedFile.java      |  5 +-
 .../apache/beam/sdk/io/LocalFileSystemTest.java |  3 +-
 .../beam/sdk/io/gcp/storage/GcsFileSystem.java  |  4 +-
 7 files changed, 45 insertions(+), 56 deletions(-)
----------------------------------------------------------------------