This is an automated email from the ASF dual-hosted git repository. tdsilva pushed a commit to branch 4.14-HBase-1.4 in repository https://gitbox.apache.org/repos/asf/phoenix.git
commit 0eeb6c935da3c66fc8326d56128a1a4dbf0d4b0b Author: Geoffrey Jacoby <gjac...@apache.org> AuthorDate: Tue Mar 12 11:17:50 2019 -0700 PHOENIX-5188 - IndexedKeyValue should populate KeyValue fields --- .../phoenix/hbase/index/wal/IndexedKeyValue.java | 25 ++++++++ .../regionserver/wal/IndexedKeyValueTest.java | 67 ++++++++++++++++++++++ 2 files changed, 92 insertions(+) diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/wal/IndexedKeyValue.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/wal/IndexedKeyValue.java index b04cf0a..f01dc06 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/wal/IndexedKeyValue.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/wal/IndexedKeyValue.java @@ -53,6 +53,7 @@ public class IndexedKeyValue extends KeyValue { public IndexedKeyValue() {} public IndexedKeyValue(byte[] bs, Mutation mutation) { + super(mutation.getRow(), 0, mutation.getRow().length); this.indexTableName = new ImmutableBytesPtr(bs); this.mutation = mutation; this.hashCode = calcHashCode(indexTableName, mutation); @@ -117,6 +118,24 @@ public class IndexedKeyValue extends KeyValue { } @Override + public int getRowOffset() { + return this.offset; + } + + @Override + public short getRowLength() { + return (short) this.length; + } + + @Override + public int getKeyLength(){ + //normally the key is row key + other key fields such as timestamp, + // but those aren't defined here because a Mutation can contain multiple, + // so we just return the length of the row key + return this.length; + } + + @Override public String toString() { return "IndexWrite:\n\ttable: " + indexTableName + "\n\tmutation:" + mutation; } @@ -179,6 +198,12 @@ public class IndexedKeyValue extends KeyValue { MutationProto mProto = MutationProto.parseFrom(mutationData); this.mutation = org.apache.hadoop.hbase.protobuf.ProtobufUtil.toMutation(mProto); this.hashCode = calcHashCode(indexTableName, mutation); + if (mutation != null){ + bytes = mutation.getRow(); + offset = 0; + length = bytes.length; + } + } public boolean getBatchFinished() { diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/IndexedKeyValueTest.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/IndexedKeyValueTest.java new file mode 100644 index 0000000..7f34fcd --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/IndexedKeyValueTest.java @@ -0,0 +1,67 @@ +/* + * 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.hadoop.hbase.regionserver.wal; + +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.phoenix.hbase.index.wal.IndexedKeyValue; +import org.apache.phoenix.hbase.index.wal.KeyValueCodec; +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; + + +public class IndexedKeyValueTest { + + @Test + public void testIndexedKeyValuePopulatesKVFields() throws Exception { + byte[] row = Bytes.toBytes("foo"); + byte[] tableNameBytes = Bytes.toBytes("MyTableName"); + Mutation mutation = new Put(row); + IndexedKeyValue indexedKeyValue = new IndexedKeyValue(tableNameBytes, mutation); + testIndexedKeyValueHelper(indexedKeyValue, row, tableNameBytes, mutation); + + //now serialize the IndexedKeyValue and make sure the deserialized copy also + //has all the right fields + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos); + KeyValueCodec.write(out, indexedKeyValue); + + IndexedKeyValue deSerializedKV = (IndexedKeyValue) + KeyValueCodec.readKeyValue(new DataInputStream( + new ByteArrayInputStream(baos.toByteArray()))); + testIndexedKeyValueHelper(deSerializedKV, row, tableNameBytes, mutation); + + } + + private void testIndexedKeyValueHelper(IndexedKeyValue indexedKeyValue, byte[] row, byte[] tableNameBytes, Mutation mutation) { + Assert.assertArrayEquals(row, indexedKeyValue.getRowArray()); + Assert.assertEquals(0, indexedKeyValue.getRowOffset()); + Assert.assertEquals(row.length, indexedKeyValue.getRowLength()); + Assert.assertArrayEquals(tableNameBytes, indexedKeyValue.getIndexTable()); + Assert.assertEquals(mutation.toString(), indexedKeyValue.getMutation().toString()); + Assert.assertArrayEquals(WALEdit.METAFAMILY, indexedKeyValue.getFamilyArray()); + } + +}