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

[05/19] beam git commit: Move PathValidator adjacent to other filesystem bits

Move PathValidator adjacent to other filesystem bits


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

Branch: refs/heads/master
Commit: 1d4b1ed49ce2212d961ad37325a3bfce8964f673
Parents: b61e5bb
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed May 3 19:36:45 2017 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu May 4 16:06:55 2017 -0700

----------------------------------------------------------------------
 .../beam/runners/dataflow/DataflowRunner.java   |  2 +-
 .../beam/sdk/io/fs/NoopPathValidator.java       | 52 ++++++++++++++++++
 .../apache/beam/sdk/io/fs/PathValidator.java    | 58 ++++++++++++++++++++
 .../apache/beam/sdk/util/NoopPathValidator.java |  4 +-
 .../org/apache/beam/sdk/util/PathValidator.java | 55 -------------------
 .../sdk/extensions/gcp/options/GcpOptions.java  |  2 +-
 .../sdk/extensions/gcp/options/GcsOptions.java  |  2 +-
 .../apache/beam/sdk/util/GcsPathValidator.java  |  1 +
 8 files changed, 117 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/1d4b1ed4/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index d5e650e..ce824c6 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -88,6 +88,7 @@ import org.apache.beam.sdk.io.FileSystems;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.io.UnboundedSource;
 import org.apache.beam.sdk.io.WriteFiles;
+import org.apache.beam.sdk.io.fs.PathValidator;
 import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions;
 import org.apache.beam.sdk.io.fs.ResourceId;
 import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
@@ -118,7 +119,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.InstanceBuilder;
 import org.apache.beam.sdk.util.MimeTypes;
 import org.apache.beam.sdk.util.NameUtils;
-import org.apache.beam.sdk.util.PathValidator;
 import org.apache.beam.sdk.util.ReleaseInfo;
 import org.apache.beam.sdk.util.Reshuffle;
 import org.apache.beam.sdk.util.ValueWithRecordId;

http://git-wip-us.apache.org/repos/asf/beam/blob/1d4b1ed4/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/NoopPathValidator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/NoopPathValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/NoopPathValidator.java
new file mode 100644
index 0000000..d5be8f0
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/NoopPathValidator.java
@@ -0,0 +1,52 @@
+/*
+ * 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.fs;
+
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/**
+ * <b>For internal use only; no backwards compatibility guarantees.</b>
+ *
+ * <p>Noop implementation of {@link PathValidator}. All paths are allowed and returned unchanged.
+ */
+@Internal
+public class NoopPathValidator implements PathValidator {
+
+  private NoopPathValidator() {
+  }
+
+  public static PathValidator fromOptions(
+      @SuppressWarnings("unused") PipelineOptions options) {
+    return new NoopPathValidator();
+  }
+
+  @Override
+  public void validateInputFilePatternSupported(String filepattern) {}
+
+  @Override
+  public void validateOutputFilePrefixSupported(String filePrefix) {}
+
+  @Override
+  public void validateOutputResourceSupported(ResourceId resourceId) {}
+
+  @Override
+  public String verifyPath(String path) {
+    return path;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1d4b1ed4/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/PathValidator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/PathValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/PathValidator.java
new file mode 100644
index 0000000..b88a33e
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/PathValidator.java
@@ -0,0 +1,58 @@
+/*
+ * 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.fs;
+
+import org.apache.beam.sdk.annotations.Internal;
+
+/**
+ * <b>For internal use only; no backwards compatibility guarantees.</b>
+ *
+ * <p>Interface for controlling validation of paths.
+ */
+@Internal
+public interface PathValidator {
+  /**
+   * Validate that a file pattern is conforming.
+   *
+   * @param filepattern The file pattern to verify.
+   */
+  void validateInputFilePatternSupported(String filepattern);
+
+  /**
+   * Validate that an output file prefix is conforming.
+   *
+   * @param filePrefix the file prefix to verify.
+   */
+  void validateOutputFilePrefixSupported(String filePrefix);
+
+  /**
+   * Validates that an output path is conforming.
+   *
+   * @param resourceId the file prefix to verify.
+   */
+  void validateOutputResourceSupported(ResourceId resourceId);
+
+  /**
+   * Validate that a path is a valid path and that the path
+   * is accessible.
+   *
+   * @param path The path to verify.
+   * @return The post-validation path.
+   */
+  String verifyPath(String path);
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1d4b1ed4/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopPathValidator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopPathValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopPathValidator.java
index 1f3f5a8..0015e59 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopPathValidator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopPathValidator.java
@@ -17,12 +17,14 @@
  */
 package org.apache.beam.sdk.util;
 
+import org.apache.beam.sdk.io.fs.PathValidator;
 import org.apache.beam.sdk.io.fs.ResourceId;
 import org.apache.beam.sdk.options.PipelineOptions;
 
 /**
- * Noop implementation of {@link PathValidator}. All paths are allowed and returned unchanged.
+ * @deprecated use {@link org.apache.beam.sdk.io.fs.NoopPathValidator}.
  */
+@Deprecated
 public class NoopPathValidator implements PathValidator {
 
   private NoopPathValidator() {

http://git-wip-us.apache.org/repos/asf/beam/blob/1d4b1ed4/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PathValidator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PathValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PathValidator.java
deleted file mode 100644
index e69648b..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PathValidator.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.io.fs.ResourceId;
-
-/**
- * Interface for controlling validation of paths.
- */
-public interface PathValidator {
-  /**
-   * Validate that a file pattern is conforming.
-   *
-   * @param filepattern The file pattern to verify.
-   */
-  void validateInputFilePatternSupported(String filepattern);
-
-  /**
-   * Validate that an output file prefix is conforming.
-   *
-   * @param filePrefix the file prefix to verify.
-   */
-  void validateOutputFilePrefixSupported(String filePrefix);
-
-  /**
-   * Validates that an output path is conforming.
-   *
-   * @param resourceId the file prefix to verify.
-   */
-  void validateOutputResourceSupported(ResourceId resourceId);
-
-  /**
-   * Validate that a path is a valid path and that the path
-   * is accessible.
-   *
-   * @param path The path to verify.
-   * @return The post-validation path.
-   */
-  String verifyPath(String path);
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/1d4b1ed4/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java
index 445f00f..126b795 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java
@@ -48,13 +48,13 @@ import javax.annotation.Nullable;
 import org.apache.beam.sdk.extensions.gcp.auth.CredentialFactory;
 import org.apache.beam.sdk.extensions.gcp.auth.GcpCredentialFactory;
 import org.apache.beam.sdk.extensions.gcp.auth.NullCredentialInitializer;
+import org.apache.beam.sdk.io.fs.PathValidator;
 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.apache.beam.sdk.util.FluentBackoff;
 import org.apache.beam.sdk.util.InstanceBuilder;
-import org.apache.beam.sdk.util.PathValidator;
 import org.apache.beam.sdk.util.RetryHttpRequestInitializer;
 import org.apache.beam.sdk.util.Transport;
 import org.apache.beam.sdk.util.gcsfs.GcsPath;

http://git-wip-us.apache.org/repos/asf/beam/blob/1d4b1ed4/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcsOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcsOptions.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcsOptions.java
index 954092c..78e233e 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcsOptions.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcsOptions.java
@@ -26,6 +26,7 @@ import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.Nullable;
+import org.apache.beam.sdk.io.fs.PathValidator;
 import org.apache.beam.sdk.options.ApplicationNameOptions;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.DefaultValueFactory;
@@ -35,7 +36,6 @@ import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.util.GcsPathValidator;
 import org.apache.beam.sdk.util.GcsUtil;
 import org.apache.beam.sdk.util.InstanceBuilder;
-import org.apache.beam.sdk.util.PathValidator;
 
 /**
  * Options used to configure Google Cloud Storage.

http://git-wip-us.apache.org/repos/asf/beam/blob/1d4b1ed4/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java
index 9ad4152..c4e557b 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java
@@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkArgument;
 
 import java.io.IOException;
 import org.apache.beam.sdk.extensions.gcp.options.GcsOptions;
+import org.apache.beam.sdk.io.fs.PathValidator;
 import org.apache.beam.sdk.io.fs.ResourceId;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.util.gcsfs.GcsPath;