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