[
https://issues.apache.org/jira/browse/BEAM-2750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16279615#comment-16279615
]
ASF GitHub Bot commented on BEAM-2750:
--
jkff closed pull request #3717: [BEAM-2750][BEAM-2751] Implement WholeFileIO
URL: https://github.com/apache/beam/pull/3717
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git
a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WholeFileIO.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WholeFileIO.java
new file mode 100644
index 000..8560e052fe7
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WholeFileIO.java
@@ -0,0 +1,347 @@
+/*
+ * 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;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static
org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions.RESOLVE_DIRECTORY;
+import static
org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions.RESOLVE_FILE;
+import static org.apache.beam.sdk.util.MimeTypes.BINARY;
+
+import com.google.auto.value.AutoValue;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.channels.Channels;
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicLong;
+
+import javax.annotation.Nullable;
+
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.fs.MatchResult;
+import org.apache.beam.sdk.io.fs.MoveOptions;
+import org.apache.beam.sdk.io.fs.ResolveOptions;
+import org.apache.beam.sdk.io.fs.ResourceId;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.util.StreamUtils;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.joda.time.Instant;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link PTransform}s for reading and writing files as {@link KV} pairs of
filename {@link String}s
+ * and byte arrays.
+ *
+ * To read a {@link PCollection} of one or more files as {@link KV}s, use
+ * {@code WholeFileIO.read()} to instantiate a transform and use
+ * {@link WholeFileIO.Read#from(String)} to specify the path of the file(s) to
be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link
+ * PCollection}, apply {@link WholeFileIO#readAll()}.
+ *
+ * Method {@link #read} returns a {@link PCollection} of {@code KV}s,
+ * each corresponding to one file's filename and contents.
+ *
+ * The filepatterns are expanded only once.
+ *
+ * Example 1: reading a file or filepattern (or file glob).
+ *
+ * {@code
+ * Pipeline p = ...;
+ *
+ * // A Read of a local file (only runs locally):
+ * PCollection> oneFile = p.apply(
+ *
WholeFileIO.read().from("/local/path/to/file.txt"));
+ *
+ * // A Read of local files in a directory (only runs locally):
+ * PCollection> manyFiles = p.apply(
+ *
WholeFileIO.read().from("/local/path/to/files/*"));
+ *
+ * // A Read of local files in nested directories (only runs locally):
+ * PCollection> manyFiles = p.apply(
+ *
WholeFileIO.read().from("/local/path/to/nested/dirs/**"));
+ * // ^ The KV's String corresponding to filename retains only the last term
of the file path
+ * // (i.e. it retains the filename and ignores intermediate directory names)
+ * }
+ *
+ * Example 2: reading a PCollection