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.Iterator; 022import java.util.NavigableSet; 023 024import org.apache.hadoop.hbase.Cell; 025import org.apache.hadoop.hbase.CellComparator; 026import org.apache.hadoop.hbase.CellUtil; 027import org.apache.hadoop.hbase.HConstants; 028import org.apache.hadoop.hbase.KeyValue; 029import org.apache.hadoop.hbase.KeyValue.Type; 030import org.apache.hadoop.hbase.KeyValueUtil; 031import org.apache.hadoop.hbase.PrivateCellUtil; 032import org.apache.hadoop.hbase.Tag; 033import org.apache.hadoop.hbase.TagType; 034import org.apache.hadoop.hbase.client.Scan; 035import org.apache.hadoop.hbase.filter.Filter; 036import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; 037import org.apache.hadoop.hbase.regionserver.ScanInfo; 038import org.apache.hadoop.hbase.regionserver.ShipperListener; 039import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker.DeleteResult; 040import org.apache.hadoop.hbase.security.visibility.VisibilityNewVersionBehaivorTracker; 041import org.apache.hadoop.hbase.security.visibility.VisibilityScanDeleteTracker; 042import org.apache.hadoop.hbase.util.Bytes; 043import org.apache.hadoop.hbase.util.Pair; 044import org.apache.yetus.audience.InterfaceAudience; 045 046/** 047 * A query matcher that is specifically designed for the scan case. 048 */ 049@InterfaceAudience.Private 050public abstract class ScanQueryMatcher implements ShipperListener { 051 052 /** 053 * {@link #match} return codes. These instruct the scanner moving through memstores and StoreFiles 054 * what to do with the current KeyValue. 055 * <p> 056 * Additionally, this contains "early-out" language to tell the scanner to move on to the next 057 * File (memstore or Storefile), or to return immediately. 058 */ 059 public static enum MatchCode { 060 /** 061 * Include KeyValue in the returned result 062 */ 063 INCLUDE, 064 065 /** 066 * Do not include KeyValue in the returned result 067 */ 068 SKIP, 069 070 /** 071 * Do not include, jump to next StoreFile or memstore (in time order) 072 */ 073 NEXT, 074 075 /** 076 * Do not include, return current result 077 */ 078 DONE, 079 080 /** 081 * These codes are used by the ScanQueryMatcher 082 */ 083 084 /** 085 * Done with the row, seek there. 086 */ 087 SEEK_NEXT_ROW, 088 089 /** 090 * Done with column, seek to next. 091 */ 092 SEEK_NEXT_COL, 093 094 /** 095 * Done with scan, thanks to the row filter. 096 */ 097 DONE_SCAN, 098 099 /** 100 * Seek to next key which is given as hint. 101 */ 102 SEEK_NEXT_USING_HINT, 103 104 /** 105 * Include KeyValue and done with column, seek to next. 106 */ 107 INCLUDE_AND_SEEK_NEXT_COL, 108 109 /** 110 * Include KeyValue and done with row, seek to next. 111 */ 112 INCLUDE_AND_SEEK_NEXT_ROW, 113 } 114 115 /** Row comparator for the region this query is for */ 116 protected final CellComparator rowComparator; 117 118 /** Key to seek to in memstore and StoreFiles */ 119 protected final Cell startKey; 120 121 /** Keeps track of columns and versions */ 122 protected final ColumnTracker columns; 123 124 /** The oldest timestamp we are interested in, based on TTL */ 125 protected final long oldestUnexpiredTS; 126 127 protected final long now; 128 129 /** Row the query is on */ 130 protected Cell currentRow; 131 132 protected ScanQueryMatcher(Cell startKey, ScanInfo scanInfo, ColumnTracker columns, 133 long oldestUnexpiredTS, long now) { 134 this.rowComparator = scanInfo.getComparator(); 135 this.startKey = startKey; 136 this.oldestUnexpiredTS = oldestUnexpiredTS; 137 this.now = now; 138 this.columns = columns; 139 } 140 141 /** 142 * @param cell 143 * @param oldestTimestamp 144 * @return true if the cell is expired 145 */ 146 private static boolean isCellTTLExpired(final Cell cell, final long oldestTimestamp, 147 final long now) { 148 // Look for a TTL tag first. Use it instead of the family setting if 149 // found. If a cell has multiple TTLs, resolve the conflict by using the 150 // first tag encountered. 151 Iterator<Tag> i = PrivateCellUtil.tagsIterator(cell); 152 while (i.hasNext()) { 153 Tag t = i.next(); 154 if (TagType.TTL_TAG_TYPE == t.getType()) { 155 // Unlike in schema cell TTLs are stored in milliseconds, no need 156 // to convert 157 long ts = cell.getTimestamp(); 158 assert t.getValueLength() == Bytes.SIZEOF_LONG; 159 long ttl = Tag.getValueAsLong(t); 160 if (ts + ttl < now) { 161 return true; 162 } 163 // Per cell TTLs cannot extend lifetime beyond family settings, so 164 // fall through to check that 165 break; 166 } 167 } 168 return false; 169 } 170 171 /** 172 * Check before the delete logic. 173 * @return null means continue. 174 */ 175 protected final MatchCode preCheck(Cell cell) { 176 if (currentRow == null) { 177 // Since the curCell is null it means we are already sure that we have moved over to the next 178 // row 179 return MatchCode.DONE; 180 } 181 // if row key is changed, then we know that we have moved over to the next row 182 if (rowComparator.compareRows(currentRow, cell) != 0) { 183 return MatchCode.DONE; 184 } 185 186 if (this.columns.done()) { 187 return MatchCode.SEEK_NEXT_ROW; 188 } 189 190 long timestamp = cell.getTimestamp(); 191 // check if this is a fake cell. The fake cell is an optimization, we should make the scanner 192 // seek to next column or next row. See StoreFileScanner.requestSeek for more details. 193 // check for early out based on timestamp alone 194 if (timestamp == HConstants.OLDEST_TIMESTAMP || columns.isDone(timestamp)) { 195 return columns.getNextRowOrNextColumn(cell); 196 } 197 // check if the cell is expired by cell TTL 198 if (isCellTTLExpired(cell, this.oldestUnexpiredTS, this.now)) { 199 return MatchCode.SKIP; 200 } 201 return null; 202 } 203 204 protected final MatchCode checkDeleted(DeleteTracker deletes, Cell cell) { 205 if (deletes.isEmpty() && !(deletes instanceof NewVersionBehaviorTracker)) { 206 return null; 207 } 208 // MvccSensitiveTracker always need check all cells to save some infos. 209 DeleteResult deleteResult = deletes.isDeleted(cell); 210 switch (deleteResult) { 211 case FAMILY_DELETED: 212 case COLUMN_DELETED: 213 if (!(deletes instanceof NewVersionBehaviorTracker)) { 214 // MvccSensitive can not seek to next because the Put with lower ts may have higher mvcc 215 return columns.getNextRowOrNextColumn(cell); 216 } 217 case VERSION_DELETED: 218 case FAMILY_VERSION_DELETED: 219 case VERSION_MASKED: 220 return MatchCode.SKIP; 221 case NOT_DELETED: 222 return null; 223 default: 224 throw new RuntimeException("Unexpected delete result: " + deleteResult); 225 } 226 } 227 228 229 /** 230 * Determines if the caller should do one of several things: 231 * <ul> 232 * <li>seek/skip to the next row (MatchCode.SEEK_NEXT_ROW)</li> 233 * <li>seek/skip to the next column (MatchCode.SEEK_NEXT_COL)</li> 234 * <li>include the current KeyValue (MatchCode.INCLUDE)</li> 235 * <li>ignore the current KeyValue (MatchCode.SKIP)</li> 236 * <li>got to the next row (MatchCode.DONE)</li> 237 * </ul> 238 * @param cell KeyValue to check 239 * @return The match code instance. 240 * @throws IOException in case there is an internal consistency problem caused by a data 241 * corruption. 242 */ 243 public abstract MatchCode match(Cell cell) throws IOException; 244 245 /** 246 * @return the start key 247 */ 248 public Cell getStartKey() { 249 return startKey; 250 } 251 252 /** 253 * @return whether there is an null column in the query 254 */ 255 public abstract boolean hasNullColumnInQuery(); 256 257 /** 258 * @return a cell represent the current row 259 */ 260 public Cell currentRow() { 261 return currentRow; 262 } 263 264 /** 265 * Make {@link #currentRow()} return null. 266 */ 267 public void clearCurrentRow() { 268 currentRow = null; 269 } 270 271 protected abstract void reset(); 272 273 /** 274 * Set the row when there is change in row 275 * @param currentRow 276 */ 277 public void setToNewRow(Cell currentRow) { 278 this.currentRow = currentRow; 279 columns.reset(); 280 reset(); 281 } 282 283 public abstract boolean isUserScan(); 284 285 /** 286 * @return Returns false if we know there are no more rows to be scanned (We've reached the 287 * <code>stopRow</code> or we are scanning on row only because this Scan is for a Get, 288 * etc. 289 */ 290 public abstract boolean moreRowsMayExistAfter(Cell cell); 291 292 public Cell getKeyForNextColumn(Cell cell) { 293 // We aren't sure whether any DeleteFamily cells exist, so we can't skip to next column. 294 // TODO: Current way disable us to seek to next column quickly. Is there any better solution? 295 // see HBASE-18471 for more details 296 // see TestFromClientSide3#testScanAfterDeletingSpecifiedRow 297 // see TestFromClientSide3#testScanAfterDeletingSpecifiedRowV2 298 if (cell.getQualifierLength() == 0) { 299 Cell nextKey = PrivateCellUtil.createNextOnRowCol(cell); 300 if (nextKey != cell) { 301 return nextKey; 302 } 303 // The cell is at the end of row/family/qualifier, so it is impossible to find any DeleteFamily cells. 304 // Let us seek to next column. 305 } 306 ColumnCount nextColumn = columns.getColumnHint(); 307 if (nextColumn == null) { 308 return PrivateCellUtil.createLastOnRowCol(cell); 309 } else { 310 return PrivateCellUtil.createFirstOnRowCol(cell, nextColumn.getBuffer(), 311 nextColumn.getOffset(), nextColumn.getLength()); 312 } 313 } 314 315 /** 316 * @param nextIndexed the key of the next entry in the block index (if any) 317 * @param currentCell The Cell we're using to calculate the seek key 318 * @return result of the compare between the indexed key and the key portion of the passed cell 319 */ 320 public int compareKeyForNextRow(Cell nextIndexed, Cell currentCell) { 321 return PrivateCellUtil.compareKeyBasedOnColHint(rowComparator, nextIndexed, currentCell, 0, 0, null, 0, 322 0, HConstants.OLDEST_TIMESTAMP, Type.Minimum.getCode()); 323 } 324 325 /** 326 * @param nextIndexed the key of the next entry in the block index (if any) 327 * @param currentCell The Cell we're using to calculate the seek key 328 * @return result of the compare between the indexed key and the key portion of the passed cell 329 */ 330 public int compareKeyForNextColumn(Cell nextIndexed, Cell currentCell) { 331 ColumnCount nextColumn = columns.getColumnHint(); 332 if (nextColumn == null) { 333 return PrivateCellUtil.compareKeyBasedOnColHint(rowComparator, nextIndexed, currentCell, 0, 0, null, 334 0, 0, HConstants.OLDEST_TIMESTAMP, Type.Minimum.getCode()); 335 } else { 336 return PrivateCellUtil.compareKeyBasedOnColHint(rowComparator, nextIndexed, currentCell, 337 currentCell.getFamilyOffset(), currentCell.getFamilyLength(), nextColumn.getBuffer(), 338 nextColumn.getOffset(), nextColumn.getLength(), HConstants.LATEST_TIMESTAMP, 339 Type.Maximum.getCode()); 340 } 341 } 342 343 /** 344 * @return the Filter 345 */ 346 public abstract Filter getFilter(); 347 348 /** 349 * Delegate to {@link Filter#getNextCellHint(Cell)}. If no filter, return {@code null}. 350 */ 351 public abstract Cell getNextKeyHint(Cell cell) throws IOException; 352 353 @Override 354 public void beforeShipped() throws IOException { 355 if (this.currentRow != null) { 356 this.currentRow = PrivateCellUtil.createFirstOnRow(CellUtil.copyRow(this.currentRow)); 357 } 358 if (columns != null) { 359 columns.beforeShipped(); 360 } 361 } 362 363 protected static Cell createStartKeyFromRow(byte[] startRow, ScanInfo scanInfo) { 364 return PrivateCellUtil.createFirstDeleteFamilyCellOnRow(startRow, scanInfo.getFamily()); 365 } 366 367 protected static Pair<DeleteTracker, ColumnTracker> getTrackers(RegionCoprocessorHost host, 368 NavigableSet<byte[]> columns, ScanInfo scanInfo, long oldestUnexpiredTS, Scan userScan) 369 throws IOException { 370 int resultMaxVersion = scanInfo.getMaxVersions(); 371 int maxVersionToCheck = resultMaxVersion; 372 if (userScan != null) { 373 if (userScan.isRaw()) { 374 resultMaxVersion = userScan.getMaxVersions(); 375 } else { 376 resultMaxVersion = Math.min(userScan.getMaxVersions(), scanInfo.getMaxVersions()); 377 } 378 maxVersionToCheck = userScan.hasFilter() ? scanInfo.getMaxVersions() : resultMaxVersion; 379 } 380 381 DeleteTracker deleteTracker; 382 if (scanInfo.isNewVersionBehavior() && (userScan == null || !userScan.isRaw())) { 383 deleteTracker = new NewVersionBehaviorTracker(columns, scanInfo.getComparator(), 384 scanInfo.getMinVersions(), scanInfo.getMaxVersions(), resultMaxVersion, 385 oldestUnexpiredTS); 386 } else { 387 deleteTracker = new ScanDeleteTracker(scanInfo.getComparator()); 388 } 389 if (host != null) { 390 deleteTracker = host.postInstantiateDeleteTracker(deleteTracker); 391 if (deleteTracker instanceof VisibilityScanDeleteTracker && scanInfo.isNewVersionBehavior()) { 392 deleteTracker = new VisibilityNewVersionBehaivorTracker(columns, scanInfo.getComparator(), 393 scanInfo.getMinVersions(), scanInfo.getMaxVersions(), resultMaxVersion, 394 oldestUnexpiredTS); 395 } 396 } 397 398 ColumnTracker columnTracker; 399 400 if (deleteTracker instanceof NewVersionBehaviorTracker) { 401 columnTracker = (NewVersionBehaviorTracker) deleteTracker; 402 } else if (columns == null || columns.size() == 0) { 403 columnTracker = new ScanWildcardColumnTracker(scanInfo.getMinVersions(), maxVersionToCheck, 404 oldestUnexpiredTS, scanInfo.getComparator()); 405 } else { 406 columnTracker = new ExplicitColumnTracker(columns, scanInfo.getMinVersions(), 407 maxVersionToCheck, oldestUnexpiredTS); 408 } 409 return new Pair<>(deleteTracker, columnTracker); 410 } 411 412 // Used only for testing purposes 413 static MatchCode checkColumn(ColumnTracker columnTracker, byte[] bytes, int offset, int length, 414 long ttl, byte type, boolean ignoreCount) throws IOException { 415 KeyValue kv = KeyValueUtil.createFirstOnRow(HConstants.EMPTY_BYTE_ARRAY, 0, 0, 416 HConstants.EMPTY_BYTE_ARRAY, 0, 0, bytes, offset, length); 417 MatchCode matchCode = columnTracker.checkColumn(kv, type); 418 if (matchCode == MatchCode.INCLUDE) { 419 return columnTracker.checkVersions(kv, ttl, type, ignoreCount); 420 } 421 return matchCode; 422 } 423}