-
Notifications
You must be signed in to change notification settings - Fork 2.7k
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
NIFI-12241 Efficient Parquet Splitting #7893
NIFI-12241 Efficient Parquet Splitting #7893
Conversation
Hey @takraj I don't know much of anything about Parquet so I'm probably not the best to really review this in terms of Parquet. But looking at what's happening here, the processor does not split Parquet at all. Instead, it clones the input and adds 'count' and 'offset' types of attributes. So the naming is problematic. If I sent in a 10 GB Parquet file to SplitParquet and I get out 10 FlowFiles, I expect each to be 1 GB. Here, each one will be 10 GB because it's a clone of the original. This would lead to a lot confusion. |
I'll giving this a try later today but yeah, the naming can certainly be better as this new processor should always be followed by a record processor using the ParquetWriter so that the actual "splitting" is happening. Could be good to leverage the new @usecase annotation to describe this. |
@markap14 , @pvillard31 Thank you for your comments. I have added use cases, and renamed the processor in 9d7fd48. Please check again. |
I did the below test:
I was expecting the second case to be much faster given that we use 8 concurrent threads. Any idea? Thoughts? |
|
The CalculateParquetOffsets step is taking less than 1 millisecond to complete: the 50 clones are not copying the data, they get the same reference to actual claim (where the data is stored in the content repo). So I think it's really around the ParquetReader and how we can "jump" to the right location as you said. Will keep an eye out for your improvement for additional testing. |
New tests with 10M records (190MB):
So definitely better. I'm a +1 from a testing pov but would be nice for someone to have a look at the code changes. @bbende maybe? |
@pvillard31 I've been working on further performance improvements in the last couple of days, and created a new variant of this processor, which calculates the offsets of the row group boundaries, and updated the reader to take these offsets and seek onto these positions in the input file. This is achieved with a special configuration option of the Parquet reader, called 'File Range', that selects only row groups that overlap with the specified start and end offset range. Hopefully this brings major improvement. Could you share how did you extract the processing times from NiFi in your benchmarks? Did you simply take the 'Tasks/Time' indicator, or is there something else that I can monitor? |
I have added a commit, which includes the alternative approach, that I have already mentioned in my previous comment. Please check. |
Thanks @takraj - giving another try to it. |
Test with 15,598,440 records (295MB):
It seems to be very similar to the previous test since the time is really related to how many records should be in every split (1M in this case). It's a 4x improvement when optimizing the number of splits versus the number of concurrent tasks so it's definitely a nice improvement. |
@pvillard31 Did you try the new You can also combine the two as I can't really do any further improvement to this, due to API limitations of the underlying library. |
Oh, I'm sorry, I didn't realize it was another processor. Will try again. |
FF with 24M records (452.85MB)
|
@@ -35,11 +35,18 @@ public class AvroParquetHDFSRecordReader implements HDFSRecordReader { | |||
private GenericRecord lastRecord; | |||
private RecordSchema recordSchema; | |||
private boolean initialized = false; | |||
private final Long count; |
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.
private final Long count; | |
private final Long recordsToRead; |
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.
Done in 9f340c9
if (count == null) { | ||
return new AvroParquetHDFSRecordReader(readerBuilder.build()); | ||
} else { | ||
return new AvroParquetHDFSRecordReader(readerBuilder.build(), count); | ||
} |
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.
if (count == null) { | |
return new AvroParquetHDFSRecordReader(readerBuilder.build()); | |
} else { | |
return new AvroParquetHDFSRecordReader(readerBuilder.build(), count); | |
} | |
return new AvroParquetHDFSRecordReader(readerBuilder.build(), count); |
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.
Done in 9f340c9
this(parquetReader, null); | ||
} | ||
|
||
public AvroParquetHDFSRecordReader(final ParquetReader<GenericRecord> parquetReader, Long count) { |
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.
public AvroParquetHDFSRecordReader(final ParquetReader<GenericRecord> parquetReader, Long count) { | |
public AvroParquetHDFSRecordReader(final ParquetReader<GenericRecord> parquetReader, final Long count) { |
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.
Done in 9f340c9
private final ParquetReader<GenericRecord> parquetReader; | ||
|
||
public ParquetRecordReader(final InputStream inputStream, final long inputLength, final Configuration configuration) throws IOException { | ||
private final Long count; |
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.
private final Long count; | |
private final Long recordsToRead; |
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.
Done in 9f340c9
@Override | ||
public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { | ||
final FlowFile original = session.get(); | ||
if (original == null) { |
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.
if (original == null) { | |
if (inputFlowFile == null) { |
"original" makes sense when used as a relationship because that is a very specific context. In the code "original" can mean too many things. Also "input..." describes more accurately what it is. (In contrast, in case of a relationship calling it "input" would be very confusing.)
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.
Done in 9f340c9
|
||
private List<FlowFile> getPartitions( | ||
ProcessSession session, | ||
FlowFile flowFile, |
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.
FlowFile flowFile, | |
FlowFile inputFlowFile, |
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.
Done in 9f340c9
long recordOffset, | ||
boolean zeroContentOutput | ||
) { | ||
final long numberOfPartitions = (recordCount / partitionSize) + ((recordCount % partitionSize) > 0 ? 1 : 0); |
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.
final long numberOfPartitions = (recordCount / partitionSize) + ((recordCount % partitionSize) > 0 ? 1 : 0); | |
final long numberOfPartitions = Math.ceilDiv(recordCount, partitionSize); |
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.
Done in 9f340c9
final File schemaFile = new File(SCHEMA_PATH); | ||
final String schemaString = IOUtils.toString(new FileInputStream(schemaFile), StandardCharsets.UTF_8); | ||
return new Schema.Parser().parse(schemaString); |
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.
final File schemaFile = new File(SCHEMA_PATH); | |
final String schemaString = IOUtils.toString(new FileInputStream(schemaFile), StandardCharsets.UTF_8); | |
return new Schema.Parser().parse(schemaString); | |
try (InputStream schemaInputStream = ParquetTestUtils.class.getClassLoader().getResourceAsStream("avro/user.avsc")) { | |
final String schemaString = IOUtils.toString(schemaInputStream, StandardCharsets.UTF_8); | |
return new Schema.Parser().parse(schemaString); | |
} |
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.
Done in 9f340c9
Pushed to nifi-1.x |
Summary
NIFI-12241
Tracking
Please complete the following tracking steps prior to pull request creation.
Issue Tracking
Pull Request Tracking
NIFI-00000
NIFI-00000
Pull Request Formatting
main
branchVerification
Please indicate the verification steps performed prior to pull request creation.
Build
mvn clean install -P contrib-check
Licensing
LICENSE
andNOTICE
filesDocumentation