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.filter; 019 020import static org.junit.Assert.assertArrayEquals; 021 022import java.io.IOException; 023import java.util.ArrayList; 024import java.util.List; 025import org.apache.hadoop.conf.Configuration; 026import org.apache.hadoop.hbase.Cell; 027import org.apache.hadoop.hbase.CellUtil; 028import org.apache.hadoop.hbase.HBaseClassTestRule; 029import org.apache.hadoop.hbase.HBaseTestingUtil; 030import org.apache.hadoop.hbase.HConstants; 031import org.apache.hadoop.hbase.PrivateCellUtil; 032import org.apache.hadoop.hbase.TableName; 033import org.apache.hadoop.hbase.client.Put; 034import org.apache.hadoop.hbase.client.Result; 035import org.apache.hadoop.hbase.client.ResultScanner; 036import org.apache.hadoop.hbase.client.Scan; 037import org.apache.hadoop.hbase.client.Table; 038import org.apache.hadoop.hbase.exceptions.DeserializationException; 039import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; 040import org.apache.hadoop.hbase.testclassification.FilterTests; 041import org.apache.hadoop.hbase.testclassification.MediumTests; 042import org.apache.hadoop.hbase.util.Bytes; 043import org.junit.AfterClass; 044import org.junit.BeforeClass; 045import org.junit.ClassRule; 046import org.junit.Rule; 047import org.junit.Test; 048import org.junit.experimental.categories.Category; 049import org.junit.rules.TestName; 050 051@Category({ FilterTests.class, MediumTests.class }) 052public class TestSeekHints { 053 054 private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); 055 private static String cf = "f"; 056 private static String cq = "q"; 057 private static String table = "t"; 058 private static Table ht; 059 060 @ClassRule 061 public static final HBaseClassTestRule CLASS_RULE = 062 HBaseClassTestRule.forClass(TestSeekHints.class); 063 064 @Rule 065 public TestName name = new TestName(); 066 067 @BeforeClass 068 public static void setUpBeforeClass() throws Exception { 069 Configuration conf = TEST_UTIL.getConfiguration(); 070 conf.setInt("hbase.client.scanner.caching", 1000); 071 conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, 072 ConstantSizeRegionSplitPolicy.class.getName()); 073 // set no splits 074 conf.setLong(HConstants.HREGION_MAX_FILESIZE, (1024L) * 1024 * 1024 * 10); 075 076 TEST_UTIL.startMiniCluster(); 077 078 // load the mini cluster with a single table with 20 rows, with rowkeys of a single byte, 0-19. 079 ht = TEST_UTIL.createTable(TableName.valueOf(table), Bytes.toBytes(cf), Integer.MAX_VALUE); 080 for (byte b = 0; b < 20; b++) { 081 Put put = new Put(new byte[] { b }).addColumn(Bytes.toBytes(cf), Bytes.toBytes(cq), 082 Bytes.toBytes("value")); 083 ht.put(put); 084 } 085 TEST_UTIL.flush(); 086 } 087 088 @AfterClass 089 public static void tearDownAfterClass() throws Exception { 090 TEST_UTIL.shutdownMiniCluster(); 091 } 092 093 @Test 094 public void testNormalScanForwardsSeek() throws IOException { 095 096 KeepAllButSeekFilter filter = new KeepAllButSeekFilter((byte) 10, (byte) 15); 097 098 Scan scan = new Scan(); 099 scan.setFilter(filter); 100 101 ResultScanner scanner = ht.getScanner(scan); 102 List<byte[]> actualRowsList = new ArrayList<>(); 103 for (Result result : scanner) { 104 actualRowsList.add(result.getRow()); 105 } 106 byte[][] actualRows = actualRowsList.toArray(new byte[][] {}); 107 108 List<byte[]> expectedRowsList = new ArrayList<>(); 109 // Start counting up from 0, as that's the first of our test rows 110 for (byte i = 0; i < 10; i++) { 111 expectedRowsList.add(new byte[] { i }); 112 } 113 // Skip rows starting from 10 and ending before 15, as the filter should remove them 114 for (byte i = 15; i < 20; i++) { 115 expectedRowsList.add(new byte[] { i }); 116 } 117 byte[][] expectedRows = expectedRowsList.toArray(new byte[][] {}); 118 119 assertArrayEquals(expectedRows, actualRows); 120 } 121 122 @Test 123 public void testReversedScanBackwardsSeek() throws IOException { 124 125 KeepAllButSeekFilter filter = new KeepAllButSeekFilter((byte) 10, (byte) 5); 126 127 Scan scan = new Scan(); 128 scan.setFilter(filter); 129 scan.setReversed(true); 130 131 ResultScanner scanner = ht.getScanner(scan); 132 List<byte[]> actualRowsList = new ArrayList<>(); 133 for (Result result : scanner) { 134 actualRowsList.add(result.getRow()); 135 } 136 byte[][] actualRows = actualRowsList.toArray(new byte[][] {}); 137 138 List<byte[]> expectedRowsList = new ArrayList<>(); 139 // Start counting down from 19, as that's the last of our test rows 140 for (byte i = 19; i > 10; i--) { 141 expectedRowsList.add(new byte[] { i }); 142 } 143 // Skip rows starting from 10 and ending before 5, as the filter should remove them 144 for (byte i = 5; i >= 0; i--) { 145 expectedRowsList.add(new byte[] { i }); 146 } 147 byte[][] expectedRows = expectedRowsList.toArray(new byte[][] {}); 148 149 assertArrayEquals(expectedRows, actualRows); 150 } 151 152 public static class KeepAllButSeekFilter extends FilterBase { 153 154 private byte seekStartRow; 155 private byte seekTargetRow; 156 157 public KeepAllButSeekFilter(byte seekStartRow, byte seekTargetRow) { 158 this.seekStartRow = seekStartRow; 159 this.seekTargetRow = seekTargetRow; 160 } 161 162 /* 163 * We return SEEK_NEXT_USING_HINT when we hit the specified row, but we return INCLUDE for all 164 * other rows. This will wind up including the rows between our "seek" row and our "hint" row 165 * only if we don't seek past them. 166 */ 167 @Override 168 public ReturnCode filterCell(final Cell c) throws IOException { 169 byte rowKeyPrefix = CellUtil.cloneRow(c)[0]; 170 if (rowKeyPrefix == seekStartRow) { 171 return ReturnCode.SEEK_NEXT_USING_HINT; 172 } 173 return ReturnCode.INCLUDE; 174 } 175 176 @Override 177 public Cell getNextCellHint(Cell currentCell) { 178 return PrivateCellUtil.createFirstOnRow(new byte[] { seekTargetRow }); 179 } 180 181 @Override 182 public byte[] toByteArray() { 183 return new byte[] { seekStartRow, seekTargetRow }; 184 } 185 186 public static KeepAllButSeekFilter parseFrom(final byte[] pbBytes) 187 throws DeserializationException { 188 189 return new KeepAllButSeekFilter(pbBytes[0], pbBytes[1]); 190 } 191 } 192 193}