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.Objects; 022import org.apache.hadoop.hbase.Cell; 023import org.apache.hadoop.hbase.exceptions.DeserializationException; 024import org.apache.yetus.audience.InterfaceAudience; 025 026import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; 027 028import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 029import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; 030 031/** 032 * A wrapper filter that returns true from {@link #filterAllRemaining()} as soon as the wrapped 033 * filters {@link Filter#filterRowKey(Cell)}, 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 returns true. 037 */ 038@InterfaceAudience.Public 039public class WhileMatchFilter extends FilterBase { 040 private boolean filterAllRemaining = false; 041 private Filter filter; 042 043 public WhileMatchFilter(Filter filter) { 044 this.filter = filter; 045 } 046 047 public Filter getFilter() { 048 return filter; 049 } 050 051 @Override 052 public void reset() throws IOException { 053 this.filter.reset(); 054 } 055 056 private void changeFAR(boolean value) { 057 filterAllRemaining = filterAllRemaining || value; 058 } 059 060 @Override 061 public boolean filterAllRemaining() throws IOException { 062 return this.filterAllRemaining || this.filter.filterAllRemaining(); 063 } 064 065 @Override 066 public boolean filterRowKey(Cell cell) throws IOException { 067 if (filterAllRemaining()) return true; 068 boolean value = filter.filterRowKey(cell); 069 changeFAR(value); 070 return value; 071 } 072 073 @Override 074 public ReturnCode filterCell(final Cell c) throws IOException { 075 ReturnCode code = filter.filterCell(c); 076 changeFAR(code != ReturnCode.INCLUDE); 077 return code; 078 } 079 080 @Override 081 public Cell transformCell(Cell v) throws IOException { 082 return filter.transformCell(v); 083 } 084 085 @Override 086 public boolean filterRow() throws IOException { 087 boolean filterRow = this.filter.filterRow(); 088 changeFAR(filterRow); 089 return filterRow; 090 } 091 092 @Override 093 public boolean hasFilterRow() { 094 return true; 095 } 096 097 /** Returns The filter serialized using pb */ 098 @Override 099 public byte[] toByteArray() throws IOException { 100 FilterProtos.WhileMatchFilter.Builder builder = FilterProtos.WhileMatchFilter.newBuilder(); 101 builder.setFilter(ProtobufUtil.toFilter(this.filter)); 102 return builder.build().toByteArray(); 103 } 104 105 /** 106 * Parse a serialized representation of {@link WhileMatchFilter} 107 * @param pbBytes A pb serialized {@link WhileMatchFilter} instance 108 * @return An instance of {@link WhileMatchFilter} made from <code>bytes</code> 109 * @throws DeserializationException if an error occurred 110 * @see #toByteArray 111 */ 112 public static WhileMatchFilter parseFrom(final byte[] pbBytes) throws DeserializationException { 113 FilterProtos.WhileMatchFilter proto; 114 try { 115 proto = FilterProtos.WhileMatchFilter.parseFrom(pbBytes); 116 } catch (InvalidProtocolBufferException e) { 117 throw new DeserializationException(e); 118 } 119 try { 120 return new WhileMatchFilter(ProtobufUtil.toFilter(proto.getFilter())); 121 } catch (IOException ioe) { 122 throw new DeserializationException(ioe); 123 } 124 } 125 126 /** 127 * Return true if and only if the fields of the filter that are serialized are equal to the 128 * corresponding fields in other. Used for testing. 129 */ 130 @Override 131 boolean areSerializedFieldsEqual(Filter o) { 132 if (o == this) { 133 return true; 134 } 135 if (!(o instanceof WhileMatchFilter)) { 136 return false; 137 } 138 WhileMatchFilter other = (WhileMatchFilter) o; 139 return getFilter().areSerializedFieldsEqual(other.getFilter()); 140 } 141 142 @Override 143 public boolean isFamilyEssential(byte[] name) throws IOException { 144 return filter.isFamilyEssential(name); 145 } 146 147 @Override 148 public String toString() { 149 return this.getClass().getSimpleName() + " " + this.filter.toString(); 150 } 151 152 @Override 153 public boolean equals(Object obj) { 154 return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); 155 } 156 157 @Override 158 public int hashCode() { 159 return Objects.hash(this.filter); 160 } 161}