[jira] [Commented] (BEAM-2750) Read whole files as one PCollection element each

2017-12-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/BEAM-2750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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 

[jira] [Commented] (BEAM-2750) Read whole files as one PCollection element each

2017-08-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/BEAM-2750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16122761#comment-16122761
 ] 

ASF GitHub Bot commented on BEAM-2750:
--

GitHub user cphbrt opened a pull request:

https://github.com/apache/beam/pull/3717

[BEAM-2750][BEAM-2751] Implement WholeFileIO

Follow this checklist to help us incorporate your contribution quickly and 
easily:

 - [x] Make sure there is a [JIRA 
issue](https://issues.apache.org/jira/projects/BEAM/issues/) filed for the 
change (usually before you start working on it).  Trivial changes like typos do 
not require a JIRA issue.  Your pull request should address just this issue, 
without pulling in other changes.
 - [x] Each commit in the pull request should have a meaningful subject 
line and body.
 - [x] Format the pull request title like `[BEAM-XXX] Fixes bug in 
ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA 
issue.
 - [x] Write a pull request description that is detailed enough to 
understand what the pull request does, how, and why.
 - [ ] Run `mvn clean verify` to make sure basic checks pass. A more 
thorough check will be performed on your pull request automatically.
 - [ ] If this contribution is large, please file an Apache [Individual 
Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).

---

## Narrative

WholeFileIO fulfills the requests of [BEAM-2750] and [BEAM-2751] for a way 
to read and write individual files as individual elements of a PCollection to 
and from specific filenames.

## Description

`WholeFileIO.Read` receives a file pattern (glob) of input files. The file 
pattern is expanded into a `PCollection` of `ResourceId`s, each pointing to a 
single file. The bytes at the file location specified by the `ResourceId`s are 
read in and attached to their originating filename in a `KV`.

`WholeFileIO.Write` receives a `PCollection` of `KV`s containing byte 
arrays and their corresponding filenames. The byte arrays are written to the 
output directory with their corresponding filename.

## Example Usage

This example pipeline will read in files according to a given file glob and 
write them to the specified output directory unmodified other than "-copy" 
appended to their filenames. If the input file glob specifies files spread 
through a directory hierarchy, they will still be written out all into the same 
flat output directory.

Example pipeline:
```java
public class WholeFileIOPipeline {

public interface FileIOOptions extends PipelineOptions {
@Description("File glob of the files to read from")
@Validation.Required
String getInputFiles();
void setInputFiles(String value);

@Description("Path of the directory to write files to")
@Validation.Required
String getOutputDir();
void setOutputDir(String value);
}

public static void main(String[] args) {
final FileIOOptions options = 
PipelineOptionsFactory.fromArgs(args).withValidation()
.as(FileIOOptions.class);
Pipeline p = Pipeline.create(options);

PCollection> files = p.apply(
"Read Bytes and filenames of input files",
WholeFileIO.read().from(options.getInputFiles())
);

PCollection> renamedFiles = files.apply(
ParDo.of(
new DoFn, KV>() {
@ProcessElement
public void processElement(ProcessContext c) {
KV file = c.element();
c.output(KV.of(file.getKey() + "-copy", 
file.getValue()));
}
}
)
);

renamedFiles.apply(
"Write Bytes to filenames in Output Directory",
WholeFileIO.write().to(options.getOutputDir())
);

p.run().waitUntilFinish();
}
}
```

Example command to run example pipeline:
```bash
mvn clean compile exec:java 
-Dexec.mainClass=com.example.WholeFileIOPipeline \
  -Dexec.args=" \
--inputFiles=/path/to/input/files/** \
--outputDir=/path/to/output/directory/ \
" \
  -Pdirect-runner
```

## ToDo

- [ ] Add comments
- [ ] Add unit tests
- [ ] Scale test for performance
- [ ] Find out if `FileSystems.resolve()` will resolve multiple 
intermediary directories if a user provides a path that doesn't fully exist 
yet. (`WholeFileIO -> Write -> expand()