/**
* Copyright 2010 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.filter;
import org.apache.hadoop.hbase.KeyValue;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.List;
import java.util.ArrayList;
import com.google.common.base.Preconditions;
/**
* Implementation of Filter interface that limits results to a specific page
* size. It terminates scanning once the number of filter-passed rows is >
* the given page size.
* <p>
* Note that this filter cannot guarantee that the number of results returned
* to a client are <= page size. This is because the filter is applied
* separately on different region servers. It does however optimize the scan of
* individual HRegions by making sure that the page size is never exceeded
* locally.
*/
public class PageFilter extends FilterBase {
private long pageSize = Long.MAX_VALUE;
private int rowsAccepted = 0;
/**
* Default constructor, filters nothing. Required though for RPC
* deserialization.
*/
public PageFilter() {
super();
}
/**
* Constructor that takes a maximum page size.
*
* @param pageSize Maximum result size.
*/
public PageFilter(final long pageSize) {
Preconditions.checkArgument(pageSize >= 0, "must be positive %s", pageSize);
this.pageSize = pageSize;
}
public long getPageSize() {
return pageSize;
}
public boolean filterAllRemaining() {
return this.rowsAccepted >= this.pageSize;
}
public boolean filterRow() {
this.rowsAccepted++;
return this.rowsAccepted > this.pageSize;
}
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
Preconditions.checkArgument(filterArguments.size() == 1,
"Expected 1 but got: %s", filterArguments.size());
long pageSize = ParseFilter.convertByteArrayToLong(filterArguments.get(0));
return new PageFilter(pageSize);
}
public void readFields(final DataInput in) throws IOException {
this.pageSize = in.readLong();
}
public void write(final DataOutput out) throws IOException {
out.writeLong(pageSize);
}
}