danny0405 commented on code in PR #8480:
URL: https://github.com/apache/hudi/pull/8480#discussion_r1172319145


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java:
##########
@@ -46,17 +49,50 @@ public class FileGroupDTO {
   TimelineDTO timeline;
 
   public static FileGroupDTO fromFileGroup(HoodieFileGroup fileGroup) {
+    return fromFileGroup(fileGroup, true);
+  }
+
+  public static List<FileGroupDTO> fromFileGroup(List<HoodieFileGroup> 
fileGroups) {
+    if (fileGroups.isEmpty()) {
+      return Collections.emptyList();
+    }
+
+    List<FileGroupDTO> fileGroupDTOs = fileGroups.stream()
+        .map(fg -> FileGroupDTO.fromFileGroup(fg, 
false)).collect(Collectors.toList());
+    // Timeline exists only in the first file group DTO. Optimisation to 
reduce payload size.
+    fileGroupDTOs.set(0, FileGroupDTO.fromFileGroup(fileGroups.get(0), true));

Review Comment:
   Can we construct the first DTS directly instead of re-construction and 
overwriting?



##########
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java:
##########
@@ -160,6 +160,8 @@ private boolean isLocalViewBehind(Context ctx) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Client [ LastTs=" + lastKnownInstantFromClient + ", 
TimelineHash=" + timelineHashFromClient
           + "], localTimeline=" + localTimeline.getInstants());
+    } else {
+      LOG.info("Client [ LastTs=" + lastKnownInstantFromClient + ", 
TimelineHash=" + timelineHashFromClient + "]");
     }

Review Comment:
   Why logging so many messages?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java:
##########
@@ -46,17 +49,50 @@ public class FileGroupDTO {
   TimelineDTO timeline;
 
   public static FileGroupDTO fromFileGroup(HoodieFileGroup fileGroup) {
+    return fromFileGroup(fileGroup, true);
+  }
+
+  public static List<FileGroupDTO> fromFileGroup(List<HoodieFileGroup> 
fileGroups) {
+    if (fileGroups.isEmpty()) {
+      return Collections.emptyList();
+    }
+
+    List<FileGroupDTO> fileGroupDTOs = fileGroups.stream()
+        .map(fg -> FileGroupDTO.fromFileGroup(fg, 
false)).collect(Collectors.toList());
+    // Timeline exists only in the first file group DTO. Optimisation to 
reduce payload size.
+    fileGroupDTOs.set(0, FileGroupDTO.fromFileGroup(fileGroups.get(0), true));
+    return fileGroupDTOs;
+  }
+
+  public static HoodieFileGroup toFileGroup(FileGroupDTO dto, 
HoodieTableMetaClient metaClient) {
+    return toFileGroup(dto, metaClient, null);
+  }
+
+  public static Stream<HoodieFileGroup> toFileGroup(List<FileGroupDTO> dtos, 
HoodieTableMetaClient metaClient) {
+    if (dtos.isEmpty()) {
+      return Stream.empty();

Review Comment:
   In which case the dtos can be empty?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java:
##########
@@ -46,17 +49,50 @@ public class FileGroupDTO {
   TimelineDTO timeline;
 
   public static FileGroupDTO fromFileGroup(HoodieFileGroup fileGroup) {
+    return fromFileGroup(fileGroup, true);
+  }
+
+  public static List<FileGroupDTO> fromFileGroup(List<HoodieFileGroup> 
fileGroups) {
+    if (fileGroups.isEmpty()) {
+      return Collections.emptyList();
+    }
+
+    List<FileGroupDTO> fileGroupDTOs = fileGroups.stream()
+        .map(fg -> FileGroupDTO.fromFileGroup(fg, 
false)).collect(Collectors.toList());
+    // Timeline exists only in the first file group DTO. Optimisation to 
reduce payload size.
+    fileGroupDTOs.set(0, FileGroupDTO.fromFileGroup(fileGroups.get(0), true));
+    return fileGroupDTOs;
+  }
+
+  public static HoodieFileGroup toFileGroup(FileGroupDTO dto, 
HoodieTableMetaClient metaClient) {
+    return toFileGroup(dto, metaClient, null);

Review Comment:
   Instead of null, can we use `Option` instead?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java:
##########
@@ -46,17 +49,50 @@ public class FileGroupDTO {
   TimelineDTO timeline;
 
   public static FileGroupDTO fromFileGroup(HoodieFileGroup fileGroup) {
+    return fromFileGroup(fileGroup, true);
+  }
+
+  public static List<FileGroupDTO> fromFileGroup(List<HoodieFileGroup> 
fileGroups) {
+    if (fileGroups.isEmpty()) {
+      return Collections.emptyList();
+    }
+
+    List<FileGroupDTO> fileGroupDTOs = fileGroups.stream()
+        .map(fg -> FileGroupDTO.fromFileGroup(fg, 
false)).collect(Collectors.toList());
+    // Timeline exists only in the first file group DTO. Optimisation to 
reduce payload size.
+    fileGroupDTOs.set(0, FileGroupDTO.fromFileGroup(fileGroups.get(0), true));
+    return fileGroupDTOs;
+  }
+
+  public static HoodieFileGroup toFileGroup(FileGroupDTO dto, 
HoodieTableMetaClient metaClient) {
+    return toFileGroup(dto, metaClient, null);
+  }
+
+  public static Stream<HoodieFileGroup> toFileGroup(List<FileGroupDTO> dtos, 
HoodieTableMetaClient metaClient) {
+    if (dtos.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // Timeline exists only in the first file group DTO. Optimisation to 
reduce payload size.
+    HoodieTimeline timeline = toFileGroup(dtos.get(0), 
metaClient).getTimeline();
+    return dtos.stream().map(dto -> toFileGroup(dto, metaClient, timeline));

Review Comment:
   Please add validation for the code path: `ValidationUtils.checkState` to 
ensure the existence of the timeline.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java:
##########
@@ -46,17 +49,50 @@ public class FileGroupDTO {
   TimelineDTO timeline;
 
   public static FileGroupDTO fromFileGroup(HoodieFileGroup fileGroup) {
+    return fromFileGroup(fileGroup, true);
+  }
+
+  public static List<FileGroupDTO> fromFileGroup(List<HoodieFileGroup> 
fileGroups) {
+    if (fileGroups.isEmpty()) {
+      return Collections.emptyList();
+    }
+
+    List<FileGroupDTO> fileGroupDTOs = fileGroups.stream()
+        .map(fg -> FileGroupDTO.fromFileGroup(fg, 
false)).collect(Collectors.toList());
+    // Timeline exists only in the first file group DTO. Optimisation to 
reduce payload size.
+    fileGroupDTOs.set(0, FileGroupDTO.fromFileGroup(fileGroups.get(0), true));
+    return fileGroupDTOs;
+  }
+
+  public static HoodieFileGroup toFileGroup(FileGroupDTO dto, 
HoodieTableMetaClient metaClient) {
+    return toFileGroup(dto, metaClient, null);
+  }
+
+  public static Stream<HoodieFileGroup> toFileGroup(List<FileGroupDTO> dtos, 
HoodieTableMetaClient metaClient) {
+    if (dtos.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // Timeline exists only in the first file group DTO. Optimisation to 
reduce payload size.
+    HoodieTimeline timeline = toFileGroup(dtos.get(0), 
metaClient).getTimeline();
+    return dtos.stream().map(dto -> toFileGroup(dto, metaClient, timeline));
+  }
+
+  private static FileGroupDTO fromFileGroup(HoodieFileGroup fileGroup, boolean 
includeTimeline) {
     FileGroupDTO dto = new FileGroupDTO();
     dto.partition = fileGroup.getPartitionPath();
     dto.id = fileGroup.getFileGroupId().getFileId();
     dto.slices = 
fileGroup.getAllRawFileSlices().map(FileSliceDTO::fromFileSlice).collect(Collectors.toList());
-    dto.timeline = TimelineDTO.fromTimeline(fileGroup.getTimeline());
+    if (includeTimeline) {
+      dto.timeline = TimelineDTO.fromTimeline(fileGroup.getTimeline());
+    }
     return dto;
   }
 
-  public static HoodieFileGroup toFileGroup(FileGroupDTO dto, 
HoodieTableMetaClient metaClient) {
+  private static HoodieFileGroup toFileGroup(FileGroupDTO dto, 
HoodieTableMetaClient metaClient, HoodieTimeline inputTimeline) {

Review Comment:
   Can we ensure that the `inputTimeline` never null, and fetch the complete 
`inputTimeline` before invoking this method?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to