CTTY commented on code in PR #17785:
URL: https://github.com/apache/hudi/pull/17785#discussion_r2692466235


##########
hudi-hadoop-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatWriter.java:
##########
@@ -42,47 +43,46 @@
 /**
  * HoodieLogFormatWriter can be used to append blocks to a log file Use 
HoodieLogFormat.WriterBuilder to construct.
  */
+@Getter
 @Slf4j
-public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
+public class HoodieLogFormatWriter extends HoodieLogFormat.Writer {
 
-  @Getter
-  private HoodieLogFile logFile;
-  private FSDataOutputStream output;
-
-  private final HoodieStorage storage;
-  @Getter
-  private final long sizeThreshold;
-  private final Integer bufferSize;
   private final Short replication;
-  private final String rolloverLogWriteToken;
-  private final LogFileCreationCallback fileCreationHook;
+  private FSDataOutputStream outputStream;
   private boolean closed = false;
   private transient Thread shutdownThread = null;
 
-  public HoodieLogFormatWriter(
-      HoodieStorage storage,
-      HoodieLogFile logFile,
+  @Builder(setterPrefix = "with")
+  private HoodieLogFormatWriter(
       Integer bufferSize,
-      Short replication,
+      HoodieStorage storage,
+      StoragePath parentPath,
+      String logFileId,
+      String fileExtension,
+      String instantTime,
+      Integer logVersion,
+      String logWriteToken,
+      String suffix,
+      Long fileLen,
       Long sizeThreshold,
-      String rolloverLogWriteToken,
-      LogFileCreationCallback fileCreationHook) {
-    this.storage = storage;
-    this.logFile = logFile;
-    this.sizeThreshold = sizeThreshold;
-    this.bufferSize = bufferSize != null ? bufferSize : 
storage.getDefaultBufferSize();
+      LogFileCreationCallback fileCreationCallback,
+      HoodieTableVersion tableVersion,
+      FSDataOutputStream outputStream,
+      Short replication
+  ) throws IOException {
+    super(bufferSize, storage, parentPath, logFileId, fileExtension, 
instantTime, logVersion, logWriteToken,
+        suffix, fileLen, sizeThreshold, fileCreationCallback, tableVersion);
+    this.outputStream = outputStream;

Review Comment:
   This doesn't seem right, `outputStream` should be initialized lazily rather 
than set like this



##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java:
##########
@@ -60,241 +60,166 @@ public interface HoodieLogFormat {
 
   String DEFAULT_WRITE_TOKEN = "0-0-0";
 
-  String DEFAULT_LOG_FORMAT_WRITER = 
"org.apache.hudi.common.table.log.HoodieLogFormatWriter";
-
-  /**
-   * Writer interface to allow appending block to this file format.
-   */
-  interface Writer extends Closeable {
-
-    /**
-     * @return the path to the current {@link HoodieLogFile} being written to.
-     */
-    HoodieLogFile getLogFile();
-
-    /**
-     * Append Block to a log file.
-     * @return {@link AppendResult} containing result of the append.
-     */
-    AppendResult appendBlock(HoodieLogBlock block) throws IOException, 
InterruptedException;
-
-    /**
-     * Appends the list of blocks to a logfile.
-     * @return {@link AppendResult} containing result of the append.
-     */
-    AppendResult appendBlocks(List<HoodieLogBlock> blocks) throws IOException, 
InterruptedException;
-
-    long getCurrentSize() throws IOException;
-  }
-
-  /**
-   * Reader interface which is an Iterator of HoodieLogBlock.
-   */
-  interface Reader extends Closeable, Iterator<HoodieLogBlock> {
-
-    /**
-     * @return the path to this {@link HoodieLogFormat}
-     */
-    HoodieLogFile getLogFile();
-
-    /**
-     * Read log file in reverse order and check if prev block is present.
-     * 
-     * @return {@code true} if previous block is present, {@code false} 
otherwise.
-     */
-    boolean hasPrev();
-
-    /**
-     * Read log file in reverse order and return prev block if present.
-     * 
-     * @return {@link HoodieLogBlock} the previous block
-     * @throws IOException
-     */
-    HoodieLogBlock prev() throws IOException;
-  }
-
   /**
-   * Builder class to construct the default log format writer.
+   * Abstract base class for appending blocks to the Hoodie log format.
+   * Subclasses provide specific implementations for writing to different 
storage layers.
    */
-  class WriterBuilder {
+  @Getter
+  @Slf4j
+  abstract class Writer implements Closeable {
 
-    private static final Logger LOG = 
LoggerFactory.getLogger(WriterBuilder.class);
     // Default max log file size 512 MB
     public static final long DEFAULT_SIZE_THRESHOLD = 512 * 1024 * 1024L;
 
     // Buffer size
-    private Integer bufferSize;
+    protected Integer bufferSize;
     // FileSystem
-    private HoodieStorage storage;
+    protected HoodieStorage storage;
     // Size threshold for the log file. Useful when used with a rolling log 
appender
-    private Long sizeThreshold;
+    protected Long sizeThreshold;
     // Log File extension. Could be .avro.delta or .avro.commits etc
-    private String fileExtension;
+    protected String fileExtension;
     // File Id
-    private String logFileId;
+    protected String logFileId;
     // File Commit Time stamp
-    private String instantTime;
+    protected String instantTime;
     // version number for this log file. If not specified, then the current 
version will be
     // computed by inspecting the file system
-    private Integer logVersion;
+    protected Integer logVersion;
     // file len of this log file
-    private Long fileLen = 0L;
+    protected Long fileLen = 0L;

Review Comment:
   I think we should use `fileSize` not `fileLen` since most APIs like 
`HoodieLogFile` still refer to it as `getFileSize`



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