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/20 04:45:53 UTC

[GitHub] [beam] rezarokni commented on a change in pull request #12645: [BEAM-10124] Add ContextualTextIO

rezarokni commented on a change in pull request #12645:
URL: https://github.com/apache/beam/pull/12645#discussion_r473578722



##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java
##########
@@ -130,7 +130,7 @@ protected FileBasedSource(
    *
    * @throws IllegalArgumentException if this source is in {@link Mode#FILEPATTERN} mode.
    */
-  protected final MatchResult.Metadata getSingleFileMetadata() {
+  public final MatchResult.Metadata getSingleFileMetadata() {

Review comment:
       @tvalentyn what do you think about this and the FileIO changes being done in a different commit / pr . As its a change that is potentially useful to other sources api dev and maybe easier to reason about then if tied to this larger commit?

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java
##########
@@ -0,0 +1,594 @@
+/*
+ * 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 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.checkerframework.checker.nullness.qual.Nullable;
+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 RecordWithMetadata RecordWithMetadata},
+ * 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 file:
+ * PCollection<RecordWithMetadata> 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<RecordWithMetadata> 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<RecordWithMetadata> 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<RecordWithMetadata> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasMultilineCSVRecords(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<RecordWithMetadata> 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#withHasMultilineCSVRecords(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 RecordWithMetadata}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setWithoutLineNumMetadata(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasMultilineCSVRecords(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)
+        .setHasMultilineCSVRecords(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<RecordWithMetadata>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract boolean getWithoutLineNumMetadata();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasMultilineCSVRecords();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract byte @Nullable [] 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 setWithoutLineNumMetadata(boolean withoutLineNumMetadata);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte @Nullable [] delimiter);
+
+      abstract Builder setHasMultilineCSVRecords(Boolean hasMultilineCSVRecords);
+
+      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 withHasMultilineCSVRecords(Boolean hasMultilineCSVRecords) {
+      return toBuilder().setHasMultilineCSVRecords(hasMultilineCSVRecords).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();
+    }
+
+    /**
+     * Allows the user the opt out of getting recordNums associated with each record.
+     *
+     * <p>ContextualTextIO uses a shuffle step to assemble the recordNums for each record which may

Review comment:
       I think it would be better to add something like, this will introduce a shuffle step, which will increase the resources used by the pipeline.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java
##########
@@ -0,0 +1,594 @@
+/*
+ * 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 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.checkerframework.checker.nullness.qual.Nullable;
+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 RecordWithMetadata RecordWithMetadata},
+ * 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 file:
+ * PCollection<RecordWithMetadata> 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<RecordWithMetadata> 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<RecordWithMetadata> 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

Review comment:
       Show example of what the field may look like, I think the example from the RFC is fine.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOSource.java
##########
@@ -0,0 +1,363 @@
+/*
+ * 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 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;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * 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<RecordWithMetadata> {
+  byte[] delimiter;
+
+  // Used to Override isSplittable
+  private boolean hasMultilineCSVRecords;
+
+  @Override
+  protected boolean isSplittable() throws Exception {
+    if (hasMultilineCSVRecords) {

Review comment:
       Create comment explaing why we return false 

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java
##########
@@ -0,0 +1,594 @@
+/*
+ * 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 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.checkerframework.checker.nullness.qual.Nullable;
+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 RecordWithMetadata RecordWithMetadata},
+ * each corresponding to one line of an inout UTF-8 text file (split into lines delimited by '\n',

Review comment:
       sorry not sure if inout is a typo?

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java
##########
@@ -0,0 +1,594 @@
+/*
+ * 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 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.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
+
+/**
+ * {@link PTransform}s that read text files and collect contextual information of the elements in

Review comment:
       Add note that ' if contextual information or multiline file reading  is not needed, then make use of TextIO'

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java
##########
@@ -0,0 +1,594 @@
+/*
+ * 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 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.checkerframework.checker.nullness.qual.Nullable;
+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 RecordWithMetadata RecordWithMetadata},
+ * 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 file:
+ * PCollection<RecordWithMetadata> 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<RecordWithMetadata> 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<RecordWithMetadata> 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<RecordWithMetadata> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasMultilineCSVRecords(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<RecordWithMetadata> 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#withHasMultilineCSVRecords(Boolean)} introduces a

Review comment:
       maybe change to : 
   when this option is used a single reader will be used to process the file, rather than multiple readers which can read from different offsets. For a large file this can result in lower performance.

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java
##########
@@ -0,0 +1,594 @@
+/*
+ * 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 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.checkerframework.checker.nullness.qual.Nullable;
+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 RecordWithMetadata RecordWithMetadata},
+ * 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 file:
+ * PCollection<RecordWithMetadata> 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<RecordWithMetadata> 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<RecordWithMetadata> 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<RecordWithMetadata> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasMultilineCSVRecords(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<RecordWithMetadata> 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#withHasMultilineCSVRecords(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

Review comment:
       explain the other config options used in the example, WithoutLineNumMetadata etc... Also as we have changed from lineContext to Record Contexr should this now be WithoutRecordMetadata?
   
   @tvalentyn fyi

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java
##########
@@ -0,0 +1,594 @@
+/*
+ * 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 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.checkerframework.checker.nullness.qual.Nullable;
+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 RecordWithMetadata RecordWithMetadata},
+ * 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 file:
+ * PCollection<RecordWithMetadata> 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<RecordWithMetadata> 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<RecordWithMetadata> 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<RecordWithMetadata> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasMultilineCSVRecords(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<RecordWithMetadata> 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#withHasMultilineCSVRecords(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 RecordWithMetadata}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setWithoutLineNumMetadata(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasMultilineCSVRecords(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() {

Review comment:
       If the default is false, do we need to set it for every call?

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOSource.java
##########
@@ -0,0 +1,363 @@
+/*
+ * 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 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;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * 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<RecordWithMetadata> {
+  byte[] delimiter;
+
+  // Used to Override isSplittable
+  private boolean hasMultilineCSVRecords;
+
+  @Override
+  protected boolean isSplittable() throws Exception {
+    if (hasMultilineCSVRecords) {
+      return false;
+    }
+    return super.isSplittable();
+  }
+
+  ContextualTextIOSource(
+      ValueProvider<String> fileSpec,
+      EmptyMatchTreatment emptyMatchTreatment,
+      byte[] delimiter,
+      boolean hasMultilineCSVRecords) {
+    super(fileSpec, emptyMatchTreatment, 1L);
+    this.delimiter = delimiter;
+    this.hasMultilineCSVRecords = hasMultilineCSVRecords;
+  }
+
+  private ContextualTextIOSource(
+      MatchResult.Metadata metadata,
+      long start,
+      long end,
+      byte[] delimiter,
+      boolean hasMultilineCSVRecords) {
+    super(metadata, 1L, start, end);
+    this.delimiter = delimiter;
+    this.hasMultilineCSVRecords = hasMultilineCSVRecords;
+  }
+
+  @Override
+  protected FileBasedSource<RecordWithMetadata> createForSubrangeOfFile(
+      MatchResult.Metadata metadata, long start, long end) {
+    return new ContextualTextIOSource(metadata, start, end, delimiter, hasMultilineCSVRecords);
+  }
+
+  @Override
+  protected FileBasedReader<RecordWithMetadata> createSingleFileReader(PipelineOptions options) {
+    return new MultiLineTextBasedReader(this, delimiter, hasMultilineCSVRecords);
+  }
+
+  @Override
+  public Coder<RecordWithMetadata> getOutputCoder() {
+    SchemaCoder<RecordWithMetadata> coder = null;
+    try {
+      coder = SchemaRegistry.createDefault().getSchemaCoder(RecordWithMetadata.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<RecordWithMetadata> {
+    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 RecordWithMetadata currentValue;
+    private @Nullable ReadableByteChannel inChannel;
+    private byte @Nullable [] 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 RecordWithMetadata 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#hasMultilineCSVRecords} 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#hasMultilineCSVRecords} 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++;
+      // The Complete FileName (with uri if this is a web url eg: temp/abc.txt) is:
+      String fileName = getCurrentSource().getSingleFileMetadata().resourceId().toString();

Review comment:
       @tvalentyn @abhiy13 what do you think of having a getUri field and a getFileName field, where the filename is just the filename,
   
   My concern is that if we have a very long URI with a file that contains a couple of cols of ints, we are massively expanding the size of the output. In the future having a id + sideinput with the filenames would solve this issue better. But this would be a nice way to reduce this problem? 

##########
File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java
##########
@@ -0,0 +1,594 @@
+/*
+ * 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 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.checkerframework.checker.nullness.qual.Nullable;
+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 RecordWithMetadata RecordWithMetadata},
+ * 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 file:
+ * PCollection<RecordWithMetadata> 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<RecordWithMetadata> 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<RecordWithMetadata> 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<RecordWithMetadata> lines = p.apply(ContextualTextIO.read()
+ *     .from("/local/path/to/files/*.csv")
+ *      .withHasMultilineCSVRecords(true));
+ * }</pre>
+ *
+ * <p>Example 5: reading while watching for new files
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<RecordWithMetadata> 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#withHasMultilineCSVRecords(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 RecordWithMetadata}element for each line of the input files.
+   */
+  public static Read read() {
+    return new AutoValue_ContextualTextIO_Read.Builder()
+        .setCompression(Compression.AUTO)
+        .setHintMatchesManyFiles(false)
+        .setWithoutLineNumMetadata(false)
+        .setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.DISALLOW))
+        .setHasMultilineCSVRecords(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)
+        .setHasMultilineCSVRecords(false)
+        .build();
+  }
+
+  /** Implementation of {@link #read}. */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<RecordWithMetadata>> {
+    abstract @Nullable ValueProvider<String> getFilepattern();
+
+    abstract MatchConfiguration getMatchConfiguration();
+
+    abstract boolean getHintMatchesManyFiles();
+
+    abstract boolean getWithoutLineNumMetadata();
+
+    abstract Compression getCompression();
+
+    abstract @Nullable Boolean getHasMultilineCSVRecords();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract byte @Nullable [] 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 setWithoutLineNumMetadata(boolean withoutLineNumMetadata);
+
+      abstract Builder setCompression(Compression compression);
+
+      abstract Builder setDelimiter(byte @Nullable [] delimiter);
+
+      abstract Builder setHasMultilineCSVRecords(Boolean hasMultilineCSVRecords);
+
+      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 withHasMultilineCSVRecords(Boolean hasMultilineCSVRecords) {
+      return toBuilder().setHasMultilineCSVRecords(hasMultilineCSVRecords).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();
+    }
+
+    /**
+     * Allows the user the opt out of getting recordNums associated with each record.
+     *
+     * <p>ContextualTextIO uses a shuffle step to assemble the recordNums for each record which may
+     * result in some performance loss.
+     *
+     * <p>Use this when metadata like fileNames are required and their position/order can be
+     * ignored.
+     */
+    public Read withoutLineNumMetadata() {
+      return toBuilder().setWithoutLineNumMetadata(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();
+    }
+
+    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<RecordWithMetadata> expand(PBegin input) {
+      checkNotNull(
+          getFilepattern(), "need to set the filepattern of a ContextualTextIO.Read transform");
+      PCollection<RecordWithMetadata> 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()));
+      }
+
+      // Check if the user decided to opt out of recordNums associated with records
+      if (getWithoutLineNumMetadata()) {
+        return lines;
+      }
+
+      // At this point the line number in RecordWithMetadata 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.
+
+      PCollection<KV<KV<String, Long>, RecordWithMetadata>> linesGroupedByFileAndRange =
+          lines.apply("AddFileNameAndRange", ParDo.of(new AddFileNameAndRange()));
+
+      PCollectionView<Map<KV<String, Long>, Long>> rangeSizes =
+          linesGroupedByFileAndRange
+              .apply("CountLinesForEachFileRange", Count.perKey())
+              .apply("SizesAsView", View.asMap());
+
+      // Get Pipeline to create a dummy PCollection with one element to help compute the lines
+      // before each Range
+      PCollection<Integer> singletonPcoll =
+          input.getPipeline().apply("CreateSingletonPcoll", Create.of(Arrays.asList(1)));
+
+      // For each (File, Offset) 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>> numLinesBeforeEachRange =
+          singletonPcoll
+              .apply(
+                  "ComputeLinesBeforeRange",
+                  ParDo.of(new ComputeLinesBeforeEachRange(rangeSizes)).withSideInputs(rangeSizes))
+              .apply("NumLinesBeforeEachRangeAsView", View.asMap());
+
+      return linesGroupedByFileAndRange.apply(
+          "AssignLineNums",
+          ParDo.of(new AssignLineNums(numLinesBeforeEachRange))
+              .withSideInputs(numLinesBeforeEachRange));
+    }
+
+    @VisibleForTesting
+    static class AddFileNameAndRange
+        extends DoFn<RecordWithMetadata, KV<KV<String, Long>, RecordWithMetadata>> {
+      @ProcessElement
+      public void processElement(
+          @Element RecordWithMetadata line,
+          OutputReceiver<KV<KV<String, Long>, RecordWithMetadata>> out) {
+        out.output(KV.of(KV.of(line.getFileName(), line.getRange().getRangeNum()), line));
+      }
+    }
+
+    /**
+     * Helper class for computing number of lines in the File preceding the beginning of the Range
+     * in this file.
+     */
+    @VisibleForTesting
+    static class ComputeLinesBeforeEachRange extends DoFn<Integer, KV<KV<String, Long>, Long>> {
+      private final PCollectionView<Map<KV<String, Long>, Long>> rangeSizes;
+
+      public ComputeLinesBeforeEachRange(PCollectionView<Map<KV<String, Long>, Long>> rangeSizes) {
+        this.rangeSizes = rangeSizes;
+      }
+
+      // 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> rangeSizesMap = p.sideInput(rangeSizes);
+
+        // 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 : rangeSizesMap.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);
+        }
+      }
+    }
+
+    static class AssignLineNums
+        extends DoFn<KV<KV<String, Long>, RecordWithMetadata>, RecordWithMetadata> {
+      PCollectionView<Map<KV<String, Long>, Long>> numLinesBeforeEachRange;
+
+      public AssignLineNums(PCollectionView<Map<KV<String, Long>, Long>> numLinesBeforeEachRange) {
+        this.numLinesBeforeEachRange = numLinesBeforeEachRange;
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext p) {
+        Long range = p.element().getKey().getValue();
+        String file = p.element().getKey().getKey();
+        RecordWithMetadata line = p.element().getValue();
+        Long linesLessThanThisRange = p.sideInput(numLinesBeforeEachRange).get(KV.of(file, range));
+        RecordWithMetadata newLine =
+            RecordWithMetadata.newBuilder()
+                .setRecordValue(line.getRecordValue())
+                .setRecordNum(line.getRange().getRangeLineNum() + linesLessThanThisRange)
+                .setFileName(line.getFileName())
+                .setRange(line.getRange())
+                .build();
+        p.output(newLine);
+      }
+    }
+
+    // Helper to create a source specific to the requested compression type.
+    protected FileBasedSource<RecordWithMetadata> getSource() {
+      return CompressedSource.from(
+              new ContextualTextIOSource(
+                  getFilepattern(),
+                  getMatchConfiguration().getEmptyMatchTreatment(),
+                  getDelimiter(),
+                  getHasMultilineCSVRecords()))
+          .withCompression(getCompression());
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      super.populateDisplayData(builder);
+      builder
+          .add(
+              DisplayData.item("compressionType", getCompression().toString())
+                  .withLabel("Compression Type"))
+          .addIfNotNull(DisplayData.item("filePattern", getFilepattern()).withLabel("File Pattern"))
+          .include("matchConfiguration", getMatchConfiguration())
+          .addIfNotNull(
+              DisplayData.item("delimiter", Arrays.toString(getDelimiter()))
+                  .withLabel("Custom delimiter to split records"))
+          .addIfNotNull(
+              DisplayData.item("hasMultilineCSVRecords", getHasMultilineCSVRecords())
+                  .withLabel("Has RFC4180 MultiLineCSV Records"));
+    }
+  }
+
+  /** Implementation of {@link #readFiles}. */
+  @AutoValue
+  public abstract static class ReadFiles
+      extends PTransform<PCollection<FileIO.ReadableFile>, PCollection<RecordWithMetadata>> {
+    abstract long getDesiredBundleSizeBytes();
+
+    @SuppressWarnings("mutable") // this returns an array that can be mutated by the caller
+    abstract byte @Nullable [] getDelimiter();
+
+    abstract boolean getHasMultilineCSVRecords();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setDesiredBundleSizeBytes(long desiredBundleSizeBytes);
+
+      abstract Builder setHasMultilineCSVRecords(boolean hasMultilineCSVRecords);
+
+      abstract Builder setDelimiter(byte @Nullable [] delimiter);
+
+      abstract ReadFiles build();
+    }
+
+    @VisibleForTesting
+    ReadFiles withDesiredBundleSizeBytes(long desiredBundleSizeBytes) {
+      return toBuilder().setDesiredBundleSizeBytes(desiredBundleSizeBytes).build();
+    }
+
+    /** Like {@link Read#withDelimiter}. */
+    public ReadFiles withDelimiter(byte[] delimiter) {
+      return toBuilder().setDelimiter(delimiter).build();
+    }
+
+    @Override
+    public PCollection<RecordWithMetadata> expand(PCollection<FileIO.ReadableFile> input) {
+      SchemaCoder<RecordWithMetadata> coder = null;
+      try {
+        coder = input.getPipeline().getSchemaRegistry().getSchemaCoder(RecordWithMetadata.class);
+      } catch (NoSuchSchemaException e) {
+        System.out.println("No Coder!");

Review comment:
       Need to use LOGGER




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