abhiy13 commented on a change in pull request #12645: URL: https://github.com/apache/beam/pull/12645#discussion_r492474704
########## File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOSource.java ########## @@ -0,0 +1,364 @@ +/* + * 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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 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; + + private static final Logger LOG = LoggerFactory.getLogger(ContextualTextIOSource.class); + + // Used to Override isSplittable + private boolean hasMultilineCSVRecords; + + @Override + protected boolean isSplittable() throws Exception { + if (hasMultilineCSVRecords) { + // When Having Multiline CSV Records, + // Splitting the file may cause a split to be within a record, + // Disabling split prevents this from happening + 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) { + LOG.error("No Coder Found for RecordWithMetadata"); + } + 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 hasMultilineCSVRecords; + + private long startingOffset; + private long totalRecordCount; + + private MultiLineTextBasedReader( + ContextualTextIOSource source, byte[] delimiter, boolean hasMultilineCSVRecords) { + super(source); + buffer = ByteString.EMPTY; + this.delimiter = delimiter; + this.hasMultilineCSVRecords = hasMultilineCSVRecords; + 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 (hasMultilineCSVRecords) { + // 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()); + } + + // The line num is: + Long recordUniqueNum = totalRecordCount++; + // The Complete FileName (with uri if this is a web url eg: temp/abc.txt) is: + String fileName = getCurrentSource().getSingleFileMetadata().resourceId().toString(); Review comment: @rezarokni thanks ! I was facing this same issue that is why could not make it part of the previous commit. FileName does seem to be the best option for now. ########## File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/RecordWithMetadata.java ########## @@ -0,0 +1,85 @@ +/* + * 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.io.fs.ResourceId; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; + +/** + * Helper Class based on {@link AutoValueSchema}, it provides Metadata associated with each Record + * when reading from file(s) using {@link ContextualTextIO}. + * + * <h3>Fields:</h3> + * + * <ul> + * <li>recordOffset: The offset of a record (the byte at which the record begins) in a file. This + * information can be useful if you wish to reconstruct the file. {@link + * RecordWithMetadata#getRecordOffset()} + * <li>recordNum: The ordinal number of the record in its file. {@link + * RecordWithMetadata#getRecordNum()} + * <li>recordValue: The value / contents of the record {@link RecordWithMetadata#getValue()} + * <li>rangeOffset: The starting offset of the range (split), which contained the record, when the + * record was read. {@link RecordWithMetadata#getRangeOffset()} + * <li>recordNumInOffset: The record number relative to the Range. (line number within the range) + * {@link RecordWithMetadata#getRecordNumInOffset()} + * <li>fileName: Name of the file to which the record belongs (this is the full filename, + * eg:path/to/file.txt) {@link RecordWithMetadata#getFileName()} + * </ul> + */ +@Experimental(Experimental.Kind.SCHEMAS) +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class RecordWithMetadata { + public abstract long getRecordOffset(); + + public abstract long getRecordNum(); + + public abstract String getValue(); + + public abstract long getRangeOffset(); + + public abstract long getRecordNumInOffset(); + + public abstract Builder toBuilder(); + + public abstract String getFileName(); + + public static Builder newBuilder() { + return new AutoValue_RecordWithMetadata.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setRecordNum(long lineNum); + + public abstract Builder setRecordOffset(long recordOffset); + + public abstract Builder setValue(String Value); Review comment: Done. ########## File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java ########## @@ -0,0 +1,631 @@ +/* + * 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.TextIO; +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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link PTransform}s that read text files and collect contextual information of the elements in + * the input. + * + * <p>Use {@link TextIO} when not reading file with Multiline Records or additional metadata is not + * required. + * + * <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 input 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> records = 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> records = + * 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> records = 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. + * + * Example of such a file could be: + * + * "aaa","b CRLF + * bb","ccc" CRLF + * zzz,yyy,xxx + * + * <pre>{@code + * Pipeline p = ...; + * + * PCollection<RecordWithMetadata> records = 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> records = p.apply(FileIO.match() + * .filepattern("filepattern") + * .continuously( + * Duration.millis(100), + * Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3)))) + * .apply(FileIO.readMatches()) + * .apply(ContextualTextIO.readFiles()); + * }</pre> + * + * <p>Example 6: reading without recordNum metadata, or only fileName associated Metadata. (the + * Objects would still contain recordNums, but these recordNums would correspond to their positions + * in their respective offsets rather than their positions within the entire file). + * + * <pre>{@code + * Pipeline p = ...; + * + * PCollection<RecordWithMetadata> records = p.apply(ContextualTextIO.read() + * .from("/local/path/to/files/*.csv") + * .setWithoutRecordNumMetadata(true)); + * }</pre> + * + * <p>NOTE: When using {@link ContextualTextIO.Read#withHasMultilineCSVRecords(Boolean)} this + * option, 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. + * + * <p>NOTE: Use {@link Read#withoutRecordNumMetadata()} when recordNum metadata is not required, for + * example, when when only filename metadata is required. Computing record positions currently + * introduces a shuffle step, which increases the resources used by the pipeline. <b> By default + * withoutRecordNumMetadata is set to false, so the shuffle step is performed.</b> + * + * <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; + private static final Logger LOG = LoggerFactory.getLogger(ContextualTextIO.class); + + /** + * 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) + .setWithoutRecordNumMetadata(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 getWithoutRecordNumMetadata(); + + 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 setWithoutRecordNumMetadata(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 to opt out of getting recordNums associated with each record. + * + * <p>When set to true, it will introduce a shuffle step to assemble the recordNums for each + * record, which will increase the resources used by the pipeline. + * + * <p>Use this when metadata like fileNames are required and their position/order can be + * ignored. + */ + public Read withoutRecordNumMetadata() { + return toBuilder().setWithoutRecordNumMetadata(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> records = null; + if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) { + records = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource())); + } else { + // All other cases go through FileIO + ReadFiles + records = + 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 (getWithoutRecordNumMetadata()) { + return records; + } + + /* + * 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>> recordsGroupedByFileAndRange = + records.apply("AddFileNameAndRange", ParDo.of(new AddFileNameAndRange())); + + PCollectionView<Map<KV<String, Long>, Long>> rangeSizes = + recordsGroupedByFileAndRange + .apply("CountRecordsForEachFileRange", 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>> numRecordsBeforeEachRange = + singletonPcoll + .apply( + "ComputeRecordsBeforeRange", + ParDo.of(new ComputeRecordsBeforeEachRange(rangeSizes)) + .withSideInputs(rangeSizes)) + .apply("NumRecordsBeforeEachRangeAsView", View.asMap()); + + return recordsGroupedByFileAndRange.apply( + "AssignLineNums", + ParDo.of(new AssignRecordNums(numRecordsBeforeEachRange)) + .withSideInputs(numRecordsBeforeEachRange)); + } + + @VisibleForTesting + static class AddFileNameAndRange + extends DoFn<RecordWithMetadata, KV<KV<String, Long>, RecordWithMetadata>> { + @ProcessElement + public void processElement( + @Element RecordWithMetadata record, + OutputReceiver<KV<KV<String, Long>, RecordWithMetadata>> out) { + out.output(KV.of(KV.of(record.getFileName().toString(), record.getRangeOffset()), record)); + } + } + + /** + * Helper class for computing number of record in the File preceding the beginning of the Range + * in this file. + */ + @VisibleForTesting + static class ComputeRecordsBeforeEachRange extends DoFn<Integer, KV<KV<String, Long>, Long>> { + private final PCollectionView<Map<KV<String, Long>, Long>> rangeSizes; + + public ComputeRecordsBeforeEachRange( + 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 number of records passed for each file + Map<String, Long> pastRecords = new HashMap<>(); + + // For each (File, Range) Pair, compute the number of records before it + for (Map.Entry entry : sorted.entrySet()) { Review comment: Done ########## File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java ########## @@ -0,0 +1,631 @@ +/* + * 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.TextIO; +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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link PTransform}s that read text files and collect contextual information of the elements in + * the input. + * + * <p>Use {@link TextIO} when not reading file with Multiline Records or additional metadata is not + * required. + * + * <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 input 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> records = 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> records = + * 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> records = 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. + * + * Example of such a file could be: + * + * "aaa","b CRLF + * bb","ccc" CRLF + * zzz,yyy,xxx + * + * <pre>{@code + * Pipeline p = ...; + * + * PCollection<RecordWithMetadata> records = 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> records = p.apply(FileIO.match() + * .filepattern("filepattern") + * .continuously( + * Duration.millis(100), + * Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3)))) + * .apply(FileIO.readMatches()) + * .apply(ContextualTextIO.readFiles()); + * }</pre> + * + * <p>Example 6: reading without recordNum metadata, or only fileName associated Metadata. (the + * Objects would still contain recordNums, but these recordNums would correspond to their positions + * in their respective offsets rather than their positions within the entire file). + * + * <pre>{@code + * Pipeline p = ...; + * + * PCollection<RecordWithMetadata> records = p.apply(ContextualTextIO.read() + * .from("/local/path/to/files/*.csv") + * .setWithoutRecordNumMetadata(true)); + * }</pre> + * + * <p>NOTE: When using {@link ContextualTextIO.Read#withHasMultilineCSVRecords(Boolean)} this + * option, 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. + * + * <p>NOTE: Use {@link Read#withoutRecordNumMetadata()} when recordNum metadata is not required, for + * example, when when only filename metadata is required. Computing record positions currently + * introduces a shuffle step, which increases the resources used by the pipeline. <b> By default + * withoutRecordNumMetadata is set to false, so the shuffle step is performed.</b> + * + * <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; + private static final Logger LOG = LoggerFactory.getLogger(ContextualTextIO.class); + + /** + * 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) + .setWithoutRecordNumMetadata(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 getWithoutRecordNumMetadata(); + + 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 setWithoutRecordNumMetadata(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 to opt out of getting recordNums associated with each record. + * + * <p>When set to true, it will introduce a shuffle step to assemble the recordNums for each + * record, which will increase the resources used by the pipeline. + * + * <p>Use this when metadata like fileNames are required and their position/order can be + * ignored. + */ + public Read withoutRecordNumMetadata() { + return toBuilder().setWithoutRecordNumMetadata(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> records = null; + if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) { + records = input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource())); + } else { + // All other cases go through FileIO + ReadFiles + records = + 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 (getWithoutRecordNumMetadata()) { + return records; + } + + /* + * 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>> recordsGroupedByFileAndRange = + records.apply("AddFileNameAndRange", ParDo.of(new AddFileNameAndRange())); + + PCollectionView<Map<KV<String, Long>, Long>> rangeSizes = + recordsGroupedByFileAndRange + .apply("CountRecordsForEachFileRange", 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>> numRecordsBeforeEachRange = + singletonPcoll + .apply( + "ComputeRecordsBeforeRange", + ParDo.of(new ComputeRecordsBeforeEachRange(rangeSizes)) + .withSideInputs(rangeSizes)) + .apply("NumRecordsBeforeEachRangeAsView", View.asMap()); + + return recordsGroupedByFileAndRange.apply( + "AssignLineNums", + ParDo.of(new AssignRecordNums(numRecordsBeforeEachRange)) + .withSideInputs(numRecordsBeforeEachRange)); + } + + @VisibleForTesting + static class AddFileNameAndRange + extends DoFn<RecordWithMetadata, KV<KV<String, Long>, RecordWithMetadata>> { + @ProcessElement + public void processElement( + @Element RecordWithMetadata record, + OutputReceiver<KV<KV<String, Long>, RecordWithMetadata>> out) { + out.output(KV.of(KV.of(record.getFileName().toString(), record.getRangeOffset()), record)); + } + } + + /** + * Helper class for computing number of record in the File preceding the beginning of the Range + * in this file. + */ + @VisibleForTesting + static class ComputeRecordsBeforeEachRange extends DoFn<Integer, KV<KV<String, Long>, Long>> { + private final PCollectionView<Map<KV<String, Long>, Long>> rangeSizes; + + public ComputeRecordsBeforeEachRange( + 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 number of records passed for each file + Map<String, Long> pastRecords = new HashMap<>(); + + // For each (File, Range) Pair, compute the number of records before it + for (Map.Entry entry : sorted.entrySet()) { + Long numRecords = (long) entry.getValue(); + KV<String, Long> fileRange = (KV<String, Long>) entry.getKey(); + String file = fileRange.getKey(); + Long numRecordsBefore = 0L; + if (pastRecords.containsKey(file)) { + numRecordsBefore = pastRecords.get(file); + } + p.output(KV.of(fileRange, numRecordsBefore)); + pastRecords.put(file, numRecordsBefore + numRecords); + } + } + } + + static class AssignRecordNums + extends DoFn<KV<KV<String, Long>, RecordWithMetadata>, RecordWithMetadata> { + PCollectionView<Map<KV<String, Long>, Long>> numRecordsBeforeEachRange; + + public AssignRecordNums( + PCollectionView<Map<KV<String, Long>, Long>> numRecordsBeforeEachRange) { + this.numRecordsBeforeEachRange = numRecordsBeforeEachRange; + } + + @ProcessElement + public void processElement(ProcessContext p) { + Long range = p.element().getKey().getValue(); + String file = p.element().getKey().getKey(); + RecordWithMetadata record = p.element().getValue(); + Long numRecordsLessThanThisRange = + p.sideInput(numRecordsBeforeEachRange).get(KV.of(file, range)); + RecordWithMetadata newLine = + RecordWithMetadata.newBuilder() Review comment: Thanks ! Done. ########## File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/RecordWithMetadata.java ########## @@ -0,0 +1,84 @@ +/* + * 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.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; + +/** + * Helper Class based on {@link AutoValueSchema}, it provides Metadata associated with each Record + * when reading from file(s) using {@link ContextualTextIO}. + * + * <h3>Fields:</h3> + * + * <ul> + * <li>recordOffset: The offset of a record (the byte at which the record begins) in a file. This + * information can be useful if you wish to reconstruct the file. {@link + * RecordWithMetadata#getRecordOffset()} + * <li>recordNum: The ordinal number of the record in its file. {@link + * RecordWithMetadata#getRecordNum()} + * <li>recordValue: The value / contents of the record {@link RecordWithMetadata#getRecordValue()} + * <li>rangeOffset: The starting offset of the range (split), which contained the record, when the + * record was read. {@link RecordWithMetadata#getRangeOffset()} + * <li>recordNumInOffset: The record number relative to the Range. (line number within the range) + * {@link RecordWithMetadata#getRecordNumInOffset()} + * <li>fileName: Name of the file to which the record belongs (this is the full filename, + * eg:path/to/file.txt) {@link RecordWithMetadata#getFileName()} + * </ul> + */ +@Experimental(Experimental.Kind.SCHEMAS) +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class RecordWithMetadata { + public abstract Long getRecordOffset(); Review comment: Done ########## File path: sdks/java/io/contextual-text-io/build.gradle ########## @@ -0,0 +1,41 @@ +/* + * 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', + enableChecker: false, + ignoreRawtypeErrors: true) + +description = "Apache Beam :: SDKs :: Java :: Contextual-Text-IO" +ext.summary = "Context-aware Text IO." + +dependencies { + + compile library.java.vendored_guava_26_0_jre + compile library.java.protobuf_java + compile project(path: ":sdks:java:core", configuration: "shadow") + testCompile project(path: ":sdks:java:core", configuration: "shadowTest") + + testCompile library.java.guava_testlib + testCompile library.java.junit + testCompile library.java.hamcrest_core + testRuntimeOnly library.java.slf4j_jdk14 + testCompile project(path: ":runners:direct-java", configuration: "shadow") + +} Review comment: Done ########## File path: sdks/java/io/contextual-text-io/build.gradle ########## @@ -0,0 +1,41 @@ +/* + * 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', + enableChecker: false, + ignoreRawtypeErrors: true) + +description = "Apache Beam :: SDKs :: Java :: Contextual-Text-IO" +ext.summary = "Context-aware Text IO." + +dependencies { + Review comment: Done ########## File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/RecordWithMetadata.java ########## @@ -0,0 +1,85 @@ +/* + * 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.io.fs.ResourceId; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; + +/** + * Helper Class based on {@link AutoValueSchema}, it provides Metadata associated with each Record + * when reading from file(s) using {@link ContextualTextIO}. + * + * <h3>Fields:</h3> + * + * <ul> + * <li>recordOffset: The offset of a record (the byte at which the record begins) in a file. This + * information can be useful if you wish to reconstruct the file. {@link + * RecordWithMetadata#getRecordOffset()} + * <li>recordNum: The ordinal number of the record in its file. {@link + * RecordWithMetadata#getRecordNum()} + * <li>recordValue: The value / contents of the record {@link RecordWithMetadata#getValue()} + * <li>rangeOffset: The starting offset of the range (split), which contained the record, when the + * record was read. {@link RecordWithMetadata#getRangeOffset()} + * <li>recordNumInOffset: The record number relative to the Range. (line number within the range) + * {@link RecordWithMetadata#getRecordNumInOffset()} + * <li>fileName: Name of the file to which the record belongs (this is the full filename, + * eg:path/to/file.txt) {@link RecordWithMetadata#getFileName()} + * </ul> + */ +@Experimental(Experimental.Kind.SCHEMAS) +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class RecordWithMetadata { + public abstract long getRecordOffset(); Review comment: Done ########## File path: sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOSource.java ########## @@ -0,0 +1,364 @@ +/* + * 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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 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; + + private static final Logger LOG = LoggerFactory.getLogger(ContextualTextIOSource.class); + + // Used to Override isSplittable + private boolean hasMultilineCSVRecords; + + @Override + protected boolean isSplittable() throws Exception { + if (hasMultilineCSVRecords) { + // When Having Multiline CSV Records, + // Splitting the file may cause a split to be within a record, + // Disabling split prevents this from happening + 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() { Review comment: Done ---------------------------------------------------------------- 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: [email protected]
