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 */
018
019package org.apache.hadoop.hbase.regionserver.querymatcher;
020
021import java.io.IOException;
022import java.util.SortedSet;
023import java.util.TreeSet;
024
025import org.apache.yetus.audience.InterfaceAudience;
026import org.apache.hadoop.hbase.Cell;
027import org.apache.hadoop.hbase.CellComparator;
028import org.apache.hadoop.hbase.CellUtil;
029import org.apache.hadoop.hbase.KeyValue;
030import org.apache.hadoop.hbase.KeyValueUtil;
031import org.apache.hadoop.hbase.util.Bytes;
032
033/**
034 * This class is responsible for the tracking and enforcement of Deletes during the course of a Scan
035 * operation. It only has to enforce Delete and DeleteColumn, since the DeleteFamily is handled at a
036 * higher level.
037 * <p>
038 * This class is utilized through three methods:
039 * <ul>
040 * <li>{@link #add} when encountering a Delete or DeleteColumn</li>
041 * <li>{@link #isDeleted} when checking if a Put Cell has been deleted</li>
042 * <li>{@link #update} when reaching the end of a StoreFile or row for scans</li>
043 * </ul>
044 * <p>
045 * This class is NOT thread-safe as queries are never multi-threaded
046 */
047@InterfaceAudience.Private
048public class ScanDeleteTracker implements DeleteTracker {
049
050  protected boolean hasFamilyStamp = false;
051  protected long familyStamp = 0L;
052  protected SortedSet<Long> familyVersionStamps = new TreeSet<Long>();
053  protected Cell deleteCell = null;
054  protected byte[] deleteBuffer = null;
055  protected int deleteOffset = 0;
056  protected int deleteLength = 0;
057  protected byte deleteType = 0;
058  protected long deleteTimestamp = 0L;
059  protected final CellComparator comparator;
060
061  public ScanDeleteTracker(CellComparator comparator) {
062    this.comparator = comparator;
063  }
064
065  /**
066   * Add the specified Cell to the list of deletes to check against for this row operation.
067   * <p>
068   * This is called when a Delete is encountered.
069   * @param cell - the delete cell
070   */
071  @Override
072  public void add(Cell cell) {
073    long timestamp = cell.getTimestamp();
074    byte type = cell.getTypeByte();
075    if (!hasFamilyStamp || timestamp > familyStamp) {
076      if (type == KeyValue.Type.DeleteFamily.getCode()) {
077        hasFamilyStamp = true;
078        familyStamp = timestamp;
079        return;
080      } else if (type == KeyValue.Type.DeleteFamilyVersion.getCode()) {
081        familyVersionStamps.add(timestamp);
082        return;
083      }
084
085      if (deleteCell != null && type < deleteType) {
086        // same column, so ignore less specific delete
087        if (CellUtil.matchingQualifier(cell, deleteCell)) {
088          return;
089        }
090      }
091      // new column, or more general delete type
092      deleteCell = cell;
093      deleteType = type;
094      deleteTimestamp = timestamp;
095    }
096    // missing else is never called.
097  }
098
099  /**
100   * Check if the specified Cell buffer has been deleted by a previously seen delete.
101   * @param cell - current cell to check if deleted by a previously seen delete
102   * @return deleteResult
103   */
104  @Override
105  public DeleteResult isDeleted(Cell cell) {
106    long timestamp = cell.getTimestamp();
107    if (hasFamilyStamp && timestamp <= familyStamp) {
108      return DeleteResult.FAMILY_DELETED;
109    }
110
111    if (familyVersionStamps.contains(Long.valueOf(timestamp))) {
112      return DeleteResult.FAMILY_VERSION_DELETED;
113    }
114
115    if (deleteCell != null) {
116      int ret = -(this.comparator.compareQualifiers(cell, deleteCell));
117      if (ret == 0) {
118        if (deleteType == KeyValue.Type.DeleteColumn.getCode()) {
119          return DeleteResult.COLUMN_DELETED;
120        }
121        // Delete (aka DeleteVersion)
122        // If the timestamp is the same, keep this one
123        if (timestamp == deleteTimestamp) {
124          return DeleteResult.VERSION_DELETED;
125        }
126        // use assert or not?
127        assert timestamp < deleteTimestamp;
128
129        // different timestamp, let's clear the buffer.
130        deleteCell = null;
131      } else if (ret < 0) {
132        // Next column case.
133        deleteCell = null;
134      } else {
135        throw new IllegalStateException("isDelete failed: deleteBuffer="
136            + Bytes.toStringBinary(deleteCell.getQualifierArray(),
137                    deleteCell.getQualifierOffset(), deleteCell.getQualifierLength())
138            + ", qualifier="
139            + Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
140              cell.getQualifierLength())
141            + ", timestamp=" + timestamp + ", comparison result: " + ret);
142      }
143    }
144
145    return DeleteResult.NOT_DELETED;
146  }
147
148  @Override
149  public boolean isEmpty() {
150    return deleteCell == null && !hasFamilyStamp && familyVersionStamps.isEmpty();
151  }
152
153  @Override
154  // called between every row.
155  public void reset() {
156    hasFamilyStamp = false;
157    familyStamp = 0L;
158    familyVersionStamps.clear();
159    deleteCell = null;
160  }
161
162  @Override
163  // should not be called at all even (!)
164  public void update() {
165    this.reset();
166  }
167
168  @Override
169  public void beforeShipped() throws IOException {
170    if (deleteCell != null) {
171      deleteCell = KeyValueUtil.toNewKeyCell(deleteCell);
172    }
173  }
174
175  @Override
176  public CellComparator getCellComparator() {
177    return this.comparator;
178  }
179}