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 org.apache.hadoop.hbase.Cell;
021import org.apache.hadoop.hbase.KeepDeletedCells;
022import org.apache.hadoop.hbase.regionserver.ScanInfo;
023import org.apache.yetus.audience.InterfaceAudience;
024
025/**
026 * A query matcher for compaction which can drop delete markers.
027 */
028@InterfaceAudience.Private
029public abstract class DropDeletesCompactionScanQueryMatcher extends CompactionScanQueryMatcher {
030
031  /**
032   * By default, when hbase.hstore.time.to.purge.deletes is 0ms, a delete marker is always removed
033   * during a major compaction. If set to non-zero value then major compaction will try to keep a
034   * delete marker around for the given number of milliseconds. We want to keep the delete markers
035   * around a bit longer because old puts might appear out-of-order. For example, during log
036   * replication between two clusters.
037   * <p>
038   * If the delete marker has lived longer than its column-family's TTL then the delete marker will
039   * be removed even if time.to.purge.deletes has not passed. This is because all the Puts that this
040   * delete marker can influence would have also expired. (Removing of delete markers on col family
041   * TTL will not happen if min-versions is set to non-zero)
042   * <p>
043   * But, if time.to.purge.deletes has not expired then a delete marker will not be removed just
044   * because there are no Puts that it is currently influencing. This is because Puts, that this
045   * delete can influence. may appear out of order.
046   */
047  protected final long timeToPurgeDeletes;
048
049  /**
050   * Oldest put in any of the involved store files Used to decide whether it is ok to delete family
051   * delete marker of this store keeps deleted KVs.
052   */
053  protected final long earliestPutTs;
054
055  protected DropDeletesCompactionScanQueryMatcher(ScanInfo scanInfo, DeleteTracker deletes,
056    ColumnTracker columns, long readPointToUse, long earliestPutTs, long oldestUnexpiredTS,
057    long now) {
058    super(scanInfo, deletes, columns, readPointToUse, oldestUnexpiredTS, now);
059    this.timeToPurgeDeletes = scanInfo.getTimeToPurgeDeletes();
060    this.earliestPutTs = earliestPutTs;
061  }
062
063  protected final MatchCode tryDropDelete(Cell cell) {
064    long timestamp = cell.getTimestamp();
065    // If it is not the time to drop the delete marker, just return
066    if (timeToPurgeDeletes > 0 && now - timestamp <= timeToPurgeDeletes) {
067      return MatchCode.INCLUDE;
068    }
069    if (
070      keepDeletedCells == KeepDeletedCells.TRUE
071        || (keepDeletedCells == KeepDeletedCells.TTL && timestamp >= oldestUnexpiredTS)
072    ) {
073      // If keepDeletedCell is true, or the delete marker is not expired yet, we should include it
074      // in version counting to see if we can drop it. The only exception is that, we can make
075      // sure that no put is older than this delete marker. And under this situation, all later
076      // cells of this column(must be delete markers) can be skipped.
077      if (timestamp < earliestPutTs) {
078        return columns.getNextRowOrNextColumn(cell);
079      } else {
080        return null;
081      }
082    } else {
083      return MatchCode.SKIP;
084    }
085  }
086}