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  @Deprecated
101  public ReturnCode filterKeyValue(final Cell c) {
102    return filterCell(c);
103  }
104
105  @Override
106  public ReturnCode filterCell(final Cell c) {
107    if (columnOffset != null) {
108      if (count >= limit) {
109        return ReturnCode.NEXT_ROW;
110      }
111      int cmp = 0;
112      // Only compare if no KV's have been seen so far.
113      if (count == 0) {
114        cmp = CellUtil.compareQualifiers(c, this.columnOffset, 0, this.columnOffset.length);
115      }
116      if (cmp < 0) {
117        return ReturnCode.SEEK_NEXT_USING_HINT;
118      } else {
119        count++;
120        return ReturnCode.INCLUDE_AND_NEXT_COL;
121      }
122    } else {
123      if (count >= offset + limit) {
124        return ReturnCode.NEXT_ROW;
125      }
126
127      ReturnCode code = count < offset ? ReturnCode.NEXT_COL : ReturnCode.INCLUDE_AND_NEXT_COL;
128      count++;
129      return code;
130    }
131  }
132
133  @Override
134  public Cell getNextCellHint(Cell cell) {
135    return PrivateCellUtil.createFirstOnRowCol(cell, columnOffset, 0, columnOffset.length);
136  }
137
138  @Override
139  public void reset() {
140    this.count = 0;
141  }
142
143  public static Filter createFilterFromArguments(ArrayList<byte[]> filterArguments) {
144    Preconditions.checkArgument(filterArguments.size() == 2, "Expected 2 but got: %s",
145      filterArguments.size());
146    int limit = ParseFilter.convertByteArrayToInt(filterArguments.get(0));
147    int offset = ParseFilter.convertByteArrayToInt(filterArguments.get(1));
148    return new ColumnPaginationFilter(limit, offset);
149  }
150
151  /** Returns The filter serialized using pb */
152  @Override
153  public byte[] toByteArray() {
154    FilterProtos.ColumnPaginationFilter.Builder builder =
155      FilterProtos.ColumnPaginationFilter.newBuilder();
156    builder.setLimit(this.limit);
157    if (this.offset >= 0) {
158      builder.setOffset(this.offset);
159    }
160    if (this.columnOffset != null) {
161      builder.setColumnOffset(UnsafeByteOperations.unsafeWrap(this.columnOffset));
162    }
163    return builder.build().toByteArray();
164  }
165
166  /**
167   * Parse a serialized representation of {@link ColumnPaginationFilter}
168   * @param pbBytes A pb serialized {@link ColumnPaginationFilter} instance
169   * @return An instance of {@link ColumnPaginationFilter} made from <code>bytes</code>
170   * @throws DeserializationException if an error occurred
171   * @see #toByteArray
172   */
173  public static ColumnPaginationFilter parseFrom(final byte[] pbBytes)
174    throws DeserializationException {
175    FilterProtos.ColumnPaginationFilter proto;
176    try {
177      proto = FilterProtos.ColumnPaginationFilter.parseFrom(pbBytes);
178    } catch (InvalidProtocolBufferException e) {
179      throw new DeserializationException(e);
180    }
181    if (proto.hasColumnOffset()) {
182      return new ColumnPaginationFilter(proto.getLimit(), proto.getColumnOffset().toByteArray());
183    }
184    return new ColumnPaginationFilter(proto.getLimit(), proto.getOffset());
185  }
186
187  /**
188   * Returns true if and only if the fields of the filter that are serialized are equal to the
189   * corresponding fields in other. Used for testing.
190   */
191  @Override
192  boolean areSerializedFieldsEqual(Filter o) {
193    if (o == this) {
194      return true;
195    }
196    if (!(o instanceof ColumnPaginationFilter)) {
197      return false;
198    }
199    ColumnPaginationFilter other = (ColumnPaginationFilter) o;
200    if (this.columnOffset != null) {
201      return this.getLimit() == other.getLimit()
202        && Bytes.equals(this.getColumnOffset(), other.getColumnOffset());
203    }
204    return this.getLimit() == other.getLimit() && this.getOffset() == other.getOffset();
205  }
206
207  @Override
208  public String toString() {
209    if (this.columnOffset != null) {
210      return (this.getClass().getSimpleName() + "(" + this.limit + ", "
211        + Bytes.toStringBinary(this.columnOffset) + ")");
212    }
213    return String.format("%s (%d, %d)", this.getClass().getSimpleName(), this.limit, this.offset);
214  }
215
216  @Override
217  public boolean equals(Object obj) {
218    return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj);
219  }
220
221  @Override
222  public int hashCode() {
223    return columnOffset == null
224      ? Objects.hash(this.limit, this.offset)
225      : Objects.hash(this.limit, Bytes.hashCode(this.columnOffset));
226  }
227}