yihua commented on code in PR #12826:
URL: https://github.com/apache/hudi/pull/12826#discussion_r1974360404


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ArchivedTimelineV2.java:
##########
@@ -249,4 +251,9 @@ public HoodieTimeline getWriteTimeline() {
             readCommits.containsKey(i.requestedTime()))
         .filter(s -> validActions.contains(s.getAction())), instantReader);
   }
+
+  @Override
+  public boolean isEmpty(HoodieInstant instant) {
+    return getInstantDetails(instant).isEmpty();

Review Comment:
   Should this use the new API?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ArchivedTimelineV2.java:
##########
@@ -156,7 +155,11 @@ public Option<byte[]> getInstantDetails(HoodieInstant 
instant) {
 
   @Override
   public InputStream getContentStream(HoodieInstant instant) {
-    return new ByteArrayInputStream(getInstantDetails(instant).orElseGet(() -> 
new byte[0]));
+    Option<InputStream> stream = getInputStreamOptionLegacy(this, instant);
+    if (stream.isEmpty()) {
+      return new ByteArrayInputStream(new byte[]{});
+    }
+    return stream.get();

Review Comment:
   Similarly could this avoid byte array?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/ArchivedTimelineV1.java:
##########
@@ -166,7 +168,11 @@ public Option<byte[]> getInstantDetails(HoodieInstant 
instant) {
 
   @Override
   public InputStream getContentStream(HoodieInstant instant) {
-    return new ByteArrayInputStream(getInstantDetails(instant).orElseGet(() -> 
new byte[0]));
+    Option<InputStream> stream = getInputStreamOptionLegacy(this, instant);
+    if (stream.isEmpty()) {
+      return new ByteArrayInputStream(new byte[]{});
+    }
+    return stream.get();

Review Comment:
   Could this use new API without going through the byte array?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/CommitMetadataSerDeV1.java:
##########
@@ -24,30 +24,41 @@
 import org.apache.hudi.common.util.JsonUtils;
 import org.apache.hudi.common.util.Option;
 
+import org.apache.avro.specific.SpecificRecordBase;
+
 import java.io.IOException;
+import java.io.InputStream;
+import java.util.function.BooleanSupplier;
 
-import static org.apache.hudi.common.util.StringUtils.fromUTF8Bytes;
+import static 
org.apache.hudi.common.table.timeline.TimelineMetadataUtils.deserializeAvroMetadata;
 
 public class CommitMetadataSerDeV1 implements CommitMetadataSerDe {
 
   @Override
-  public <T> T deserialize(HoodieInstant instant, byte[] bytes, Class<T> 
clazz) throws IOException {
+  public <T> T deserialize(HoodieInstant instant, InputStream inputStream, 
BooleanSupplier isEmptyInstant, Class<T> clazz) throws IOException {
     try {
-      if (bytes.length == 0) {
-        return clazz.newInstance();
+      // For commit metadata we need special case handling as they are using 
non avro type in memory.

Review Comment:
   ```suggestion
         // For commit metadata we need special case handling as they are 
serialized to JSON in V1
   ```



-- 
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