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