You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2016/09/29 06:57:14 UTC

[29/50] [abbrv] hbase git commit: HBASE-16691 Optimize KeyOnlyFilter by utilizing KeyOnlyCell - revert due to TestFilter failure

HBASE-16691 Optimize KeyOnlyFilter by utilizing KeyOnlyCell - revert due to TestFilter failure


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/aa4a678e
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/aa4a678e
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/aa4a678e

Branch: refs/heads/hbase-14439
Commit: aa4a678ebf7e54f84432ec36f3a3e4dd6359315f
Parents: f196a8c
Author: tedyu <yu...@gmail.com>
Authored: Tue Sep 27 03:37:54 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Sep 27 03:37:54 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/filter/KeyOnlyFilter.java      | 294 +------------------
 .../hadoop/hbase/filter/TestKeyOnlyFilter.java  | 127 --------
 2 files changed, 15 insertions(+), 406 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/aa4a678e/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
index 1a0d2af..2fd5aba 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
@@ -20,12 +20,11 @@ package org.apache.hadoop.hbase.filter;
 
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 
-import org.apache.hadoop.hbase.ByteBufferedCell;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -62,11 +61,19 @@ public class KeyOnlyFilter extends FilterBase {
   }
 
   private Cell createKeyOnlyCell(Cell c) {
-    if (c instanceof ByteBufferedCell) {
-      return new KeyOnlyByteBufferedCell((ByteBufferedCell) c, lenAsVal);
-    } else {
-      return new KeyOnlyCell(c, lenAsVal);
-    }
+    // KV format: <keylen:4><valuelen:4><key:keylen><value:valuelen>
+    // Rebuild as: <keylen:4><0:4><key:keylen>
+    int dataLen = lenAsVal ? Bytes.SIZEOF_INT : 0;
+    int keyOffset = (2 * Bytes.SIZEOF_INT);
+    int keyLen = KeyValueUtil.keyLength(c);
+    byte[] newBuffer = new byte[keyLen + keyOffset + dataLen];
+    Bytes.putInt(newBuffer, 0, keyLen);
+    Bytes.putInt(newBuffer, Bytes.SIZEOF_INT, dataLen);
+    KeyValueUtil.appendKeyTo(c, newBuffer, keyOffset);
+    if (lenAsVal) {
+      Bytes.putInt(newBuffer, newBuffer.length - dataLen, c.getValueLength());
+    }
+    return new KeyValue(newBuffer);
   }
 
   @Override
@@ -123,275 +130,4 @@ public class KeyOnlyFilter extends FilterBase {
     KeyOnlyFilter other = (KeyOnlyFilter)o;
     return this.lenAsVal == other.lenAsVal;
   }
-
-  static class KeyOnlyCell implements Cell {
-    private Cell cell;
-    private boolean lenAsVal;
-
-    public KeyOnlyCell(Cell c, boolean lenAsVal) {
-      this.cell = c;
-      this.lenAsVal = lenAsVal;
-    }
-
-    @Override
-    public byte[] getRowArray() {
-      return cell.getRowArray();
-    }
-
-    @Override
-    public int getRowOffset() {
-      return cell.getRowOffset();
-    }
-
-    @Override
-    public short getRowLength() {
-      return cell.getRowLength();
-    }
-
-    @Override
-    public byte[] getFamilyArray() {
-      return cell.getFamilyArray();
-    }
-
-    @Override
-    public int getFamilyOffset() {
-      return cell.getFamilyOffset();
-    }
-
-    @Override
-    public byte getFamilyLength() {
-      return cell.getFamilyLength();
-    }
-
-    @Override
-    public byte[] getQualifierArray() {
-      return cell.getQualifierArray();
-    }
-
-    @Override
-    public int getQualifierOffset() {
-      return cell.getQualifierOffset();
-    }
-
-    @Override
-    public int getQualifierLength() {
-      return cell.getQualifierLength();
-    }
-
-    @Override
-    public long getTimestamp() {
-      return cell.getTimestamp();
-    }
-
-    @Override
-    public byte getTypeByte() {
-      return cell.getTypeByte();
-    }
-
-    @Override
-    public long getSequenceId() {
-      return 0;
-    }
-
-    @Override
-    public byte[] getValueArray() {
-      if (lenAsVal) {
-        return Bytes.toBytes(cell.getValueLength());
-      } else {
-        return HConstants.EMPTY_BYTE_ARRAY;
-      }
-    }
-
-    @Override
-    public int getValueOffset() {
-      return 0;
-    }
-
-    @Override
-    public int getValueLength() {
-      if (lenAsVal) {
-        return Bytes.SIZEOF_INT;
-      } else {
-        return 0;
-      }
-    }
-
-    @Override
-    public byte[] getTagsArray() {
-      return HConstants.EMPTY_BYTE_ARRAY;
-    }
-
-    @Override
-    public int getTagsOffset() {
-      return 0;
-    }
-
-    @Override
-    public int getTagsLength() {
-      return 0;
-    }
-  }
-
-  static class KeyOnlyByteBufferedCell extends ByteBufferedCell {
-    private ByteBufferedCell cell;
-    private boolean lenAsVal;
-
-    public KeyOnlyByteBufferedCell(ByteBufferedCell c, boolean lenAsVal) {
-      this.cell = c;
-      this.lenAsVal = lenAsVal;
-    }
-
-    @Override
-    public byte[] getRowArray() {
-      return cell.getRowArray();
-    }
-
-    @Override
-    public int getRowOffset() {
-      return cell.getRowOffset();
-    }
-
-    @Override
-    public short getRowLength() {
-      return cell.getRowLength();
-    }
-
-    @Override
-    public byte[] getFamilyArray() {
-      return cell.getFamilyArray();
-    }
-
-    @Override
-    public int getFamilyOffset() {
-      return cell.getFamilyOffset();
-    }
-
-    @Override
-    public byte getFamilyLength() {
-      return cell.getFamilyLength();
-    }
-
-    @Override
-    public byte[] getQualifierArray() {
-      return cell.getQualifierArray();
-    }
-
-    @Override
-    public int getQualifierOffset() {
-      return cell.getQualifierOffset();
-    }
-
-    @Override
-    public int getQualifierLength() {
-      return cell.getQualifierLength();
-    }
-
-    @Override
-    public long getTimestamp() {
-      return cell.getTimestamp();
-    }
-
-    @Override
-    public byte getTypeByte() {
-      return cell.getTypeByte();
-    }
-
-    @Override
-    public long getSequenceId() {
-      return 0;
-    }
-
-    @Override
-    public byte[] getValueArray() {
-      if (lenAsVal) {
-        return Bytes.toBytes(cell.getValueLength());
-      } else {
-        return HConstants.EMPTY_BYTE_ARRAY;
-      }
-    }
-
-    @Override
-    public int getValueOffset() {
-      return 0;
-    }
-
-    @Override
-    public int getValueLength() {
-      if (lenAsVal) {
-        return Bytes.SIZEOF_INT;
-      } else {
-        return 0;
-      }
-    }
-
-    @Override
-    public byte[] getTagsArray() {
-      return HConstants.EMPTY_BYTE_ARRAY;
-    }
-
-    @Override
-    public int getTagsOffset() {
-      return 0;
-    }
-
-    @Override
-    public int getTagsLength() {
-      return 0;
-    }
-
-    @Override
-    public ByteBuffer getRowByteBuffer() {
-      return cell.getRowByteBuffer();
-    }
-
-    @Override
-    public int getRowPosition() {
-      return cell.getRowPosition();
-    }
-
-    @Override
-    public ByteBuffer getFamilyByteBuffer() {
-      return cell.getFamilyByteBuffer();
-    }
-
-    @Override
-    public int getFamilyPosition() {
-      return cell.getFamilyPosition();
-    }
-
-    @Override
-    public ByteBuffer getQualifierByteBuffer() {
-      return cell.getQualifierByteBuffer();
-    }
-
-    @Override
-    public int getQualifierPosition() {
-      return cell.getQualifierPosition();
-    }
-
-    @Override
-    public ByteBuffer getValueByteBuffer() {
-      if (lenAsVal) {
-        return ByteBuffer.wrap(Bytes.toBytes(cell.getValueLength()));
-      } else {
-        return HConstants.EMPTY_BYTE_BUFFER;
-      }
-    }
-
-    @Override
-    public int getValuePosition() {
-      return 0;
-    }
-
-    @Override
-    public ByteBuffer getTagsByteBuffer() {
-      return HConstants.EMPTY_BYTE_BUFFER;
-    }
-
-    @Override
-    public int getTagsPosition() {
-      return 0;
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa4a678e/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
deleted file mode 100644
index 31f7904..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/**
- * 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.filter;
-
-import static org.junit.Assert.assertTrue;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.TestCellUtil.ByteBufferedCellImpl;
-import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyByteBufferedCell;
-import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyCell;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-@Category({ MiscTests.class, SmallTests.class })
-@RunWith(Parameterized.class)
-public class TestKeyOnlyFilter {
-
-  private final boolean lenAsVal;
-
-  @Parameters
-  public static Collection<Object[]> parameters() {
-    List<Object[]> paramList = new ArrayList<Object[]>();
-    {
-      paramList.add(new Object[] { false });
-      paramList.add(new Object[] { true });
-    }
-    return paramList;
-  }
-
-  public TestKeyOnlyFilter(boolean lenAsVal) {
-    this.lenAsVal = lenAsVal;
-  }
-
-  @Test
-  public void testKeyOnly() throws Exception {
-    byte[] r = Bytes.toBytes("row1");
-    byte[] f = Bytes.toBytes("cf1");
-    byte[] q = Bytes.toBytes("qual1");
-    byte[] v = Bytes.toBytes("val1");
-    byte[] tags = Bytes.toBytes("tag1");
-    KeyValue kv = new KeyValue(r, f, q, 0, q.length, 1234L, Type.Put, v, 0,
-        v.length, tags);
-
-    ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
-    ByteBufferedCellImpl bbCell = new ByteBufferedCellImpl(buffer, 0,
-        buffer.remaining());
-
-    // KV format: <keylen:4><valuelen:4><key:keylen><value:valuelen>
-    // Rebuild as: <keylen:4><0:4><key:keylen>
-    int dataLen = lenAsVal ? Bytes.SIZEOF_INT : 0;
-    int keyOffset = (2 * Bytes.SIZEOF_INT);
-    int keyLen = KeyValueUtil.keyLength(kv);
-    byte[] newBuffer = new byte[keyLen + keyOffset + dataLen];
-    Bytes.putInt(newBuffer, 0, keyLen);
-    Bytes.putInt(newBuffer, Bytes.SIZEOF_INT, dataLen);
-    KeyValueUtil.appendKeyTo(kv, newBuffer, keyOffset);
-    if (lenAsVal) {
-      Bytes.putInt(newBuffer, newBuffer.length - dataLen, kv.getValueLength());
-    }
-    KeyValue KeyOnlyKeyValue = new KeyValue(newBuffer);
-
-    KeyOnlyCell keyOnlyCell = new KeyOnlyCell(kv, lenAsVal);
-    KeyOnlyByteBufferedCell keyOnlyByteBufferedCell = new KeyOnlyByteBufferedCell(
-        bbCell, lenAsVal);
-
-    assertTrue(CellUtil.matchingRows(KeyOnlyKeyValue, keyOnlyCell));
-    assertTrue(CellUtil.matchingRows(KeyOnlyKeyValue, keyOnlyByteBufferedCell));
-
-    assertTrue(CellUtil.matchingFamily(KeyOnlyKeyValue, keyOnlyCell));
-    assertTrue(CellUtil
-        .matchingFamily(KeyOnlyKeyValue, keyOnlyByteBufferedCell));
-
-    assertTrue(CellUtil.matchingQualifier(KeyOnlyKeyValue, keyOnlyCell));
-    assertTrue(CellUtil.matchingQualifier(KeyOnlyKeyValue,
-        keyOnlyByteBufferedCell));
-
-    assertTrue(CellUtil.matchingValue(KeyOnlyKeyValue, keyOnlyCell));
-    assertTrue(KeyOnlyKeyValue.getValueLength() == keyOnlyByteBufferedCell
-        .getValueLength());
-    if (keyOnlyByteBufferedCell.getValueLength() > 0) {
-      assertTrue(CellUtil.matchingValue(KeyOnlyKeyValue,
-          keyOnlyByteBufferedCell));
-    }
-
-    assertTrue(KeyOnlyKeyValue.getTimestamp() == keyOnlyCell.getTimestamp());
-    assertTrue(KeyOnlyKeyValue.getTimestamp() == keyOnlyByteBufferedCell
-        .getTimestamp());
-
-    assertTrue(KeyOnlyKeyValue.getTypeByte() == keyOnlyCell.getTypeByte());
-    assertTrue(KeyOnlyKeyValue.getTypeByte() == keyOnlyByteBufferedCell
-        .getTypeByte());
-
-    assertTrue(KeyOnlyKeyValue.getTagsLength() == keyOnlyCell.getTagsLength());
-    assertTrue(KeyOnlyKeyValue.getTagsLength() == keyOnlyByteBufferedCell
-        .getTagsLength());
-  }
-
-}