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


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java:
##########
@@ -46,17 +48,28 @@ public class FileGroupDTO {
   TimelineDTO timeline;
 
   public static FileGroupDTO fromFileGroup(HoodieFileGroup fileGroup) {

Review Comment:
   This method can be removed



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DTOUtils.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.hudi.common.table.timeline.dto;
+
+import org.apache.hudi.common.model.HoodieFileGroup;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.ValidationUtils.checkState;
+
+/**
+ * DTO utils to hold batch apis.
+ */
+public class DTOUtils {
+
+  public static List<FileGroupDTO> 
fileGroupDTOsfromFileGroups(List<HoodieFileGroup> fileGroups) {
+    if (fileGroups.isEmpty()) {
+      return Collections.emptyList();
+    } else if (fileGroups.size() == 1) {
+      return 
Collections.singletonList(FileGroupDTO.fromFileGroup(fileGroups.get(0), true));
+    } else {
+      List<FileGroupDTO> fileGroupDTOS = new ArrayList<>();
+      fileGroupDTOS.add(FileGroupDTO.fromFileGroup(fileGroups.get(0), true));
+      fileGroupDTOS.addAll(fileGroups.subList(1, fileGroups.size()).stream()
+          .map(fg -> FileGroupDTO.fromFileGroup(fg, 
false)).collect(Collectors.toList()));
+      return fileGroupDTOS;
+    }
+  }
+
+  public static Stream<HoodieFileGroup> 
fileGroupDTOsToFileGroups(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 = FileGroupDTO.toFileGroup(dtos.get(0), 
metaClient).getTimeline();

Review Comment:
   Does this code make sense to you?
   
   ```java
       checkState(dtos.get(0).timeline != null, "Timeline is expected to be set 
for the first FileGroupDTO");
       HoodieTimeline timeline = TimelineDTO.toTimeline(dtos.get(0).timeline, 
metaClient);
       return dtos.stream().map(dto -> FileGroupDTO.toFileGroup(dto, timeline));
   ```



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java:
##########
@@ -46,17 +48,28 @@ public class FileGroupDTO {
   TimelineDTO timeline;
 
   public static FileGroupDTO fromFileGroup(HoodieFileGroup fileGroup) {
+    return fromFileGroup(fileGroup, true);
+  }
+
+  public static HoodieFileGroup toFileGroup(FileGroupDTO dto, 
HoodieTableMetaClient metaClient) {

Review Comment:
   This method can be removed



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java:
##########
@@ -46,17 +48,28 @@ public class FileGroupDTO {
   TimelineDTO timeline;
 
   public static FileGroupDTO fromFileGroup(HoodieFileGroup fileGroup) {
+    return fromFileGroup(fileGroup, true);
+  }
+
+  public static HoodieFileGroup toFileGroup(FileGroupDTO dto, 
HoodieTableMetaClient metaClient) {
+    return toFileGroup(dto, metaClient, Option.empty());
+  }
+
+  public 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) {
+  public static HoodieFileGroup toFileGroup(FileGroupDTO dto, 
HoodieTableMetaClient metaClient, Option<HoodieTimeline> inputTimeline) {
+    HoodieTimeline fgTimeline = inputTimeline.isPresent() ? 
inputTimeline.get() : TimelineDTO.toTimeline(dto.timeline, metaClient);
     HoodieFileGroup fileGroup =

Review Comment:
   The signature can be simplified to:
   
   ```java
   public static HoodieFileGroup toFileGroup(FileGroupDTO dto, HoodieTimeline 
fgTimeline)
   ```



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to