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; 023import java.util.ArrayList; 024import java.util.Iterator; 025import java.util.List; 026 027import org.apache.hadoop.hbase.Cell; 028import org.apache.hadoop.hbase.CellUtil; 029import org.apache.hadoop.hbase.CompareOperator; 030import org.apache.yetus.audience.InterfaceAudience; 031import org.apache.hadoop.hbase.exceptions.DeserializationException; 032import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; 033import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 034import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; 035import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; 036 037/** 038 * A {@link Filter} that checks a single column value, but does not emit the 039 * tested column. This will enable a performance boost over 040 * {@link SingleColumnValueFilter}, if the tested column value is not actually 041 * needed as input (besides for the filtering itself). 042 */ 043@InterfaceAudience.Public 044public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { 045 046 /** 047 * Constructor for binary compare of the value of a single column. If the 048 * column is found and the condition passes, all columns of the row will be 049 * emitted; except for the tested column value. If the column is not found or 050 * the condition fails, the row will not be emitted. 051 * 052 * @param family name of column family 053 * @param qualifier name of column qualifier 054 * @param compareOp operator 055 * @param value value to compare column values against 056 * {@link #SingleColumnValueExcludeFilter(byte[], byte[], CompareOperator, byte[])} 057 */ 058 @Deprecated 059 public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier, 060 CompareOp compareOp, byte[] value) { 061 super(family, qualifier, compareOp, value); 062 } 063 064 /** 065 * Constructor for binary compare of the value of a single column. If the 066 * column is found and the condition passes, all columns of the row will be 067 * emitted; except for the tested column value. If the column is not found or 068 * the condition fails, the row will not be emitted. 069 * 070 * @param family name of column family 071 * @param qualifier name of column qualifier 072 * @param op operator 073 * @param value value to compare column values against 074 */ 075 public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier, 076 CompareOperator op, byte[] value) { 077 super(family, qualifier, op, value); 078 } 079 080 /** 081 * Constructor for binary compare of the value of a single column. If the 082 * column is found and the condition passes, all columns of the row will be 083 * emitted; except for the tested column value. If the condition fails, the 084 * row will not be emitted. 085 * <p> 086 * Use the filterIfColumnMissing flag to set whether the rest of the columns 087 * in a row will be emitted if the specified column to check is not found in 088 * the row. 089 * 090 * @param family name of column family 091 * @param qualifier name of column qualifier 092 * @param compareOp operator 093 * @param comparator Comparator to use. 094 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use 095 * {@link #SingleColumnValueExcludeFilter(byte[], byte[], CompareOperator, ByteArrayComparable)} 096 */ 097 @Deprecated 098 public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier, 099 CompareOp compareOp, ByteArrayComparable comparator) { 100 super(family, qualifier, compareOp, comparator); 101 } 102 103 /** 104 * Constructor for binary compare of the value of a single column. If the 105 * column is found and the condition passes, all columns of the row will be 106 * emitted; except for the tested column value. If the condition fails, the 107 * row will not be emitted. 108 * <p> 109 * Use the filterIfColumnMissing flag to set whether the rest of the columns 110 * in a row will be emitted if the specified column to check is not found in 111 * the row. 112 * 113 * @param family name of column family 114 * @param qualifier name of column qualifier 115 * @param op operator 116 * @param comparator Comparator to use. 117 */ 118 public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier, 119 CompareOperator op, ByteArrayComparable comparator) { 120 super(family, qualifier, op, comparator); 121 } 122 123 124 /** 125 * Constructor for protobuf deserialization only. 126 * @param family 127 * @param qualifier 128 * @param compareOp 129 * @param comparator 130 * @param filterIfMissing 131 * @param latestVersionOnly 132 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use 133 * {@link #SingleColumnValueExcludeFilter(byte[], byte[], CompareOperator, ByteArrayComparable, boolean, boolean)} 134 */ 135 @Deprecated 136 protected SingleColumnValueExcludeFilter(final byte[] family, final byte[] qualifier, 137 final CompareOp compareOp, ByteArrayComparable comparator, final boolean filterIfMissing, 138 final boolean latestVersionOnly) { 139 this(family, qualifier, CompareOperator.valueOf(compareOp.name()), comparator, 140 filterIfMissing, latestVersionOnly); 141 } 142 143 /** 144 * Constructor for protobuf deserialization only. 145 * @param family 146 * @param qualifier 147 * @param op 148 * @param comparator 149 * @param filterIfMissing 150 * @param latestVersionOnly 151 */ 152 protected SingleColumnValueExcludeFilter(final byte[] family, final byte[] qualifier, 153 final CompareOperator op, ByteArrayComparable comparator, final boolean filterIfMissing, 154 final boolean latestVersionOnly) { 155 super(family, qualifier, op, comparator, filterIfMissing, latestVersionOnly); 156 } 157 158 // We cleaned result row in FilterRow to be consistent with scanning process. 159 @Override 160 public boolean hasFilterRow() { 161 return true; 162 } 163 164 // Here we remove from row all key values from testing column 165 @Override 166 public void filterRowCells(List<Cell> kvs) { 167 Iterator<? extends Cell> it = kvs.iterator(); 168 while (it.hasNext()) { 169 // If the current column is actually the tested column, 170 // we will skip it instead. 171 if (CellUtil.matchingColumn(it.next(), this.columnFamily, this.columnQualifier)) { 172 it.remove(); 173 } 174 } 175 } 176 177 public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) { 178 SingleColumnValueFilter tempFilter = (SingleColumnValueFilter) 179 SingleColumnValueFilter.createFilterFromArguments(filterArguments); 180 SingleColumnValueExcludeFilter filter = new SingleColumnValueExcludeFilter ( 181 tempFilter.getFamily(), tempFilter.getQualifier(), 182 tempFilter.getOperator(), tempFilter.getComparator()); 183 184 if (filterArguments.size() == 6) { 185 filter.setFilterIfMissing(tempFilter.getFilterIfMissing()); 186 filter.setLatestVersionOnly(tempFilter.getLatestVersionOnly()); 187 } 188 return filter; 189 } 190 191 /** 192 * @return The filter serialized using pb 193 */ 194 @Override 195 public byte [] toByteArray() { 196 FilterProtos.SingleColumnValueExcludeFilter.Builder builder = 197 FilterProtos.SingleColumnValueExcludeFilter.newBuilder(); 198 builder.setSingleColumnValueFilter(super.convert()); 199 return builder.build().toByteArray(); 200 } 201 202 /** 203 * @param pbBytes A pb serialized {@link SingleColumnValueExcludeFilter} instance 204 * @return An instance of {@link SingleColumnValueExcludeFilter} made from <code>bytes</code> 205 * @throws DeserializationException 206 * @see #toByteArray 207 */ 208 public static SingleColumnValueExcludeFilter parseFrom(final byte [] pbBytes) 209 throws DeserializationException { 210 FilterProtos.SingleColumnValueExcludeFilter proto; 211 try { 212 proto = FilterProtos.SingleColumnValueExcludeFilter.parseFrom(pbBytes); 213 } catch (InvalidProtocolBufferException e) { 214 throw new DeserializationException(e); 215 } 216 217 FilterProtos.SingleColumnValueFilter parentProto = proto.getSingleColumnValueFilter(); 218 final CompareOperator compareOp = 219 CompareOperator.valueOf(parentProto.getCompareOp().name()); 220 final ByteArrayComparable comparator; 221 try { 222 comparator = ProtobufUtil.toComparator(parentProto.getComparator()); 223 } catch (IOException ioe) { 224 throw new DeserializationException(ioe); 225 } 226 227 return new SingleColumnValueExcludeFilter(parentProto.hasColumnFamily() ? parentProto 228 .getColumnFamily().toByteArray() : null, parentProto.hasColumnQualifier() ? parentProto 229 .getColumnQualifier().toByteArray() : null, compareOp, comparator, parentProto 230 .getFilterIfMissing(), parentProto.getLatestVersionOnly()); 231 } 232 233 /** 234 * @return true if and only if the fields of the filter that are serialized 235 * are equal to the corresponding fields in other. Used for testing. 236 */ 237 @Override 238 boolean areSerializedFieldsEqual(Filter o) { 239 if (o == this) return true; 240 if (!(o instanceof SingleColumnValueExcludeFilter)) return false; 241 242 return super.areSerializedFieldsEqual(o); 243 } 244 245 @Override 246 public boolean equals(Object obj) { 247 return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); 248 } 249 250 @Override 251 public int hashCode() { 252 return super.hashCode(); 253 } 254}