-
Notifications
You must be signed in to change notification settings - Fork 4.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Add header removal for TextIO #29202
Changes from 17 commits
1d917d8
e3e547d
63f3dec
c4f3d6c
79015dc
ae385f9
85034e4
a647724
2153404
20aeb2c
d4755d6
c29672c
d9499b2
6ef369b
e8db3e7
fbe5f5f
f3b61fe
7903169
5f99fbf
62baa30
edc8a75
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -56,26 +56,43 @@ | |
public class TextSource extends FileBasedSource<String> { | ||
byte[] delimiter; | ||
|
||
int skipHeaderLines; | ||
|
||
public TextSource( | ||
ValueProvider<String> fileSpec, EmptyMatchTreatment emptyMatchTreatment, byte[] delimiter) { | ||
ValueProvider<String> fileSpec, | ||
EmptyMatchTreatment emptyMatchTreatment, | ||
byte[] delimiter, | ||
int skipHeaderLines) { | ||
super(fileSpec, emptyMatchTreatment, 1L); | ||
this.delimiter = delimiter; | ||
this.skipHeaderLines = skipHeaderLines; | ||
} | ||
|
||
public TextSource(MatchResult.Metadata metadata, long start, long end, byte[] delimiter) { | ||
public TextSource( | ||
ValueProvider<String> fileSpec, EmptyMatchTreatment emptyMatchTreatment, byte[] delimiter) { | ||
this(fileSpec, emptyMatchTreatment, delimiter, 0); | ||
} | ||
|
||
public TextSource( | ||
MatchResult.Metadata metadata, long start, long end, byte[] delimiter, int skipHeaderLines) { | ||
super(metadata, 1L, start, end); | ||
this.delimiter = delimiter; | ||
this.skipHeaderLines = skipHeaderLines; | ||
} | ||
|
||
public TextSource(MatchResult.Metadata metadata, long start, long end, byte[] delimiter) { | ||
this(metadata, start, end, delimiter, 0); | ||
} | ||
|
||
@Override | ||
protected FileBasedSource<String> createForSubrangeOfFile( | ||
MatchResult.Metadata metadata, long start, long end) { | ||
return new TextSource(metadata, start, end, delimiter); | ||
return new TextSource(metadata, start, end, delimiter, skipHeaderLines); | ||
} | ||
|
||
@Override | ||
protected FileBasedReader<String> createSingleFileReader(PipelineOptions options) { | ||
return new TextBasedReader(this, delimiter); | ||
return new TextBasedReader(this, delimiter, skipHeaderLines); | ||
} | ||
|
||
@Override | ||
|
@@ -98,6 +115,7 @@ static class TextBasedReader extends FileBasedReader<String> { | |
private static final byte LF = '\n'; | ||
|
||
private final byte @Nullable [] delimiter; | ||
private final int skipHeaderLines; | ||
private final ByteArrayOutputStream str; | ||
private final byte[] buffer; | ||
private final ByteBuffer byteBuffer; | ||
|
@@ -112,11 +130,16 @@ static class TextBasedReader extends FileBasedReader<String> { | |
private boolean skipLineFeedAtStart; // skip an LF if at the start of the next buffer | ||
|
||
private TextBasedReader(TextSource source, byte[] delimiter) { | ||
this(source, delimiter, 0); | ||
} | ||
|
||
private TextBasedReader(TextSource source, byte[] delimiter, int skipHeaderLines) { | ||
super(source); | ||
this.buffer = new byte[READ_BUFFER_SIZE]; | ||
this.str = new ByteArrayOutputStream(); | ||
this.byteBuffer = ByteBuffer.wrap(buffer); | ||
this.delimiter = delimiter; | ||
this.skipHeaderLines = skipHeaderLines; | ||
} | ||
|
||
@Override | ||
|
@@ -171,21 +194,42 @@ protected void startReading(ReadableByteChannel channel) throws IOException { | |
} else { | ||
startOfNextRecord = bufferPosn = (int) requiredPosition; | ||
} | ||
skipHeader(skipHeaderLines, true); | ||
} else { | ||
((SeekableByteChannel) channel).position(requiredPosition); | ||
startOfNextRecord = requiredPosition; | ||
skipHeader(skipHeaderLines, false); | ||
if (requiredPosition > startOfNextRecord) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am wondering what will happen if multiple readers having requiredPosition <= startOfNextRecord here. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. good point There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nvm. In this case, among those readers, only one will have an end position outside of the header lines, and will continue to read. So this should not be an issue then. |
||
((SeekableByteChannel) channel).position(requiredPosition); | ||
startOfNextRecord = requiredPosition; | ||
bufferLength = bufferPosn = 0; | ||
// Read and discard the next record ensuring that startOfNextRecord and bufferPosn point | ||
// to the beginning of the next record. | ||
readNextRecord(); | ||
currentValue = null; | ||
} | ||
} | ||
|
||
// Read and discard the next record ensuring that startOfNextRecord and bufferPosn point | ||
// to the beginning of the next record. | ||
readNextRecord(); | ||
currentValue = null; | ||
} else { | ||
// Check to see if we start with the UTF_BOM bytes skipping them if present. | ||
if (fileStartsWithBom()) { | ||
startOfNextRecord = bufferPosn = UTF8_BOM.size(); | ||
} | ||
skipHeader(skipHeaderLines, false); | ||
} | ||
} | ||
|
||
private void skipHeader(int headerLines, boolean skipFirstLine) throws IOException { | ||
if (headerLines == 1) { | ||
readNextRecord(); | ||
} else if (headerLines > 1) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could you also add a test to cover the skipping of multiple header lines? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ok |
||
// this will be expensive | ||
((SeekableByteChannel) inChannel).position(0); | ||
for (int line = 0; line < headerLines; ++line) { | ||
readNextRecord(); | ||
} | ||
} else if (headerLines == 0 && skipFirstLine) { | ||
readNextRecord(); | ||
} | ||
currentValue = null; | ||
} | ||
|
||
private boolean fileStartsWithBom() throws IOException { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this would go to beam 2.53.0. Also, link to the GitHub issue
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please append the the original issue link to this line, like
TextIO
supports skipping header. (Java)(#17990).