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.regionserver.querymatcher; 019 020import java.io.IOException; 021import java.util.NavigableSet; 022import org.apache.hadoop.hbase.Cell; 023import org.apache.hadoop.hbase.CellUtil; 024import org.apache.hadoop.hbase.DoNotRetryIOException; 025import org.apache.hadoop.hbase.ExtendedCell; 026import org.apache.hadoop.hbase.KeyValueUtil; 027import org.apache.hadoop.hbase.PrivateCellUtil; 028import org.apache.hadoop.hbase.client.Scan; 029import org.apache.hadoop.hbase.filter.Filter; 030import org.apache.hadoop.hbase.filter.Filter.ReturnCode; 031import org.apache.hadoop.hbase.io.TimeRange; 032import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; 033import org.apache.hadoop.hbase.regionserver.ScanInfo; 034import org.apache.hadoop.hbase.util.Pair; 035import org.apache.yetus.audience.InterfaceAudience; 036 037/** 038 * Query matcher for user scan. 039 * <p> 040 * We do not consider mvcc here because 041 * {@link org.apache.hadoop.hbase.regionserver.StoreFileScanner} and 042 * {@link org.apache.hadoop.hbase.regionserver.SegmentScanner} will only return a cell whose mvcc is 043 * less than or equal to given read point. For 044 * {@link org.apache.hadoop.hbase.client.IsolationLevel#READ_UNCOMMITTED}, we just set the read 045 * point to {@link Long#MAX_VALUE}, i.e. still do not need to consider it. 046 */ 047@InterfaceAudience.Private 048public abstract class UserScanQueryMatcher extends ScanQueryMatcher { 049 050 protected final boolean hasNullColumn; 051 052 protected final Filter filter; 053 054 protected final byte[] stopRow; 055 056 protected final TimeRange tr; 057 058 private final int versionsAfterFilter; 059 060 private int count = 0; 061 062 private ExtendedCell curColCell = null; 063 064 /** 065 * Holds a seek-hint produced by {@link org.apache.hadoop.hbase.filter.Filter#getSkipHint(Cell)} 066 * at one of the structural short-circuit points in {@link #matchColumn}. When non-null this is 067 * returned by {@link #getNextKeyHint} instead of delegating to 068 * {@link org.apache.hadoop.hbase.filter.Filter#getNextCellHint}, because the hint was computed 069 * for a cell that never reached {@code filterCell}. Cleared on every {@link #getNextKeyHint} call 070 * so it cannot leak across multiple seek-hint cycles. 071 */ 072 private ExtendedCell pendingSkipHint = null; 073 074 private final boolean reversed; 075 076 private static ExtendedCell createStartKey(Scan scan, ScanInfo scanInfo) { 077 if (scan.includeStartRow()) { 078 return createStartKeyFromRow(scan.getStartRow(), scanInfo); 079 } else { 080 return PrivateCellUtil.createLastOnRow(scan.getStartRow()); 081 } 082 } 083 084 protected UserScanQueryMatcher(Scan scan, ScanInfo scanInfo, ColumnTracker columns, 085 boolean hasNullColumn, long oldestUnexpiredTS, long now) { 086 super(createStartKey(scan, scanInfo), scanInfo, columns, oldestUnexpiredTS, now); 087 this.hasNullColumn = hasNullColumn; 088 this.filter = scan.getFilter(); 089 if (this.filter != null) { 090 this.versionsAfterFilter = scan.isRaw() 091 ? scan.getMaxVersions() 092 : Math.min(scan.getMaxVersions(), scanInfo.getMaxVersions()); 093 } else { 094 this.versionsAfterFilter = 0; 095 } 096 this.stopRow = scan.getStopRow(); 097 this.reversed = scan.isReversed(); 098 TimeRange timeRange = scan.getColumnFamilyTimeRange().get(scanInfo.getFamily()); 099 if (timeRange == null) { 100 this.tr = scan.getTimeRange(); 101 } else { 102 this.tr = timeRange; 103 } 104 } 105 106 @Override 107 public boolean hasNullColumnInQuery() { 108 return hasNullColumn; 109 } 110 111 @Override 112 public boolean isUserScan() { 113 return true; 114 } 115 116 @Override 117 public Filter getFilter() { 118 return filter; 119 } 120 121 @Override 122 public ExtendedCell getNextKeyHint(ExtendedCell cell) throws IOException { 123 if (pendingSkipHint != null) { 124 ExtendedCell hint = pendingSkipHint; 125 pendingSkipHint = null; 126 return hint; 127 } 128 if (filter == null) { 129 return null; 130 } else { 131 Cell hint = filter.getNextCellHint(cell); 132 if (hint == null || hint instanceof ExtendedCell) { 133 return (ExtendedCell) hint; 134 } else { 135 throw new DoNotRetryIOException("Incorrect filter implementation, " 136 + "the Cell returned by getNextKeyHint is not an ExtendedCell. Filter class: " 137 + filter.getClass().getName()); 138 } 139 } 140 } 141 142 @Override 143 public void beforeShipped() throws IOException { 144 super.beforeShipped(); 145 if (curColCell != null) { 146 this.curColCell = KeyValueUtil.toNewKeyCell(this.curColCell); 147 } 148 if (pendingSkipHint != null) { 149 this.pendingSkipHint = KeyValueUtil.toNewKeyCell(this.pendingSkipHint); 150 } 151 } 152 153 @Override 154 public void setToNewRow(ExtendedCell currentRow) { 155 pendingSkipHint = null; 156 super.setToNewRow(currentRow); 157 } 158 159 @Override 160 public void clearCurrentRow() { 161 pendingSkipHint = null; 162 super.clearCurrentRow(); 163 } 164 165 // At each structural short-circuit below (time-range, column-exclusion, version-exhaustion), 166 // the filter is consulted via resolveSkipHint() before falling back to the default skip/seek 167 // code. This lets filters provide a forward seek target even when filterCell is never called. 168 protected final MatchCode matchColumn(ExtendedCell cell, long timestamp, byte typeByte) 169 throws IOException { 170 int tsCmp = tr.compare(timestamp); 171 if (tsCmp > 0) { 172 if (resolveSkipHint(cell)) { 173 return MatchCode.SEEK_NEXT_USING_HINT; 174 } 175 return MatchCode.SKIP; 176 } 177 if (tsCmp < 0) { 178 if (resolveSkipHint(cell)) { 179 return MatchCode.SEEK_NEXT_USING_HINT; 180 } 181 return columns.getNextRowOrNextColumn(cell); 182 } 183 // STEP 1: Check if the column is part of the requested columns 184 MatchCode matchCode = columns.checkColumn(cell, typeByte); 185 if (matchCode != MatchCode.INCLUDE) { 186 if (resolveSkipHint(cell)) { 187 return MatchCode.SEEK_NEXT_USING_HINT; 188 } 189 return matchCode; 190 } 191 /* 192 * STEP 2: check the number of versions needed. This method call returns SKIP, SEEK_NEXT_COL, 193 * INCLUDE, INCLUDE_AND_SEEK_NEXT_COL, or INCLUDE_AND_SEEK_NEXT_ROW. 194 */ 195 matchCode = columns.checkVersions(cell, timestamp, typeByte, false); 196 switch (matchCode) { 197 case SKIP: 198 if (resolveSkipHint(cell)) { 199 return MatchCode.SEEK_NEXT_USING_HINT; 200 } 201 return MatchCode.SKIP; 202 case SEEK_NEXT_COL: 203 if (resolveSkipHint(cell)) { 204 return MatchCode.SEEK_NEXT_USING_HINT; 205 } 206 return MatchCode.SEEK_NEXT_COL; 207 default: 208 // It means it is INCLUDE, INCLUDE_AND_SEEK_NEXT_COL or INCLUDE_AND_SEEK_NEXT_ROW. 209 assert matchCode == MatchCode.INCLUDE || matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_COL 210 || matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_ROW; 211 break; 212 } 213 214 return filter == null 215 ? matchCode 216 : mergeFilterResponse(cell, matchCode, filter.filterCell(cell)); 217 } 218 219 /** 220 * Asks the current filter for a seek hint via 221 * {@link org.apache.hadoop.hbase.filter.Filter#getSkipHint(Cell)}, validates the returned cell 222 * type, and if non-null stores it in {@link #pendingSkipHint} so that {@link #getNextKeyHint} can 223 * return it when the scan pipeline asks for the seek target after receiving 224 * {@link ScanQueryMatcher.MatchCode#SEEK_NEXT_USING_HINT}. 225 * <p> 226 * This is only called from the structural short-circuit branches of {@link #matchColumn}, where 227 * {@code filterCell} has <em>not</em> been called, in accordance with the stateless contract of 228 * {@code Filter#getSkipHint}. The filter-null guard is included here so call-sites need no 229 * boilerplate. 230 * @param cell the cell that triggered the structural short-circuit 231 * @return {@code true} if the filter returned a valid hint (stored in {@link #pendingSkipHint}), 232 * {@code false} if no filter is set or the filter returned {@code null} 233 * @throws DoNotRetryIOException if the filter returns a non-{@link ExtendedCell} instance 234 * @throws IOException if the filter signals an I/O failure 235 */ 236 private boolean resolveSkipHint(ExtendedCell cell) throws IOException { 237 if (filter == null) { 238 return false; 239 } 240 Cell raw = filter.getSkipHint(cell); 241 if (raw == null) { 242 return false; 243 } 244 if (!(raw instanceof ExtendedCell)) { 245 throw new DoNotRetryIOException( 246 "Incorrect filter implementation: the Cell returned by getSkipHint " 247 + "is not an ExtendedCell. Filter class: " + filter.getClass().getName()); 248 } 249 ExtendedCell hint = (ExtendedCell) raw; 250 // Full-key compare is intentional: skip hints can advance within the same row 251 // (e.g., to a later column), not just across rows. 252 int cmp = rowComparator.compare(hint, cell); 253 if ((!reversed && cmp <= 0) || (reversed && cmp >= 0)) { 254 return false; 255 } 256 pendingSkipHint = hint; 257 return true; 258 } 259 260 /** 261 * Call this when scan has filter. Decide the desired behavior by checkVersions's MatchCode and 262 * filterCell's ReturnCode. Cell may be skipped by filter, so the column versions in result may be 263 * less than user need. It need to check versions again when filter and columnTracker both include 264 * the cell. <br/> 265 * 266 * <pre> 267 * ColumnChecker FilterResponse Desired behavior 268 * INCLUDE SKIP SKIP 269 * INCLUDE NEXT_COL SEEK_NEXT_COL or SEEK_NEXT_ROW 270 * INCLUDE NEXT_ROW SEEK_NEXT_ROW 271 * INCLUDE SEEK_NEXT_USING_HINT SEEK_NEXT_USING_HINT 272 * INCLUDE INCLUDE INCLUDE 273 * INCLUDE INCLUDE_AND_NEXT_COL INCLUDE_AND_SEEK_NEXT_COL 274 * INCLUDE INCLUDE_AND_SEEK_NEXT_ROW INCLUDE_AND_SEEK_NEXT_ROW 275 * INCLUDE_AND_SEEK_NEXT_COL SKIP SEEK_NEXT_COL 276 * INCLUDE_AND_SEEK_NEXT_COL NEXT_COL SEEK_NEXT_COL or SEEK_NEXT_ROW 277 * INCLUDE_AND_SEEK_NEXT_COL NEXT_ROW SEEK_NEXT_ROW 278 * INCLUDE_AND_SEEK_NEXT_COL SEEK_NEXT_USING_HINT SEEK_NEXT_USING_HINT 279 * INCLUDE_AND_SEEK_NEXT_COL INCLUDE INCLUDE_AND_SEEK_NEXT_COL 280 * INCLUDE_AND_SEEK_NEXT_COL INCLUDE_AND_NEXT_COL INCLUDE_AND_SEEK_NEXT_COL 281 * INCLUDE_AND_SEEK_NEXT_COL INCLUDE_AND_SEEK_NEXT_ROW INCLUDE_AND_SEEK_NEXT_ROW 282 * INCLUDE_AND_SEEK_NEXT_ROW SKIP SEEK_NEXT_ROW 283 * INCLUDE_AND_SEEK_NEXT_ROW NEXT_COL SEEK_NEXT_ROW 284 * INCLUDE_AND_SEEK_NEXT_ROW NEXT_ROW SEEK_NEXT_ROW 285 * INCLUDE_AND_SEEK_NEXT_ROW SEEK_NEXT_USING_HINT SEEK_NEXT_USING_HINT 286 * INCLUDE_AND_SEEK_NEXT_ROW INCLUDE INCLUDE_AND_SEEK_NEXT_ROW 287 * INCLUDE_AND_SEEK_NEXT_ROW INCLUDE_AND_NEXT_COL INCLUDE_AND_SEEK_NEXT_ROW 288 * INCLUDE_AND_SEEK_NEXT_ROW INCLUDE_AND_SEEK_NEXT_ROW INCLUDE_AND_SEEK_NEXT_ROW 289 * </pre> 290 */ 291 private final MatchCode mergeFilterResponse(ExtendedCell cell, MatchCode matchCode, 292 ReturnCode filterResponse) { 293 switch (filterResponse) { 294 case SKIP: 295 if (matchCode == MatchCode.INCLUDE) { 296 return MatchCode.SKIP; 297 } else if (matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_COL) { 298 return MatchCode.SEEK_NEXT_COL; 299 } else if (matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) { 300 return MatchCode.SEEK_NEXT_ROW; 301 } 302 break; 303 case NEXT_COL: 304 if (matchCode == MatchCode.INCLUDE || matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_COL) { 305 return columns.getNextRowOrNextColumn(cell); 306 } else if (matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) { 307 return MatchCode.SEEK_NEXT_ROW; 308 } 309 break; 310 case NEXT_ROW: 311 return MatchCode.SEEK_NEXT_ROW; 312 case SEEK_NEXT_USING_HINT: 313 return MatchCode.SEEK_NEXT_USING_HINT; 314 case INCLUDE: 315 break; 316 case INCLUDE_AND_NEXT_COL: 317 if (matchCode == MatchCode.INCLUDE) { 318 matchCode = MatchCode.INCLUDE_AND_SEEK_NEXT_COL; 319 } 320 break; 321 case INCLUDE_AND_SEEK_NEXT_ROW: 322 matchCode = MatchCode.INCLUDE_AND_SEEK_NEXT_ROW; 323 break; 324 default: 325 throw new RuntimeException("UNEXPECTED"); 326 } 327 328 // It means it is INCLUDE, INCLUDE_AND_SEEK_NEXT_COL or INCLUDE_AND_SEEK_NEXT_ROW. 329 assert matchCode == MatchCode.INCLUDE || matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_COL 330 || matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_ROW; 331 332 // We need to make sure that the number of cells returned will not exceed max version in scan 333 // when the match code is INCLUDE* case. 334 if (curColCell == null || !CellUtil.matchingRowColumn(cell, curColCell)) { 335 count = 0; 336 curColCell = cell; 337 } 338 count += 1; 339 340 if (count > versionsAfterFilter) { 341 // when the number of cells exceed max version in scan, we should return SEEK_NEXT_COL match 342 // code, but if current code is INCLUDE_AND_SEEK_NEXT_ROW, we can optimize to choose the max 343 // step between SEEK_NEXT_COL and INCLUDE_AND_SEEK_NEXT_ROW, which is SEEK_NEXT_ROW. 344 if (matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) { 345 matchCode = MatchCode.SEEK_NEXT_ROW; 346 } else { 347 matchCode = MatchCode.SEEK_NEXT_COL; 348 } 349 } 350 if (matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_COL || matchCode == MatchCode.SEEK_NEXT_COL) { 351 // Update column tracker to next column, As we use the column hint from the tracker to seek 352 // to next cell (HBASE-19749) 353 columns.doneWithColumn(cell); 354 } 355 return matchCode; 356 } 357 358 protected abstract boolean isGet(); 359 360 protected abstract boolean moreRowsMayExistsAfter(int cmpToStopRow); 361 362 @Override 363 public boolean moreRowsMayExistAfter(ExtendedCell cell) { 364 // If a 'get' Scan -- we are doing a Get (every Get is a single-row Scan in implementation) -- 365 // then we are looking at one row only, the one specified in the Get coordinate..so we know 366 // for sure that there are no more rows on this Scan 367 if (isGet()) { 368 return false; 369 } 370 // If no stopRow, return that there may be more rows. The tests that follow depend on a 371 // non-empty, non-default stopRow so this little test below short-circuits out doing the 372 // following compares. 373 if (this.stopRow == null || this.stopRow.length == 0) { 374 return true; 375 } 376 return moreRowsMayExistsAfter(rowComparator.compareRows(cell, stopRow, 0, stopRow.length)); 377 } 378 379 public static UserScanQueryMatcher create(Scan scan, ScanInfo scanInfo, 380 NavigableSet<byte[]> columns, long oldestUnexpiredTS, long now, 381 RegionCoprocessorHost regionCoprocessorHost) throws IOException { 382 boolean hasNullColumn = 383 !(columns != null && columns.size() != 0 && columns.first().length != 0); 384 Pair<DeleteTracker, ColumnTracker> trackers = 385 getTrackers(regionCoprocessorHost, columns, scanInfo, oldestUnexpiredTS, scan); 386 DeleteTracker deleteTracker = trackers.getFirst(); 387 ColumnTracker columnTracker = trackers.getSecond(); 388 if (scan.isRaw()) { 389 return RawScanQueryMatcher.create(scan, scanInfo, columnTracker, hasNullColumn, 390 oldestUnexpiredTS, now); 391 } else { 392 return NormalUserScanQueryMatcher.create(scan, scanInfo, columnTracker, deleteTracker, 393 hasNullColumn, oldestUnexpiredTS, now); 394 } 395 } 396}