-
Notifications
You must be signed in to change notification settings - Fork 198
feat(hudi): enable concurrent hudi s3 footer collection #815
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
base: main
Are you sure you want to change the base?
Changes from all commits
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 |
|---|---|---|
|
|
@@ -39,6 +39,7 @@ | |
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.stream.Collectors; | ||
| import java.util.stream.IntStream; | ||
| import java.util.stream.Stream; | ||
|
|
||
| import org.apache.avro.Conversions; | ||
|
|
@@ -199,6 +200,50 @@ void columnStatsWithoutMetadataTable(@TempDir Path tempDir) throws IOException { | |
| validateOutput(output); | ||
| } | ||
|
|
||
| @Test | ||
| void columnStatsWithoutMetadataTable_parallelFooterReadsAreThreadSafe(@TempDir Path tempDir) | ||
| throws IOException { | ||
| Path file = tempDir.resolve("tmp.parquet"); | ||
| GenericData genericData = GenericData.get(); | ||
| genericData.addLogicalTypeConversion(new Conversions.DecimalConversion()); | ||
| try (ParquetWriter<GenericRecord> writer = | ||
| AvroParquetWriter.<GenericRecord>builder( | ||
| HadoopOutputFile.fromPath( | ||
| new org.apache.hadoop.fs.Path(file.toUri()), configuration)) | ||
| .withSchema(AVRO_SCHEMA) | ||
| .withDataModel(genericData) | ||
| .build()) { | ||
| for (GenericRecord record : getRecords()) { | ||
| writer.write(record); | ||
| } | ||
| } | ||
|
|
||
| HoodieTableMetaClient mockMetaClient = mock(HoodieTableMetaClient.class); | ||
| when(mockMetaClient.getHadoopConf()).thenReturn(configuration); | ||
| HudiFileStatsExtractor fileStatsExtractor = new HudiFileStatsExtractor(mockMetaClient); | ||
|
|
||
| List<InternalDataFile> inputFiles = | ||
| IntStream.range(0, 200) | ||
| .mapToObj( | ||
| i -> | ||
| InternalDataFile.builder() | ||
| .physicalPath(file.toString()) | ||
| .columnStats(Collections.emptyList()) | ||
| .fileFormat(FileFormat.APACHE_PARQUET) | ||
| .lastModified(1234L) | ||
| .fileSizeBytes(4321L) | ||
| .recordCount(0) | ||
| .build()) | ||
| .collect(Collectors.toList()); | ||
|
|
||
| List<InternalDataFile> output = | ||
| fileStatsExtractor.addStatsToFiles(null, inputFiles.stream(), schema).collect(Collectors.toList()); | ||
|
Contributor
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. nit: Line exceeds style limit. Split: List<InternalDataFile> output =
fileStatsExtractor
.addStatsToFiles(null, inputFiles.stream(), schema)
.collect(Collectors.toList()); |
||
|
|
||
| assertEquals(200, output.size()); | ||
| assertTrue(output.stream().allMatch(fileWithStats -> fileWithStats.getRecordCount() == 2)); | ||
|
Contributor
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. The parallel test only checks |
||
| assertTrue(output.stream().allMatch(fileWithStats -> fileWithStats.getColumnStats().size() == 9)); | ||
| } | ||
|
|
||
| private void validateOutput(List<InternalDataFile> output) { | ||
| assertEquals(1, output.size()); | ||
| InternalDataFile fileWithStats = output.get(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.
Stream.parallel()uses the commonForkJoinPool, which is shared across the entire JVM. SincereadRangeFromParquetMetadatais a blocking I/O call (especially on S3), this can starve other parallel streams andCompletableFuture.supplyAsynccalls that share the same pool.ForkJoinPoolis designed for CPU-bound divide-and-conquer work — for blocking I/O, a dedicatedExecutorService(fixed thread pool) is the right tool. Consider injecting an executor and usingCompletableFuture.supplyAsync:The executor can be a constructor parameter (e.g.,
Executors.newFixedThreadPool(n)) so callers control threading.Also — is
UTILS(ParquetUtils, the static singleton on line 73) thread-safe?readRangeFromParquetMetadatais now called concurrently from multiple threads on the same instance. This should be verified before relying on it here.