-
Notifications
You must be signed in to change notification settings - Fork 2.1k
[Feature][connectors-v2/connector-file]Local file connector supports parquet file type logical segmentation reading #10239
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
Open
chl-wxp
wants to merge
2
commits into
apache:dev
Choose a base branch
from
chl-wxp:add-ParquetFileSplitStrategy
base: dev
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
118 changes: 118 additions & 0 deletions
118
...org/apache/seatunnel/connectors/seatunnel/file/source/split/ParquetFileSplitStrategy.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,118 @@ | ||
| /* | ||
| * 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.seatunnel.connectors.seatunnel.file.source.split; | ||
|
|
||
| import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; | ||
| import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorErrorCode; | ||
|
|
||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.parquet.hadoop.ParquetFileReader; | ||
| import org.apache.parquet.hadoop.metadata.BlockMetaData; | ||
| import org.apache.parquet.hadoop.util.HadoopInputFile; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.ArrayList; | ||
| import java.util.List; | ||
|
|
||
| /** | ||
| * {@link ParquetFileSplitStrategy} defines a split strategy for Parquet files based on Parquet | ||
| * physical storage units (RowGroups). | ||
| * | ||
| * <p>This strategy uses {@code RowGroup} as the minimum indivisible split unit and generates {@link | ||
| * FileSourceSplit}s by merging one or more contiguous RowGroups according to the configured split | ||
| * size. A split will never break a RowGroup, ensuring correctness and compatibility with Parquet | ||
| * readers. | ||
| * | ||
| * <p>The generated split range ({@code start}, {@code length}) represents a byte range covering | ||
| * complete RowGroups. The actual row-level reading and decoding are delegated to the Parquet reader | ||
| * implementation. | ||
| * | ||
| * <p>This design enables efficient parallel reading of Parquet files while preserving Parquet | ||
| * format semantics and avoiding invalid byte-level splits. | ||
| */ | ||
| public class ParquetFileSplitStrategy implements FileSplitStrategy { | ||
|
|
||
| private final long splitSizeBytes; | ||
|
|
||
| public ParquetFileSplitStrategy(long splitSizeBytes) { | ||
| if (splitSizeBytes <= 0) { | ||
| throw new SeaTunnelRuntimeException( | ||
| FileConnectorErrorCode.FILE_SPLIT_SIZE_ILLEGAL, | ||
| "SplitSizeBytes must be greater than 0"); | ||
| } | ||
| this.splitSizeBytes = splitSizeBytes; | ||
| } | ||
|
|
||
| @Override | ||
| public List<FileSourceSplit> split(String tableId, String filePath) { | ||
| try { | ||
| return splitByRowGroups(tableId, filePath, readRowGroups(filePath)); | ||
| } catch (IOException e) { | ||
| throw new SeaTunnelRuntimeException(FileConnectorErrorCode.FILE_SPLIT_FAIL, e); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Core split logic based on row group metadata. This method is IO-free and unit-test friendly. | ||
| */ | ||
| List<FileSourceSplit> splitByRowGroups( | ||
| String tableId, String filePath, List<BlockMetaData> rowGroups) { | ||
| List<FileSourceSplit> splits = new ArrayList<>(); | ||
| if (rowGroups == null || rowGroups.isEmpty()) { | ||
| return splits; | ||
| } | ||
| long currentStart = 0; | ||
| long currentLength = 0; | ||
| boolean hasOpenSplit = false; | ||
| for (BlockMetaData block : rowGroups) { | ||
| long rgStart = block.getStartingPos(); | ||
| long rgSize = block.getCompressedSize(); | ||
| // start a new split | ||
| if (!hasOpenSplit) { | ||
| currentStart = rgStart; | ||
| currentLength = rgSize; | ||
| hasOpenSplit = true; | ||
| continue; | ||
| } | ||
| // exceeds threshold, close current split | ||
| if (currentLength + rgSize > splitSizeBytes) { | ||
| splits.add(new FileSourceSplit(tableId, filePath, currentStart, currentLength)); | ||
| // start next split | ||
| currentStart = rgStart; | ||
| currentLength = rgSize; | ||
| } else { | ||
| currentLength += rgSize; | ||
| } | ||
| } | ||
| // last split | ||
| if (hasOpenSplit && currentLength > 0) { | ||
| splits.add(new FileSourceSplit(tableId, filePath, currentStart, currentLength)); | ||
| } | ||
| return splits; | ||
| } | ||
|
|
||
| private List<BlockMetaData> readRowGroups(String filePath) throws IOException { | ||
| Path path = new Path(filePath); | ||
| Configuration conf = new Configuration(); | ||
| try (ParquetFileReader reader = | ||
| ParquetFileReader.open(HadoopInputFile.fromPath(path, conf))) { | ||
| return reader.getFooter().getBlocks(); | ||
| } | ||
| } | ||
| } |
94 changes: 94 additions & 0 deletions
94
...apache/seatunnel/connectors/seatunnel/file/source/split/ParquetFileSplitStrategyTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,94 @@ | ||
| /* | ||
| * 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.seatunnel.connectors.seatunnel.file.source.split; | ||
|
|
||
| import org.apache.parquet.hadoop.metadata.BlockMetaData; | ||
|
|
||
| import org.junit.jupiter.api.Assertions; | ||
| import org.junit.jupiter.api.Test; | ||
| import org.mockito.Mockito; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.Collections; | ||
| import java.util.List; | ||
|
|
||
| import static org.mockito.Mockito.when; | ||
|
|
||
| public class ParquetFileSplitStrategyTest { | ||
|
|
||
| private static final String TABLE_ID = "test.test_table"; | ||
| private static final String FILE_PATH = "/tmp/test.parquet"; | ||
|
|
||
| @Test | ||
| void testSplitByRowGroupsEmpty() { | ||
| ParquetFileSplitStrategy strategy = new ParquetFileSplitStrategy(100); | ||
| List<FileSourceSplit> splits = | ||
| strategy.splitByRowGroups(TABLE_ID, FILE_PATH, Collections.emptyList()); | ||
| Assertions.assertTrue(splits.isEmpty()); | ||
| } | ||
|
|
||
| @Test | ||
| void testSplitByRowGroupsSingleRowGroup() { | ||
| ParquetFileSplitStrategy strategy = new ParquetFileSplitStrategy(1000); | ||
| List<BlockMetaData> blocks = new ArrayList<>(); | ||
| blocks.add(mockBlock(0, 200)); | ||
| List<FileSourceSplit> splits = strategy.splitByRowGroups(TABLE_ID, FILE_PATH, blocks); | ||
| Assertions.assertEquals(1, splits.size()); | ||
| FileSourceSplit split = splits.get(0); | ||
| Assertions.assertEquals(0, split.getStart()); | ||
| Assertions.assertEquals(200, split.getLength()); | ||
| } | ||
|
|
||
| @Test | ||
| void testSplitByRowGroupsMergeRowGroups() { | ||
| ParquetFileSplitStrategy strategy = new ParquetFileSplitStrategy(500); | ||
| List<BlockMetaData> blocks = new ArrayList<>(); | ||
| blocks.add(mockBlock(0, 100)); | ||
| blocks.add(mockBlock(100, 150)); | ||
| blocks.add(mockBlock(250, 200)); | ||
| List<FileSourceSplit> splits = strategy.splitByRowGroups(TABLE_ID, FILE_PATH, blocks); | ||
| // 100 + 150 + 200 = 450 < 500 | ||
| Assertions.assertEquals(1, splits.size()); | ||
| FileSourceSplit split = splits.get(0); | ||
| Assertions.assertEquals(0, split.getStart()); | ||
| Assertions.assertEquals(450, split.getLength()); | ||
| } | ||
|
|
||
| @Test | ||
| void testSplitByRowGroupsSplitWhenExceedsThreshold() { | ||
| ParquetFileSplitStrategy strategy = new ParquetFileSplitStrategy(300); | ||
| List<BlockMetaData> blocks = new ArrayList<>(); | ||
| blocks.add(mockBlock(0, 100)); | ||
| blocks.add(mockBlock(100, 150)); | ||
| blocks.add(mockBlock(250, 200)); | ||
| List<FileSourceSplit> splits = strategy.splitByRowGroups(TABLE_ID, FILE_PATH, blocks); | ||
| Assertions.assertEquals(2, splits.size()); | ||
| FileSourceSplit first = splits.get(0); | ||
| Assertions.assertEquals(0, first.getStart()); | ||
| Assertions.assertEquals(250, first.getLength()); | ||
| FileSourceSplit second = splits.get(1); | ||
| Assertions.assertEquals(250, second.getStart()); | ||
| Assertions.assertEquals(200, second.getLength()); | ||
| } | ||
|
|
||
| private BlockMetaData mockBlock(long start, long compressedSize) { | ||
| BlockMetaData block = Mockito.mock(BlockMetaData.class); | ||
| when(block.getStartingPos()).thenReturn(start); | ||
| when(block.getCompressedSize()).thenReturn(compressedSize); | ||
| return block; | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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 use English punctuation marks