apurtell commented on code in PR #2133: URL: https://github.com/apache/phoenix/pull/2133#discussion_r2069081631
########## phoenix-core-server/src/main/java/org/apache/phoenix/replication/log/LogFileCodec.java: ########## @@ -0,0 +1,288 @@ +/* + * 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.ByteArrayOutputStream; +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.io.ByteBuffInputStream; +import org.apache.hadoop.hbase.nio.ByteBuff; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.WritableUtils; + +/** + * Default Codec for encoding and decoding ReplicationLog Records within a block buffer. + * This implementation uses standard Java DataInput/DataOutput for serialization. + * + * Record Format within a block: + * <pre> + * +--------------------------------------------+ + * | RECORD LENGTH (vint) | + * +--------------------------------------------+ + * | RECORD HEADER | + * | - Mutation type (byte) | + * | - HBase table name length (vint) | + * | - HBase table name (byte[]) | + * | - Transaction/SCN or commit ID (vint) | + * +--------------------------------------------+ + * | ROW KEY LENGTH (vint) | + * | ROW KEY (byte[]) | + * +--------------------------------------------+ + * | MUTATION TIMESTAMP (vint) | + * +--------------------------------------------+ + * | NUMBER OF COLUMN FAMILIES CHANGED (vint) | + * +--------------------------------------------+ + * | PER-FAMILY DATA (repeated) | + * | +--------------------------------------+ | + * | | COLUMN FAMILY NAME LENGTH (vint) | | + * | | COLUMN FAMILY NAME (byte[]) | | + * | | NUMBER OF CELLS IN FAMILY (vint) | | + * | +--------------------------------------+ | + * | | PER-CELL DATA (repeated) | | + * | | +––––––––––––----------------–--–+ | | + * | | | COLUMN QUALIFIER LENGTH (vint) | | | + * | | | COLUMN QUALIFIER (byte[]) | | | + * | | | VALUE LENGTH (vint) | | | + * | | | VALUE (byte[]) | | | + * | | +–––––––––––––--------------––--–+ | | + * | +--------------------------------------+ | + * +--------------------------------------------+ + * </pre> + */ +public class LogFileCodec implements LogFile.Codec { + + @Override + public Encoder getEncoder(DataOutput out) { + return new RecordEncoder(out); + } + + @Override + public Decoder getDecoder(DataInput in) { + return new RecordDecoder(in); + } + + @Override + public Decoder getDecoder(ByteBuffer buffer) { + // We use HBase ByteBuff and ByteBuffInputStream which avoids copying, because our buffers + // are known to be heap based. + ByteBuff wrapByteBuf = ByteBuff.wrap(buffer); + try { + return new RecordDecoder(new DataInputStream(new ByteBuffInputStream(wrapByteBuf))); + } finally { + wrapByteBuf.release(); + } + } + + private static class RecordEncoder implements LogFile.Codec.Encoder { + private final DataOutput out; + private final ByteArrayOutputStream currentRecord; + + RecordEncoder(DataOutput out) { + this.out = out; + currentRecord = new ByteArrayOutputStream(); + } + + + @Override + public void write(LogFile.Record record) throws IOException { + + DataOutput recordOut = new DataOutputStream(currentRecord); + + // Write record fields + + Mutation mutation = record.getMutation(); + LogFileRecord.MutationType mutationType = LogFileRecord.MutationType.get(mutation); + recordOut.writeByte(mutationType.getCode()); + byte[] nameBytes = record.getHBaseTableName().getBytes(StandardCharsets.UTF_8); + WritableUtils.writeVInt(recordOut, nameBytes.length); + recordOut.write(nameBytes); + WritableUtils.writeVLong(recordOut, record.getCommitId()); + byte[] rowKey = mutation.getRow(); + WritableUtils.writeVInt(recordOut, rowKey.length); + recordOut.write(rowKey); + recordOut.writeLong(mutation.getTimestamp()); + + Map<byte[], List<Cell>> familyMap = mutation.getFamilyCellMap(); + int cfCount = familyMap.size(); + WritableUtils.writeVInt(recordOut, cfCount); + + for (Map.Entry<byte[], List<Cell>> entry: familyMap.entrySet()) { + byte[] columnFamily = entry.getKey(); + WritableUtils.writeVInt(recordOut, columnFamily.length); + recordOut.write(columnFamily); + List<Cell> cells = entry.getValue(); + WritableUtils.writeVInt(recordOut, cells.size()); + for (Cell cell: cells) { + WritableUtils.writeVInt(recordOut, cell.getQualifierLength()); + recordOut.write(cell.getQualifierArray(), cell.getQualifierOffset(), + cell.getQualifierLength()); + WritableUtils.writeVInt(recordOut, cell.getValueLength()); + if (cell.getValueLength() > 0) { + recordOut.write(cell.getValueArray(), cell.getValueOffset(), + cell.getValueLength()); + } + } + } + + byte[] currentRecordBytes = currentRecord.toByteArray(); + // Write total record length + WritableUtils.writeVInt(out, currentRecordBytes.length); + // Write the record + out.write(currentRecordBytes); + + // Set the size (including the vint prefix) on the record object + ((LogFileRecord) record).setSerializedLength(currentRecordBytes.length + + WritableUtils.getVIntSize(currentRecordBytes.length)); + + // Reset the ByteArrayOutputStream to release resources + currentRecord.reset(); + } + } + + private static class RecordDecoder implements LogFile.Codec.Decoder { + private final DataInput in; + // A reference to the object populated by the last successful advance() + private LogFileRecord current = null; + + RecordDecoder(DataInput in) { + this.in = in; + } + + @Override + public boolean advance(LogFile.Record reuse) throws IOException { + try { + int recordDataLength = WritableUtils.readVInt(in); Review Comment: The EOFException is only for the short read case. I.e. the file is corrupt and truncated. So we would unexpectedly hit the EOF condition even if tracking an expected position/length. -- 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: issues-unsubscr...@phoenix.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org