You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2010/12/20 18:30:44 UTC

svn commit: r1051218 - in /hbase/trunk/src: main/java/org/apache/hadoop/hbase/filter/BitComparator.java test/java/org/apache/hadoop/hbase/filter/TestBitComparator.java

Author: stack
Date: Mon Dec 20 17:30:44 2010
New Revision: 1051218

URL: http://svn.apache.org/viewvc?rev=1051218&view=rev
Log:
HBASE-3335 Add BitComparator for filtering

Added:
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestBitComparator.java

Added: hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java?rev=1051218&view=auto
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java (added)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java Mon Dec 20 17:30:44 2010
@@ -0,0 +1,92 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * A bit comparator which performs the specified bitwise operation on each of the bytes
+ * with the specified byte array. Then returns whether the result is non-zero.
+ */
+public class BitComparator extends WritableByteArrayComparable {
+
+  /** Nullary constructor for Writable, do not use */
+  public BitComparator() {}
+
+  /** Bit operators. */
+  public enum BitwiseOp {
+    /** and */
+    AND,
+    /** or */
+    OR,
+    /** xor */
+    XOR
+  }
+  protected BitwiseOp bitOperator;
+
+  /**
+   * Constructor
+   * @param value value
+   * @param BitwiseOp bitOperator - the operator to use on the bit comparison
+   */
+  public BitComparator(byte[] value, BitwiseOp bitOperator) {
+    super(value);
+    this.bitOperator = bitOperator;
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    bitOperator = BitwiseOp.valueOf(in.readUTF());
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    out.writeUTF(bitOperator.name());
+  }
+
+  @Override
+  public int compareTo(byte[] value) {
+    if (value.length != this.value.length) {
+      return 1;
+    }
+    int b = 0;
+    //Iterating backwards is faster because we can quit after one non-zero byte.
+    for (int i = value.length - 1; i >= 0 && b == 0; i--) {
+      switch (bitOperator) {
+        case AND:
+          b = (this.value[i] & value[i]) & 0xff;
+          break;
+        case OR:
+          b = (this.value[i] | value[i]) & 0xff;
+          break;
+        case XOR:
+          b = (this.value[i] ^ value[i]) & 0xff;
+          break;
+      }
+    }
+    return b == 0 ? 1 : 0;
+  }
+}
+

Added: hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestBitComparator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestBitComparator.java?rev=1051218&view=auto
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestBitComparator.java (added)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestBitComparator.java Mon Dec 20 17:30:44 2010
@@ -0,0 +1,64 @@
+/**
+ *  Copyright 2010 The Apache Software Foundation
+ * 
+ *  Licensed 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.
+ *  under the License.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import junit.framework.TestCase;
+
+/**
+ * Tests for the bit comparator
+ */
+public class TestBitComparator extends TestCase {
+
+  private static byte[] zeros = new byte[]{0, 0, 0, 0, 0, 0};
+  private static byte[] ones = new byte[]{1, 1, 1, 1, 1, 1};
+  private static byte[] data0 = new byte[]{0, 1, 2, 4, 8, 15};
+  private static byte[] data1 = new byte[]{15, 0, 0, 0, 0, 0};
+  private static byte[] data2 = new byte[]{0, 0, 0, 0, 0, 15};
+  private static byte[] data3 = new byte[]{15, 15, 15, 15, 15};
+  private final int Equal = 0;
+  private final int NotEqual = 1;
+
+  public void testANDOperation() {
+    testOperation(zeros, ones, BitComparator.BitwiseOp.AND, NotEqual);
+    testOperation(data1, ones, BitComparator.BitwiseOp.AND, Equal);
+    testOperation(data1, data0, BitComparator.BitwiseOp.AND, NotEqual);
+    testOperation(data2, data1, BitComparator.BitwiseOp.AND, NotEqual);
+    testOperation(ones, data0, BitComparator.BitwiseOp.AND, Equal);
+    testOperation(ones, data3, BitComparator.BitwiseOp.AND, NotEqual);
+  }
+
+  public void testOROperation() {
+    testOperation(ones, zeros, BitComparator.BitwiseOp.OR, Equal);
+    testOperation(zeros, zeros, BitComparator.BitwiseOp.OR, NotEqual);
+    testOperation(data1, zeros, BitComparator.BitwiseOp.OR, Equal);
+    testOperation(data2, data1, BitComparator.BitwiseOp.OR, Equal);
+    testOperation(ones, data3, BitComparator.BitwiseOp.OR, NotEqual);
+  }
+
+  public void testXOROperation() {
+    testOperation(ones, zeros, BitComparator.BitwiseOp.XOR, Equal);
+    testOperation(zeros, zeros, BitComparator.BitwiseOp.XOR, NotEqual);
+    testOperation(ones, ones, BitComparator.BitwiseOp.XOR, NotEqual);
+    testOperation(data2, data1, BitComparator.BitwiseOp.XOR, Equal);
+    testOperation(ones, data3, BitComparator.BitwiseOp.XOR, NotEqual);
+  }
+
+  private void testOperation(byte[] data, byte[] comparatorBytes, BitComparator.BitwiseOp operator, int expected) {
+    BitComparator comparator = new BitComparator(comparatorBytes, operator);
+    assertEquals(comparator.compareTo(data), expected);
+  }
+}