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.filter;
019
020import java.io.IOException;
021import java.util.ArrayList;
022import java.util.Objects;
023import org.apache.hadoop.hbase.Cell;
024import org.apache.hadoop.hbase.CellUtil;
025import org.apache.hadoop.hbase.PrivateCellUtil;
026import org.apache.hadoop.hbase.exceptions.DeserializationException;
027import org.apache.hadoop.hbase.util.Bytes;
028import org.apache.yetus.audience.InterfaceAudience;
029
030import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
031import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
032import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
033
034import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
035
036/**
037 * A filter, based on the ColumnCountGetFilter, takes two arguments: limit and offset. This filter
038 * can be used for row-based indexing, where references to other tables are stored across many
039 * columns, in order to efficient lookups and paginated results for end users. Only most recent
040 * versions are considered for pagination.
041 */
042@InterfaceAudience.Public
043public class ColumnPaginationFilter extends FilterBase {
044
045  private int limit = 0;
046  private int offset = -1;
047  private byte[] columnOffset = null;
048  private int count = 0;
049
050  /**
051   * Initializes filter with an integer offset and limit. The offset is arrived at scanning
052   * sequentially and skipping entries. @limit number of columns are then retrieved. If multiple
053   * column families are involved, the columns may be spread across them.
054   * @param limit  Max number of columns to return.
055   * @param offset The integer offset where to start pagination.
056   */
057  public ColumnPaginationFilter(final int limit, final int offset) {
058    Preconditions.checkArgument(limit >= 0, "limit must be positive %s", limit);
059    Preconditions.checkArgument(offset >= 0, "offset must be positive %s", offset);
060    this.limit = limit;
061    this.offset = offset;
062  }
063
064  /**
065   * Initializes filter with a string/bookmark based offset and limit. The offset is arrived at, by
066   * seeking to it using scanner hints. If multiple column families are involved, pagination starts
067   * at the first column family which contains @columnOffset. Columns are then retrieved
068   * sequentially upto @limit number of columns which maybe spread across multiple column families,
069   * depending on how the scan is setup.
070   * @param limit        Max number of columns to return.
071   * @param columnOffset The string/bookmark offset on where to start pagination.
072   */
073  public ColumnPaginationFilter(final int limit, final byte[] columnOffset) {
074    Preconditions.checkArgument(limit >= 0, "limit must be positive %s", limit);
075    Preconditions.checkArgument(columnOffset != null, "columnOffset must be non-null %s",
076      columnOffset);
077    this.limit = limit;
078    this.columnOffset = columnOffset;
079  }
080
081  public int getLimit() {
082    return limit;
083  }
084
085  public int getOffset() {
086    return offset;
087  }
088
089  public byte[] getColumnOffset() {
090    return columnOffset;
091  }
092
093  @Override
094  public boolean filterRowKey(Cell cell) throws IOException {
095    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
096    return false;
097  }
098
099  @Override
100  public ReturnCode filterCell(final Cell c) {
101    if (columnOffset != null) {
102      if (count >= limit) {
103        return ReturnCode.NEXT_ROW;
104      }
105      int cmp = 0;
106      // Only compare if no KV's have been seen so far.
107      if (count == 0) {
108        cmp = CellUtil.compareQualifiers(c, this.columnOffset, 0, this.columnOffset.length);
109      }
110      if (cmp < 0) {
111        return ReturnCode.SEEK_NEXT_USING_HINT;
112      } else {
113        count++;
114        return ReturnCode.INCLUDE_AND_NEXT_COL;
115      }
116    } else {
117      if (count >= offset + limit) {
118        return ReturnCode.NEXT_ROW;
119      }
120
121      ReturnCode code = count < offset ? ReturnCode.NEXT_COL : ReturnCode.INCLUDE_AND_NEXT_COL;
122      count++;
123      return code;
124    }
125  }
126
127  @Override
128  public Cell getNextCellHint(Cell cell) {
129    return PrivateCellUtil.createFirstOnRowCol(cell, columnOffset, 0, columnOffset.length);
130  }
131
132  @Override
133  public void reset() {
134    this.count = 0;
135  }
136
137  public static Filter createFilterFromArguments(ArrayList<byte[]> filterArguments) {
138    Preconditions.checkArgument(filterArguments.size() == 2, "Expected 2 but got: %s",
139      filterArguments.size());
140    int limit = ParseFilter.convertByteArrayToInt(filterArguments.get(0));
141    int offset = ParseFilter.convertByteArrayToInt(filterArguments.get(1));
142    return new ColumnPaginationFilter(limit, offset);
143  }
144
145  /** Returns The filter serialized using pb */
146  @Override
147  public byte[] toByteArray() {
148    FilterProtos.ColumnPaginationFilter.Builder builder =
149      FilterProtos.ColumnPaginationFilter.newBuilder();
150    builder.setLimit(this.limit);
151    if (this.offset >= 0) {
152      builder.setOffset(this.offset);
153    }
154    if (this.columnOffset != null) {
155      builder.setColumnOffset(UnsafeByteOperations.unsafeWrap(this.columnOffset));
156    }
157    return builder.build().toByteArray();
158  }
159
160  /**
161   * Parse a serialized representation of {@link ColumnPaginationFilter}
162   * @param pbBytes A pb serialized {@link ColumnPaginationFilter} instance
163   * @return An instance of {@link ColumnPaginationFilter} made from <code>bytes</code>
164   * @throws DeserializationException if an error occurred
165   * @see #toByteArray
166   */
167  public static ColumnPaginationFilter parseFrom(final byte[] pbBytes)
168    throws DeserializationException {
169    FilterProtos.ColumnPaginationFilter proto;
170    try {
171      proto = FilterProtos.ColumnPaginationFilter.parseFrom(pbBytes);
172    } catch (InvalidProtocolBufferException e) {
173      throw new DeserializationException(e);
174    }
175    if (proto.hasColumnOffset()) {
176      return new ColumnPaginationFilter(proto.getLimit(), proto.getColumnOffset().toByteArray());
177    }
178    return new ColumnPaginationFilter(proto.getLimit(), proto.getOffset());
179  }
180
181  /**
182   * Returns true if and only if the fields of the filter that are serialized are equal to the
183   * corresponding fields in other. Used for testing.
184   */
185  @Override
186  boolean areSerializedFieldsEqual(Filter o) {
187    if (o == this) {
188      return true;
189    }
190    if (!(o instanceof ColumnPaginationFilter)) {
191      return false;
192    }
193    ColumnPaginationFilter other = (ColumnPaginationFilter) o;
194    if (this.columnOffset != null) {
195      return this.getLimit() == other.getLimit()
196        && Bytes.equals(this.getColumnOffset(), other.getColumnOffset());
197    }
198    return this.getLimit() == other.getLimit() && this.getOffset() == other.getOffset();
199  }
200
201  @Override
202  public String toString() {
203    if (this.columnOffset != null) {
204      return (this.getClass().getSimpleName() + "(" + this.limit + ", "
205        + Bytes.toStringBinary(this.columnOffset) + ")");
206    }
207    return String.format("%s (%d, %d)", this.getClass().getSimpleName(), this.limit, this.offset);
208  }
209
210  @Override
211  public boolean equals(Object obj) {
212    return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj);
213  }
214
215  @Override
216  public int hashCode() {
217    return columnOffset == null
218      ? Objects.hash(this.limit, this.offset)
219      : Objects.hash(this.limit, Bytes.hashCode(this.columnOffset));
220  }
221}