001/* 002 * 003 * Licensed to the Apache Software Foundation (ASF) under one 004 * or more contributor license agreements. See the NOTICE file 005 * distributed with this work for additional information 006 * regarding copyright ownership. The ASF licenses this file 007 * to you under the Apache License, Version 2.0 (the 008 * "License"); you may not use this file except in compliance 009 * with the License. You may obtain a copy of the License at 010 * 011 * http://www.apache.org/licenses/LICENSE-2.0 012 * 013 * Unless required by applicable law or agreed to in writing, software 014 * distributed under the License is distributed on an "AS IS" BASIS, 015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 016 * See the License for the specific language governing permissions and 017 * limitations under the License. 018 */ 019 020package org.apache.hadoop.hbase.filter; 021 022import java.io.IOException; 023 024import org.apache.hadoop.hbase.Cell; 025import org.apache.yetus.audience.InterfaceAudience; 026import org.apache.hadoop.hbase.exceptions.DeserializationException; 027import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 028import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; 029import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; 030 031/** 032 * A wrapper filter that returns true from {@link #filterAllRemaining()} as soon 033 * as the wrapped filters {@link Filter#filterRowKey(byte[], int, int)}, 034 * {@link Filter#filterCell(org.apache.hadoop.hbase.Cell)}, 035 * {@link org.apache.hadoop.hbase.filter.Filter#filterRow()} or 036 * {@link org.apache.hadoop.hbase.filter.Filter#filterAllRemaining()} methods 037 * returns true. 038 */ 039@InterfaceAudience.Public 040public class WhileMatchFilter extends FilterBase { 041 private boolean filterAllRemaining = false; 042 private Filter filter; 043 044 public WhileMatchFilter(Filter filter) { 045 this.filter = filter; 046 } 047 048 public Filter getFilter() { 049 return filter; 050 } 051 052 @Override 053 public void reset() throws IOException { 054 this.filter.reset(); 055 } 056 057 private void changeFAR(boolean value) { 058 filterAllRemaining = filterAllRemaining || value; 059 } 060 061 @Override 062 public boolean filterAllRemaining() throws IOException { 063 return this.filterAllRemaining || this.filter.filterAllRemaining(); 064 } 065 066 @Override 067 public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException { 068 boolean value = filter.filterRowKey(buffer, offset, length); 069 changeFAR(value); 070 return value; 071 } 072 073 @Override 074 public boolean filterRowKey(Cell cell) throws IOException { 075 if (filterAllRemaining()) return true; 076 boolean value = filter.filterRowKey(cell); 077 changeFAR(value); 078 return value; 079 } 080 081 @Deprecated 082 @Override 083 public ReturnCode filterKeyValue(final Cell c) throws IOException { 084 return filterCell(c); 085 } 086 087 @Override 088 public ReturnCode filterCell(final Cell c) throws IOException { 089 ReturnCode code = filter.filterCell(c); 090 changeFAR(code != ReturnCode.INCLUDE); 091 return code; 092 } 093 094 @Override 095 public Cell transformCell(Cell v) throws IOException { 096 return filter.transformCell(v); 097 } 098 099 @Override 100 public boolean filterRow() throws IOException { 101 boolean filterRow = this.filter.filterRow(); 102 changeFAR(filterRow); 103 return filterRow; 104 } 105 106 @Override 107 public boolean hasFilterRow() { 108 return true; 109 } 110 111 /** 112 * @return The filter serialized using pb 113 */ 114 @Override 115 public byte[] toByteArray() throws IOException { 116 FilterProtos.WhileMatchFilter.Builder builder = 117 FilterProtos.WhileMatchFilter.newBuilder(); 118 builder.setFilter(ProtobufUtil.toFilter(this.filter)); 119 return builder.build().toByteArray(); 120 } 121 122 /** 123 * @param pbBytes A pb serialized {@link WhileMatchFilter} instance 124 * @return An instance of {@link WhileMatchFilter} made from <code>bytes</code> 125 * @throws org.apache.hadoop.hbase.exceptions.DeserializationException 126 * @see #toByteArray 127 */ 128 public static WhileMatchFilter parseFrom(final byte [] pbBytes) 129 throws DeserializationException { 130 FilterProtos.WhileMatchFilter proto; 131 try { 132 proto = FilterProtos.WhileMatchFilter.parseFrom(pbBytes); 133 } catch (InvalidProtocolBufferException e) { 134 throw new DeserializationException(e); 135 } 136 try { 137 return new WhileMatchFilter(ProtobufUtil.toFilter(proto.getFilter())); 138 } catch (IOException ioe) { 139 throw new DeserializationException(ioe); 140 } 141 } 142 143 /** 144 * @param o the other filter to compare with 145 * @return true if and only if the fields of the filter that are serialized 146 * are equal to the corresponding fields in other. Used for testing. 147 */ 148 @Override 149 boolean areSerializedFieldsEqual(Filter o) { 150 if (o == this) return true; 151 if (!(o instanceof WhileMatchFilter)) return false; 152 153 WhileMatchFilter other = (WhileMatchFilter)o; 154 return getFilter().areSerializedFieldsEqual(other.getFilter()); 155 } 156 157 @Override 158 public boolean isFamilyEssential(byte[] name) throws IOException { 159 return filter.isFamilyEssential(name); 160 } 161 162 @Override 163 public String toString() { 164 return this.getClass().getSimpleName() + " " + this.filter.toString(); 165 } 166}