|
24 | 24 | import org.apache.hudi.common.util.collection.Pair; |
25 | 25 |
|
26 | 26 | import com.fasterxml.jackson.annotation.JsonIgnoreProperties; |
| 27 | +import com.fasterxml.jackson.databind.JsonNode; |
| 28 | +import com.fasterxml.jackson.databind.node.ArrayNode; |
| 29 | + |
27 | 30 | import org.apache.hadoop.conf.Configuration; |
28 | 31 | import org.apache.hadoop.fs.FileStatus; |
29 | 32 | import org.apache.hadoop.fs.Path; |
| 33 | +import org.apache.hudi.exception.HoodieException; |
30 | 34 | import org.apache.log4j.LogManager; |
31 | 35 | import org.apache.log4j.Logger; |
32 | 36 |
|
|
37 | 41 | import java.util.Collection; |
38 | 42 | import java.util.HashMap; |
39 | 43 | import java.util.HashSet; |
| 44 | +import java.util.Iterator; |
40 | 45 | import java.util.List; |
41 | 46 | import java.util.Map; |
42 | 47 | import java.util.stream.Collectors; |
@@ -236,6 +241,44 @@ public static <T> T fromJsonString(String jsonStr, Class<T> clazz) throws Except |
236 | 241 | return JsonUtils.getObjectMapper().readValue(jsonStr, clazz); |
237 | 242 | } |
238 | 243 |
|
| 244 | + /** |
| 245 | + * parse the bytes of deltacommit, and get the base file and the log files belonging to this |
| 246 | + * provided file group. |
| 247 | + */ |
| 248 | + // TODO: refactor this method to avoid doing the json tree walking (HUDI-4822). |
| 249 | + public static Option<Pair<String, List<String>>> getFileSliceForFileGroupFromDeltaCommit( |
| 250 | + byte[] bytes, HoodieFileGroupId fileGroupId) { |
| 251 | + String jsonStr = new String(bytes, StandardCharsets.UTF_8); |
| 252 | + if (jsonStr.isEmpty()) { |
| 253 | + return Option.empty(); |
| 254 | + } |
| 255 | + |
| 256 | + try { |
| 257 | + JsonNode ptToWriteStatsMap = JsonUtils.getObjectMapper().readTree(jsonStr).get("partitionToWriteStats"); |
| 258 | + Iterator<Map.Entry<String, JsonNode>> pts = ptToWriteStatsMap.fields(); |
| 259 | + while (pts.hasNext()) { |
| 260 | + Map.Entry<String, JsonNode> ptToWriteStats = pts.next(); |
| 261 | + if (ptToWriteStats.getValue().isArray()) { |
| 262 | + for (JsonNode writeStat : ptToWriteStats.getValue()) { |
| 263 | + HoodieFileGroupId fgId = new HoodieFileGroupId(ptToWriteStats.getKey(), writeStat.get("fileId").asText()); |
| 264 | + if (fgId.equals(fileGroupId)) { |
| 265 | + String baseFile = writeStat.get("baseFile").asText(); |
| 266 | + ArrayNode logFilesNode = (ArrayNode) writeStat.get("logFiles"); |
| 267 | + List<String> logFiles = new ArrayList<>(); |
| 268 | + for (JsonNode logFile : logFilesNode) { |
| 269 | + logFiles.add(logFile.asText()); |
| 270 | + } |
| 271 | + return Option.of(Pair.of(baseFile, logFiles)); |
| 272 | + } |
| 273 | + } |
| 274 | + } |
| 275 | + } |
| 276 | + return Option.empty(); |
| 277 | + } catch (Exception e) { |
| 278 | + throw new HoodieException("Fail to parse the base file and log files from DeltaCommit", e); |
| 279 | + } |
| 280 | + } |
| 281 | + |
239 | 282 | // Here the functions are named "fetch" instead of "get", to get avoid of the json conversion. |
240 | 283 | public long fetchTotalPartitionsWritten() { |
241 | 284 | return partitionToWriteStats.size(); |
|
0 commit comments