[
https://issues.apache.org/jira/browse/BEAM-10124?focusedWorklogId=473210&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-473210
]
ASF GitHub Bot logged work on BEAM-10124:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 21/Aug/20 05:52
Start Date: 21/Aug/20 05:52
Worklog Time Spent: 10m
Work Description: tvalentyn commented on a change in pull request #12645:
URL: https://github.com/apache/beam/pull/12645#discussion_r474411845
##########
File path:
sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java
##########
@@ -151,8 +157,27 @@
* .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.
+ * <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 or
+ * when only metadata associated with filenames is required, Not using this
option introduces a
+ * shuffle step which increases the resources used by the pipeline. <b>This
option is set to false
+ * by default. Meaning that the shuffle step will be performed, set it to
false to avoid the shuffle
Review comment:
s/false/true
how about:
```
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>
```
##########
File path:
sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/RecordWithMetadata.java
##########
@@ -19,21 +19,45 @@
import com.google.auto.value.AutoValue;
import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.sdk.schemas.AutoValueSchema;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
-@Internal
+/**
+ * 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 record number of the record from its ordinal file.
{@link
Review comment:
The ordinal number of the record in its file.
##########
File path:
sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/RecordWithMetadata.java
##########
@@ -19,21 +19,45 @@
import com.google.auto.value.AutoValue;
import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.sdk.schemas.AutoValueSchema;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
-@Internal
+/**
+ * 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 record number of the record from its ordinal file.
{@link
+ * RecordWithMetadata#getRecordNum()}
+ * <li>recordValue: The value / contents of the records {@link
Review comment:
of the record
##########
File path:
sdks/java/io/contextual-text-io/src/main/java/org/apache/beam/sdk/io/contextualtextio/RecordWithMetadata.java
##########
@@ -19,21 +19,45 @@
import com.google.auto.value.AutoValue;
import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.sdk.schemas.AutoValueSchema;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
-@Internal
+/**
+ * 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 record number of the record from its ordinal file.
{@link
+ * RecordWithMetadata#getRecordNum()}
+ * <li>recordValue: The value / contents of the records {@link
+ * RecordWithMetadata#getRecordValue()}
+ * <li>startingOffset: The starting offset of the range to which the record
belongs. {@link
Review comment:
this name may be confusing with recordOffset.
Consider:
`rangeOffset`
`recordNumInRange`
`The starting offset of the range (split), which contained the record, when
the record was read.`
----------------------------------------------------------------
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]
Issue Time Tracking
-------------------
Worklog Id: (was: 473210)
Remaining Estimate: 1,337h 20m (was: 1,337.5h)
Time Spent: 6h 40m (was: 6.5h)
> ContextualTextIO - An IO that is provides metadata about the line.
> -------------------------------------------------------------------
>
> Key: BEAM-10124
> URL: https://issues.apache.org/jira/browse/BEAM-10124
> Project: Beam
> Issue Type: New Feature
> Components: io-ideas
> Reporter: Reza ardeshir rokni
> Priority: P2
> Original Estimate: 1,344h
> Time Spent: 6h 40m
> Remaining Estimate: 1,337h 20m
>
> There are two important Source IO's that allow for dealing with text files
> FileIO and TextIO. When the requirements go beyond the scope of TextIO we ask
> the end user to make use of FileIO and go it on their own.
> We want to correct this experience by creating a more feature rich TextIO
> which can return along with each line of data contextual information about
> the line. For example the file that it came from, and the ordinal position of
> the line within the file.
> Another area that we would like to deal with is more formal support for CSV
> files by allowing compliance to RFC4180 files. Specifically the RFC allows
> for line breaks (CRLF) to be used within fields within double quotes.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)