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