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/06 23:09:59 UTC

[GitHub] [beam] abhiy13 opened a new pull request #12490: [BEAM-10124] Add ContextualTextIO

abhiy13 opened a new pull request #12490:
URL: https://github.com/apache/beam/pull/12490


   [[BEAM-10124]](https://issues.apache.org/jira/browse/BEAM-10124) Added ContextualTextIO.
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [x] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [x] 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.
   
   
   R: @tvalentyn @rezarokni 


----------------------------------------------------------------
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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");
+      PCollection<LineContext> output = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        output = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        output =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // Output Contains LineContext Objects Without Correct Line Numbers
+      // The following operation assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =
+          output
+              .apply(
+                  "Convert LineContext to KV<KV<File,Range>, LineContext>",
+                  ParDo.of(
+                      new DoFn<LineContext, KV<KV<String, Long>, LineContext>>() {
+                        @ProcessElement
+                        public void processElement(
+                            @Element LineContext line,
+                            OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+                          out.output(
+                              KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+                        }
+                      }))
+              .apply("Apply GBK to PColl<KV<KV<File, Range>, LineCtx>>", GroupByKey.create());
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          groupedOutput
+              .apply(
+                  "KV<KV<File, Range>, Iter<LineCtx>> to KV<Range, Sizeof(Iter<LineCtx>)>",
+                  ParDo.of(

Review comment:
       Ack.




----------------------------------------------------------------
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] tvalentyn commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context

Review comment:
       How about: 
   {@link PTransform}s that read text files and collect contextual information of the elements in the input.

##########
File path: sdks/java/io/contextual-text-io/build.gradle
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+plugins { id 'org.apache.beam.module' }
+applyJavaNature(
+        automaticModuleName: 'org.apache.beam.sdk.io.contextual-text-io')
+
+description = "Apache Beam :: SDKs :: Java :: Contextual-Text-IO"
+ext.summary = "Context aware Text IO."

Review comment:
       nit: Context-aware

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):

Review comment:
       Can this read a file from location supported by Beam file systems (like GCS)?
   If so, we can remove `(only runs locally)`?

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")

Review comment:
       Should we say: path/to/files/*.csv?
   
   

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));

Review comment:
       Note that RFC4180 uses term 'field', not columns.  

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();

Review comment:
       Why is this field nullable? Do we interpret `null` value somehow?
   Also note that ContextualTextIOSource takes `boolean`, not `Boolean`.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */

Review comment:
       I am seeing both `withRFC4180MultiLineColumn` and `withHasRFC4180MultiLineColumn`. Is there a reason that have need both? If not, let's keep withRFC4180MultiLineColumn only.
   
   Also this one has a wrong docstring.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");
+      PCollection<LineContext> output = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        output = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        output =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // Output Contains LineContext Objects Without Correct Line Numbers
+      // The following operation assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =
+          output
+              .apply(
+                  "Convert LineContext to KV<KV<File,Range>, LineContext>",
+                  ParDo.of(
+                      new DoFn<LineContext, KV<KV<String, Long>, LineContext>>() {
+                        @ProcessElement
+                        public void processElement(
+                            @Element LineContext line,
+                            OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+                          out.output(
+                              KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+                        }
+                      }))
+              .apply("Apply GBK to PColl<KV<KV<File, Range>, LineCtx>>", GroupByKey.create());
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          groupedOutput
+              .apply(
+                  "KV<KV<File, Range>, Iter<LineCtx>> to KV<Range, Sizeof(Iter<LineCtx>)>",
+                  ParDo.of(
+                      new DoFn<
+                          KV<KV<String, Long>, Iterable<LineContext>>,
+                          KV<KV<String, Long>, Long>>() {
+                        @ProcessElement
+                        public void processElement(
+                            @Element KV<KV<String, Long>, Iterable<LineContext>> elem,
+                            OutputReceiver<KV<KV<String, Long>, Long>> out) {
+                          out.output(KV.of(elem.getKey(), (long) Iterables.size(elem.getValue())));
+                        }
+                      }))
+              .apply("Convert Sizes to PCollView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element so that
+      // prefix sums can be computed in one pass

Review comment:
       The concept of prefix sums` may not be intuitive to a reader. If we start using 'chunk' terminology, perhaps we can use use `chunk offsets`/`chunk absolute offset` `offset of each chunk within the file`.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}

Review comment:
       Break this sentence in two.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.

Review comment:
       Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may contain line breaks.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");

Review comment:
       s/TextIO/ContextualTextIO

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a

Review comment:
       Wording suggestion:
   
    * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
    * performance penalty: when this option is enabled, the input cannot be split and read in parallel.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.

Review comment:
       Reads text from the file(s) ?

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/LineContext.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.ContextualTextIO;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+
+@Internal
+@Experimental(Experimental.Kind.SCHEMAS)
+@DefaultSchema(AutoValueSchema.class)
+@AutoValue
+public abstract class LineContext {
+  public abstract Range getRange();

Review comment:
       Should `range` be `protected`?

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or

Review comment:
       These should not be only options. We can point the user to what file options are supported by Beam file system, and not need to list them all. Giving examples is OK, but currently it sounds that we have only two options. Even if we had only two options,more can be added in the future.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {

Review comment:
        We plan to mark this as experimental, right? If so can we do that and also add next steps on how this IO should be integrated into FileIO + pointer to the doc that discusses this alternative?
   cc: @rezarokni 

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");
+      PCollection<LineContext> output = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        output = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));

Review comment:
       This is a Read transform. Using 'output' as the pcollection that stores intermediate read result is not natural.
   How about `lines` -> `linesGroupedByFileAndRange` -> `linesWithFullContext`?

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */

Review comment:
       This is only relevant if the file is a CSV file, right? Can we explain that? Would this be appropriate:
   
       /** When reading RFC4180 CSV files that have values that span multiple lines, set this to true. Note: this reduces the read performance (see: {@link to the description above}...).*/
   

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");
+      PCollection<LineContext> output = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        output = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        output =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // Output Contains LineContext Objects Without Correct Line Numbers
+      // The following operation assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =
+          output
+              .apply(
+                  "Convert LineContext to KV<KV<File,Range>, LineContext>",
+                  ParDo.of(
+                      new DoFn<LineContext, KV<KV<String, Long>, LineContext>>() {
+                        @ProcessElement
+                        public void processElement(
+                            @Element LineContext line,
+                            OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+                          out.output(
+                              KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+                        }
+                      }))
+              .apply("Apply GBK to PColl<KV<KV<File, Range>, LineCtx>>", GroupByKey.create());
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          groupedOutput
+              .apply(
+                  "KV<KV<File, Range>, Iter<LineCtx>> to KV<Range, Sizeof(Iter<LineCtx>)>",

Review comment:
       `...to KV<KV<File, Range>, ...>`?

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");
+      PCollection<LineContext> output = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        output = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        output =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // Output Contains LineContext Objects Without Correct Line Numbers
+      // The following operation assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =
+          output
+              .apply(
+                  "Convert LineContext to KV<KV<File,Range>, LineContext>",
+                  ParDo.of(
+                      new DoFn<LineContext, KV<KV<String, Long>, LineContext>>() {
+                        @ProcessElement
+                        public void processElement(
+                            @Element LineContext line,
+                            OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+                          out.output(
+                              KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+                        }
+                      }))
+              .apply("Apply GBK to PColl<KV<KV<File, Range>, LineCtx>>", GroupByKey.create());
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          groupedOutput
+              .apply(
+                  "KV<KV<File, Range>, Iter<LineCtx>> to KV<Range, Sizeof(Iter<LineCtx>)>",
+                  ParDo.of(

Review comment:
       Can we use `Count.perKey()` combiner? 

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");
+      PCollection<LineContext> output = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        output = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        output =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // Output Contains LineContext Objects Without Correct Line Numbers

Review comment:
       It would be helpful to explain why LineContext objects do not have correct line numbers. Would the following be appropriate:
   // `At this point the line number in LineContext contains the relative line offset from the beginning of the read range. To compute the absolute position from the beginning of the input, we group the lines within the same ranges, and evaluate the size of each range.` 
   
   nit: treat comments like sentences in documentation, e.g. `Use sentence case followed by a period.`
   

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");
+      PCollection<LineContext> output = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        output = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));

Review comment:
       Also perhaps we can call (File, Range) tuple a `Chunk`, and use terms like `linesGroupedByChunk`, `chunkSizes` instead of `sizes`, etc. 
   




----------------------------------------------------------------
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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.

Review comment:
       Ack.




----------------------------------------------------------------
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] tvalentyn commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIOSource.java
##########
@@ -0,0 +1,357 @@
+/*
+ * 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.ContextualTextIO;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.util.NoSuchElementException;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.io.fs.MatchResult;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+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.Preconditions;
+
+/**
+ * Implementation detail of {@link ContextualTextIO.Read}.
+ *
+ * <p>A {@link FileBasedSource} which can decode records delimited by newline characters.
+ *
+ * <p>This source splits the data into records using {@code UTF-8} {@code \n}, {@code \r}, or {@code
+ * \r\n} as the delimiter. This source is not strict and supports decoding the last record even if
+ * it is not delimited. Finally, no records are decoded if the stream is empty.
+ *
+ * <p>This source supports reading from any arbitrary byte position within the stream. If the
+ * starting position is not {@code 0}, then bytes are skipped until the first delimiter is found
+ * representing the beginning of the first record to be decoded.
+ */
+@VisibleForTesting
+class ContextualTextIOSource extends FileBasedSource<LineContext> {
+  byte[] delimiter;
+
+  // Used to Override isSplittable
+  private boolean hasRFC4180MultiLineColumn;
+
+  @Override
+  protected boolean isSplittable() throws Exception {
+    if (hasRFC4180MultiLineColumn) return false;
+    return super.isSplittable();
+  }
+
+  ContextualTextIOSource(
+      ValueProvider<String> fileSpec,
+      EmptyMatchTreatment emptyMatchTreatment,
+      byte[] delimiter,
+      boolean hasRFC4180MultiLineColumn) {
+    super(fileSpec, emptyMatchTreatment, 1L);
+    this.delimiter = delimiter;
+    this.hasRFC4180MultiLineColumn = hasRFC4180MultiLineColumn;
+  }
+
+  private ContextualTextIOSource(
+      MatchResult.Metadata metadata,
+      long start,
+      long end,
+      byte[] delimiter,
+      boolean hasRFC4180MultiLineColumn) {
+    super(metadata, 1L, start, end);
+    this.delimiter = delimiter;
+    this.hasRFC4180MultiLineColumn = hasRFC4180MultiLineColumn;
+  }
+
+  @Override
+  protected FileBasedSource<LineContext> createForSubrangeOfFile(
+      MatchResult.Metadata metadata, long start, long end) {
+    return new ContextualTextIOSource(metadata, start, end, delimiter, hasRFC4180MultiLineColumn);
+  }
+
+  @Override
+  protected FileBasedReader<LineContext> createSingleFileReader(PipelineOptions options) {
+    return new MultiLineTextBasedReader(this, delimiter, hasRFC4180MultiLineColumn);
+  }
+
+  @Override
+  public Coder<LineContext> getOutputCoder() {
+    SchemaCoder<LineContext> coder = null;
+    try {
+      coder = SchemaRegistry.createDefault().getSchemaCoder(LineContext.class);
+    } catch (NoSuchSchemaException e) {
+      System.out.println("No Coder!");
+    }
+    return coder;
+  }
+
+  /**
+   * A {@link FileBasedReader FileBasedReader} which can decode records delimited by delimiter
+   * characters.
+   *
+   * <p>See {@link ContextualTextIOSource } for further details.
+   */
+  @VisibleForTesting
+  static class MultiLineTextBasedReader extends FileBasedReader<LineContext> {
+    public static final int READ_BUFFER_SIZE = 8192;
+    private static final ByteString UTF8_BOM =
+        ByteString.copyFrom(new byte[] {(byte) 0xEF, (byte) 0xBB, (byte) 0xBF});
+    private final ByteBuffer readBuffer = ByteBuffer.allocate(READ_BUFFER_SIZE);
+    private ByteString buffer;
+    private int startOfDelimiterInBuffer;
+    private int endOfDelimiterInBuffer;
+    private long startOfRecord;
+    private volatile long startOfNextRecord;
+    private volatile boolean eof;
+    private volatile boolean elementIsPresent;
+    private @Nullable LineContext currentValue;
+    private @Nullable ReadableByteChannel inChannel;
+    private @Nullable byte[] delimiter;
+
+    // Add to override the isSplittable
+    private boolean hasRFC4180MultiLineColumn;
+
+    private long startingOffset;
+    private long readerlineNum;
+
+    private MultiLineTextBasedReader(
+        ContextualTextIOSource source, byte[] delimiter, boolean hasRFC4180MultiLineColumn) {
+      super(source);
+      buffer = ByteString.EMPTY;
+      this.delimiter = delimiter;
+      this.hasRFC4180MultiLineColumn = hasRFC4180MultiLineColumn;
+      startingOffset = getCurrentSource().getStartOffset(); // Start offset;
+    }
+
+    @Override
+    protected long getCurrentOffset() throws NoSuchElementException {
+      if (!elementIsPresent) throw new NoSuchElementException();
+      return startOfRecord;
+    }
+
+    @Override
+    public long getSplitPointsRemaining() {
+      if (isStarted() && startOfNextRecord >= getCurrentSource().getEndOffset()) {
+        return isDone() ? 0 : 1;
+      }
+      return super.getSplitPointsRemaining();
+    }
+
+    @Override
+    public LineContext getCurrent() throws NoSuchElementException {
+      if (!elementIsPresent) {
+        throw new NoSuchElementException();
+      }
+      return currentValue;
+    }
+
+    @Override
+    protected void startReading(ReadableByteChannel channel) throws IOException {
+      this.inChannel = channel;
+      // If the first offset is greater than zero, we need to skip bytes until we see our
+      // first delimiter.
+      long startOffset = getCurrentSource().getStartOffset();
+      if (startOffset > 0) {
+        Preconditions.checkState(
+            channel instanceof SeekableByteChannel,
+            "%s only supports reading from a SeekableByteChannel when given a start offset"
+                + " greater than 0.",
+            ContextualTextIOSource.class.getSimpleName());
+        long requiredPosition = startOffset - 1;
+        if (delimiter != null && startOffset >= delimiter.length) {
+          // we need to move back the offset of at worse delimiter.size to be sure to see
+          // all the bytes of the delimiter in the call to findDelimiterBounds() below
+          requiredPosition = startOffset - delimiter.length;
+        }
+        ((SeekableByteChannel) channel).position(requiredPosition);
+        findDelimiterBoundsWithMultiLineCheck();
+        buffer = buffer.substring(endOfDelimiterInBuffer);
+        startOfNextRecord = requiredPosition + endOfDelimiterInBuffer;
+        endOfDelimiterInBuffer = 0;
+        startOfDelimiterInBuffer = 0;
+      }
+    }
+
+    private void findDelimiterBoundsWithMultiLineCheck() throws IOException {
+      findDelimiterBounds();
+    }
+
+    /**
+     * Locates the start position and end position of the next delimiter. Will consume the channel
+     * till either EOF or the delimiter bounds are found.
+     *
+     * <p>If {@link ContextualTextIOSource#hasRFC4180MultiLineColumn} is set then the behaviour will
+     * change from the standard read seen in {@link org.apache.beam.sdk.io.TextIO}. The assumption
+     * when {@link ContextualTextIOSource#hasRFC4180MultiLineColumn} is set is that the file is
+     * being read with a single thread.
+     *
+     * <p>This fills the buffer and updates the positions as follows:
+     *
+     * <pre>{@code
+     * ------------------------------------------------------
+     * | element bytes | delimiter bytes | unconsumed bytes |
+     * ------------------------------------------------------
+     * 0            start of          end of              buffer
+     *              delimiter         delimiter           size
+     *              in buffer         in buffer
+     * }</pre>
+     */
+    private void findDelimiterBounds() throws IOException {
+      int bytePositionInBuffer = 0;
+      boolean doubleQuoteClosed = true;
+
+      while (true) {
+        if (!tryToEnsureNumberOfBytesInBuffer(bytePositionInBuffer + 1)) {
+          startOfDelimiterInBuffer = endOfDelimiterInBuffer = bytePositionInBuffer;
+          break;
+        }
+
+        byte currentByte = buffer.byteAt(bytePositionInBuffer);
+        if (hasRFC4180MultiLineColumn) {
+          // Check if we are inside an open Quote
+          if (currentByte == '"') {
+            doubleQuoteClosed = !doubleQuoteClosed;
+          }
+        } else {
+          doubleQuoteClosed = true;
+        }
+
+        if (delimiter == null) {
+          // default delimiter
+          if (currentByte == '\n') {
+            startOfDelimiterInBuffer = bytePositionInBuffer;
+            endOfDelimiterInBuffer = startOfDelimiterInBuffer + 1;
+            if (doubleQuoteClosed) {
+              break;
+            }
+          } else if (currentByte == '\r') {
+            startOfDelimiterInBuffer = bytePositionInBuffer;
+            endOfDelimiterInBuffer = startOfDelimiterInBuffer + 1;
+            if (tryToEnsureNumberOfBytesInBuffer(bytePositionInBuffer + 2)) {
+              currentByte = buffer.byteAt(bytePositionInBuffer + 1);
+              if (currentByte == '\n') {
+                endOfDelimiterInBuffer += 1;
+              }
+            }
+            if (doubleQuoteClosed) {
+              break;
+            }
+          }
+        } else {
+          // when the user defines a delimiter
+          int i = 0;
+          startOfDelimiterInBuffer = endOfDelimiterInBuffer = bytePositionInBuffer;
+          while ((i < delimiter.length) && (currentByte == delimiter[i])) {
+            // read next byte;
+            i++;
+            if (tryToEnsureNumberOfBytesInBuffer(bytePositionInBuffer + i + 1)) {
+              currentByte = buffer.byteAt(bytePositionInBuffer + i);
+            } else {
+              // corner case: delimiter truncate at the end of file
+              startOfDelimiterInBuffer = endOfDelimiterInBuffer = bytePositionInBuffer;
+              break;
+            }
+          }
+          if (i == delimiter.length) {
+            endOfDelimiterInBuffer = bytePositionInBuffer + i;
+            if (doubleQuoteClosed) break;
+          }
+        }
+        bytePositionInBuffer += 1;
+      }
+    }
+
+    @Override
+    protected boolean readNextRecord() throws IOException {
+      startOfRecord = startOfNextRecord;
+
+      findDelimiterBoundsWithMultiLineCheck();
+
+      // If we have reached EOF file and consumed all of the buffer then we know
+      // that there are no more records.
+      if (eof && buffer.isEmpty()) {
+        elementIsPresent = false;
+        return false;
+      }
+
+      decodeCurrentElement();
+      startOfNextRecord = startOfRecord + endOfDelimiterInBuffer;
+      return true;
+    }
+
+    /**
+     * Decodes the current element updating the buffer to only contain the unconsumed bytes.
+     *
+     * <p>This invalidates the currently stored {@code startOfDelimiterInBuffer} and {@code
+     * endOfDelimiterInBuffer}.
+     */
+    private void decodeCurrentElement() throws IOException {
+      ByteString dataToDecode = buffer.substring(0, startOfDelimiterInBuffer);
+      // If present, the UTF8 Byte Order Mark (BOM) will be removed.
+      if (startOfRecord == 0 && dataToDecode.startsWith(UTF8_BOM)) {
+        dataToDecode = dataToDecode.substring(UTF8_BOM.size());
+      }
+
+      /////////////////////////////////////////////
+
+      //      Data of the Current Line
+      //      dataToDecode.toStringUtf8();
+
+      // The line num is:
+      Long lineUniqueLineNum = readerlineNum++;

Review comment:
       uniqueRecordNum / recordNum ?




----------------------------------------------------------------
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] tvalentyn commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -328,117 +334,127 @@ static boolean isSelfOverlapping(byte[] s) {
                 .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
       }
 
-      // Output Contains LineContext Objects Without Correct Line Numbers
-      // The following operation assigns line numbers to all LineContext Objects
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
 
-      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =
-          output
-              .apply(
-                  "Convert LineContext to KV<KV<File,Range>, LineContext>",
-                  ParDo.of(
-                      new DoFn<LineContext, KV<KV<String, Long>, LineContext>>() {
-                        @ProcessElement
-                        public void processElement(
-                            @Element LineContext line,
-                            OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
-                          out.output(
-                              KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
-                        }
-                      }))
-              .apply("Apply GBK to PColl<KV<KV<File, Range>, LineCtx>>", GroupByKey.create());
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
 
       PCollectionView<Map<KV<String, Long>, Long>> sizes =
-          groupedOutput
-              .apply(
-                  "KV<KV<File, Range>, Iter<LineCtx>> to KV<Range, Sizeof(Iter<LineCtx>)>",
-                  ParDo.of(
-                      new DoFn<
-                          KV<KV<String, Long>, Iterable<LineContext>>,
-                          KV<KV<String, Long>, Long>>() {
-                        @ProcessElement
-                        public void processElement(
-                            @Element KV<KV<String, Long>, Iterable<LineContext>> elem,
-                            OutputReceiver<KV<KV<String, Long>, Long>> out) {
-                          out.output(KV.of(elem.getKey(), (long) Iterables.size(elem.getValue())));
-                        }
-                      }))
-              .apply("Convert Sizes to PCollView", View.asMap());
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
 
       // Get Pipeline to create a dummy PCollection with one element so that
-      // prefix sums can be computed in one pass
-      PCollection<Integer> p =
-          input.getPipeline().apply("Create Dummy Pcoll", Create.of(Arrays.asList(1)));
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Range) pair, calculate the number of lines occurring before the Range for
+      // each File
+
+      // After computing the number of lines before each range, we can find the line number in
+      // original file as numLiesBeforeOffset + lineNumInCurrentOffset

Review comment:
       TODO(offline): discuss naming of these.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -328,117 +334,127 @@ static boolean isSelfOverlapping(byte[] s) {
                 .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
       }
 
-      // Output Contains LineContext Objects Without Correct Line Numbers
-      // The following operation assigns line numbers to all LineContext Objects
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
 
-      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =
-          output
-              .apply(
-                  "Convert LineContext to KV<KV<File,Range>, LineContext>",
-                  ParDo.of(
-                      new DoFn<LineContext, KV<KV<String, Long>, LineContext>>() {
-                        @ProcessElement
-                        public void processElement(
-                            @Element LineContext line,
-                            OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
-                          out.output(
-                              KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
-                        }
-                      }))
-              .apply("Apply GBK to PColl<KV<KV<File, Range>, LineCtx>>", GroupByKey.create());
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
 
       PCollectionView<Map<KV<String, Long>, Long>> sizes =
-          groupedOutput
-              .apply(
-                  "KV<KV<File, Range>, Iter<LineCtx>> to KV<Range, Sizeof(Iter<LineCtx>)>",
-                  ParDo.of(
-                      new DoFn<
-                          KV<KV<String, Long>, Iterable<LineContext>>,
-                          KV<KV<String, Long>, Long>>() {
-                        @ProcessElement
-                        public void processElement(
-                            @Element KV<KV<String, Long>, Iterable<LineContext>> elem,
-                            OutputReceiver<KV<KV<String, Long>, Long>> out) {
-                          out.output(KV.of(elem.getKey(), (long) Iterables.size(elem.getValue())));
-                        }
-                      }))
-              .apply("Convert Sizes to PCollView", View.asMap());
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
 
       // Get Pipeline to create a dummy PCollection with one element so that

Review comment:
       Interrupted comment

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -90,7 +91,7 @@
  * <pre>{@code
  * Pipeline p = ...;
  *
- * // A simple Read of a local file (only runs locally):
+ * // A simple Read of a local file (only runs locally when the filepath is on system):

Review comment:
       How about: 
   
   // A simple Read of a file:
   // PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/path/to/file.txt"));
   
   

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(
+          getFilepattern(), "need to set the filepattern of a ContextualTextIO.Read transform");
+      PCollection<LineContext> lines = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        lines = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        lines =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
+
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =

Review comment:
       how about rangeSizes?

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");
+      PCollection<LineContext> output = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        output = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        output =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // Output Contains LineContext Objects Without Correct Line Numbers
+      // The following operation assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =
+          output
+              .apply(
+                  "Convert LineContext to KV<KV<File,Range>, LineContext>",
+                  ParDo.of(
+                      new DoFn<LineContext, KV<KV<String, Long>, LineContext>>() {
+                        @ProcessElement
+                        public void processElement(
+                            @Element LineContext line,
+                            OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+                          out.output(
+                              KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+                        }
+                      }))
+              .apply("Apply GBK to PColl<KV<KV<File, Range>, LineCtx>>", GroupByKey.create());
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          groupedOutput
+              .apply(
+                  "KV<KV<File, Range>, Iter<LineCtx>> to KV<Range, Sizeof(Iter<LineCtx>)>",
+                  ParDo.of(
+                      new DoFn<
+                          KV<KV<String, Long>, Iterable<LineContext>>,
+                          KV<KV<String, Long>, Long>>() {
+                        @ProcessElement
+                        public void processElement(
+                            @Element KV<KV<String, Long>, Iterable<LineContext>> elem,
+                            OutputReceiver<KV<KV<String, Long>, Long>> out) {
+                          out.output(KV.of(elem.getKey(), (long) Iterables.size(elem.getValue())));
+                        }
+                      }))
+              .apply("Convert Sizes to PCollView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element so that
+      // prefix sums can be computed in one pass
+      PCollection<Integer> p =
+          input.getPipeline().apply("Create Dummy Pcoll", Create.of(Arrays.asList(1)));

Review comment:
       Note, this is somewhat user-facing (visible in the UI). How about 'Create a Singleton' 

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -328,117 +334,127 @@ static boolean isSelfOverlapping(byte[] s) {
                 .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
       }
 
-      // Output Contains LineContext Objects Without Correct Line Numbers
-      // The following operation assigns line numbers to all LineContext Objects
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
 
-      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =
-          output
-              .apply(
-                  "Convert LineContext to KV<KV<File,Range>, LineContext>",
-                  ParDo.of(
-                      new DoFn<LineContext, KV<KV<String, Long>, LineContext>>() {
-                        @ProcessElement
-                        public void processElement(
-                            @Element LineContext line,
-                            OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
-                          out.output(
-                              KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
-                        }
-                      }))
-              .apply("Apply GBK to PColl<KV<KV<File, Range>, LineCtx>>", GroupByKey.create());
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
 
       PCollectionView<Map<KV<String, Long>, Long>> sizes =
-          groupedOutput
-              .apply(
-                  "KV<KV<File, Range>, Iter<LineCtx>> to KV<Range, Sizeof(Iter<LineCtx>)>",
-                  ParDo.of(
-                      new DoFn<
-                          KV<KV<String, Long>, Iterable<LineContext>>,
-                          KV<KV<String, Long>, Long>>() {
-                        @ProcessElement
-                        public void processElement(
-                            @Element KV<KV<String, Long>, Iterable<LineContext>> elem,
-                            OutputReceiver<KV<KV<String, Long>, Long>> out) {
-                          out.output(KV.of(elem.getKey(), (long) Iterables.size(elem.getValue())));
-                        }
-                      }))
-              .apply("Convert Sizes to PCollView", View.asMap());
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
 
       // Get Pipeline to create a dummy PCollection with one element so that
-      // prefix sums can be computed in one pass
-      PCollection<Integer> p =
-          input.getPipeline().apply("Create Dummy Pcoll", Create.of(Arrays.asList(1)));
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Range) pair, calculate the number of lines occurring before the Range for
+      // each File
+
+      // After computing the number of lines before each range, we can find the line number in
+      // original file as numLiesBeforeOffset + lineNumInCurrentOffset

Review comment:
       TODO(offline): discuss the comment.
         // After computing the number of lines before each range, we can find the line number in
   

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -328,117 +334,127 @@ static boolean isSelfOverlapping(byte[] s) {
                 .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
       }
 
-      // Output Contains LineContext Objects Without Correct Line Numbers
-      // The following operation assigns line numbers to all LineContext Objects
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
 
-      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =
-          output
-              .apply(
-                  "Convert LineContext to KV<KV<File,Range>, LineContext>",
-                  ParDo.of(
-                      new DoFn<LineContext, KV<KV<String, Long>, LineContext>>() {
-                        @ProcessElement
-                        public void processElement(
-                            @Element LineContext line,
-                            OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
-                          out.output(
-                              KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
-                        }
-                      }))
-              .apply("Apply GBK to PColl<KV<KV<File, Range>, LineCtx>>", GroupByKey.create());
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
 
       PCollectionView<Map<KV<String, Long>, Long>> sizes =
-          groupedOutput
-              .apply(
-                  "KV<KV<File, Range>, Iter<LineCtx>> to KV<Range, Sizeof(Iter<LineCtx>)>",
-                  ParDo.of(
-                      new DoFn<
-                          KV<KV<String, Long>, Iterable<LineContext>>,
-                          KV<KV<String, Long>, Long>>() {
-                        @ProcessElement
-                        public void processElement(
-                            @Element KV<KV<String, Long>, Iterable<LineContext>> elem,
-                            OutputReceiver<KV<KV<String, Long>, Long>> out) {
-                          out.output(KV.of(elem.getKey(), (long) Iterables.size(elem.getValue())));
-                        }
-                      }))
-              .apply("Convert Sizes to PCollView", View.asMap());
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
 
       // Get Pipeline to create a dummy PCollection with one element so that
-      // prefix sums can be computed in one pass
-      PCollection<Integer> p =
-          input.getPipeline().apply("Create Dummy Pcoll", Create.of(Arrays.asList(1)));
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));

Review comment:
       CreateSingletonPcoll

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -328,117 +334,127 @@ static boolean isSelfOverlapping(byte[] s) {
                 .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
       }
 
-      // Output Contains LineContext Objects Without Correct Line Numbers
-      // The following operation assigns line numbers to all LineContext Objects
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
 
-      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =
-          output
-              .apply(
-                  "Convert LineContext to KV<KV<File,Range>, LineContext>",
-                  ParDo.of(
-                      new DoFn<LineContext, KV<KV<String, Long>, LineContext>>() {
-                        @ProcessElement
-                        public void processElement(
-                            @Element LineContext line,
-                            OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
-                          out.output(
-                              KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
-                        }
-                      }))
-              .apply("Apply GBK to PColl<KV<KV<File, Range>, LineCtx>>", GroupByKey.create());
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
 
       PCollectionView<Map<KV<String, Long>, Long>> sizes =
-          groupedOutput
-              .apply(
-                  "KV<KV<File, Range>, Iter<LineCtx>> to KV<Range, Sizeof(Iter<LineCtx>)>",
-                  ParDo.of(
-                      new DoFn<
-                          KV<KV<String, Long>, Iterable<LineContext>>,
-                          KV<KV<String, Long>, Long>>() {
-                        @ProcessElement
-                        public void processElement(
-                            @Element KV<KV<String, Long>, Iterable<LineContext>> elem,
-                            OutputReceiver<KV<KV<String, Long>, Long>> out) {
-                          out.output(KV.of(elem.getKey(), (long) Iterables.size(elem.getValue())));
-                        }
-                      }))
-              .apply("Convert Sizes to PCollView", View.asMap());
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
 
       // Get Pipeline to create a dummy PCollection with one element so that
-      // prefix sums can be computed in one pass
-      PCollection<Integer> p =
-          input.getPipeline().apply("Create Dummy Pcoll", Create.of(Arrays.asList(1)));
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Range) pair, calculate the number of lines occurring before the Range for
+      // each File
+
+      // After computing the number of lines before each range, we can find the line number in
+      // original file as numLiesBeforeOffset + lineNumInCurrentOffset
       PCollectionView<Map<KV<String, Long>, Long>> sizesOrdered =
-          p.apply(
-                  "Create Map for Line Nums with prefix sums",
-                  ParDo.of(
-                          new DoFn<Integer, KV<KV<String, Long>, Long>>() {
-                            @ProcessElement
-                            public void processElement(ProcessContext p) {
-                              Map<KV<String, Long>, Long> sizeMap = p.sideInput(sizes);
-
-                              // Ensure sorting by Range
-                              SortedMap<KV<String, Long>, Long> sorted =
-                                  new TreeMap<>(
-                                      (a, b) -> {
-                                        // Add custom comparator as KV<K, V> is not comparable by
-                                        // default
-                                        if (a.getKey().compareTo(b.getKey()) == 0) {
-                                          return a.getValue().compareTo(b.getValue());
-                                        }
-                                        return a.getKey().compareTo(b.getKey());
-                                      });
-
-                              // Initialize sorted map
-                              for (Map.Entry<KV<String, Long>, Long> entry : sizeMap.entrySet()) {
-                                sorted.put(entry.getKey(), entry.getValue());
-                              }
-
-                              // tracks lines passed for each file
-                              Map<String, Long> pastLines = new HashMap<>();
-
-                              for (Map.Entry entry : sorted.entrySet()) {
-                                Long lines = (long) entry.getValue();
-                                KV<String, Long> FileRange = (KV<String, Long>) entry.getKey();
-                                String file = FileRange.getKey();
-                                Long linesBefore = 0L;
-                                if (pastLines.containsKey(file)) {
-                                  linesBefore = pastLines.get(file);
-                                }
-                                p.output(KV.of(FileRange, linesBefore));
-                                pastLines.put(file, linesBefore + lines);
-                              }
-                            }
-                          })
-                      .withSideInputs(sizes))
-              .apply("Convert Sorted Sizes Map to PCollView", View.asMap());
-
-      return groupedOutput.apply(
-          "Set Line Nums for all LineContext Objects",
-          ParDo.of(
-                  new DoFn<KV<KV<String, Long>, Iterable<LineContext>>, LineContext>() {
-                    @ProcessElement
-                    public void processElement(ProcessContext p) {
-                      Long Range = p.element().getKey().getValue();
-                      String File = p.element().getKey().getKey();
-                      Iterable<LineContext> lines = p.element().getValue();
-                      Long linesLessThanThisRange =
-                          p.sideInput(sizesOrdered).get(KV.of(File, Range));
-                      lines.forEach(
-                          (LineContext line) -> {
-                            LineContext newLine =
-                                LineContext.newBuilder()
-                                    .setLine(line.getLine())
-                                    .setLineNum(
-                                        line.getRange().getRangeLineNum() + linesLessThanThisRange)
-                                    .setFile(line.getFile())
-                                    .setRange(line.getRange())
-                                    .build();
-                            p.output(newLine);
-                          });
-                    }
-                  })
-              .withSideInputs(sizesOrdered));
+          dummyPcoll
+              .apply(
+                  "computeLinesBeforeRange",
+                  ParDo.of(new computeLinesBeforeEachRange(sizes)).withSideInputs(sizes))
+              .apply("", View.asMap());
+
+      return linesGroupedByFileAndRange.apply(
+          "assignLineNums",
+          ParDo.of(new assignLineNums(sizesOrdered)).withSideInputs(sizesOrdered));
+    }
+
+    protected static class addFileNameAndRange
+        extends DoFn<LineContext, KV<KV<String, Long>, LineContext>> {
+      @ProcessElement
+      public void processElement(
+          @Element LineContext line, OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+        out.output(KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+      }
+    }
+
+    /** Helper class for computing Number of Lines preceding each Pair of (File, Range) */
+    protected static class computeLinesBeforeEachRange
+        extends DoFn<Integer, KV<KV<String, Long>, Long>> {
+      private final PCollectionView<Map<KV<String, Long>, Long>> sizes;
+
+      public computeLinesBeforeEachRange(PCollectionView<Map<KV<String, Long>, Long>> sizes) {
+        this.sizes = sizes;
+      }
+
+      // Add custom comparator as KV<K, V> is not comparable by default
+      private static class FileRangeComparator<K extends Comparable<K>, V extends Comparable<V>>
+          implements Comparator<KV<K, V>> {
+        @Override
+        public int compare(KV<K, V> a, KV<K, V> b) {
+          if (a.getKey().compareTo(b.getKey()) == 0) {
+            return a.getValue().compareTo(b.getValue());
+          }
+          return a.getKey().compareTo(b.getKey());
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext p) {
+        // Get the Map Containing the size from side-input
+        Map<KV<String, Long>, Long> sizeMap = p.sideInput(sizes);
+
+        // The FileRange Pair must be sorted
+        SortedMap<KV<String, Long>, Long> sorted = new TreeMap<>(new FileRangeComparator<>());
+
+        // Initialize sorted map with values
+        for (Map.Entry<KV<String, Long>, Long> entry : sizeMap.entrySet()) {
+          sorted.put(entry.getKey(), entry.getValue());
+        }
+
+        // HashMap that tracks lines passed for each file
+        Map<String, Long> pastLines = new HashMap<>();
+
+        // For each (File, Range) Pair, compute the number of lines before it
+        for (Map.Entry entry : sorted.entrySet()) {
+          Long lines = (long) entry.getValue();
+          KV<String, Long> FileRange = (KV<String, Long>) entry.getKey();
+          String file = FileRange.getKey();
+          Long linesBefore = 0L;
+          if (pastLines.containsKey(file)) {
+            linesBefore = pastLines.get(file);
+          }
+          p.output(KV.of(FileRange, linesBefore));
+          pastLines.put(file, linesBefore + lines);
+        }
+      }
+    }
+
+    protected static class assignLineNums

Review comment:
       The convention in Java is to capitalize class names.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(
+          getFilepattern(), "need to set the filepattern of a ContextualTextIO.Read transform");
+      PCollection<LineContext> lines = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        lines = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        lines =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
+
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects

Review comment:
       Seems out of place or unnecessary.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(
+          getFilepattern(), "need to set the filepattern of a ContextualTextIO.Read transform");
+      PCollection<LineContext> lines = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        lines = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        lines =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
+
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));

Review comment:
       nit: AddFileNameAndRange for consistency with other DisplayData entires.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/LineContext.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.ContextualTextIO;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+
+@Internal
+@Experimental(Experimental.Kind.SCHEMAS)
+@DefaultSchema(AutoValueSchema.class)
+@AutoValue
+public abstract class LineContext {

Review comment:
       What are your thoughts on calling this `Line` or `Record` instead of `LineContext` cc: @rezarokni 

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(
+          getFilepattern(), "need to set the filepattern of a ContextualTextIO.Read transform");
+      PCollection<LineContext> lines = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        lines = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        lines =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
+
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element so that
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Range) pair, calculate the number of lines occurring before the Range for
+      // each File
+
+      // After computing the number of lines before each range, we can find the line number in
+      // original file as numLiesBeforeOffset + lineNumInCurrentOffset
+      PCollectionView<Map<KV<String, Long>, Long>> sizesOrdered =
+          dummyPcoll
+              .apply(
+                  "computeLinesBeforeRange",
+                  ParDo.of(new computeLinesBeforeEachRange(sizes)).withSideInputs(sizes))
+              .apply("", View.asMap());
+
+      return linesGroupedByFileAndRange.apply(
+          "assignLineNums",
+          ParDo.of(new assignLineNums(sizesOrdered)).withSideInputs(sizesOrdered));
+    }
+
+    protected static class addFileNameAndRange
+        extends DoFn<LineContext, KV<KV<String, Long>, LineContext>> {
+      @ProcessElement
+      public void processElement(
+          @Element LineContext line, OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+        out.output(KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+      }
+    }
+
+    /** Helper class for computing Number of Lines preceding each Pair of (File, Range) */
+    protected static class computeLinesBeforeEachRange
+        extends DoFn<Integer, KV<KV<String, Long>, Long>> {
+      private final PCollectionView<Map<KV<String, Long>, Long>> sizes;
+
+      public computeLinesBeforeEachRange(PCollectionView<Map<KV<String, Long>, Long>> sizes) {
+        this.sizes = sizes;
+      }
+
+      // Add custom comparator as KV<K, V> is not comparable by default
+      private static class FileRangeComparator<K extends Comparable<K>, V extends Comparable<V>>
+          implements Comparator<KV<K, V>> {
+        @Override
+        public int compare(KV<K, V> a, KV<K, V> b) {
+          if (a.getKey().compareTo(b.getKey()) == 0) {
+            return a.getValue().compareTo(b.getValue());
+          }
+          return a.getKey().compareTo(b.getKey());
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext p) {
+        // Get the Map Containing the size from side-input

Review comment:
               // Get the map containing the sizes of all read ranges from the side-input.
   

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(
+          getFilepattern(), "need to set the filepattern of a ContextualTextIO.Read transform");
+      PCollection<LineContext> lines = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        lines = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        lines =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
+
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element so that
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Range) pair, calculate the number of lines occurring before the Range for
+      // each File
+
+      // After computing the number of lines before each range, we can find the line number in
+      // original file as numLiesBeforeOffset + lineNumInCurrentOffset
+      PCollectionView<Map<KV<String, Long>, Long>> sizesOrdered =
+          dummyPcoll
+              .apply(
+                  "computeLinesBeforeRange",
+                  ParDo.of(new computeLinesBeforeEachRange(sizes)).withSideInputs(sizes))
+              .apply("", View.asMap());
+
+      return linesGroupedByFileAndRange.apply(
+          "assignLineNums",
+          ParDo.of(new assignLineNums(sizesOrdered)).withSideInputs(sizesOrdered));
+    }
+
+    protected static class addFileNameAndRange
+        extends DoFn<LineContext, KV<KV<String, Long>, LineContext>> {
+      @ProcessElement
+      public void processElement(
+          @Element LineContext line, OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+        out.output(KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+      }
+    }
+
+    /** Helper class for computing Number of Lines preceding each Pair of (File, Range) */
+    protected static class computeLinesBeforeEachRange
+        extends DoFn<Integer, KV<KV<String, Long>, Long>> {
+      private final PCollectionView<Map<KV<String, Long>, Long>> sizes;
+
+      public computeLinesBeforeEachRange(PCollectionView<Map<KV<String, Long>, Long>> sizes) {
+        this.sizes = sizes;
+      }
+
+      // Add custom comparator as KV<K, V> is not comparable by default
+      private static class FileRangeComparator<K extends Comparable<K>, V extends Comparable<V>>
+          implements Comparator<KV<K, V>> {
+        @Override
+        public int compare(KV<K, V> a, KV<K, V> b) {
+          if (a.getKey().compareTo(b.getKey()) == 0) {
+            return a.getValue().compareTo(b.getValue());
+          }
+          return a.getKey().compareTo(b.getKey());
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext p) {
+        // Get the Map Containing the size from side-input
+        Map<KV<String, Long>, Long> sizeMap = p.sideInput(sizes);
+
+        // The FileRange Pair must be sorted
+        SortedMap<KV<String, Long>, Long> sorted = new TreeMap<>(new FileRangeComparator<>());
+
+        // Initialize sorted map with values

Review comment:
       This is unnecessary.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(
+          getFilepattern(), "need to set the filepattern of a ContextualTextIO.Read transform");
+      PCollection<LineContext> lines = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        lines = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        lines =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
+
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element so that
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Range) pair, calculate the number of lines occurring before the Range for
+      // each File
+
+      // After computing the number of lines before each range, we can find the line number in
+      // original file as numLiesBeforeOffset + lineNumInCurrentOffset
+      PCollectionView<Map<KV<String, Long>, Long>> sizesOrdered =
+          dummyPcoll
+              .apply(
+                  "computeLinesBeforeRange",
+                  ParDo.of(new computeLinesBeforeEachRange(sizes)).withSideInputs(sizes))
+              .apply("", View.asMap());
+
+      return linesGroupedByFileAndRange.apply(
+          "assignLineNums",
+          ParDo.of(new assignLineNums(sizesOrdered)).withSideInputs(sizesOrdered));
+    }
+
+    protected static class addFileNameAndRange
+        extends DoFn<LineContext, KV<KV<String, Long>, LineContext>> {
+      @ProcessElement
+      public void processElement(
+          @Element LineContext line, OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+        out.output(KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+      }
+    }
+
+    /** Helper class for computing Number of Lines preceding each Pair of (File, Range) */
+    protected static class computeLinesBeforeEachRange
+        extends DoFn<Integer, KV<KV<String, Long>, Long>> {
+      private final PCollectionView<Map<KV<String, Long>, Long>> sizes;
+
+      public computeLinesBeforeEachRange(PCollectionView<Map<KV<String, Long>, Long>> sizes) {
+        this.sizes = sizes;
+      }
+
+      // Add custom comparator as KV<K, V> is not comparable by default
+      private static class FileRangeComparator<K extends Comparable<K>, V extends Comparable<V>>
+          implements Comparator<KV<K, V>> {
+        @Override
+        public int compare(KV<K, V> a, KV<K, V> b) {
+          if (a.getKey().compareTo(b.getKey()) == 0) {
+            return a.getValue().compareTo(b.getValue());
+          }
+          return a.getKey().compareTo(b.getKey());
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext p) {
+        // Get the Map Containing the size from side-input
+        Map<KV<String, Long>, Long> sizeMap = p.sideInput(sizes);

Review comment:
       How about `rangeSizes` instead of `sizeMap`.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(
+          getFilepattern(), "need to set the filepattern of a ContextualTextIO.Read transform");
+      PCollection<LineContext> lines = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        lines = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        lines =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
+
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element so that
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Range) pair, calculate the number of lines occurring before the Range for
+      // each File
+
+      // After computing the number of lines before each range, we can find the line number in
+      // original file as numLiesBeforeOffset + lineNumInCurrentOffset
+      PCollectionView<Map<KV<String, Long>, Long>> sizesOrdered =
+          dummyPcoll
+              .apply(
+                  "computeLinesBeforeRange",
+                  ParDo.of(new computeLinesBeforeEachRange(sizes)).withSideInputs(sizes))
+              .apply("", View.asMap());
+
+      return linesGroupedByFileAndRange.apply(
+          "assignLineNums",
+          ParDo.of(new assignLineNums(sizesOrdered)).withSideInputs(sizesOrdered));
+    }
+
+    protected static class addFileNameAndRange
+        extends DoFn<LineContext, KV<KV<String, Long>, LineContext>> {
+      @ProcessElement
+      public void processElement(
+          @Element LineContext line, OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+        out.output(KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+      }
+    }
+
+    /** Helper class for computing Number of Lines preceding each Pair of (File, Range) */
+    protected static class computeLinesBeforeEachRange
+        extends DoFn<Integer, KV<KV<String, Long>, Long>> {
+      private final PCollectionView<Map<KV<String, Long>, Long>> sizes;
+
+      public computeLinesBeforeEachRange(PCollectionView<Map<KV<String, Long>, Long>> sizes) {
+        this.sizes = sizes;
+      }
+
+      // Add custom comparator as KV<K, V> is not comparable by default
+      private static class FileRangeComparator<K extends Comparable<K>, V extends Comparable<V>>
+          implements Comparator<KV<K, V>> {
+        @Override
+        public int compare(KV<K, V> a, KV<K, V> b) {
+          if (a.getKey().compareTo(b.getKey()) == 0) {
+            return a.getValue().compareTo(b.getValue());
+          }
+          return a.getKey().compareTo(b.getKey());
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext p) {
+        // Get the Map Containing the size from side-input
+        Map<KV<String, Long>, Long> sizeMap = p.sideInput(sizes);
+
+        // The FileRange Pair must be sorted
+        SortedMap<KV<String, Long>, Long> sorted = new TreeMap<>(new FileRangeComparator<>());
+
+        // Initialize sorted map with values
+        for (Map.Entry<KV<String, Long>, Long> entry : sizeMap.entrySet()) {
+          sorted.put(entry.getKey(), entry.getValue());
+        }
+
+        // HashMap that tracks lines passed for each file
+        Map<String, Long> pastLines = new HashMap<>();
+
+        // For each (File, Range) Pair, compute the number of lines before it
+        for (Map.Entry entry : sorted.entrySet()) {
+          Long lines = (long) entry.getValue();
+          KV<String, Long> FileRange = (KV<String, Long>) entry.getKey();
+          String file = FileRange.getKey();
+          Long linesBefore = 0L;
+          if (pastLines.containsKey(file)) {
+            linesBefore = pastLines.get(file);
+          }
+          p.output(KV.of(FileRange, linesBefore));
+          pastLines.put(file, linesBefore + lines);
+        }
+      }
+    }
+
+    protected static class assignLineNums
+        extends DoFn<KV<KV<String, Long>, LineContext>, LineContext> {
+      PCollectionView<Map<KV<String, Long>, Long>> sizesOrdered;

Review comment:
       TODO(offline): What does this map represent? Does it have to be ordered?
   
   How about:
   fileRangeStartingOffset
   fileRangeStartingLine

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(
+          getFilepattern(), "need to set the filepattern of a ContextualTextIO.Read transform");
+      PCollection<LineContext> lines = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        lines = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        lines =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
+
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element so that
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Range) pair, calculate the number of lines occurring before the Range for
+      // each File
+
+      // After computing the number of lines before each range, we can find the line number in
+      // original file as numLiesBeforeOffset + lineNumInCurrentOffset
+      PCollectionView<Map<KV<String, Long>, Long>> sizesOrdered =
+          dummyPcoll
+              .apply(
+                  "computeLinesBeforeRange",
+                  ParDo.of(new computeLinesBeforeEachRange(sizes)).withSideInputs(sizes))
+              .apply("", View.asMap());
+
+      return linesGroupedByFileAndRange.apply(
+          "assignLineNums",
+          ParDo.of(new assignLineNums(sizesOrdered)).withSideInputs(sizesOrdered));
+    }
+
+    protected static class addFileNameAndRange
+        extends DoFn<LineContext, KV<KV<String, Long>, LineContext>> {
+      @ProcessElement
+      public void processElement(
+          @Element LineContext line, OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+        out.output(KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+      }
+    }
+
+    /** Helper class for computing Number of Lines preceding each Pair of (File, Range) */
+    protected static class computeLinesBeforeEachRange
+        extends DoFn<Integer, KV<KV<String, Long>, Long>> {
+      private final PCollectionView<Map<KV<String, Long>, Long>> sizes;
+
+      public computeLinesBeforeEachRange(PCollectionView<Map<KV<String, Long>, Long>> sizes) {
+        this.sizes = sizes;
+      }
+
+      // Add custom comparator as KV<K, V> is not comparable by default
+      private static class FileRangeComparator<K extends Comparable<K>, V extends Comparable<V>>
+          implements Comparator<KV<K, V>> {
+        @Override
+        public int compare(KV<K, V> a, KV<K, V> b) {
+          if (a.getKey().compareTo(b.getKey()) == 0) {
+            return a.getValue().compareTo(b.getValue());
+          }
+          return a.getKey().compareTo(b.getKey());
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext p) {
+        // Get the Map Containing the size from side-input
+        Map<KV<String, Long>, Long> sizeMap = p.sideInput(sizes);
+
+        // The FileRange Pair must be sorted
+        SortedMap<KV<String, Long>, Long> sorted = new TreeMap<>(new FileRangeComparator<>());
+
+        // Initialize sorted map with values
+        for (Map.Entry<KV<String, Long>, Long> entry : sizeMap.entrySet()) {
+          sorted.put(entry.getKey(), entry.getValue());
+        }
+
+        // HashMap that tracks lines passed for each file
+        Map<String, Long> pastLines = new HashMap<>();

Review comment:
       Do we actually need this map given that we iterate over sorted file ranges? 

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/LineContext.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.ContextualTextIO;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+
+@Internal
+@Experimental(Experimental.Kind.SCHEMAS)
+@DefaultSchema(AutoValueSchema.class)
+@AutoValue
+public abstract class LineContext {
+  public abstract Range getRange();
+
+  public abstract Long getLineNum();

Review comment:
       How do we define line number when RFC4180 multiline setting is enabled?  Should we use a different term? `position` perhaps? 
   cc: @rezarokni 

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(
+          getFilepattern(), "need to set the filepattern of a ContextualTextIO.Read transform");
+      PCollection<LineContext> lines = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        lines = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        lines =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
+
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element so that
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Range) pair, calculate the number of lines occurring before the Range for
+      // each File
+
+      // After computing the number of lines before each range, we can find the line number in
+      // original file as numLiesBeforeOffset + lineNumInCurrentOffset
+      PCollectionView<Map<KV<String, Long>, Long>> sizesOrdered =
+          dummyPcoll
+              .apply(
+                  "computeLinesBeforeRange",
+                  ParDo.of(new computeLinesBeforeEachRange(sizes)).withSideInputs(sizes))
+              .apply("", View.asMap());
+
+      return linesGroupedByFileAndRange.apply(
+          "assignLineNums",
+          ParDo.of(new assignLineNums(sizesOrdered)).withSideInputs(sizesOrdered));
+    }
+
+    protected static class addFileNameAndRange
+        extends DoFn<LineContext, KV<KV<String, Long>, LineContext>> {
+      @ProcessElement
+      public void processElement(
+          @Element LineContext line, OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+        out.output(KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+      }
+    }
+
+    /** Helper class for computing Number of Lines preceding each Pair of (File, Range) */
+    protected static class computeLinesBeforeEachRange
+        extends DoFn<Integer, KV<KV<String, Long>, Long>> {
+      private final PCollectionView<Map<KV<String, Long>, Long>> sizes;
+
+      public computeLinesBeforeEachRange(PCollectionView<Map<KV<String, Long>, Long>> sizes) {
+        this.sizes = sizes;
+      }
+
+      // Add custom comparator as KV<K, V> is not comparable by default
+      private static class FileRangeComparator<K extends Comparable<K>, V extends Comparable<V>>

Review comment:
       @rezarokni @abhiy13 would it make sense to define `class FileRange extends KV<String, Long> implements Comparable<FileRange>` ?

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(
+          getFilepattern(), "need to set the filepattern of a ContextualTextIO.Read transform");
+      PCollection<LineContext> lines = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        lines = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        lines =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
+
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element so that
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Range) pair, calculate the number of lines occurring before the Range for
+      // each File
+
+      // After computing the number of lines before each range, we can find the line number in
+      // original file as numLiesBeforeOffset + lineNumInCurrentOffset
+      PCollectionView<Map<KV<String, Long>, Long>> sizesOrdered =
+          dummyPcoll
+              .apply(
+                  "computeLinesBeforeRange",
+                  ParDo.of(new computeLinesBeforeEachRange(sizes)).withSideInputs(sizes))
+              .apply("", View.asMap());
+
+      return linesGroupedByFileAndRange.apply(
+          "assignLineNums",
+          ParDo.of(new assignLineNums(sizesOrdered)).withSideInputs(sizesOrdered));
+    }
+
+    protected static class addFileNameAndRange
+        extends DoFn<LineContext, KV<KV<String, Long>, LineContext>> {
+      @ProcessElement
+      public void processElement(
+          @Element LineContext line, OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+        out.output(KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+      }
+    }
+
+    /** Helper class for computing Number of Lines preceding each Pair of (File, Range) */

Review comment:
       Wording suggestion: "Helper class for computing number of lines in the File preceding the beginning of the Range in this file."

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -328,117 +334,127 @@ static boolean isSelfOverlapping(byte[] s) {
                 .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
       }
 
-      // Output Contains LineContext Objects Without Correct Line Numbers
-      // The following operation assigns line numbers to all LineContext Objects
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
 
-      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =
-          output
-              .apply(
-                  "Convert LineContext to KV<KV<File,Range>, LineContext>",
-                  ParDo.of(
-                      new DoFn<LineContext, KV<KV<String, Long>, LineContext>>() {
-                        @ProcessElement
-                        public void processElement(
-                            @Element LineContext line,
-                            OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
-                          out.output(
-                              KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
-                        }
-                      }))
-              .apply("Apply GBK to PColl<KV<KV<File, Range>, LineCtx>>", GroupByKey.create());
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
 
       PCollectionView<Map<KV<String, Long>, Long>> sizes =
-          groupedOutput
-              .apply(
-                  "KV<KV<File, Range>, Iter<LineCtx>> to KV<Range, Sizeof(Iter<LineCtx>)>",
-                  ParDo.of(
-                      new DoFn<
-                          KV<KV<String, Long>, Iterable<LineContext>>,
-                          KV<KV<String, Long>, Long>>() {
-                        @ProcessElement
-                        public void processElement(
-                            @Element KV<KV<String, Long>, Iterable<LineContext>> elem,
-                            OutputReceiver<KV<KV<String, Long>, Long>> out) {
-                          out.output(KV.of(elem.getKey(), (long) Iterables.size(elem.getValue())));
-                        }
-                      }))
-              .apply("Convert Sizes to PCollView", View.asMap());
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
 
       // Get Pipeline to create a dummy PCollection with one element so that
-      // prefix sums can be computed in one pass
-      PCollection<Integer> p =
-          input.getPipeline().apply("Create Dummy Pcoll", Create.of(Arrays.asList(1)));
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Range) pair, calculate the number of lines occurring before the Range for
+      // each File
+
+      // After computing the number of lines before each range, we can find the line number in
+      // original file as numLiesBeforeOffset + lineNumInCurrentOffset

Review comment:
       Lines before the beginning of the range.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(
+          getFilepattern(), "need to set the filepattern of a ContextualTextIO.Read transform");
+      PCollection<LineContext> lines = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        lines = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        lines =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
+
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element so that
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Range) pair, calculate the number of lines occurring before the Range for
+      // each File
+
+      // After computing the number of lines before each range, we can find the line number in
+      // original file as numLiesBeforeOffset + lineNumInCurrentOffset
+      PCollectionView<Map<KV<String, Long>, Long>> sizesOrdered =
+          dummyPcoll
+              .apply(
+                  "computeLinesBeforeRange",
+                  ParDo.of(new computeLinesBeforeEachRange(sizes)).withSideInputs(sizes))
+              .apply("", View.asMap());
+
+      return linesGroupedByFileAndRange.apply(
+          "assignLineNums",
+          ParDo.of(new assignLineNums(sizesOrdered)).withSideInputs(sizesOrdered));
+    }
+
+    protected static class addFileNameAndRange
+        extends DoFn<LineContext, KV<KV<String, Long>, LineContext>> {
+      @ProcessElement
+      public void processElement(
+          @Element LineContext line, OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+        out.output(KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+      }
+    }
+
+    /** Helper class for computing Number of Lines preceding each Pair of (File, Range) */
+    protected static class computeLinesBeforeEachRange
+        extends DoFn<Integer, KV<KV<String, Long>, Long>> {

Review comment:
       @rezarokni what is the style to document input, sideinputs and outputs of custom DoFn classes?
   
   @abhiy13 we could mention that main input is not used so that readers don't try to guess what it is before reading the code.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(
+          getFilepattern(), "need to set the filepattern of a ContextualTextIO.Read transform");
+      PCollection<LineContext> lines = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        lines = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        lines =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
+
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element so that
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Range) pair, calculate the number of lines occurring before the Range for
+      // each File
+
+      // After computing the number of lines before each range, we can find the line number in
+      // original file as numLiesBeforeOffset + lineNumInCurrentOffset
+      PCollectionView<Map<KV<String, Long>, Long>> sizesOrdered =
+          dummyPcoll
+              .apply(
+                  "computeLinesBeforeRange",
+                  ParDo.of(new computeLinesBeforeEachRange(sizes)).withSideInputs(sizes))
+              .apply("", View.asMap());
+
+      return linesGroupedByFileAndRange.apply(
+          "assignLineNums",
+          ParDo.of(new assignLineNums(sizesOrdered)).withSideInputs(sizesOrdered));
+    }
+
+    protected static class addFileNameAndRange
+        extends DoFn<LineContext, KV<KV<String, Long>, LineContext>> {
+      @ProcessElement
+      public void processElement(
+          @Element LineContext line, OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+        out.output(KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+      }
+    }
+
+    /** Helper class for computing Number of Lines preceding each Pair of (File, Range) */
+    protected static class computeLinesBeforeEachRange
+        extends DoFn<Integer, KV<KV<String, Long>, Long>> {
+      private final PCollectionView<Map<KV<String, Long>, Long>> sizes;
+
+      public computeLinesBeforeEachRange(PCollectionView<Map<KV<String, Long>, Long>> sizes) {
+        this.sizes = sizes;
+      }
+
+      // Add custom comparator as KV<K, V> is not comparable by default
+      private static class FileRangeComparator<K extends Comparable<K>, V extends Comparable<V>>
+          implements Comparator<KV<K, V>> {
+        @Override
+        public int compare(KV<K, V> a, KV<K, V> b) {
+          if (a.getKey().compareTo(b.getKey()) == 0) {
+            return a.getValue().compareTo(b.getValue());
+          }
+          return a.getKey().compareTo(b.getKey());
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext p) {
+        // Get the Map Containing the size from side-input
+        Map<KV<String, Long>, Long> sizeMap = p.sideInput(sizes);
+
+        // The FileRange Pair must be sorted
+        SortedMap<KV<String, Long>, Long> sorted = new TreeMap<>(new FileRangeComparator<>());
+
+        // Initialize sorted map with values
+        for (Map.Entry<KV<String, Long>, Long> entry : sizeMap.entrySet()) {
+          sorted.put(entry.getKey(), entry.getValue());
+        }
+
+        // HashMap that tracks lines passed for each file
+        Map<String, Long> pastLines = new HashMap<>();
+
+        // For each (File, Range) Pair, compute the number of lines before it
+        for (Map.Entry entry : sorted.entrySet()) {
+          Long lines = (long) entry.getValue();
+          KV<String, Long> FileRange = (KV<String, Long>) entry.getKey();
+          String file = FileRange.getKey();
+          Long linesBefore = 0L;
+          if (pastLines.containsKey(file)) {
+            linesBefore = pastLines.get(file);
+          }
+          p.output(KV.of(FileRange, linesBefore));
+          pastLines.put(file, linesBefore + lines);
+        }
+      }
+    }
+
+    protected static class assignLineNums
+        extends DoFn<KV<KV<String, Long>, LineContext>, LineContext> {

Review comment:
       Same comment as above: should we document the inputs & outputs for this DoFn?

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(
+          getFilepattern(), "need to set the filepattern of a ContextualTextIO.Read transform");
+      PCollection<LineContext> lines = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        lines = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        lines =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // At this point the line number in LineContext contains the relative line offset from the
+      // beginning of the read range.
+
+      // To compute the absolute position from the beginning of the input,
+      // we group the lines within the same ranges, and evaluate the size of each range.
+
+      // The following operations will assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, LineContext>> linesGroupedByFileAndRange =
+          lines.apply("addFileNameAndRange", ParDo.of(new addFileNameAndRange()));
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          linesGroupedByFileAndRange
+              .apply("countLinesForEachFileRange", Count.perKey())
+              .apply("sizesAsView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element so that
+      PCollection<Integer> dummyPcoll =
+          input.getPipeline().apply("CreateDummyPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Range) pair, calculate the number of lines occurring before the Range for
+      // each File
+
+      // After computing the number of lines before each range, we can find the line number in
+      // original file as numLiesBeforeOffset + lineNumInCurrentOffset
+      PCollectionView<Map<KV<String, Long>, Long>> sizesOrdered =
+          dummyPcoll
+              .apply(
+                  "computeLinesBeforeRange",
+                  ParDo.of(new computeLinesBeforeEachRange(sizes)).withSideInputs(sizes))
+              .apply("", View.asMap());
+
+      return linesGroupedByFileAndRange.apply(
+          "assignLineNums",
+          ParDo.of(new assignLineNums(sizesOrdered)).withSideInputs(sizesOrdered));
+    }
+
+    protected static class addFileNameAndRange
+        extends DoFn<LineContext, KV<KV<String, Long>, LineContext>> {
+      @ProcessElement
+      public void processElement(
+          @Element LineContext line, OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+        out.output(KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+      }
+    }
+
+    /** Helper class for computing Number of Lines preceding each Pair of (File, Range) */
+    protected static class computeLinesBeforeEachRange
+        extends DoFn<Integer, KV<KV<String, Long>, Long>> {
+      private final PCollectionView<Map<KV<String, Long>, Long>> sizes;
+
+      public computeLinesBeforeEachRange(PCollectionView<Map<KV<String, Long>, Long>> sizes) {
+        this.sizes = sizes;
+      }
+
+      // Add custom comparator as KV<K, V> is not comparable by default
+      private static class FileRangeComparator<K extends Comparable<K>, V extends Comparable<V>>
+          implements Comparator<KV<K, V>> {
+        @Override
+        public int compare(KV<K, V> a, KV<K, V> b) {
+          if (a.getKey().compareTo(b.getKey()) == 0) {
+            return a.getValue().compareTo(b.getValue());
+          }
+          return a.getKey().compareTo(b.getKey());
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext p) {
+        // Get the Map Containing the size from side-input
+        Map<KV<String, Long>, Long> sizeMap = p.sideInput(sizes);
+
+        // The FileRange Pair must be sorted

Review comment:
       It is easy to see from the code that we are sorting entries from the map passed from the side input, so this comment does not add new information. You could say _why_ we need to do the sorting or remove the comment.




----------------------------------------------------------------
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] rezarokni commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/LineContext.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.ContextualTextIO;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+
+@Internal
+@Experimental(Experimental.Kind.SCHEMAS)
+@DefaultSchema(AutoValueSchema.class)
+@AutoValue
+public abstract class LineContext {

Review comment:
       Good point, yes so really this is RecordWithMetadata, which is verbose but very descriptive. 




----------------------------------------------------------------
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] tvalentyn commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {

Review comment:
       @abhiy13 @rezarokni  what do you think about replacing `withHasRFC4180MultiLineColumn` with
   
   `withMultilineCSV`
   
   and add something like:
   ```withMultilineCSV allows readling CSV files with multiline fields, however it disables read parallelism. This  setting requires input values with line breaks be encompassed in double quotes, and double quotes in values be escaped with a preceding double quote as defined in RFC4180.```

##########
File path: sdks/java/io/contextual-text-io/src/test/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIOTest.java
##########
@@ -0,0 +1,1271 @@
+/*
+ * 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.ContextualTextIO;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertFalse;
+import static junit.framework.Assert.assertNotNull;
+import static junit.framework.Assert.assertTrue;
+import static org.apache.beam.sdk.TestUtils.LINES_ARRAY;
+import static org.apache.beam.sdk.TestUtils.NO_LINES_ARRAY;
+import static org.apache.beam.sdk.io.Compression.AUTO;
+import static org.apache.beam.sdk.io.Compression.BZIP2;
+import static org.apache.beam.sdk.io.Compression.DEFLATE;
+import static org.apache.beam.sdk.io.Compression.GZIP;
+import static org.apache.beam.sdk.io.Compression.UNCOMPRESSED;
+import static org.apache.beam.sdk.io.Compression.ZIP;
+import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+import static org.apache.beam.sdk.values.TypeDescriptors.strings;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.startsWith;
+import static org.junit.Assume.assumeFalse;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.io.Writer;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.zip.GZIPOutputStream;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.GenerateSequence;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ExperimentalOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.testing.NeedsRunner;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.SourceTestUtils;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo;
+import org.apache.beam.sdk.testing.ValidatesRunner;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.ToString;
+import org.apache.beam.sdk.transforms.Watch;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;
+import org.apache.beam.sdk.transforms.display.DisplayDataMatchers;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
+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.Iterables;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream;
+import org.apache.commons.compress.compressors.deflate.DeflateCompressorOutputStream;
+import org.joda.time.Duration;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.junit.runners.Parameterized;
+
+/** Tests for {@link ContextualTextIO.Read} */
+public class ContextualTextIOTest {
+  private static final int NUM_LINES_FOR_LARGE = 1024;
+
+  private static final List<String> EMPTY = Collections.emptyList();
+
+  private static final List<String> TINY = Arrays.asList("ABC", "DEF", "HIJ");
+
+  private static final List<String> LARGE = makeLines(NUM_LINES_FOR_LARGE);
+
+  private static File writeToFile(
+      List<String> lines, TemporaryFolder folder, String fileName, Compression compression)
+      throws IOException {
+    File file = folder.getRoot().toPath().resolve(fileName).toFile();
+    OutputStream output = new FileOutputStream(file);
+    switch (compression) {
+      case UNCOMPRESSED:
+        break;
+      case GZIP:
+        output = new GZIPOutputStream(output);
+        break;
+      case BZIP2:
+        output = new BZip2CompressorOutputStream(output);
+        break;
+      case ZIP:
+        ZipOutputStream zipOutput = new ZipOutputStream(output);
+        zipOutput.putNextEntry(new ZipEntry("entry"));
+        output = zipOutput;
+        break;
+      case DEFLATE:
+        output = new DeflateCompressorOutputStream(output);
+        break;
+      default:
+        throw new UnsupportedOperationException(compression.toString());
+    }
+    writeToStreamAndClose(lines, output);
+    return file;
+  }
+
+  /**
+   * Helper that writes the given lines (adding a newline in between) to a stream, then closes the
+   * stream.
+   */
+  private static void writeToStreamAndClose(List<String> lines, OutputStream outputStream) {
+    try (PrintStream writer = new PrintStream(outputStream)) {
+      for (String line : lines) {
+        writer.println(line);
+      }
+    }
+  }
+
+  /** Helper to make an array of compressible strings. Returns ["line" i] for i in range(0,n). */
+  private static List<String> makeLines(int n) {
+    List<String> lines = new ArrayList<>();
+    for (int i = 0; i < n; ++i) {
+      lines.add("Line " + i);
+    }
+    return lines;
+  }
+
+  private static class convertLineContextToString extends DoFn<LineContext, String> {
+    @ProcessElement
+    public void processElement(@Element LineContext L, OutputReceiver<String> out) {
+      String file = L.getFile().substring(L.getFile().lastIndexOf('/') + 1);
+      out.output(file + " " + L.getLineNum() + " " + L.getLine());
+    }
+  }
+
+  /**
+   * Helper method that runs a variety of ways to read a single file using ContextualTextIO and
+   * checks that they all match the given expected output.
+   *
+   * <p>The transforms being verified are:
+   *
+   * <ul>
+   *   <li>ContextualTextIO.read().from(filename).withCompression(compressionType).withHintMatchesManyFiles()
+   *   <li>ContextualTextIO.read().from(filename).withCompression(compressionType)
+   *   <li>ContextualTextIO.read().from(filename).withCompression(compressionType).with
+   *   <li>ContextualTextIO.readFiles().withCompression(compressionType)
+   * </ul>
+   */
+  private static void assertReadingCompressedFileMatchesExpected(
+      File file, Compression compression, List<String> expected, Pipeline p) {
+
+    ContextualTextIO.Read read =
+        ContextualTextIO.read().from(file.getPath()).withCompression(compression);
+
+    // Convert the expected output into LineContext output Format
+    List<String> expectedOutput = new ArrayList<>();
+    for (int lineNum = 0; lineNum < expected.size(); ++lineNum) {
+      expectedOutput.add(file.getName() + " " + lineNum + " " + expected.get(lineNum));
+    }
+
+    PAssert.that(
+            p.apply("Read_" + file + "_" + compression.toString(), read)
+                .apply("ConvertLineContextToString", ParDo.of(new convertLineContextToString())))
+        .containsInAnyOrder(expectedOutput);
+    PAssert.that(
+            p.apply(
+                    "Read_" + file + "_" + compression.toString() + "_many",
+                    read.withHintMatchesManyFiles())
+                .apply(
+                    "ConvertLineContextToString" + "_many",
+                    ParDo.of(new convertLineContextToString())))
+        .containsInAnyOrder(expectedOutput);
+
+    PAssert.that(
+            p.apply(
+                    "Read_" + file + "_" + compression.toString() + "_withRFC4180",
+                    read.withHasRFC4180MultiLineColumn(true))
+                .apply(
+                    "ConvertLineContextToString" + "_withRFC4180",
+                    ParDo.of(new convertLineContextToString())))
+        .containsInAnyOrder(expectedOutput);
+
+    PAssert.that(
+            p.apply("Create_Paths_ReadFiles_" + file, Create.of(file.getPath()))
+                .apply("Match_" + file, FileIO.matchAll())
+                .apply("ReadMatches_" + file, FileIO.readMatches().withCompression(compression))
+                .apply("ReadFiles_" + compression.toString(), ContextualTextIO.readFiles())
+                .apply(
+                    "ConvertLineContextToStringWithFileIO",
+                    ParDo.of(new convertLineContextToString())))
+        .containsInAnyOrder(expectedOutput);
+  }
+
+  /**
+   * Create a zip file with the given lines.
+   *
+   * @param expected A list of expected lines, populated in the zip file.
+   * @param folder A temporary folder used to create files.
+   * @param filename Optionally zip file name (can be null).
+   * @param fieldsEntries Fields to write in zip entries.
+   * @return The zip filename.
+   * @throws Exception In case of a failure during zip file creation.
+   */
+  private static File createZipFile(
+      List<String> expected, TemporaryFolder folder, String filename, String[]... fieldsEntries)
+      throws Exception {
+    File tmpFile = folder.getRoot().toPath().resolve(filename).toFile();
+
+    ZipOutputStream out = new ZipOutputStream(new FileOutputStream(tmpFile));
+    PrintStream writer = new PrintStream(out, true /* auto-flush on write */);
+
+    int index = 0;
+    for (String[] entry : fieldsEntries) {
+      out.putNextEntry(new ZipEntry(Integer.toString(index)));
+      for (String field : entry) {
+        writer.println(field);
+        expected.add(field);
+      }
+      out.closeEntry();
+      index++;
+    }
+
+    writer.close();
+    out.close();
+
+    return tmpFile;
+  }
+
+  private static ContextualTextIOSource prepareSource(
+      TemporaryFolder temporaryFolder, byte[] data, byte[] delimiter, boolean hasRFC4180Multiline)
+      throws IOException {
+    Path path = temporaryFolder.newFile().toPath();
+    Files.write(path, data);
+    return new ContextualTextIOSource(
+        ValueProvider.StaticValueProvider.of(path.toString()),
+        EmptyMatchTreatment.DISALLOW,
+        delimiter,
+        hasRFC4180Multiline);
+  }
+
+  private static String getFileSuffix(Compression compression) {
+    switch (compression) {
+      case UNCOMPRESSED:
+        return ".txt";
+      case GZIP:
+        return ".gz";
+      case BZIP2:
+        return ".bz2";
+      case ZIP:
+        return ".zip";
+      case DEFLATE:
+        return ".deflate";
+      default:
+        return "";
+    }
+  }
+  /** Tests for reading from different size of files with various Compression. */
+  @RunWith(Parameterized.class)
+  public static class CompressedReadTest {
+    @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+    @Rule public TestPipeline p = TestPipeline.create();
+
+    @Parameterized.Parameters(name = "{index}: {1}")
+    public static Iterable<Object[]> data() {
+      return ImmutableList.<Object[]>builder()
+          .add(new Object[] {EMPTY, UNCOMPRESSED})
+          .add(new Object[] {EMPTY, GZIP})
+          .add(new Object[] {EMPTY, BZIP2})
+          .add(new Object[] {EMPTY, ZIP})
+          .add(new Object[] {EMPTY, DEFLATE})
+          .add(new Object[] {TINY, UNCOMPRESSED})
+          .add(new Object[] {TINY, GZIP})
+          .add(new Object[] {TINY, BZIP2})
+          .add(new Object[] {TINY, ZIP})
+          .add(new Object[] {TINY, DEFLATE})
+          .add(new Object[] {LARGE, UNCOMPRESSED})
+          .add(new Object[] {LARGE, GZIP})
+          .add(new Object[] {LARGE, BZIP2})
+          .add(new Object[] {LARGE, ZIP})
+          .add(new Object[] {LARGE, DEFLATE})
+          .build();
+    }
+
+    @Parameterized.Parameter(0)
+    public List<String> lines;
+
+    @Parameterized.Parameter(1)
+    public Compression compression;
+
+    /** Tests reading from a small, compressed file with no extension. */
+    @Test
+    @Category(NeedsRunner.class)
+    public void testCompressedReadWithoutExtension() throws Exception {
+      String fileName = lines.size() + "_" + compression + "_no_extension";
+      File fileWithNoExtension = writeToFile(lines, tempFolder, fileName, compression);
+      assertReadingCompressedFileMatchesExpected(fileWithNoExtension, compression, lines, p);
+      p.run();
+    }
+
+    @Test
+    @Category(NeedsRunner.class)
+    public void testCompressedReadWithExtension() throws Exception {
+      String fileName =
+          lines.size() + "_" + compression + "_no_extension" + getFileSuffix(compression);
+      File fileWithExtension = writeToFile(lines, tempFolder, fileName, compression);
+
+      // Sanity check that we're properly testing compression.
+      if (lines.size() == NUM_LINES_FOR_LARGE && !compression.equals(UNCOMPRESSED)) {
+        File uncompressedFile = writeToFile(lines, tempFolder, "large.txt", UNCOMPRESSED);
+        assertThat(uncompressedFile.length(), greaterThan(fileWithExtension.length()));
+      }
+
+      assertReadingCompressedFileMatchesExpected(fileWithExtension, compression, lines, p);
+      p.run();
+    }
+
+    @Test
+    @Category(NeedsRunner.class)
+    public void testReadWithAuto() throws Exception {
+      // Files with non-compressed extensions should work in AUTO and UNCOMPRESSED modes.
+      String fileName =
+          lines.size() + "_" + compression + "_no_extension" + getFileSuffix(compression);
+      File fileWithExtension = writeToFile(lines, tempFolder, fileName, compression);
+      assertReadingCompressedFileMatchesExpected(fileWithExtension, AUTO, lines, p);
+      p.run();
+    }
+  }
+
+  /** Tests for reading files with various delimiters. */
+  @RunWith(Parameterized.class)
+  public static class ReadWithDelimiterTest {
+    private static final ImmutableList<String> EXPECTED = ImmutableList.of("asdf", "hjkl", "xyz");
+    @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+
+    @Parameterized.Parameters(name = "{index}: {0}")
+    public static Iterable<Object[]> data() {
+      return ImmutableList.<Object[]>builder()
+          //          .add(new Object[] {"\n\n\n", ImmutableList.of("", "", "")})
+          .add(new Object[] {"asdf\nhjkl\nxyz\n", EXPECTED})
+          .add(new Object[] {"asdf\rhjkl\rxyz\r", EXPECTED})
+          .add(new Object[] {"asdf\r\nhjkl\r\nxyz\r\n", EXPECTED})
+          .add(new Object[] {"asdf\rhjkl\r\nxyz\n", EXPECTED})
+          .add(new Object[] {"asdf\nhjkl\nxyz", EXPECTED})
+          .add(new Object[] {"asdf\rhjkl\rxyz", EXPECTED})
+          .add(new Object[] {"asdf\r\nhjkl\r\nxyz", EXPECTED})
+          .add(new Object[] {"asdf\rhjkl\r\nxyz", EXPECTED})
+          .build();
+    }
+
+    @Parameterized.Parameter(0)
+    public String line;
+
+    @Parameterized.Parameter(1)
+    public ImmutableList<String> expected;
+
+    @Test
+    public void testReadLinesWithDelimiter() throws Exception {
+      runTestReadWithData(line.getBytes(UTF_8), expected);
+    }
+
+    private ContextualTextIOSource prepareSource(byte[] data, boolean hasRFC4180Multiline)
+        throws IOException {
+      return ContextualTextIOTest.prepareSource(tempFolder, data, null, hasRFC4180Multiline);
+    }
+
+    private void runTestReadWithData(byte[] data, List<String> expectedResults) throws Exception {
+      ContextualTextIOSource source = prepareSource(data, false);
+      List<LineContext> actual =
+          SourceTestUtils.readFromSource(source, PipelineOptionsFactory.create());
+      List<String> actualOutput = new ArrayList<>();
+      actual.forEach(
+          (LineContext L) -> {
+            String file = L.getFile().substring(L.getFile().lastIndexOf('/') + 1);
+            actualOutput.add(L.getLine());
+          });
+      assertThat(
+          actualOutput,
+          containsInAnyOrder(new ArrayList<>(expectedResults).toArray(new String[0])));
+    }
+  }
+
+  @RunWith(Parameterized.class)
+  public static class ReadWithDelimiterAndRFC4180 {
+    static final ImmutableList<String> Expected = ImmutableList.of("\"asdf\nhjkl\nmnop\"", "xyz");
+    @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+
+    @Parameterized.Parameters(name = "{index}: {0}")
+    public static Iterable<Object[]> data() {
+      return ImmutableList.<Object[]>builder()
+          .add(new Object[] {"\n\n\n", ImmutableList.of("", "", "")})
+          .add(new Object[] {"\"asdf\nhjkl\"\nxyz\n", ImmutableList.of("\"asdf\nhjkl\"", "xyz")})
+          .add(new Object[] {"\"asdf\nhjkl\nmnop\"\nxyz\n", Expected})
+          .add(new Object[] {"\"asdf\nhjkl\nmnop\"\nxyz\r", Expected})
+          .add(new Object[] {"\"asdf\nhjkl\nmnop\"\r\nxyz\n", Expected})
+          .add(new Object[] {"\"asdf\nhjkl\nmnop\"\r\nxyz\r\n", Expected})
+          .add(new Object[] {"\"asdf\nhjkl\nmnop\"\rxyz\r\n", Expected})
+          .build();
+    }
+
+    @Parameterized.Parameter(0)
+    public String line;
+
+    @Parameterized.Parameter(1)
+    public ImmutableList<String> expected;
+
+    @Test
+    public void testReadLinesWithDelimiter() throws Exception {
+      runTestReadWithData(line.getBytes(UTF_8), expected);
+    }
+
+    private ContextualTextIOSource prepareSource(byte[] data, boolean hasRFC4180Multiline)
+        throws IOException {
+      return ContextualTextIOTest.prepareSource(tempFolder, data, null, hasRFC4180Multiline);
+    }
+
+    private void runTestReadWithData(byte[] data, List<String> expectedResults) throws Exception {
+      ContextualTextIOSource source = prepareSource(data, true);
+      List<LineContext> actual =
+          SourceTestUtils.readFromSource(source, PipelineOptionsFactory.create());
+      List<String> actualOutput = new ArrayList<>();
+      actual.forEach(
+          (LineContext L) -> {
+            String file = L.getFile().substring(L.getFile().lastIndexOf('/') + 1);
+            actualOutput.add(L.getLine());
+          });
+      assertThat(
+          actualOutput,
+          containsInAnyOrder(new ArrayList<>(expectedResults).toArray(new String[0])));
+    }
+  }
+
+  /** Tests Specific for checking functionality of ContextualTextIO */
+  @RunWith(JUnit4.class)
+  public static class ContextualTextIOSpecificTests {
+    @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+    @Rule public TestPipeline p = TestPipeline.create();
+
+    public static final char CR = (char) 0x0D;
+    public static final char LF = (char) 0x0A;
+
+    public static final String CRLF = "" + CR + LF;
+
+    public String createFiles(List<String> input) throws Exception {

Review comment:
       nit: createFile




----------------------------------------------------------------
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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");

Review comment:
       Ack.




----------------------------------------------------------------
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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */

Review comment:
       Ack.




----------------------------------------------------------------
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] rezarokni commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {

Review comment:
       sgtm




----------------------------------------------------------------
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] rezarokni commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/LineContext.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.ContextualTextIO;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+
+@Internal
+@Experimental(Experimental.Kind.SCHEMAS)
+@DefaultSchema(AutoValueSchema.class)
+@AutoValue
+public abstract class LineContext {
+  public abstract Range getRange();
+
+  public abstract Long getLineNum();

Review comment:
       It would still be line x even if there is a \n in the cell no?




----------------------------------------------------------------
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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")

Review comment:
       Ack.




----------------------------------------------------------------
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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a

Review comment:
       Ack.




----------------------------------------------------------------
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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");
+      PCollection<LineContext> output = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        output = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));

Review comment:
       Great suggestions. 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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):

Review comment:
       Made changes to it.
   PTAL.




----------------------------------------------------------------
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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {

Review comment:
       `@Experimental(Experimental.Kind.UNSPECIFIED)`
   `package org.apache.beam.sdk.io.ContextualTextIO;`
   
   I've marked the package as experimental.
   Please check [this](https://github.com/apache/beam/blob/46504b28f7e6a984b398d50dcb5bc48071bf9a82/sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/package-info.java).
   
   Should I still add the `@Experimental` annotation here too ?




----------------------------------------------------------------
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] tvalentyn commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Count;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in
+ * the input.
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code
+ * ContextualTextIO.read()}. To instantiate a transform use {@link
+ * ContextualTextIO.Read#from(String)} and specify the path of the file(s) to be read.
+ * Alternatively, if the filenames to be read are themselves in a {@link PCollection} you can use
+ * {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally when the filepath is on system):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or file pattern of RFC4180-compliant CSV files with fields that may
+ * contain line breaks.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty: when this option is enabled, the input cannot be split and read in parallel.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /**
+     * When reading RFC4180 CSV files that have values that span multiple lines, set this to true.
+     * Note: this reduces the read performance (see: {@link ContextualTextIO}).
+     */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {

Review comment:
       @abhiy13 @rezarokni  what do you think about replacing `withHasRFC4180MultiLineColumn` with
   
   `withMultilineCSV`
   
   and add something like:
   ```The withMultilineCSV setting allows reading CSV files with multiline fields, however it disables read parallelism. This  setting requires input values with line breaks be encompassed in double quotes, and double quotes in values be escaped with a preceding double quote as defined in RFC4180.```




----------------------------------------------------------------
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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}

Review comment:
       Ack.




----------------------------------------------------------------
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] tvalentyn closed pull request #12490: [BEAM-10124] Add ContextualTextIO

Posted by GitBox <gi...@apache.org>.
tvalentyn closed pull request #12490:
URL: https://github.com/apache/beam/pull/12490


   


----------------------------------------------------------------
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] rezarokni commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/LineContext.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.ContextualTextIO;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+
+@Internal
+@Experimental(Experimental.Kind.SCHEMAS)
+@DefaultSchema(AutoValueSchema.class)
+@AutoValue
+public abstract class LineContext {

Review comment:
       RecordMetadata would be more descriptive.




----------------------------------------------------------------
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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));

Review comment:
       `@DefaultSchema(AutoValueSchema.class)`
   `@AutoValue`
   `public abstract class LineContext {`
   
   I made the class based on Schema, this should be returning an object based on a Row.
   Am I missing something ? 




----------------------------------------------------------------
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] tvalentyn commented on pull request #12490: [BEAM-10124] Add ContextualTextIO

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


   Closing in favor of https://github.com/apache/beam/pull/12645.


----------------------------------------------------------------
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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/LineContext.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.ContextualTextIO;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+
+@Internal
+@Experimental(Experimental.Kind.SCHEMAS)
+@DefaultSchema(AutoValueSchema.class)
+@AutoValue
+public abstract class LineContext {
+  public abstract Range getRange();

Review comment:
       Sounds Good.
   cc: @rezarokni 




----------------------------------------------------------------
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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/build.gradle
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+plugins { id 'org.apache.beam.module' }
+applyJavaNature(
+        automaticModuleName: 'org.apache.beam.sdk.io.contextual-text-io')
+
+description = "Apache Beam :: SDKs :: Java :: Contextual-Text-IO"
+ext.summary = "Context aware Text IO."

Review comment:
       Ack.
   




----------------------------------------------------------------
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] tvalentyn commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/LineContext.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.ContextualTextIO;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+
+@Internal
+@Experimental(Experimental.Kind.SCHEMAS)
+@DefaultSchema(AutoValueSchema.class)
+@AutoValue
+public abstract class LineContext {

Review comment:
       Would it be confusing that RecordMetadata also includes the value itself (so it's data + metadata)?




----------------------------------------------------------------
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] rezarokni commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));

Review comment:
       Its a nicety for the user, using the AutoValue class will require the use of a Convert .toRows if the user wants to use any of the Row analytical functions or things like Group. But if we give them a Row then they avoid that Convert step. 




----------------------------------------------------------------
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] tvalentyn commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIOSource.java
##########
@@ -0,0 +1,359 @@
+/*
+ * 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.ContextualTextIO;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.util.NoSuchElementException;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.io.fs.MatchResult;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+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.Preconditions;
+
+/**
+ * Implementation detail of {@link ContextualTextIO.Read}.
+ *
+ * <p>A {@link FileBasedSource} which can decode records delimited by newline characters.
+ *
+ * <p>This source splits the data into records using {@code UTF-8} {@code \n}, {@code \r}, or {@code
+ * \r\n} as the delimiter. This source is not strict and supports decoding the last record even if
+ * it is not delimited. Finally, no records are decoded if the stream is empty.
+ *
+ * <p>This source supports reading from any arbitrary byte position within the stream. If the

Review comment:
       Unless Multiline setting is enabled? 

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIOSource.java
##########
@@ -0,0 +1,359 @@
+/*
+ * 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.ContextualTextIO;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.util.NoSuchElementException;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.io.fs.MatchResult;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+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.Preconditions;
+
+/**
+ * Implementation detail of {@link ContextualTextIO.Read}.
+ *
+ * <p>A {@link FileBasedSource} which can decode records delimited by newline characters.
+ *
+ * <p>This source splits the data into records using {@code UTF-8} {@code \n}, {@code \r}, or {@code
+ * \r\n} as the delimiter. This source is not strict and supports decoding the last record even if
+ * it is not delimited. Finally, no records are decoded if the stream is empty.
+ *
+ * <p>This source supports reading from any arbitrary byte position within the stream. If the
+ * starting position is not {@code 0}, then bytes are skipped until the first delimiter is found
+ * representing the beginning of the first record to be decoded.
+ */
+@VisibleForTesting
+class ContextualTextIOSource extends FileBasedSource<LineContext> {
+  byte[] delimiter;
+
+  // Used to Override isSplittable
+  private boolean hasRFC4180MultiLineColumn;
+
+  @Override
+  protected boolean isSplittable() throws Exception {
+    if (hasRFC4180MultiLineColumn) return false;
+    return super.isSplittable();
+  }
+
+  ContextualTextIOSource(
+      ValueProvider<String> fileSpec,
+      EmptyMatchTreatment emptyMatchTreatment,
+      byte[] delimiter,
+      boolean hasRFC4180MultiLineColumn) {
+    super(fileSpec, emptyMatchTreatment, 1L);
+    this.delimiter = delimiter;
+    this.hasRFC4180MultiLineColumn = hasRFC4180MultiLineColumn;
+  }
+
+  private ContextualTextIOSource(
+      MatchResult.Metadata metadata,
+      long start,
+      long end,
+      byte[] delimiter,
+      boolean hasRFC4180MultiLineColumn) {
+    super(metadata, 1L, start, end);
+    this.delimiter = delimiter;
+    this.hasRFC4180MultiLineColumn = hasRFC4180MultiLineColumn;
+  }
+
+  @Override
+  protected FileBasedSource<LineContext> createForSubrangeOfFile(
+      MatchResult.Metadata metadata, long start, long end) {
+    return new ContextualTextIOSource(metadata, start, end, delimiter, hasRFC4180MultiLineColumn);
+  }
+
+  @Override
+  protected FileBasedReader<LineContext> createSingleFileReader(PipelineOptions options) {
+    return new MultiLineTextBasedReader(this, delimiter, hasRFC4180MultiLineColumn);
+  }
+
+  @Override
+  public Coder<LineContext> getOutputCoder() {
+    SchemaCoder<LineContext> coder = null;
+    try {
+      coder = SchemaRegistry.createDefault().getSchemaCoder(LineContext.class);
+    } catch (NoSuchSchemaException e) {
+      System.out.println("No Coder!");
+    }
+    return coder;
+  }
+
+  /**
+   * A {@link FileBasedReader FileBasedReader} which can decode records delimited by delimiter
+   * characters.
+   *
+   * <p>See {@link ContextualTextIOSource } for further details.
+   */
+  @VisibleForTesting
+  static class MultiLineTextBasedReader extends FileBasedReader<LineContext> {
+    public static final int READ_BUFFER_SIZE = 8192;
+    private static final ByteString UTF8_BOM =
+        ByteString.copyFrom(new byte[] {(byte) 0xEF, (byte) 0xBB, (byte) 0xBF});
+    private final ByteBuffer readBuffer = ByteBuffer.allocate(READ_BUFFER_SIZE);
+    private ByteString buffer;
+    private int startOfDelimiterInBuffer;
+    private int endOfDelimiterInBuffer;
+    private long startOfRecord;
+    private volatile long startOfNextRecord;
+    private volatile boolean eof;
+    private volatile boolean elementIsPresent;
+    private @Nullable LineContext currentValue;
+    private @Nullable ReadableByteChannel inChannel;
+    private @Nullable byte[] delimiter;
+
+    // Add to override the isSplittable
+    private boolean hasRFC4180MultiLineColumn;
+
+    private long startingOffset;
+    private long readerlineNum;
+
+    private MultiLineTextBasedReader(
+        ContextualTextIOSource source, byte[] delimiter, boolean hasRFC4180MultiLineColumn) {
+      super(source);
+      buffer = ByteString.EMPTY;
+      this.delimiter = delimiter;
+      this.hasRFC4180MultiLineColumn = hasRFC4180MultiLineColumn;
+      startingOffset = getCurrentSource().getStartOffset(); // Start offset;
+    }
+
+    @Override
+    protected long getCurrentOffset() throws NoSuchElementException {
+      if (!elementIsPresent) throw new NoSuchElementException();
+      return startOfRecord;
+    }
+
+    @Override
+    public long getSplitPointsRemaining() {
+      if (isStarted() && startOfNextRecord >= getCurrentSource().getEndOffset()) {
+        return isDone() ? 0 : 1;
+      }
+      return super.getSplitPointsRemaining();
+    }
+
+    @Override
+    public LineContext getCurrent() throws NoSuchElementException {
+      if (!elementIsPresent) {
+        throw new NoSuchElementException();
+      }
+      return currentValue;
+    }
+
+    @Override
+    protected void startReading(ReadableByteChannel channel) throws IOException {
+      this.inChannel = channel;
+      // If the first offset is greater than zero, we need to skip bytes until we see our
+      // first delimiter.
+      long startOffset = getCurrentSource().getStartOffset();
+      if (startOffset > 0) {
+        Preconditions.checkState(
+            channel instanceof SeekableByteChannel,
+            "%s only supports reading from a SeekableByteChannel when given a start offset"
+                + " greater than 0.",
+            ContextualTextIOSource.class.getSimpleName());
+        long requiredPosition = startOffset - 1;
+        if (delimiter != null && startOffset >= delimiter.length) {
+          // we need to move back the offset of at worse delimiter.size to be sure to see
+          // all the bytes of the delimiter in the call to findDelimiterBounds() below
+          requiredPosition = startOffset - delimiter.length;
+        }
+        ((SeekableByteChannel) channel).position(requiredPosition);
+        findDelimiterBoundsWithMultiLineCheck();
+        buffer = buffer.substring(endOfDelimiterInBuffer);
+        startOfNextRecord = requiredPosition + endOfDelimiterInBuffer;
+        endOfDelimiterInBuffer = 0;
+        startOfDelimiterInBuffer = 0;
+      }
+    }
+
+    private void findDelimiterBoundsWithMultiLineCheck() throws IOException {
+      findDelimiterBounds();
+    }
+
+    /**
+     * Locates the start position and end position of the next delimiter. Will consume the channel
+     * till either EOF or the delimiter bounds are found.
+     *
+     * <p>If {@link ContextualTextIOSource#hasRFC4180MultiLineColumn} is set then the behaviour will
+     * change from the standard read seen in {@link org.apache.beam.sdk.io.TextIO}. The assumption
+     * when {@link ContextualTextIOSource#hasRFC4180MultiLineColumn} is set is that the file is
+     * being read with a single thread.
+     *
+     * <p>This fills the buffer and updates the positions as follows:
+     *
+     * <pre>{@code
+     * ------------------------------------------------------
+     * | element bytes | delimiter bytes | unconsumed bytes |
+     * ------------------------------------------------------
+     * 0            start of          end of              buffer
+     *              delimiter         delimiter           size
+     *              in buffer         in buffer
+     * }</pre>
+     */
+    private void findDelimiterBounds() throws IOException {
+      int bytePositionInBuffer = 0;
+      boolean doubleQuoteClosed = true;
+      boolean insideOpenQuote = true;

Review comment:
       Why is the value for  `insideOpenQuote` `true` in the beginning?

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIOSource.java
##########
@@ -0,0 +1,359 @@
+/*
+ * 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.ContextualTextIO;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.util.NoSuchElementException;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.io.fs.MatchResult;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+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.Preconditions;
+
+/**
+ * Implementation detail of {@link ContextualTextIO.Read}.
+ *
+ * <p>A {@link FileBasedSource} which can decode records delimited by newline characters.
+ *
+ * <p>This source splits the data into records using {@code UTF-8} {@code \n}, {@code \r}, or {@code
+ * \r\n} as the delimiter. This source is not strict and supports decoding the last record even if
+ * it is not delimited. Finally, no records are decoded if the stream is empty.
+ *
+ * <p>This source supports reading from any arbitrary byte position within the stream. If the
+ * starting position is not {@code 0}, then bytes are skipped until the first delimiter is found
+ * representing the beginning of the first record to be decoded.
+ */
+@VisibleForTesting
+class ContextualTextIOSource extends FileBasedSource<LineContext> {
+  byte[] delimiter;
+
+  // Used to Override isSplittable
+  private boolean hasRFC4180MultiLineColumn;
+
+  @Override
+  protected boolean isSplittable() throws Exception {
+    if (hasRFC4180MultiLineColumn) return false;
+    return super.isSplittable();
+  }
+
+  ContextualTextIOSource(
+      ValueProvider<String> fileSpec,
+      EmptyMatchTreatment emptyMatchTreatment,
+      byte[] delimiter,
+      boolean hasRFC4180MultiLineColumn) {
+    super(fileSpec, emptyMatchTreatment, 1L);
+    this.delimiter = delimiter;
+    this.hasRFC4180MultiLineColumn = hasRFC4180MultiLineColumn;
+  }
+
+  private ContextualTextIOSource(
+      MatchResult.Metadata metadata,
+      long start,
+      long end,
+      byte[] delimiter,
+      boolean hasRFC4180MultiLineColumn) {
+    super(metadata, 1L, start, end);
+    this.delimiter = delimiter;
+    this.hasRFC4180MultiLineColumn = hasRFC4180MultiLineColumn;
+  }
+
+  @Override
+  protected FileBasedSource<LineContext> createForSubrangeOfFile(
+      MatchResult.Metadata metadata, long start, long end) {
+    return new ContextualTextIOSource(metadata, start, end, delimiter, hasRFC4180MultiLineColumn);
+  }
+
+  @Override
+  protected FileBasedReader<LineContext> createSingleFileReader(PipelineOptions options) {
+    return new MultiLineTextBasedReader(this, delimiter, hasRFC4180MultiLineColumn);
+  }
+
+  @Override
+  public Coder<LineContext> getOutputCoder() {
+    SchemaCoder<LineContext> coder = null;
+    try {
+      coder = SchemaRegistry.createDefault().getSchemaCoder(LineContext.class);
+    } catch (NoSuchSchemaException e) {
+      System.out.println("No Coder!");

Review comment:
       We should use loggers to handle errors like this, e.g. https://github.com/apache/beam/blob/71c7760f4b5c5bf0d91e2c8403fae99216308a3e/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java#L897

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIOSource.java
##########
@@ -0,0 +1,359 @@
+/*
+ * 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.ContextualTextIO;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.util.NoSuchElementException;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.io.fs.MatchResult;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+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.Preconditions;
+
+/**
+ * Implementation detail of {@link ContextualTextIO.Read}.
+ *
+ * <p>A {@link FileBasedSource} which can decode records delimited by newline characters.
+ *
+ * <p>This source splits the data into records using {@code UTF-8} {@code \n}, {@code \r}, or {@code
+ * \r\n} as the delimiter. This source is not strict and supports decoding the last record even if
+ * it is not delimited. Finally, no records are decoded if the stream is empty.
+ *
+ * <p>This source supports reading from any arbitrary byte position within the stream. If the
+ * starting position is not {@code 0}, then bytes are skipped until the first delimiter is found
+ * representing the beginning of the first record to be decoded.
+ */
+@VisibleForTesting
+class ContextualTextIOSource extends FileBasedSource<LineContext> {
+  byte[] delimiter;
+
+  // Used to Override isSplittable
+  private boolean hasRFC4180MultiLineColumn;
+
+  @Override
+  protected boolean isSplittable() throws Exception {
+    if (hasRFC4180MultiLineColumn) return false;
+    return super.isSplittable();
+  }
+
+  ContextualTextIOSource(
+      ValueProvider<String> fileSpec,
+      EmptyMatchTreatment emptyMatchTreatment,
+      byte[] delimiter,
+      boolean hasRFC4180MultiLineColumn) {
+    super(fileSpec, emptyMatchTreatment, 1L);
+    this.delimiter = delimiter;
+    this.hasRFC4180MultiLineColumn = hasRFC4180MultiLineColumn;
+  }
+
+  private ContextualTextIOSource(
+      MatchResult.Metadata metadata,
+      long start,
+      long end,
+      byte[] delimiter,
+      boolean hasRFC4180MultiLineColumn) {
+    super(metadata, 1L, start, end);
+    this.delimiter = delimiter;
+    this.hasRFC4180MultiLineColumn = hasRFC4180MultiLineColumn;
+  }
+
+  @Override
+  protected FileBasedSource<LineContext> createForSubrangeOfFile(
+      MatchResult.Metadata metadata, long start, long end) {
+    return new ContextualTextIOSource(metadata, start, end, delimiter, hasRFC4180MultiLineColumn);
+  }
+
+  @Override
+  protected FileBasedReader<LineContext> createSingleFileReader(PipelineOptions options) {
+    return new MultiLineTextBasedReader(this, delimiter, hasRFC4180MultiLineColumn);
+  }
+
+  @Override
+  public Coder<LineContext> getOutputCoder() {
+    SchemaCoder<LineContext> coder = null;
+    try {
+      coder = SchemaRegistry.createDefault().getSchemaCoder(LineContext.class);
+    } catch (NoSuchSchemaException e) {
+      System.out.println("No Coder!");
+    }
+    return coder;
+  }
+
+  /**
+   * A {@link FileBasedReader FileBasedReader} which can decode records delimited by delimiter
+   * characters.
+   *
+   * <p>See {@link ContextualTextIOSource } for further details.
+   */
+  @VisibleForTesting
+  static class MultiLineTextBasedReader extends FileBasedReader<LineContext> {
+    public static final int READ_BUFFER_SIZE = 8192;
+    private static final ByteString UTF8_BOM =
+        ByteString.copyFrom(new byte[] {(byte) 0xEF, (byte) 0xBB, (byte) 0xBF});
+    private final ByteBuffer readBuffer = ByteBuffer.allocate(READ_BUFFER_SIZE);
+    private ByteString buffer;
+    private int startOfDelimiterInBuffer;
+    private int endOfDelimiterInBuffer;
+    private long startOfRecord;
+    private volatile long startOfNextRecord;
+    private volatile boolean eof;
+    private volatile boolean elementIsPresent;
+    private @Nullable LineContext currentValue;
+    private @Nullable ReadableByteChannel inChannel;
+    private @Nullable byte[] delimiter;
+
+    // Add to override the isSplittable
+    private boolean hasRFC4180MultiLineColumn;
+
+    private long startingOffset;
+    private long readerlineNum;
+
+    private MultiLineTextBasedReader(
+        ContextualTextIOSource source, byte[] delimiter, boolean hasRFC4180MultiLineColumn) {
+      super(source);
+      buffer = ByteString.EMPTY;
+      this.delimiter = delimiter;
+      this.hasRFC4180MultiLineColumn = hasRFC4180MultiLineColumn;
+      startingOffset = getCurrentSource().getStartOffset(); // Start offset;
+    }
+
+    @Override
+    protected long getCurrentOffset() throws NoSuchElementException {
+      if (!elementIsPresent) throw new NoSuchElementException();
+      return startOfRecord;
+    }
+
+    @Override
+    public long getSplitPointsRemaining() {
+      if (isStarted() && startOfNextRecord >= getCurrentSource().getEndOffset()) {
+        return isDone() ? 0 : 1;
+      }
+      return super.getSplitPointsRemaining();
+    }
+
+    @Override
+    public LineContext getCurrent() throws NoSuchElementException {
+      if (!elementIsPresent) {
+        throw new NoSuchElementException();
+      }
+      return currentValue;
+    }
+
+    @Override
+    protected void startReading(ReadableByteChannel channel) throws IOException {
+      this.inChannel = channel;
+      // If the first offset is greater than zero, we need to skip bytes until we see our
+      // first delimiter.
+      long startOffset = getCurrentSource().getStartOffset();
+      if (startOffset > 0) {
+        Preconditions.checkState(
+            channel instanceof SeekableByteChannel,
+            "%s only supports reading from a SeekableByteChannel when given a start offset"
+                + " greater than 0.",
+            ContextualTextIOSource.class.getSimpleName());
+        long requiredPosition = startOffset - 1;
+        if (delimiter != null && startOffset >= delimiter.length) {
+          // we need to move back the offset of at worse delimiter.size to be sure to see
+          // all the bytes of the delimiter in the call to findDelimiterBounds() below
+          requiredPosition = startOffset - delimiter.length;
+        }
+        ((SeekableByteChannel) channel).position(requiredPosition);
+        findDelimiterBoundsWithMultiLineCheck();
+        buffer = buffer.substring(endOfDelimiterInBuffer);
+        startOfNextRecord = requiredPosition + endOfDelimiterInBuffer;
+        endOfDelimiterInBuffer = 0;
+        startOfDelimiterInBuffer = 0;
+      }
+    }
+
+    private void findDelimiterBoundsWithMultiLineCheck() throws IOException {

Review comment:
       Do we need this helper?

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIOSource.java
##########
@@ -0,0 +1,359 @@
+/*
+ * 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.ContextualTextIO;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.util.NoSuchElementException;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.io.fs.MatchResult;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+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.Preconditions;
+
+/**
+ * Implementation detail of {@link ContextualTextIO.Read}.
+ *
+ * <p>A {@link FileBasedSource} which can decode records delimited by newline characters.
+ *
+ * <p>This source splits the data into records using {@code UTF-8} {@code \n}, {@code \r}, or {@code
+ * \r\n} as the delimiter. This source is not strict and supports decoding the last record even if
+ * it is not delimited. Finally, no records are decoded if the stream is empty.
+ *
+ * <p>This source supports reading from any arbitrary byte position within the stream. If the
+ * starting position is not {@code 0}, then bytes are skipped until the first delimiter is found
+ * representing the beginning of the first record to be decoded.
+ */
+@VisibleForTesting
+class ContextualTextIOSource extends FileBasedSource<LineContext> {
+  byte[] delimiter;
+
+  // Used to Override isSplittable
+  private boolean hasRFC4180MultiLineColumn;
+
+  @Override
+  protected boolean isSplittable() throws Exception {
+    if (hasRFC4180MultiLineColumn) return false;
+    return super.isSplittable();
+  }
+
+  ContextualTextIOSource(
+      ValueProvider<String> fileSpec,
+      EmptyMatchTreatment emptyMatchTreatment,
+      byte[] delimiter,
+      boolean hasRFC4180MultiLineColumn) {
+    super(fileSpec, emptyMatchTreatment, 1L);
+    this.delimiter = delimiter;
+    this.hasRFC4180MultiLineColumn = hasRFC4180MultiLineColumn;
+  }
+
+  private ContextualTextIOSource(
+      MatchResult.Metadata metadata,
+      long start,
+      long end,
+      byte[] delimiter,
+      boolean hasRFC4180MultiLineColumn) {
+    super(metadata, 1L, start, end);
+    this.delimiter = delimiter;
+    this.hasRFC4180MultiLineColumn = hasRFC4180MultiLineColumn;
+  }
+
+  @Override
+  protected FileBasedSource<LineContext> createForSubrangeOfFile(
+      MatchResult.Metadata metadata, long start, long end) {
+    return new ContextualTextIOSource(metadata, start, end, delimiter, hasRFC4180MultiLineColumn);
+  }
+
+  @Override
+  protected FileBasedReader<LineContext> createSingleFileReader(PipelineOptions options) {
+    return new MultiLineTextBasedReader(this, delimiter, hasRFC4180MultiLineColumn);
+  }
+
+  @Override
+  public Coder<LineContext> getOutputCoder() {
+    SchemaCoder<LineContext> coder = null;
+    try {
+      coder = SchemaRegistry.createDefault().getSchemaCoder(LineContext.class);
+    } catch (NoSuchSchemaException e) {
+      System.out.println("No Coder!");
+    }
+    return coder;
+  }
+
+  /**
+   * A {@link FileBasedReader FileBasedReader} which can decode records delimited by delimiter
+   * characters.
+   *
+   * <p>See {@link ContextualTextIOSource } for further details.
+   */
+  @VisibleForTesting
+  static class MultiLineTextBasedReader extends FileBasedReader<LineContext> {
+    public static final int READ_BUFFER_SIZE = 8192;
+    private static final ByteString UTF8_BOM =
+        ByteString.copyFrom(new byte[] {(byte) 0xEF, (byte) 0xBB, (byte) 0xBF});
+    private final ByteBuffer readBuffer = ByteBuffer.allocate(READ_BUFFER_SIZE);
+    private ByteString buffer;
+    private int startOfDelimiterInBuffer;
+    private int endOfDelimiterInBuffer;
+    private long startOfRecord;
+    private volatile long startOfNextRecord;
+    private volatile boolean eof;
+    private volatile boolean elementIsPresent;
+    private @Nullable LineContext currentValue;
+    private @Nullable ReadableByteChannel inChannel;
+    private @Nullable byte[] delimiter;
+
+    // Add to override the isSplittable
+    private boolean hasRFC4180MultiLineColumn;
+
+    private long startingOffset;
+    private long readerlineNum;
+
+    private MultiLineTextBasedReader(
+        ContextualTextIOSource source, byte[] delimiter, boolean hasRFC4180MultiLineColumn) {
+      super(source);
+      buffer = ByteString.EMPTY;
+      this.delimiter = delimiter;
+      this.hasRFC4180MultiLineColumn = hasRFC4180MultiLineColumn;
+      startingOffset = getCurrentSource().getStartOffset(); // Start offset;
+    }
+
+    @Override
+    protected long getCurrentOffset() throws NoSuchElementException {
+      if (!elementIsPresent) throw new NoSuchElementException();
+      return startOfRecord;
+    }
+
+    @Override
+    public long getSplitPointsRemaining() {
+      if (isStarted() && startOfNextRecord >= getCurrentSource().getEndOffset()) {
+        return isDone() ? 0 : 1;
+      }
+      return super.getSplitPointsRemaining();
+    }
+
+    @Override
+    public LineContext getCurrent() throws NoSuchElementException {
+      if (!elementIsPresent) {
+        throw new NoSuchElementException();
+      }
+      return currentValue;
+    }
+
+    @Override
+    protected void startReading(ReadableByteChannel channel) throws IOException {
+      this.inChannel = channel;
+      // If the first offset is greater than zero, we need to skip bytes until we see our
+      // first delimiter.
+      long startOffset = getCurrentSource().getStartOffset();
+      if (startOffset > 0) {
+        Preconditions.checkState(
+            channel instanceof SeekableByteChannel,
+            "%s only supports reading from a SeekableByteChannel when given a start offset"
+                + " greater than 0.",
+            ContextualTextIOSource.class.getSimpleName());
+        long requiredPosition = startOffset - 1;
+        if (delimiter != null && startOffset >= delimiter.length) {
+          // we need to move back the offset of at worse delimiter.size to be sure to see
+          // all the bytes of the delimiter in the call to findDelimiterBounds() below
+          requiredPosition = startOffset - delimiter.length;
+        }
+        ((SeekableByteChannel) channel).position(requiredPosition);
+        findDelimiterBoundsWithMultiLineCheck();
+        buffer = buffer.substring(endOfDelimiterInBuffer);
+        startOfNextRecord = requiredPosition + endOfDelimiterInBuffer;
+        endOfDelimiterInBuffer = 0;
+        startOfDelimiterInBuffer = 0;
+      }
+    }
+
+    private void findDelimiterBoundsWithMultiLineCheck() throws IOException {
+      findDelimiterBounds();
+    }
+
+    /**
+     * Locates the start position and end position of the next delimiter. Will consume the channel
+     * till either EOF or the delimiter bounds are found.
+     *
+     * <p>If {@link ContextualTextIOSource#hasRFC4180MultiLineColumn} is set then the behaviour will
+     * change from the standard read seen in {@link org.apache.beam.sdk.io.TextIO}. The assumption
+     * when {@link ContextualTextIOSource#hasRFC4180MultiLineColumn} is set is that the file is
+     * being read with a single thread.
+     *
+     * <p>This fills the buffer and updates the positions as follows:
+     *
+     * <pre>{@code
+     * ------------------------------------------------------
+     * | element bytes | delimiter bytes | unconsumed bytes |
+     * ------------------------------------------------------
+     * 0            start of          end of              buffer
+     *              delimiter         delimiter           size
+     *              in buffer         in buffer
+     * }</pre>
+     */
+    private void findDelimiterBounds() throws IOException {
+      int bytePositionInBuffer = 0;
+      boolean doubleQuoteClosed = true;
+      boolean insideOpenQuote = true;
+
+      while (true) {
+        if (!tryToEnsureNumberOfBytesInBuffer(bytePositionInBuffer + 1)) {
+          startOfDelimiterInBuffer = endOfDelimiterInBuffer = bytePositionInBuffer;
+          break;
+        }
+
+        byte currentByte = buffer.byteAt(bytePositionInBuffer);
+        if (hasRFC4180MultiLineColumn) {
+          // Check if we are inside an open Quote
+          if (currentByte == '"') {
+            doubleQuoteClosed = !doubleQuoteClosed;

Review comment:
       Would anything change if we remove `doubleQuoteClosed` and write:             
   `insideOpenQuote = !insideOpenQuote;` here?
   

##########
File path: sdks/java/io/contextual-text-io/src/test/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIOTest.java
##########
@@ -0,0 +1,1271 @@
+/*
+ * 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.ContextualTextIO;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertFalse;
+import static junit.framework.Assert.assertNotNull;
+import static junit.framework.Assert.assertTrue;
+import static org.apache.beam.sdk.TestUtils.LINES_ARRAY;
+import static org.apache.beam.sdk.TestUtils.NO_LINES_ARRAY;
+import static org.apache.beam.sdk.io.Compression.AUTO;
+import static org.apache.beam.sdk.io.Compression.BZIP2;
+import static org.apache.beam.sdk.io.Compression.DEFLATE;
+import static org.apache.beam.sdk.io.Compression.GZIP;
+import static org.apache.beam.sdk.io.Compression.UNCOMPRESSED;
+import static org.apache.beam.sdk.io.Compression.ZIP;
+import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+import static org.apache.beam.sdk.values.TypeDescriptors.strings;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.startsWith;
+import static org.junit.Assume.assumeFalse;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.io.Writer;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.zip.GZIPOutputStream;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.GenerateSequence;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ExperimentalOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.testing.NeedsRunner;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.SourceTestUtils;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo;
+import org.apache.beam.sdk.testing.ValidatesRunner;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.ToString;
+import org.apache.beam.sdk.transforms.Watch;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;
+import org.apache.beam.sdk.transforms.display.DisplayDataMatchers;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
+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.Iterables;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream;
+import org.apache.commons.compress.compressors.deflate.DeflateCompressorOutputStream;
+import org.joda.time.Duration;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.junit.runners.Parameterized;
+
+/** Tests for {@link ContextualTextIO.Read} */
+public class ContextualTextIOTest {
+  private static final int NUM_LINES_FOR_LARGE = 1024;
+
+  private static final List<String> EMPTY = Collections.emptyList();
+
+  private static final List<String> TINY = Arrays.asList("ABC", "DEF", "HIJ");
+
+  private static final List<String> LARGE = makeLines(NUM_LINES_FOR_LARGE);
+
+  private static File writeToFile(
+      List<String> lines, TemporaryFolder folder, String fileName, Compression compression)
+      throws IOException {
+    File file = folder.getRoot().toPath().resolve(fileName).toFile();
+    OutputStream output = new FileOutputStream(file);
+    switch (compression) {
+      case UNCOMPRESSED:
+        break;
+      case GZIP:
+        output = new GZIPOutputStream(output);
+        break;
+      case BZIP2:
+        output = new BZip2CompressorOutputStream(output);
+        break;
+      case ZIP:
+        ZipOutputStream zipOutput = new ZipOutputStream(output);
+        zipOutput.putNextEntry(new ZipEntry("entry"));
+        output = zipOutput;
+        break;
+      case DEFLATE:
+        output = new DeflateCompressorOutputStream(output);
+        break;
+      default:
+        throw new UnsupportedOperationException(compression.toString());
+    }
+    writeToStreamAndClose(lines, output);
+    return file;
+  }
+
+  /**
+   * Helper that writes the given lines (adding a newline in between) to a stream, then closes the
+   * stream.
+   */
+  private static void writeToStreamAndClose(List<String> lines, OutputStream outputStream) {
+    try (PrintStream writer = new PrintStream(outputStream)) {
+      for (String line : lines) {
+        writer.println(line);
+      }
+    }
+  }
+
+  /** Helper to make an array of compressible strings. Returns ["line" i] for i in range(0,n). */
+  private static List<String> makeLines(int n) {
+    List<String> lines = new ArrayList<>();
+    for (int i = 0; i < n; ++i) {
+      lines.add("Line " + i);
+    }
+    return lines;
+  }
+
+  private static class convertLineContextToString extends DoFn<LineContext, String> {
+    @ProcessElement
+    public void processElement(@Element LineContext L, OutputReceiver<String> out) {
+      String file = L.getFile().substring(L.getFile().lastIndexOf('/') + 1);
+      out.output(file + " " + L.getLineNum() + " " + L.getLine());
+    }
+  }
+
+  /**
+   * Helper method that runs a variety of ways to read a single file using ContextualTextIO and
+   * checks that they all match the given expected output.
+   *
+   * <p>The transforms being verified are:
+   *
+   * <ul>
+   *   <li>ContextualTextIO.read().from(filename).withCompression(compressionType).withHintMatchesManyFiles()
+   *   <li>ContextualTextIO.read().from(filename).withCompression(compressionType)
+   *   <li>ContextualTextIO.read().from(filename).withCompression(compressionType).with
+   *   <li>ContextualTextIO.readFiles().withCompression(compressionType)
+   * </ul>
+   */
+  private static void assertReadingCompressedFileMatchesExpected(
+      File file, Compression compression, List<String> expected, Pipeline p) {
+
+    ContextualTextIO.Read read =
+        ContextualTextIO.read().from(file.getPath()).withCompression(compression);
+
+    // Convert the expected output into LineContext output Format
+    List<String> expectedOutput = new ArrayList<>();
+    for (int lineNum = 0; lineNum < expected.size(); ++lineNum) {
+      expectedOutput.add(file.getName() + " " + lineNum + " " + expected.get(lineNum));
+    }
+
+    PAssert.that(
+            p.apply("Read_" + file + "_" + compression.toString(), read)
+                .apply("ConvertLineContextToString", ParDo.of(new convertLineContextToString())))
+        .containsInAnyOrder(expectedOutput);
+    PAssert.that(
+            p.apply(
+                    "Read_" + file + "_" + compression.toString() + "_many",
+                    read.withHintMatchesManyFiles())
+                .apply(
+                    "ConvertLineContextToString" + "_many",
+                    ParDo.of(new convertLineContextToString())))
+        .containsInAnyOrder(expectedOutput);
+
+    PAssert.that(
+            p.apply(
+                    "Read_" + file + "_" + compression.toString() + "_withRFC4180",
+                    read.withHasRFC4180MultiLineColumn(true))
+                .apply(
+                    "ConvertLineContextToString" + "_withRFC4180",
+                    ParDo.of(new convertLineContextToString())))
+        .containsInAnyOrder(expectedOutput);
+
+    PAssert.that(
+            p.apply("Create_Paths_ReadFiles_" + file, Create.of(file.getPath()))
+                .apply("Match_" + file, FileIO.matchAll())
+                .apply("ReadMatches_" + file, FileIO.readMatches().withCompression(compression))
+                .apply("ReadFiles_" + compression.toString(), ContextualTextIO.readFiles())
+                .apply(
+                    "ConvertLineContextToStringWithFileIO",
+                    ParDo.of(new convertLineContextToString())))
+        .containsInAnyOrder(expectedOutput);
+  }
+
+  /**
+   * Create a zip file with the given lines.
+   *
+   * @param expected A list of expected lines, populated in the zip file.
+   * @param folder A temporary folder used to create files.
+   * @param filename Optionally zip file name (can be null).
+   * @param fieldsEntries Fields to write in zip entries.
+   * @return The zip filename.
+   * @throws Exception In case of a failure during zip file creation.
+   */
+  private static File createZipFile(
+      List<String> expected, TemporaryFolder folder, String filename, String[]... fieldsEntries)
+      throws Exception {
+    File tmpFile = folder.getRoot().toPath().resolve(filename).toFile();
+
+    ZipOutputStream out = new ZipOutputStream(new FileOutputStream(tmpFile));
+    PrintStream writer = new PrintStream(out, true /* auto-flush on write */);
+
+    int index = 0;
+    for (String[] entry : fieldsEntries) {
+      out.putNextEntry(new ZipEntry(Integer.toString(index)));
+      for (String field : entry) {
+        writer.println(field);
+        expected.add(field);
+      }
+      out.closeEntry();
+      index++;
+    }
+
+    writer.close();
+    out.close();
+
+    return tmpFile;
+  }
+
+  private static ContextualTextIOSource prepareSource(
+      TemporaryFolder temporaryFolder, byte[] data, byte[] delimiter, boolean hasRFC4180Multiline)
+      throws IOException {
+    Path path = temporaryFolder.newFile().toPath();
+    Files.write(path, data);
+    return new ContextualTextIOSource(
+        ValueProvider.StaticValueProvider.of(path.toString()),
+        EmptyMatchTreatment.DISALLOW,
+        delimiter,
+        hasRFC4180Multiline);
+  }
+
+  private static String getFileSuffix(Compression compression) {
+    switch (compression) {
+      case UNCOMPRESSED:
+        return ".txt";
+      case GZIP:
+        return ".gz";
+      case BZIP2:
+        return ".bz2";
+      case ZIP:
+        return ".zip";
+      case DEFLATE:
+        return ".deflate";
+      default:
+        return "";
+    }
+  }
+  /** Tests for reading from different size of files with various Compression. */
+  @RunWith(Parameterized.class)
+  public static class CompressedReadTest {
+    @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+    @Rule public TestPipeline p = TestPipeline.create();
+
+    @Parameterized.Parameters(name = "{index}: {1}")
+    public static Iterable<Object[]> data() {
+      return ImmutableList.<Object[]>builder()
+          .add(new Object[] {EMPTY, UNCOMPRESSED})
+          .add(new Object[] {EMPTY, GZIP})
+          .add(new Object[] {EMPTY, BZIP2})
+          .add(new Object[] {EMPTY, ZIP})
+          .add(new Object[] {EMPTY, DEFLATE})
+          .add(new Object[] {TINY, UNCOMPRESSED})
+          .add(new Object[] {TINY, GZIP})
+          .add(new Object[] {TINY, BZIP2})
+          .add(new Object[] {TINY, ZIP})
+          .add(new Object[] {TINY, DEFLATE})
+          .add(new Object[] {LARGE, UNCOMPRESSED})
+          .add(new Object[] {LARGE, GZIP})
+          .add(new Object[] {LARGE, BZIP2})
+          .add(new Object[] {LARGE, ZIP})
+          .add(new Object[] {LARGE, DEFLATE})
+          .build();
+    }
+
+    @Parameterized.Parameter(0)
+    public List<String> lines;
+
+    @Parameterized.Parameter(1)
+    public Compression compression;
+
+    /** Tests reading from a small, compressed file with no extension. */
+    @Test
+    @Category(NeedsRunner.class)
+    public void testCompressedReadWithoutExtension() throws Exception {
+      String fileName = lines.size() + "_" + compression + "_no_extension";
+      File fileWithNoExtension = writeToFile(lines, tempFolder, fileName, compression);
+      assertReadingCompressedFileMatchesExpected(fileWithNoExtension, compression, lines, p);
+      p.run();
+    }
+
+    @Test
+    @Category(NeedsRunner.class)
+    public void testCompressedReadWithExtension() throws Exception {
+      String fileName =
+          lines.size() + "_" + compression + "_no_extension" + getFileSuffix(compression);
+      File fileWithExtension = writeToFile(lines, tempFolder, fileName, compression);
+
+      // Sanity check that we're properly testing compression.
+      if (lines.size() == NUM_LINES_FOR_LARGE && !compression.equals(UNCOMPRESSED)) {
+        File uncompressedFile = writeToFile(lines, tempFolder, "large.txt", UNCOMPRESSED);
+        assertThat(uncompressedFile.length(), greaterThan(fileWithExtension.length()));
+      }
+
+      assertReadingCompressedFileMatchesExpected(fileWithExtension, compression, lines, p);
+      p.run();
+    }
+
+    @Test
+    @Category(NeedsRunner.class)
+    public void testReadWithAuto() throws Exception {
+      // Files with non-compressed extensions should work in AUTO and UNCOMPRESSED modes.
+      String fileName =
+          lines.size() + "_" + compression + "_no_extension" + getFileSuffix(compression);
+      File fileWithExtension = writeToFile(lines, tempFolder, fileName, compression);
+      assertReadingCompressedFileMatchesExpected(fileWithExtension, AUTO, lines, p);
+      p.run();
+    }
+  }
+
+  /** Tests for reading files with various delimiters. */
+  @RunWith(Parameterized.class)
+  public static class ReadWithDelimiterTest {
+    private static final ImmutableList<String> EXPECTED = ImmutableList.of("asdf", "hjkl", "xyz");
+    @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+
+    @Parameterized.Parameters(name = "{index}: {0}")
+    public static Iterable<Object[]> data() {
+      return ImmutableList.<Object[]>builder()
+          //          .add(new Object[] {"\n\n\n", ImmutableList.of("", "", "")})
+          .add(new Object[] {"asdf\nhjkl\nxyz\n", EXPECTED})
+          .add(new Object[] {"asdf\rhjkl\rxyz\r", EXPECTED})
+          .add(new Object[] {"asdf\r\nhjkl\r\nxyz\r\n", EXPECTED})
+          .add(new Object[] {"asdf\rhjkl\r\nxyz\n", EXPECTED})
+          .add(new Object[] {"asdf\nhjkl\nxyz", EXPECTED})
+          .add(new Object[] {"asdf\rhjkl\rxyz", EXPECTED})
+          .add(new Object[] {"asdf\r\nhjkl\r\nxyz", EXPECTED})
+          .add(new Object[] {"asdf\rhjkl\r\nxyz", EXPECTED})
+          .build();
+    }
+
+    @Parameterized.Parameter(0)
+    public String line;
+
+    @Parameterized.Parameter(1)
+    public ImmutableList<String> expected;
+
+    @Test
+    public void testReadLinesWithDelimiter() throws Exception {
+      runTestReadWithData(line.getBytes(UTF_8), expected);
+    }
+
+    private ContextualTextIOSource prepareSource(byte[] data, boolean hasRFC4180Multiline)
+        throws IOException {
+      return ContextualTextIOTest.prepareSource(tempFolder, data, null, hasRFC4180Multiline);
+    }
+
+    private void runTestReadWithData(byte[] data, List<String> expectedResults) throws Exception {
+      ContextualTextIOSource source = prepareSource(data, false);
+      List<LineContext> actual =
+          SourceTestUtils.readFromSource(source, PipelineOptionsFactory.create());
+      List<String> actualOutput = new ArrayList<>();
+      actual.forEach(
+          (LineContext L) -> {
+            String file = L.getFile().substring(L.getFile().lastIndexOf('/') + 1);
+            actualOutput.add(L.getLine());
+          });
+      assertThat(
+          actualOutput,
+          containsInAnyOrder(new ArrayList<>(expectedResults).toArray(new String[0])));
+    }
+  }
+
+  @RunWith(Parameterized.class)
+  public static class ReadWithDelimiterAndRFC4180 {
+    static final ImmutableList<String> Expected = ImmutableList.of("\"asdf\nhjkl\nmnop\"", "xyz");
+    @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+
+    @Parameterized.Parameters(name = "{index}: {0}")
+    public static Iterable<Object[]> data() {
+      return ImmutableList.<Object[]>builder()
+          .add(new Object[] {"\n\n\n", ImmutableList.of("", "", "")})
+          .add(new Object[] {"\"asdf\nhjkl\"\nxyz\n", ImmutableList.of("\"asdf\nhjkl\"", "xyz")})
+          .add(new Object[] {"\"asdf\nhjkl\nmnop\"\nxyz\n", Expected})
+          .add(new Object[] {"\"asdf\nhjkl\nmnop\"\nxyz\r", Expected})
+          .add(new Object[] {"\"asdf\nhjkl\nmnop\"\r\nxyz\n", Expected})
+          .add(new Object[] {"\"asdf\nhjkl\nmnop\"\r\nxyz\r\n", Expected})
+          .add(new Object[] {"\"asdf\nhjkl\nmnop\"\rxyz\r\n", Expected})
+          .build();
+    }
+
+    @Parameterized.Parameter(0)
+    public String line;
+
+    @Parameterized.Parameter(1)
+    public ImmutableList<String> expected;
+
+    @Test
+    public void testReadLinesWithDelimiter() throws Exception {
+      runTestReadWithData(line.getBytes(UTF_8), expected);
+    }
+
+    private ContextualTextIOSource prepareSource(byte[] data, boolean hasRFC4180Multiline)
+        throws IOException {
+      return ContextualTextIOTest.prepareSource(tempFolder, data, null, hasRFC4180Multiline);
+    }
+
+    private void runTestReadWithData(byte[] data, List<String> expectedResults) throws Exception {
+      ContextualTextIOSource source = prepareSource(data, true);
+      List<LineContext> actual =
+          SourceTestUtils.readFromSource(source, PipelineOptionsFactory.create());
+      List<String> actualOutput = new ArrayList<>();
+      actual.forEach(
+          (LineContext L) -> {
+            String file = L.getFile().substring(L.getFile().lastIndexOf('/') + 1);
+            actualOutput.add(L.getLine());
+          });
+      assertThat(
+          actualOutput,
+          containsInAnyOrder(new ArrayList<>(expectedResults).toArray(new String[0])));
+    }
+  }
+
+  /** Tests Specific for checking functionality of ContextualTextIO */
+  @RunWith(JUnit4.class)
+  public static class ContextualTextIOSpecificTests {
+    @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+    @Rule public TestPipeline p = TestPipeline.create();
+
+    public static final char CR = (char) 0x0D;
+    public static final char LF = (char) 0x0A;
+
+    public static final String CRLF = "" + CR + LF;
+
+    public String createFiles(List<String> input) throws Exception {
+
+      File tmpFile = tempFolder.newFile();
+      String filename = tmpFile.getPath();
+
+      try (PrintStream writer = new PrintStream(new FileOutputStream(tmpFile))) {
+        for (String elem : input) {
+          byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem);
+          String line = new String(encodedElem, Charsets.UTF_8);
+          writer.println(line);
+        }
+      }
+      return filename;
+    }
+
+    @Test
+    @Category(NeedsRunner.class)
+    public void MultipleFilesTest() throws Exception {
+      List<File> files =
+          Arrays.asList(
+              tempFolder.newFile("File1"),
+              tempFolder.newFile("File2"),
+              tempFolder.newFile("File3"));
+
+      int num = 0;
+      for (File tmpFile : files) {
+        num += 2;
+        String filename = tmpFile.getPath();
+        try (PrintStream writer = new PrintStream(new FileOutputStream(tmpFile))) {
+          for (int lineNum = 0; lineNum < 10 + num; ++lineNum) {
+            String elem = filename + " " + lineNum;
+            byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem);
+            String line = new String(encodedElem, Charsets.UTF_8);
+            writer.println(line);
+          }
+        }
+      }
+      String filePath = files.get(0).getPath();
+      filePath = filePath.substring(0, filePath.lastIndexOf('/') + 1);
+      filePath += '*';
+      p.apply(ContextualTextIO.read().from(filePath))
+          .apply(
+              MapElements.into(strings())
+                  .via(
+                      (LineContext L) -> {
+                        String expectedLineNum =
+                            L.getLine().substring(L.getLine().lastIndexOf(' ') + 1);
+                        assertEquals(Long.parseLong(expectedLineNum), (long) L.getLineNum());
+                        return "";
+                      }));
+
+      p.run();
+    }
+
+    @Test
+    @Category(NeedsRunner.class)
+    public void testWithHintMatchesManyFiles() throws IOException {
+      List<File> files =
+          Arrays.asList(
+              tempFolder.newFile("File1"),
+              tempFolder.newFile("File2"),
+              tempFolder.newFile("File3"));
+
+      int num = 0;
+      for (File tmpFile : files) {
+        num += 2;
+        String filename = tmpFile.getPath();
+        try (PrintStream writer = new PrintStream(new FileOutputStream(tmpFile))) {
+          for (int lineNum = 0; lineNum < 10 + num; ++lineNum) {
+            String elem = filename + " " + lineNum;
+            byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem);
+            String line = new String(encodedElem, Charsets.UTF_8);
+            writer.println(line);
+          }
+        }
+      }
+      String filePath = files.get(0).getPath();
+      filePath = filePath.substring(0, filePath.lastIndexOf('/') + 1);
+      filePath += '*';
+      p.apply(ContextualTextIO.read().from(filePath).withHintMatchesManyFiles())
+          .apply(
+              MapElements.into(strings())
+                  .via(
+                      (LineContext L) -> {
+                        String expectedLineNum =
+                            L.getLine().substring(L.getLine().lastIndexOf(' ') + 1);
+                        assertEquals(Long.parseLong(expectedLineNum), (long) L.getLineNum());
+                        return "";
+                      }));
+
+      p.run();
+    }
+
+    @Test
+    @Category(NeedsRunner.class)
+    public void runBasicReadTest() throws Exception {
+
+      List<String> input = ImmutableList.of("1", "2");
+      ContextualTextIO.Read read = ContextualTextIO.read().from(createFiles(input));
+      PCollection<LineContext> output = p.apply(read);
+
+      PCollection<String> result =
+          output.apply(MapElements.into(strings()).via(x -> String.valueOf(x.getLine())));
+
+      PAssert.that(result).containsInAnyOrder("1", "2");
+
+      p.run();
+    }
+
+    @Test
+    @Category(NeedsRunner.class)
+    public void runBasicReadTestWithRFC4180Set() throws Exception {
+
+      List<String> input = ImmutableList.of("1", "2");
+
+      ContextualTextIO.Read read =
+          ContextualTextIO.read().from(createFiles(input)).withHasRFC4180MultiLineColumn(true);
+      PCollection<LineContext> output = p.apply(read);
+
+      PCollection<String> result =
+          output.apply(MapElements.into(strings()).via(x -> String.valueOf(x.getLine())));
+
+      PAssert.that(result).containsInAnyOrder("1", "2");
+
+      p.run();
+    }
+
+    @Test
+    @Category(NeedsRunner.class)
+    /** Test to read files with using MultiLine columns as per RFC4180 */
+    public void runSmallRFC4180MultiLineReadTest() throws Exception {
+
+      // Generate lines of format "1\n1" where number changes per line.
+      List<String> input =
+          IntStream.range(0, 2)
+              .<String>mapToObj(x -> "\"" + x + CRLF + x + "\"")
+              .collect(Collectors.toList());
+
+      ContextualTextIO.Read read =
+          ContextualTextIO.read().from(createFiles(input)).withRFC4180MultiLineColumn(true);
+      PCollection<LineContext> output = p.apply(read);
+
+      PCollection<String> result =
+          output.apply(
+              MapElements.into(strings())
+                  .via(
+                      x -> {
+                        return String.valueOf(x.getLine());
+                      }));
+
+      PAssert.that(result).containsInAnyOrder(input);
+
+      p.run();
+    }
+
+    @Test
+    @Category(NeedsRunner.class)
+    /** Test to read files with using MultiLine columns as per RFC4180 */
+    public void runSmallRFC4180EscapedCharcatersReadTest() throws Exception {
+
+      // Generate lines of format  "aaa","b""bb","ccc" where number changes per line.
+      List<String> input =
+          IntStream.range(0, 2)
+              .<String>mapToObj(x -> "\"aaa\",\"b\"\"bb\",\"ccc\"")
+              .collect(Collectors.toList());
+
+      ContextualTextIO.Read read =
+          ContextualTextIO.read().from(createFiles(input)).withRFC4180MultiLineColumn(true);
+      PCollection<LineContext> output = p.apply(read);
+
+      PCollection<String> result =
+          output.apply(
+              MapElements.into(strings())
+                  .via(
+                      x -> {
+                        return String.valueOf(x.getLine());
+                      }));
+
+      PAssert.that(result).containsInAnyOrder(input);
+
+      p.run();
+    }
+
+    @Test
+    @Category(NeedsRunner.class)
+    /** Test to read files with using MultiLine columns as per RFC4180 */
+    public void runLargeRFC4180MultiLineReadTest() throws Exception {
+
+      // Generate lines of format "1\n1" where number changes per line.
+      List<String> input =
+          IntStream.range(0, 1000)
+              .<String>mapToObj(x -> "\"" + x + CRLF + x + "\"")
+              .collect(Collectors.toList());
+
+      ContextualTextIO.Read read =
+          ContextualTextIO.read().from(createFiles(input)).withHasRFC4180MultiLineColumn(true);
+      PCollection<LineContext> output = p.apply(read);
+
+      PCollection<String> result =
+          output.apply(MapElements.into(strings()).via(x -> String.valueOf(x.getLine())));
+
+      PAssert.that(result).containsInAnyOrder(input);
+
+      p.run();
+    }
+
+    @Test
+    @Category(NeedsRunner.class)
+    /** Test to read files with using MultiLine columns as per RFC4180 */
+    public void runLargeRFC4180MultiLineAndEscapedReadTest() throws Exception {
+
+      // Generate lines of format  "aaa","b""\nbb","ccc","""\nHello" where number changes per line.
+      List<String> input =
+          IntStream.range(0, 1000)
+              .<String>mapToObj(
+                  x -> "\"a" + CRLF + "aa\",\"b\"\"" + CRLF + "bb\",\"ccc\",\"\"\"\\nHello\"")
+              .collect(Collectors.toList());
+
+      ContextualTextIO.Read read =
+          ContextualTextIO.read().from(createFiles(input)).withHasRFC4180MultiLineColumn(true);
+      PCollection<LineContext> output = p.apply(read);
+
+      PCollection<String> result =
+          output.apply(MapElements.into(strings()).via(x -> String.valueOf(x.getLine())));
+
+      PAssert.that(result).containsInAnyOrder(input);
+
+      p.run();
+    }
+
+    @Test
+    @Category(NeedsRunner.class)
+    /** Test to read files with using MultiLine columns as per RFC4180 */

Review comment:
       This comment is probably not very helpful since we are copypasting it with 5 different scenarios.
   Also, a good test case communicates the scenario and expected result in the test name, so that a comment is not required.




----------------------------------------------------------------
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] rezarokni commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/LineContext.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.ContextualTextIO;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+
+@Internal
+@Experimental(Experimental.Kind.SCHEMAS)
+@DefaultSchema(AutoValueSchema.class)
+@AutoValue
+public abstract class LineContext {
+  public abstract Range getRange();

Review comment:
       It would be needed outside of the package for folks to access the Range + line tuple no? 




----------------------------------------------------------------
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] tvalentyn commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");
+      PCollection<LineContext> output = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        output = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        output =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // Output Contains LineContext Objects Without Correct Line Numbers
+      // The following operation assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =
+          output
+              .apply(
+                  "Convert LineContext to KV<KV<File,Range>, LineContext>",
+                  ParDo.of(
+                      new DoFn<LineContext, KV<KV<String, Long>, LineContext>>() {
+                        @ProcessElement
+                        public void processElement(
+                            @Element LineContext line,
+                            OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+                          out.output(
+                              KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+                        }
+                      }))
+              .apply("Apply GBK to PColl<KV<KV<File, Range>, LineCtx>>", GroupByKey.create());
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          groupedOutput
+              .apply(
+                  "KV<KV<File, Range>, Iter<LineCtx>> to KV<Range, Sizeof(Iter<LineCtx>)>",
+                  ParDo.of(
+                      new DoFn<
+                          KV<KV<String, Long>, Iterable<LineContext>>,
+                          KV<KV<String, Long>, Long>>() {
+                        @ProcessElement
+                        public void processElement(
+                            @Element KV<KV<String, Long>, Iterable<LineContext>> elem,
+                            OutputReceiver<KV<KV<String, Long>, Long>> out) {
+                          out.output(KV.of(elem.getKey(), (long) Iterables.size(elem.getValue())));
+                        }
+                      }))
+              .apply("Convert Sizes to PCollView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element so that
+      // prefix sums can be computed in one pass

Review comment:
       The concept of `prefix sums` may not be intuitive to a reader. If we start using 'chunk' terminology, perhaps we can use use `chunk offsets`/`chunk absolute offset` / `offset of each chunk within the 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] rezarokni commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.

Review comment:
       Call out that the user explicitly needs to tell our system that their files have RFC4180 characteristics, important to also point out which of those characteristics would cause the system to not provide correct results if this flag is not set. 

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");
+      PCollection<LineContext> output = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        output = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        output =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // Output Contains LineContext Objects Without Correct Line Numbers
+      // The following operation assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =

Review comment:
       This is a very nice option for the end user, but what are the performance implications of a shuffle .. If there is a large performance effect, this can become an optional choice which the user can then elect to disable. There is a nice pattern in BigQueryIO when .withextraerrorinfo is chosen which disables it when its not needed. 

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));

Review comment:
       Would be good to explore returning a Row rather than a LineContext. This will be more consistent with the general direction of travel for the java API I think. 

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {

Review comment:
       +1




----------------------------------------------------------------
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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");
+      PCollection<LineContext> output = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        output = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        output =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // Output Contains LineContext Objects Without Correct Line Numbers
+      // The following operation assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =
+          output
+              .apply(
+                  "Convert LineContext to KV<KV<File,Range>, LineContext>",
+                  ParDo.of(
+                      new DoFn<LineContext, KV<KV<String, Long>, LineContext>>() {
+                        @ProcessElement
+                        public void processElement(
+                            @Element LineContext line,
+                            OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+                          out.output(
+                              KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+                        }
+                      }))
+              .apply("Apply GBK to PColl<KV<KV<File, Range>, LineCtx>>", GroupByKey.create());
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          groupedOutput
+              .apply(
+                  "KV<KV<File, Range>, Iter<LineCtx>> to KV<Range, Sizeof(Iter<LineCtx>)>",

Review comment:
       Ack.
   PTAL.




----------------------------------------------------------------
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] tvalentyn commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");
+      PCollection<LineContext> output = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        output = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        output =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // Output Contains LineContext Objects Without Correct Line Numbers
+      // The following operation assigns line numbers to all LineContext Objects
+
+      PCollection<KV<KV<String, Long>, Iterable<LineContext>>> groupedOutput =
+          output
+              .apply(
+                  "Convert LineContext to KV<KV<File,Range>, LineContext>",
+                  ParDo.of(
+                      new DoFn<LineContext, KV<KV<String, Long>, LineContext>>() {
+                        @ProcessElement
+                        public void processElement(
+                            @Element LineContext line,
+                            OutputReceiver<KV<KV<String, Long>, LineContext>> out) {
+                          out.output(
+                              KV.of(KV.of(line.getFile(), line.getRange().getRangeNum()), line));
+                        }
+                      }))
+              .apply("Apply GBK to PColl<KV<KV<File, Range>, LineCtx>>", GroupByKey.create());
+
+      PCollectionView<Map<KV<String, Long>, Long>> sizes =
+          groupedOutput
+              .apply(
+                  "KV<KV<File, Range>, Iter<LineCtx>> to KV<Range, Sizeof(Iter<LineCtx>)>",
+                  ParDo.of(
+                      new DoFn<
+                          KV<KV<String, Long>, Iterable<LineContext>>,
+                          KV<KV<String, Long>, Long>>() {
+                        @ProcessElement
+                        public void processElement(
+                            @Element KV<KV<String, Long>, Iterable<LineContext>> elem,
+                            OutputReceiver<KV<KV<String, Long>, Long>> out) {
+                          out.output(KV.of(elem.getKey(), (long) Iterables.size(elem.getValue())));
+                        }
+                      }))
+              .apply("Convert Sizes to PCollView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element so that
+      // prefix sums can be computed in one pass

Review comment:
       The concept of `prefix sums` may not be intuitive to a reader. If we start using 'chunk' terminology, perhaps we can use use `chunk offsets`/`chunk absolute offset` `offset of each chunk within the 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] abhiy13 commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {
+  private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
+
+  /**
+   * A {@link PTransform} that reads from one or more text files and returns a bounded {@link
+   * PCollection} containing one {@link LineContext}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /**
+   * Like {@link #read}, but reads each file in a {@link PCollection} of {@link
+   * FileIO.ReadableFile}, returned by {@link FileIO#readMatches}.
+   */
+  public static ReadFiles readFiles() {
+    return new AutoValue_ContextualTextIO_ReadFiles.Builder()
+        // 64MB is a reasonable value that allows to amortize the cost of opening files,
+        // but is not so large as to exhaust a typical runner's maximum amount of output per
+        // ProcessElement call.
+        .setDesiredBundleSizeBytes(DEFAULT_BUNDLE_SIZE_BYTES)
+        .setHasRFC4180MultiLineColumn(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<LineContext>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasRFC4180MultiLineColumn();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract @Nullable byte[] getDelimiter();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setFilepattern(ValueProvider<String> filepattern);
+
+      abstract Builder setMatchConfiguration(MatchConfiguration matchConfiguration);
+
+      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte[] delimiter);
+
+      abstract Builder setHasRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn);
+
+      abstract Read build();
+    }
+
+    /**
+     * Reads text files that reads from the file(s) with the given filename or filename pattern.
+     *
+     * <p>This can be a local path (if running locally), or a Google Cloud Storage filename or
+     * filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if running locally or using
+     * remote execution service).
+     *
+     * <p>Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java
+     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
+     *
+     * <p>If it is known that the filepattern will match a very large number of files (at least tens
+     * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability.
+     */
+    public Read from(String filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return from(StaticValueProvider.of(filepattern));
+    }
+
+    /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */
+    public Read from(ValueProvider<String> filepattern) {
+      checkArgument(filepattern != null, "filepattern can not be null");
+      return toBuilder().setFilepattern(filepattern).build();
+    }
+
+    /** Sets the {@link MatchConfiguration}. */
+    public Read withMatchConfiguration(MatchConfiguration matchConfiguration) {
+      return toBuilder().setMatchConfiguration(matchConfiguration).build();
+    }
+
+    /** Sets if the file has RFC4180 MultiLineColumn. */
+    public Read withRFC4180MultiLineColumn(Boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    /**
+     * Reads from input sources using the specified compression type.
+     *
+     * <p>If no compression type is specified, the default is {@link Compression#AUTO}.
+     */
+    public Read withCompression(Compression compression) {
+      return toBuilder().setCompression(compression).build();
+    }
+
+    /**
+     * Hints that the filepattern specified in {@link #from(String)} matches a very large number of
+     * files.
+     *
+     * <p>This hint may cause a runner to execute the transform differently, in a way that improves
+     * performance for this case, but it may worsen performance if the filepattern matches only a
+     * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will
+     * happen less efficiently within individual files).
+     */
+    public Read withHintMatchesManyFiles() {
+      return toBuilder().setHintMatchesManyFiles(true).build();
+    }
+
+    /** See {@link MatchConfiguration#withEmptyMatchTreatment}. */
+    public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) {
+      return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment));
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withDelimiter(byte[] delimiter) {
+      checkArgument(delimiter != null, "delimiter can not be null");
+      checkArgument(!isSelfOverlapping(delimiter), "delimiter must not self-overlap");
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    /** Set the custom delimiter to be used in place of the default ones ('\r', '\n' or '\r\n'). */
+    public Read withHasRFC4180MultiLineColumn(boolean hasRFC4180MultiLineColumn) {
+      return toBuilder().setHasRFC4180MultiLineColumn(hasRFC4180MultiLineColumn).build();
+    }
+
+    static boolean isSelfOverlapping(byte[] s) {
+      // s self-overlaps if v exists such as s = vu = wv with u and w non empty
+      for (int i = 1; i < s.length - 1; ++i) {
+        if (ByteBuffer.wrap(s, 0, i).equals(ByteBuffer.wrap(s, s.length - i, i))) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public PCollection<LineContext> expand(PBegin input) {
+      checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform");
+      PCollection<LineContext> output = null;
+      if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) {
+        output = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource()));
+      } else {
+        // All other cases go through FileIO + ReadFiles
+        output =
+            input
+                .apply(
+                    "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of()))
+                .apply("Match All", FileIO.matchAll().withConfiguration(getMatchConfiguration()))
+                .apply(
+                    "Read Matches",
+                    FileIO.readMatches()
+                        .withCompression(getCompression())
+                        .withDirectoryTreatment(DirectoryTreatment.PROHIBIT))
+                .apply("Via ReadFiles", readFiles().withDelimiter(getDelimiter()));
+      }
+
+      // Output Contains LineContext Objects Without Correct Line Numbers

Review comment:
       Ack.




----------------------------------------------------------------
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] tvalentyn commented on a change in pull request #12490: [BEAM-10124] Add ContextualTextIO

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



##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/ContextualTextIO/ContextualTextIO.java
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ContextualTextIO;
+
+import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
+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 avro.shaded.com.google.common.collect.Iterables;
+import com.google.auto.value.AutoValue;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.CompressedSource;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileIO.MatchConfiguration;
+import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+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.transforms.View;
+import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+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.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s for reading from text files with Context
+ *
+ * <h2>Reading from text files</h2>
+ *
+ * <p>To read a {@link PCollection} from one or more text files, use {@code ContextualTextIO.read()}
+ * to instantiate a transform use {@link ContextualTextIO.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} you can use {@link FileIO} to match them and {@link ContextualTextIO#readFiles()} to
+ * read them.
+ *
+ * <p>{@link #read} returns a {@link PCollection} of {@link LineContext LineContext}, each
+ * corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n', '\r',
+ * '\r\n', or specified delimiter see {@link ContextualTextIO.Read#withDelimiter})
+ *
+ * <h3>Filepattern expansion and watching</h3>
+ *
+ * <p>By default, the filepatterns are expanded only once. The combination of {@link
+ * FileIO.Match#continuously(Duration, TerminationCondition)} and {@link #readFiles()} allow
+ * streaming of new files matching the filepattern(s).
+ *
+ * <p>By default, {@link #read} prohibits filepatterns that match no files, and {@link #readFiles()}
+ * allows them in case the filepattern contains a glob wildcard character. Use {@link
+ * ContextualTextIO.Read#withEmptyMatchTreatment} or {@link
+ * FileIO.Match#withEmptyMatchTreatment(EmptyMatchTreatment)} plus {@link #readFiles()} to configure
+ * this behavior.
+ *
+ * <p>Example 1: reading a file or filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read().from("/local/path/to/file.txt"));
+ * }</pre>
+ *
+ * <p>Example 2: reading a PCollection of filenames.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * // E.g. the filenames might be computed from other data in the pipeline, or
+ * // read from a data source.
+ * PCollection<String> filenames = ...;
+ *
+ * // Read all files in the collection.
+ * PCollection<LineContext> lines =
+ *     filenames
+ *         .apply(FileIO.matchAll())
+ *         .apply(FileIO.readMatches())
+ *         .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * <p>Example 3: streaming new files matching a filepattern.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *     .watchForNewFiles(
+ *       // Check for new files every minute
+ *       Duration.standardMinutes(1),
+ *       // Stop watching the filepattern if no new files appear within an hour
+ *       afterTimeSinceNewOutput(Duration.standardHours(1))));
+ * }</pre>
+ *
+ * <p>Example 4: reading a file or filepattern of Multiline CSV files.
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*")
+ *      .withHasRFC4180MultiLineColumn(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<LineContext> lines = p.apply(FileIO.match()
+ *      .filepattern("filepattern")
+ *      .continuously(
+ *        Duration.millis(100),
+ *        Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))
+ *      .apply(FileIO.readMatches())
+ *      .apply(ContextualTextIO.readFiles());
+ * }</pre>
+ *
+ * NOTE: Using {@link ContextualTextIO.Read#withHasRFC4180MultiLineColumn(boolean)} introduces a
+ * performance penalty, when using this option the files are not split and read on multiple workers.
+ *
+ * <h3>Reading a very large number of files</h3>
+ *
+ * <p>If it is known that the filepattern will match a very large number of files (e.g. tens of
+ * thousands or more), use {@link ContextualTextIO.Read#withHintMatchesManyFiles} for better
+ * performance and scalability. Note that it may decrease performance if the filepattern matches
+ * only a small number of files.
+ */
+public class ContextualTextIO {

Review comment:
       I see. package-level may be sufficient, I don't have authoritative opinion.
   
   I expected something like https://github.com/apache/beam/blob/3e7371eae49b7aba142c8c0e898a62f583b49276/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java#L827 
   
   Note that it has a different 'Kind'.




----------------------------------------------------------------
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