apurtell commented on code in PR #2115:
URL: https://github.com/apache/phoenix/pull/2115#discussion_r2056732045


##########
phoenix-core-server/src/main/java/org/apache/phoenix/replication/log/LogFile.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.phoenix.replication.log;
+
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Defines the structure and constants for Phoenix Replication Log files.
+ * Provides interfaces for reading and writing these logs.
+ */
+public final class LogFile {
+
+    /** Magic number for Phoenix Replication Log files */
+    public static byte[] MAGIC = Bytes.toBytes("PLOG");
+
+    /** Current major version of the replication log format */
+    public static byte VERSION_MAJOR = 1;
+    /** Current minor version of the replication log format */
+    public static byte VERSION_MINOR = 0;
+
+    /** Size of the block checksum trailer (CRC64) */
+    public static int CHECKSUM_SIZE = Bytes.SIZEOF_LONG;
+
+    /** Represents the file header */
+    public interface Header {
+        int getMajorVersion();
+        Header setMajorVersion(int majorVersion);
+        int getMinorVersion();
+        Header setMinorVersion(int minorVersion);
+        int getSerializedLength();
+        void readFields(DataInput in) throws IOException;
+        void write(DataOutput out) throws IOException;
+    }
+
+    /** Represents the header of a single block within the log file */
+    public interface BlockHeader {
+        /** Magic number for Phoenix Replication Log blocks */
+        public static final byte[] MAGIC = Bytes.toBytes("PBLK");
+        /** Current version of the replication log block header */
+        public static final byte VERSION = 1;
+        int getVersion();
+        Compression.Algorithm getCompression();
+        BlockHeader setCompression(Compression.Algorithm compression);
+        int getUncompressedSize();
+        BlockHeader setUncompressedSize(int uncompressedSize);
+        int getCompressedSize();
+        BlockHeader setCompressedSize(int compressedSize);
+        int getSerializedLength();
+        void readFields(DataInput in) throws IOException;
+        void write(DataOutput out) throws IOException;
+    }
+
+    /** Represents the file trailer */
+    public interface Trailer {
+        int getMajorVersion();
+        Trailer setMajorVersion(int majorVersion);
+        int getMinorVersion();
+        Trailer setMinorVersion(int minorVersion);
+        long getRecordCount();
+        Trailer setRecordCount(long recordCount);
+        long getBlockCount();
+        Trailer setBlockCount(long blockCount);
+        long getBlocksStartOffset();
+        Trailer setBlocksStartOffset(long offset);
+        long getTrailerStartOffset();
+        Trailer setTrailerStartOffset(long offset);
+        int getSerializedLength();
+        void readFields(DataInput in) throws IOException;
+        void write(DataOutput out) throws IOException;
+    }
+
+    /** Represents a single logical change */
+    public interface Record {

Review Comment:
   Ok, I am working on a commit for this PR that addresses this review feedback:
    - Document LogFile interfaces with javadoc
    - Directly accept HBase Mutation objects in the Writer
    - Directly provide HBase Mutation objects from the Reader
   
   Additional changes:
    - Optimize the encoding and decoding of column family and qualifiers, to 
avoid string concat/splitting, changing the file format. Spec requires update!
    - Encode records with a single pass. Saves on lots of duplicate structure 
enumeration, requiring a small copy in trade.
    - LogFileCompressionTest test fix.
    
   I will push it up soon. 



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