001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase;
019
020import static org.junit.Assert.assertEquals;
021import static org.junit.Assert.assertTrue;
022
023import java.nio.ByteBuffer;
024import java.util.Collections;
025import java.util.Set;
026import java.util.TreeSet;
027import org.apache.hadoop.hbase.testclassification.MiscTests;
028import org.apache.hadoop.hbase.testclassification.SmallTests;
029import org.apache.hadoop.hbase.util.Bytes;
030import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
031import org.junit.ClassRule;
032import org.junit.Test;
033import org.junit.experimental.categories.Category;
034
035@Category({ MiscTests.class, SmallTests.class })
036public class TestCellComparator {
037
038  @ClassRule
039  public static final HBaseClassTestRule CLASS_RULE =
040    HBaseClassTestRule.forClass(TestCellComparator.class);
041
042  private CellComparator comparator = CellComparator.getInstance();
043  private CellComparator innerStoreComparator = InnerStoreCellComparator.INNER_STORE_COMPARATOR;
044
045  byte[] row1 = Bytes.toBytes("row1");
046  byte[] row2 = Bytes.toBytes("row2");
047  byte[] row_1_0 = Bytes.toBytes("row10");
048
049  byte[] fam0 = HConstants.EMPTY_BYTE_ARRAY;
050  byte[] fam1 = Bytes.toBytes("fam1");
051  byte[] fam2 = Bytes.toBytes("fam2");
052  byte[] fam_1_2 = Bytes.toBytes("fam12");
053
054  byte[] qual1 = Bytes.toBytes("qual1");
055  byte[] qual2 = Bytes.toBytes("qual2");
056
057  byte[] val = Bytes.toBytes("val");
058
059  @Test
060  public void testCompareCells() {
061    KeyValue kv1 = new KeyValue(row1, fam1, qual1, val);
062    KeyValue kv2 = new KeyValue(row2, fam1, qual1, val);
063    assertTrue(comparator.compare(kv1, kv2) < 0);
064
065    kv1 = new KeyValue(row1, fam2, qual1, val);
066    kv2 = new KeyValue(row1, fam1, qual1, val);
067    assertTrue(comparator.compareFamilies(kv1, kv2) > 0);
068
069    kv1 = new KeyValue(row1, fam1, qual1, 1L, val);
070    kv2 = new KeyValue(row1, fam1, qual1, 2L, val);
071    assertTrue(comparator.compare(kv1, kv2) > 0);
072
073    kv1 = new KeyValue(row1, fam1, qual1, 1L, KeyValue.Type.Put);
074    kv2 = new KeyValue(row1, fam1, qual1, 1L, KeyValue.Type.Maximum);
075    assertTrue(comparator.compare(kv1, kv2) > 0);
076
077    kv1 = new KeyValue(row1, fam1, qual1, 1L, KeyValue.Type.Put);
078    kv2 = new KeyValue(row1, fam1, qual1, 1L, KeyValue.Type.Put);
079    assertTrue(CellUtil.equals(kv1, kv2));
080  }
081
082  @Test
083  public void testCompareCellsWithEmptyFamily() {
084    KeyValue kv1 = new KeyValue(row1, fam0, qual1, val);
085    KeyValue kv2 = new KeyValue(row1, fam1, qual1, val);
086    assertTrue(comparator.compare(kv1, kv2) < 0);
087    assertTrue(innerStoreComparator.compare(kv1, kv2) < 0);
088
089    kv1 = new KeyValue(row1, fam0, qual2, val);
090    kv2 = new KeyValue(row1, fam0, qual1, val);
091    assertTrue(comparator.compare(kv1, kv2) > 0);
092    assertTrue(innerStoreComparator.compare(kv1, kv2) > 0);
093
094    kv1 = new KeyValue(row1, fam0, qual2, val);
095    kv2 = new KeyValue(row1, fam0, qual1, val);
096    assertTrue(comparator.compareFamilies(kv1, kv2) == 0);
097    assertTrue(innerStoreComparator.compareFamilies(kv1, kv2) == 0);
098
099    kv1 = new KeyValue(row1, fam1, qual2, val);
100    kv2 = new KeyValue(row1, fam1, qual1, val);
101    assertTrue(comparator.compareFamilies(kv1, kv2) == 0);
102    assertTrue(innerStoreComparator.compareFamilies(kv1, kv2) == 0);
103  }
104
105  @Test
106  public void testCompareCellWithKey() throws Exception {
107    KeyValue kv1 = new KeyValue(row1, fam1, qual1, val);
108    KeyValue kv2 = new KeyValue(row2, fam1, qual1, val);
109    assertTrue(PrivateCellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length) < 0);
110
111    kv1 = new KeyValue(row1, fam2, qual1, val);
112    kv2 = new KeyValue(row1, fam1, qual1, val);
113    assertTrue(PrivateCellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length) > 0);
114
115    kv1 = new KeyValue(row1, fam1, qual1, 1L, val);
116    kv2 = new KeyValue(row1, fam1, qual1, 2L, val);
117    assertTrue(PrivateCellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length) > 0);
118
119    kv1 = new KeyValue(row1, fam1, qual1, 1L, KeyValue.Type.Put);
120    kv2 = new KeyValue(row1, fam1, qual1, 1L, KeyValue.Type.Maximum);
121    assertTrue(PrivateCellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length) > 0);
122
123    kv1 = new KeyValue(row1, fam1, qual1, 1L, KeyValue.Type.Put);
124    kv2 = new KeyValue(row1, fam1, qual1, 1L, KeyValue.Type.Put);
125    assertTrue(PrivateCellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length) == 0);
126  }
127
128  @Test
129  public void testCompareByteBufferedCell() {
130    byte[] r1 = Bytes.toBytes("row1");
131    byte[] r2 = Bytes.toBytes("row2");
132    byte[] f1 = Bytes.toBytes("cf1");
133    byte[] q1 = Bytes.toBytes("qual1");
134    byte[] q2 = Bytes.toBytes("qual2");
135    byte[] v = Bytes.toBytes("val1");
136    KeyValue kv = new KeyValue(r1, f1, q1, v);
137    ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
138    Cell bbCell1 = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
139    kv = new KeyValue(r2, f1, q1, v);
140    buffer = ByteBuffer.wrap(kv.getBuffer());
141    Cell bbCell2 = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
142    // compareColumns not on CellComparator so use Impl directly
143    assertEquals(0, CellComparatorImpl.COMPARATOR.compareColumns(bbCell1, bbCell2));
144    assertEquals(0, CellComparatorImpl.COMPARATOR.compareColumns(bbCell1, kv));
145    kv = new KeyValue(r2, f1, q2, v);
146    buffer = ByteBuffer.wrap(kv.getBuffer());
147    Cell bbCell3 = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
148    assertEquals(0, comparator.compareFamilies(bbCell2, bbCell3));
149    assertTrue(comparator.compareQualifiers(bbCell2, bbCell3) < 0);
150    assertTrue(CellComparatorImpl.COMPARATOR.compareColumns(bbCell2, bbCell3) < 0);
151
152    assertEquals(0, comparator.compareRows(bbCell2, bbCell3));
153    assertTrue(comparator.compareRows(bbCell1, bbCell2) < 0);
154  }
155
156  /**
157   * Test meta comparisons using our new ByteBufferKeyValue Cell type, the type we use everywhere in
158   * 2.0.
159   */
160  @Test
161  public void testMetaComparisons() throws Exception {
162    long now = EnvironmentEdgeManager.currentTime();
163
164    // Meta compares
165    Cell aaa = createByteBufferKeyValueFromKeyValue(
166      new KeyValue(Bytes.toBytes("TestScanMultipleVersions,row_0500,1236020145502"), now));
167    Cell bbb = createByteBufferKeyValueFromKeyValue(
168      new KeyValue(Bytes.toBytes("TestScanMultipleVersions,,99999999999999"), now));
169    CellComparator c = MetaCellComparator.META_COMPARATOR;
170    assertTrue(c.compare(bbb, aaa) < 0);
171
172    Cell ccc = createByteBufferKeyValueFromKeyValue(
173      new KeyValue(Bytes.toBytes("TestScanMultipleVersions,,1236023996656"), Bytes.toBytes("info"),
174        Bytes.toBytes("regioninfo"), 1236024396271L, (byte[]) null));
175    assertTrue(c.compare(ccc, bbb) < 0);
176
177    Cell x = createByteBufferKeyValueFromKeyValue(
178      new KeyValue(Bytes.toBytes("TestScanMultipleVersions,row_0500,1236034574162"),
179        Bytes.toBytes("info"), Bytes.toBytes(""), 9223372036854775807L, (byte[]) null));
180    Cell y = createByteBufferKeyValueFromKeyValue(
181      new KeyValue(Bytes.toBytes("TestScanMultipleVersions,row_0500,1236034574162"),
182        Bytes.toBytes("info"), Bytes.toBytes("regioninfo"), 1236034574912L, (byte[]) null));
183    assertTrue(c.compare(x, y) < 0);
184  }
185
186  private static Cell createByteBufferKeyValueFromKeyValue(KeyValue kv) {
187    ByteBuffer bb = ByteBuffer.wrap(kv.getBuffer());
188    return new ByteBufferKeyValue(bb, 0, bb.remaining());
189  }
190
191  /**
192   * More tests using ByteBufferKeyValue copied over from TestKeyValue which uses old KVs only.
193   */
194  @Test
195  public void testMetaComparisons2() {
196    long now = EnvironmentEdgeManager.currentTime();
197    CellComparator c = MetaCellComparator.META_COMPARATOR;
198    assertTrue(c.compare(
199      createByteBufferKeyValueFromKeyValue(
200        new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,1"), now)),
201      createByteBufferKeyValueFromKeyValue(
202        new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,1"), now)))
203        == 0);
204    Cell a = createByteBufferKeyValueFromKeyValue(
205      new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,1"), now));
206    Cell b = createByteBufferKeyValueFromKeyValue(
207      new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,2"), now));
208    assertTrue(c.compare(a, b) < 0);
209    assertTrue(c.compare(
210      createByteBufferKeyValueFromKeyValue(
211        new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,2"), now)),
212      createByteBufferKeyValueFromKeyValue(
213        new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,1"), now)))
214        > 0);
215    assertTrue(c.compare(
216      createByteBufferKeyValueFromKeyValue(
217        new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,1"), now)),
218      createByteBufferKeyValueFromKeyValue(
219        new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,1"), now)))
220        == 0);
221    assertTrue(c.compare(
222      createByteBufferKeyValueFromKeyValue(
223        new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,1"), now)),
224      createByteBufferKeyValueFromKeyValue(
225        new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,2"), now)))
226        < 0);
227    assertTrue(c.compare(
228      createByteBufferKeyValueFromKeyValue(
229        new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,2"), now)),
230      createByteBufferKeyValueFromKeyValue(
231        new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,1"), now)))
232        > 0);
233  }
234
235  @Test
236  public void testBinaryKeys() throws Exception {
237    Set<Cell> set = new TreeSet<>(CellComparatorImpl.COMPARATOR);
238    final byte[] fam = Bytes.toBytes("col");
239    final byte[] qf = Bytes.toBytes("umn");
240    final byte[] nb = new byte[0];
241    Cell[] keys = {
242      createByteBufferKeyValueFromKeyValue(
243        new KeyValue(Bytes.toBytes("aaaaa,\u0000\u0000,2"), fam, qf, 2, nb)),
244      createByteBufferKeyValueFromKeyValue(
245        new KeyValue(Bytes.toBytes("aaaaa,\u0001,3"), fam, qf, 3, nb)),
246      createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("aaaaa,,1"), fam, qf, 1, nb)),
247      createByteBufferKeyValueFromKeyValue(
248        new KeyValue(Bytes.toBytes("aaaaa,\u1000,5"), fam, qf, 5, nb)),
249      createByteBufferKeyValueFromKeyValue(
250        new KeyValue(Bytes.toBytes("aaaaa,a,4"), fam, qf, 4, nb)),
251      createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("a,a,0"), fam, qf, 0, nb)), };
252    // Add to set with bad comparator
253    Collections.addAll(set, keys);
254    // This will output the keys incorrectly.
255    boolean assertion = false;
256    int count = 0;
257    for (Cell k : set) {
258      if (!(count++ == k.getTimestamp())) {
259        assertion = true;
260      }
261    }
262    assertTrue(assertion);
263    // Make set with good comparator
264    set = new TreeSet<>(MetaCellComparator.META_COMPARATOR);
265    Collections.addAll(set, keys);
266    count = 0;
267    for (Cell k : set) {
268      assertTrue("count=" + count + ", " + k.toString(), count++ == k.getTimestamp());
269    }
270  }
271}