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


##########
phoenix-core-server/src/main/java/org/apache/phoenix/replication/log/LogFileCodec.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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:
+ * - Total Record Length (vint)
+ * - Mutation Type (byte)
+ * - Schema Object Name Length (vint)
+ * - Schema Object Name Bytes (byte[])
+ * - Commit ID / SCN (vlong)
+ * - Row Key Length (vint)
+ * - Row Key Bytes (byte[])
+ * - Timestamp (long)
+ * - Number of Columns (vint)
+ * - For each column:
+ *   - Column Name Length (vint)
+ *   - Column Name Bytes (byte[])
+ *   - Value Length (vint)
+ *   - Value Bytes (byte[])
+ */
+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 colCount = familyMap.size();
+            WritableUtils.writeVInt(recordOut, colCount);
+
+            for (Map.Entry<byte[], List<Cell>> entry: familyMap.entrySet()) {
+                byte[] column = entry.getKey();

Review Comment:
   This is a column family, so we should rename it to say cf or columnFamily



##########
phoenix-core-server/src/main/java/org/apache/phoenix/replication/log/LogFileCodec.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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:
+ * - Total Record Length (vint)
+ * - Mutation Type (byte)
+ * - Schema Object Name Length (vint)
+ * - Schema Object Name Bytes (byte[])
+ * - Commit ID / SCN (vlong)
+ * - Row Key Length (vint)
+ * - Row Key Bytes (byte[])
+ * - Timestamp (long)
+ * - Number of Columns (vint)
+ * - For each column:
+ *   - Column Name Length (vint)
+ *   - Column Name Bytes (byte[])
+ *   - Value Length (vint)
+ *   - Value Bytes (byte[])
+ */
+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 colCount = familyMap.size();

Review Comment:
   This should renamed to cfCount



##########
phoenix-core-server/src/main/java/org/apache/phoenix/replication/log/AsyncFSDataOutput.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.IOException;
+import java.io.InterruptedIOException;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutput;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * SyncableDataOutput implementation that delegates to a hbase-async 
AsyncFSOutput.
+ */
[email protected](value="EI_EXPOSE_REP2",
+    justification="Intentional")
+public class AsyncFSDataOutput implements SyncableDataOutput {
+
+    private final AsyncFSOutput delegate;
+
+    public AsyncFSDataOutput(AsyncFSOutput delegate) {
+        this.delegate = delegate;
+    }
+
+    @Override
+    public void sync() throws IOException {
+        try {
+            // Our sync method is synchronous, as intended.
+            delegate.flush(true).get();
+        } catch (InterruptedException e) {
+            InterruptedIOException ioe = new InterruptedIOException();
+            ioe.initCause(e);
+            throw ioe;
+        } catch (ExecutionException e) {
+            throw new IOException(e);
+        }
+    }
+
+    @Override
+    public long getPos() throws IOException {
+        return delegate.getSyncedLength() + delegate.buffered();
+    }
+
+    @Override
+    public void close() throws IOException {
+        delegate.close();
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+        delegate.writeInt(b);
+    }
+
+    @Override
+    public void write(byte[] b) throws IOException {
+        delegate.write(b);
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+        delegate.write(b, off, len);
+    }
+
+    // The below DataOutput interface methods must be adapted to AsyncFSOutput 
because
+    // AsyncFSOutput only implements write(byte[]), write(byte[], int, int) 
and writeInt(int).
+    // Most will never be called. We only need these so we can conform to the 
SyncableDataOutput
+    // interface contract. The methods we really care about for performance 
have been directly
+    // delegated above.
+
+    private byte[] byteBuf = new byte[1];
+
+    @Override
+    public void writeByte(int v) throws IOException {
+        byteBuf[0] = (byte) v;
+        delegate.write(byteBuf);
+    }
+
+    @Override
+    public void writeBoolean(boolean v) throws IOException {
+        writeByte(v ? 1 : 0);
+    }
+
+    @Override
+    public void writeShort(int v) throws IOException {
+        writeByte((v >>> 8) & 0xFF);
+        writeByte((v >>> 0) & 0xFF);
+    }
+
+    @Override
+    public void writeChar(int v) throws IOException {
+        writeByte((v >>> 8) & 0xFF);
+        writeByte((v >>> 0) & 0xFF);

Review Comment:
   Should this be writeByte(v & 0xFF)? Does unsigned right shifting mean no-op?



##########
phoenix-core-server/src/main/java/org/apache/phoenix/replication/log/LogFileCodec.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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:
+ * - Total Record Length (vint)
+ * - Mutation Type (byte)
+ * - Schema Object Name Length (vint)

Review Comment:
   HBase Table Name



##########
phoenix-core-server/src/main/java/org/apache/phoenix/replication/log/LogFileCodec.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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:
+ * - Total Record Length (vint)
+ * - Mutation Type (byte)
+ * - Schema Object Name Length (vint)
+ * - Schema Object Name Bytes (byte[])
+ * - Commit ID / SCN (vlong)
+ * - Row Key Length (vint)
+ * - Row Key Bytes (byte[])
+ * - Timestamp (long)
+ * - Number of Columns (vint)

Review Comment:
   Need to update to include a column family name before column name-values.



##########
phoenix-core-server/src/main/java/org/apache/phoenix/replication/log/AsyncFSDataOutput.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.IOException;
+import java.io.InterruptedIOException;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutput;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * SyncableDataOutput implementation that delegates to a hbase-async 
AsyncFSOutput.
+ */
[email protected](value="EI_EXPOSE_REP2",
+    justification="Intentional")
+public class AsyncFSDataOutput implements SyncableDataOutput {
+
+    private final AsyncFSOutput delegate;
+
+    public AsyncFSDataOutput(AsyncFSOutput delegate) {
+        this.delegate = delegate;
+    }
+
+    @Override
+    public void sync() throws IOException {
+        try {
+            // Our sync method is synchronous, as intended.
+            delegate.flush(true).get();
+        } catch (InterruptedException e) {
+            InterruptedIOException ioe = new InterruptedIOException();
+            ioe.initCause(e);
+            throw ioe;
+        } catch (ExecutionException e) {
+            throw new IOException(e);
+        }
+    }
+
+    @Override
+    public long getPos() throws IOException {
+        return delegate.getSyncedLength() + delegate.buffered();
+    }
+
+    @Override
+    public void close() throws IOException {
+        delegate.close();
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+        delegate.writeInt(b);
+    }
+
+    @Override
+    public void write(byte[] b) throws IOException {
+        delegate.write(b);
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+        delegate.write(b, off, len);
+    }
+
+    // The below DataOutput interface methods must be adapted to AsyncFSOutput 
because
+    // AsyncFSOutput only implements write(byte[]), write(byte[], int, int) 
and writeInt(int).
+    // Most will never be called. We only need these so we can conform to the 
SyncableDataOutput
+    // interface contract. The methods we really care about for performance 
have been directly
+    // delegated above.
+
+    private byte[] byteBuf = new byte[1];
+
+    @Override
+    public void writeByte(int v) throws IOException {
+        byteBuf[0] = (byte) v;
+        delegate.write(byteBuf);
+    }
+
+    @Override
+    public void writeBoolean(boolean v) throws IOException {
+        writeByte(v ? 1 : 0);
+    }
+
+    @Override
+    public void writeShort(int v) throws IOException {
+        writeByte((v >>> 8) & 0xFF);
+        writeByte((v >>> 0) & 0xFF);
+    }
+
+    @Override
+    public void writeChar(int v) throws IOException {
+        writeByte((v >>> 8) & 0xFF);
+        writeByte((v >>> 0) & 0xFF);
+    }
+
+    @Override
+    public void writeInt(int v) throws IOException {
+        delegate.writeInt(v);
+    }
+
+    @Override
+    public void writeLong(long v) throws IOException {
+        delegate.write(Bytes.toBytes(v));
+    }
+
+    @Override
+    public void writeFloat(float v) throws IOException {
+        delegate.write(Bytes.toBytes(v));
+    }
+
+    @Override
+    public void writeDouble(double v) throws IOException {
+        delegate.write(Bytes.toBytes(v));
+    }
+
+    @Override
+    public void writeBytes(String s) throws IOException {
+        writeUTF(s); // Simplify here by unconditionally coding strings as 
UTF-8.
+    }
+
+    @Override
+    public void writeChars(String s) throws IOException {
+        int len = s.length();
+        for (int i = 0; i < len; i++) {
+            int v = s.charAt(i);
+            writeByte((v >>> 8) & 0xFF);

Review Comment:
   Should we call writeChar(v) here?



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