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.List; 022import java.util.Objects; 023import org.apache.hadoop.hbase.Cell; 024import org.apache.hadoop.hbase.exceptions.DeserializationException; 025import org.apache.yetus.audience.InterfaceAudience; 026 027import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; 028 029import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 030import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; 031 032/** 033 * This is a Filter wrapper class which is used in the server side. Some filter related hooks can be 034 * defined in this wrapper. The only way to create a FilterWrapper instance is passing a client side 035 * Filter instance through {@link org.apache.hadoop.hbase.client.Scan#getFilter()}. 036 */ 037@InterfaceAudience.Private 038final public class FilterWrapper extends Filter { 039 Filter filter = null; 040 041 /** 042 * Constructor. 043 * @param filter filter to wrap 044 * @throws NullPointerException if {@code filter} is {@code null} 045 */ 046 public FilterWrapper(Filter filter) { 047 this.filter = Objects.requireNonNull(filter, "Cannot create FilterWrapper with null Filter"); 048 } 049 050 /** Returns The filter serialized using pb */ 051 @Override 052 public byte[] toByteArray() throws IOException { 053 FilterProtos.FilterWrapper.Builder builder = FilterProtos.FilterWrapper.newBuilder(); 054 builder.setFilter(ProtobufUtil.toFilter(this.filter)); 055 return builder.build().toByteArray(); 056 } 057 058 /** 059 * @param pbBytes A pb serialized {@link FilterWrapper} instance 060 * @return An instance of {@link FilterWrapper} made from <code>bytes</code> 061 * @throws org.apache.hadoop.hbase.exceptions.DeserializationException 062 * @see #toByteArray 063 */ 064 public static FilterWrapper parseFrom(final byte[] pbBytes) throws DeserializationException { 065 FilterProtos.FilterWrapper proto; 066 try { 067 proto = FilterProtos.FilterWrapper.parseFrom(pbBytes); 068 } catch (InvalidProtocolBufferException e) { 069 throw new DeserializationException(e); 070 } 071 try { 072 return new FilterWrapper(ProtobufUtil.toFilter(proto.getFilter())); 073 } catch (IOException ioe) { 074 throw new DeserializationException(ioe); 075 } 076 } 077 078 @Override 079 public void reset() throws IOException { 080 this.filter.reset(); 081 } 082 083 @Override 084 public boolean filterAllRemaining() throws IOException { 085 return this.filter.filterAllRemaining(); 086 } 087 088 @Override 089 public boolean filterRow() throws IOException { 090 return this.filter.filterRow(); 091 } 092 093 @Override 094 public Cell getNextCellHint(Cell currentCell) throws IOException { 095 return this.filter.getNextCellHint(currentCell); 096 } 097 098 @Override 099 public boolean filterRowKey(Cell cell) throws IOException { 100 if (filterAllRemaining()) return true; 101 return this.filter.filterRowKey(cell); 102 } 103 104 @Override 105 public ReturnCode filterCell(final Cell c) throws IOException { 106 return this.filter.filterCell(c); 107 } 108 109 @Override 110 public Cell transformCell(Cell v) throws IOException { 111 return this.filter.transformCell(v); 112 } 113 114 @Override 115 public boolean hasFilterRow() { 116 return this.filter.hasFilterRow(); 117 } 118 119 @Override 120 public void filterRowCells(List<Cell> kvs) throws IOException { 121 filterRowCellsWithRet(kvs); 122 } 123 124 public enum FilterRowRetCode { 125 NOT_CALLED, 126 INCLUDE, // corresponds to filter.filterRow() returning false 127 EXCLUDE, // corresponds to filter.filterRow() returning true 128 INCLUDE_THIS_FAMILY // exclude other families 129 } 130 131 public FilterRowRetCode filterRowCellsWithRet(List<Cell> kvs) throws IOException { 132 // To fix HBASE-6429, 133 // Filter with filterRow() returning true is incompatible with scan with limit 134 // 1. hasFilterRow() returns true, if either filterRow() or filterRow(kvs) is implemented. 135 // 2. filterRow() is merged with filterRow(kvs), 136 // so that to make all those row related filtering stuff in the same function. 137 this.filter.filterRowCells(kvs); 138 if (!kvs.isEmpty()) { 139 if (this.filter.filterRow()) { 140 kvs.clear(); 141 return FilterRowRetCode.EXCLUDE; 142 } 143 return FilterRowRetCode.INCLUDE; 144 } 145 return FilterRowRetCode.NOT_CALLED; 146 } 147 148 @Override 149 public boolean isFamilyEssential(byte[] name) throws IOException { 150 return filter.isFamilyEssential(name); 151 } 152 153 /** 154 * @param o the other filter to compare with 155 * @return true if and only if the fields of the filter that are serialized are equal to the 156 * corresponding fields in other. Used for testing. 157 */ 158 @Override 159 boolean areSerializedFieldsEqual(Filter o) { 160 if (o == this) return true; 161 if (!(o instanceof FilterWrapper)) return false; 162 163 FilterWrapper other = (FilterWrapper) o; 164 return this.filter.areSerializedFieldsEqual(other.filter); 165 } 166}