001/* 002 * Copyright The Apache Software Foundation 003 * 004 * Licensed to the Apache Software Foundation (ASF) under one 005 * or more contributor license agreements. See the NOTICE file 006 * distributed with this work for additional information 007 * regarding copyright ownership. The ASF licenses this file 008 * to you under the Apache License, Version 2.0 (the 009 * "License"); you may not use this file except in compliance 010 * with the License. You may obtain a copy of the License at 011 * 012 * http://www.apache.org/licenses/LICENSE-2.0 013 * 014 * Unless required by applicable law or agreed to in writing, software 015 * distributed under the License is distributed on an "AS IS" BASIS, 016 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 017 * See the License for the specific language governing permissions and 018 * limitations under the License. 019 */ 020package org.apache.hadoop.hbase.filter; 021 022import java.io.IOException; 023import java.util.List; 024 025import org.apache.hadoop.hbase.Cell; 026import org.apache.yetus.audience.InterfaceAudience; 027import org.apache.hadoop.hbase.exceptions.DeserializationException; 028import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 029import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; 030import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; 031 032/** 033 * This is a Filter wrapper class which is used in the server side. Some filter 034 * related hooks can be defined in this wrapper. The only way to create a 035 * FilterWrapper instance is passing a client side Filter instance through 036 * {@link org.apache.hadoop.hbase.client.Scan#getFilter()}. 037 * 038 */ 039@InterfaceAudience.Private 040final public class FilterWrapper extends Filter { 041 Filter filter = null; 042 043 public FilterWrapper( Filter filter ) { 044 if (null == filter) { 045 // ensure the filter instance is not null 046 throw new NullPointerException("Cannot create FilterWrapper with null Filter"); 047 } 048 this.filter = filter; 049 } 050 051 /** 052 * @return The filter serialized using pb 053 */ 054 @Override 055 public byte[] toByteArray() throws IOException { 056 FilterProtos.FilterWrapper.Builder builder = 057 FilterProtos.FilterWrapper.newBuilder(); 058 builder.setFilter(ProtobufUtil.toFilter(this.filter)); 059 return builder.build().toByteArray(); 060 } 061 062 /** 063 * @param pbBytes A pb serialized {@link FilterWrapper} instance 064 * @return An instance of {@link FilterWrapper} made from <code>bytes</code> 065 * @throws org.apache.hadoop.hbase.exceptions.DeserializationException 066 * @see #toByteArray 067 */ 068 public static FilterWrapper parseFrom(final byte [] pbBytes) 069 throws DeserializationException { 070 FilterProtos.FilterWrapper proto; 071 try { 072 proto = FilterProtos.FilterWrapper.parseFrom(pbBytes); 073 } catch (InvalidProtocolBufferException e) { 074 throw new DeserializationException(e); 075 } 076 try { 077 return new FilterWrapper(ProtobufUtil.toFilter(proto.getFilter())); 078 } catch (IOException ioe) { 079 throw new DeserializationException(ioe); 080 } 081 } 082 083 @Override 084 public void reset() throws IOException { 085 this.filter.reset(); 086 } 087 088 @Override 089 public boolean filterAllRemaining() throws IOException { 090 return this.filter.filterAllRemaining(); 091 } 092 093 @Override 094 public boolean filterRow() throws IOException { 095 return this.filter.filterRow(); 096 } 097 098 @Override 099 public Cell getNextCellHint(Cell currentCell) throws IOException { 100 return this.filter.getNextCellHint(currentCell); 101 } 102 103 @Override 104 public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException { 105 // No call to this. 106 if (filterAllRemaining()) return true; 107 return this.filter.filterRowKey(buffer, offset, length); 108 } 109 110 @Override 111 public boolean filterRowKey(Cell cell) throws IOException { 112 if (filterAllRemaining()) return true; 113 return this.filter.filterRowKey(cell); 114 } 115 116 @Deprecated 117 @Override 118 public ReturnCode filterKeyValue(final Cell c) throws IOException { 119 return filterCell(c); 120 } 121 122 @Override 123 public ReturnCode filterCell(final Cell c) throws IOException { 124 return this.filter.filterCell(c); 125 } 126 127 @Override 128 public Cell transformCell(Cell v) throws IOException { 129 return this.filter.transformCell(v); 130 } 131 132 @Override 133 public boolean hasFilterRow() { 134 return this.filter.hasFilterRow(); 135 } 136 137 @Override 138 public void filterRowCells(List<Cell> kvs) throws IOException { 139 filterRowCellsWithRet(kvs); 140 } 141 142 public enum FilterRowRetCode { 143 NOT_CALLED, 144 INCLUDE, // corresponds to filter.filterRow() returning false 145 EXCLUDE, // corresponds to filter.filterRow() returning true 146 INCLUDE_THIS_FAMILY // exclude other families 147 } 148 public FilterRowRetCode filterRowCellsWithRet(List<Cell> kvs) throws IOException { 149 //To fix HBASE-6429, 150 //Filter with filterRow() returning true is incompatible with scan with limit 151 //1. hasFilterRow() returns true, if either filterRow() or filterRow(kvs) is implemented. 152 //2. filterRow() is merged with filterRow(kvs), 153 //so that to make all those row related filtering stuff in the same function. 154 this.filter.filterRowCells(kvs); 155 if (!kvs.isEmpty()) { 156 if (this.filter.filterRow()) { 157 kvs.clear(); 158 return FilterRowRetCode.EXCLUDE; 159 } 160 return FilterRowRetCode.INCLUDE; 161 } 162 return FilterRowRetCode.NOT_CALLED; 163 } 164 165 @Override 166 public boolean isFamilyEssential(byte[] name) throws IOException { 167 return filter.isFamilyEssential(name); 168 } 169 170 /** 171 * @param o the other filter to compare with 172 * @return true if and only if the fields of the filter that are serialized 173 * are equal to the corresponding fields in other. Used for testing. 174 */ 175 @Override 176 boolean areSerializedFieldsEqual(Filter o) { 177 if (o == this) return true; 178 if (!(o instanceof FilterWrapper)) return false; 179 180 FilterWrapper other = (FilterWrapper)o; 181 return this.filter.areSerializedFieldsEqual(other.filter); 182 } 183}