-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-3478] Implement CDC Read in Spark #6727
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
Changes from all commits
957e477
41a25f7
5348c9c
279654f
4d290f6
e298986
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 |
|---|---|---|
|
|
@@ -24,9 +24,13 @@ | |
| import org.apache.hudi.common.util.collection.Pair; | ||
|
|
||
| import com.fasterxml.jackson.annotation.JsonIgnoreProperties; | ||
| import com.fasterxml.jackson.databind.JsonNode; | ||
| import com.fasterxml.jackson.databind.node.ArrayNode; | ||
|
|
||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.FileStatus; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.hudi.exception.HoodieException; | ||
| import org.apache.log4j.LogManager; | ||
| import org.apache.log4j.Logger; | ||
|
|
||
|
|
@@ -37,6 +41,7 @@ | |
| import java.util.Collection; | ||
| import java.util.HashMap; | ||
| import java.util.HashSet; | ||
| import java.util.Iterator; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.stream.Collectors; | ||
|
|
@@ -236,6 +241,44 @@ public static <T> T fromJsonString(String jsonStr, Class<T> clazz) throws Except | |
| return JsonUtils.getObjectMapper().readValue(jsonStr, clazz); | ||
| } | ||
|
|
||
| /** | ||
| * parse the bytes of deltacommit, and get the base file and the log files belonging to this | ||
| * provided file group. | ||
| */ | ||
| // TODO: refactor this method to avoid doing the json tree walking (HUDI-4822). | ||
| public static Option<Pair<String, List<String>>> getFileSliceForFileGroupFromDeltaCommit( | ||
| byte[] bytes, HoodieFileGroupId fileGroupId) { | ||
| String jsonStr = new String(bytes, StandardCharsets.UTF_8); | ||
| if (jsonStr.isEmpty()) { | ||
| return Option.empty(); | ||
| } | ||
|
|
||
| try { | ||
| JsonNode ptToWriteStatsMap = JsonUtils.getObjectMapper().readTree(jsonStr).get("partitionToWriteStats"); | ||
|
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. Actually, why do we even need this method? We have the utils to deser the
Contributor
Author
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. we don't have any usable method to get the log files and base file from the commit metadata.
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. @YannByron we might not have the method to get all the log files but we shouldn't be producing new methods for deserializing |
||
| Iterator<Map.Entry<String, JsonNode>> pts = ptToWriteStatsMap.fields(); | ||
| while (pts.hasNext()) { | ||
| Map.Entry<String, JsonNode> ptToWriteStats = pts.next(); | ||
| if (ptToWriteStats.getValue().isArray()) { | ||
| for (JsonNode writeStat : ptToWriteStats.getValue()) { | ||
| HoodieFileGroupId fgId = new HoodieFileGroupId(ptToWriteStats.getKey(), writeStat.get("fileId").asText()); | ||
| if (fgId.equals(fileGroupId)) { | ||
| String baseFile = writeStat.get("baseFile").asText(); | ||
| ArrayNode logFilesNode = (ArrayNode) writeStat.get("logFiles"); | ||
| List<String> logFiles = new ArrayList<>(); | ||
| for (JsonNode logFile : logFilesNode) { | ||
| logFiles.add(logFile.asText()); | ||
| } | ||
| return Option.of(Pair.of(baseFile, logFiles)); | ||
| } | ||
| } | ||
| } | ||
| } | ||
| return Option.empty(); | ||
| } catch (Exception e) { | ||
| throw new HoodieException("Fail to parse the base file and log files from DeltaCommit", e); | ||
| } | ||
| } | ||
|
|
||
| // Here the functions are named "fetch" instead of "get", to get avoid of the json conversion. | ||
| public long fetchTotalPartitionsWritten() { | ||
| return partitionToWriteStats.size(); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
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 fix the import sequences of all the files.
org.apache.hudipackage should be in the front.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