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 static org.junit.Assert.assertTrue; 021import static org.junit.Assert.fail; 022 023import java.nio.charset.Charset; 024import java.util.ArrayList; 025import java.util.Collections; 026import java.util.LinkedList; 027import java.util.List; 028import org.apache.commons.io.IOUtils; 029import org.apache.commons.text.StringSubstitutor; 030import org.apache.hadoop.conf.Configuration; 031import org.apache.hadoop.hbase.CompareOperator; 032import org.apache.hadoop.hbase.DoNotRetryIOException; 033import org.apache.hadoop.hbase.HBaseClassTestRule; 034import org.apache.hadoop.hbase.HBaseCommonTestingUtil; 035import org.apache.hadoop.hbase.HBaseConfiguration; 036import org.apache.hadoop.hbase.HBaseTestingUtil; 037import org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange; 038import org.apache.hadoop.hbase.testclassification.FilterTests; 039import org.apache.hadoop.hbase.testclassification.MediumTests; 040import org.apache.hadoop.hbase.util.Bytes; 041import org.apache.hadoop.hbase.util.ClassLoaderTestHelper; 042import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 043import org.apache.hadoop.hbase.util.Pair; 044import org.junit.AfterClass; 045import org.junit.ClassRule; 046import org.junit.Test; 047import org.junit.experimental.categories.Category; 048import org.junit.runner.RunWith; 049import org.junit.runners.Parameterized; 050 051import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 052import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; 053 054@RunWith(Parameterized.class) 055@Category({ FilterTests.class, MediumTests.class }) 056public class TestFilterSerialization { 057 058 @ClassRule 059 public static final HBaseClassTestRule CLASS_RULE = 060 HBaseClassTestRule.forClass(TestFilterSerialization.class); 061 062 @Parameterized.Parameter(0) 063 public boolean allowFastReflectionFallthrough; 064 065 @Parameterized.Parameters(name = "{index}: allowFastReflectionFallthrough={0}") 066 public static Iterable<Object[]> data() { 067 return HBaseCommonTestingUtil.BOOLEAN_PARAMETERIZED; 068 } 069 070 @AfterClass 071 public static void afterClass() throws Exception { 072 // set back to true so that it doesn't affect any other tests 073 ProtobufUtil.setAllowFastReflectionFallthrough(true); 074 } 075 076 @Test 077 public void testColumnCountGetFilter() throws Exception { 078 ColumnCountGetFilter columnCountGetFilter = new ColumnCountGetFilter(1); 079 assertTrue(columnCountGetFilter.areSerializedFieldsEqual( 080 ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnCountGetFilter)))); 081 } 082 083 @Test 084 public void testColumnPaginationFilter() throws Exception { 085 ColumnPaginationFilter columnPaginationFilter = new ColumnPaginationFilter(1, 7); 086 assertTrue(columnPaginationFilter.areSerializedFieldsEqual( 087 ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnPaginationFilter)))); 088 } 089 090 @Test 091 public void testColumnPrefixFilter() throws Exception { 092 // empty string 093 ColumnPrefixFilter columnPrefixFilter = new ColumnPrefixFilter(Bytes.toBytes("")); 094 assertTrue(columnPrefixFilter 095 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnPrefixFilter)))); 096 097 // non-empty string 098 columnPrefixFilter = new ColumnPrefixFilter(Bytes.toBytes("")); 099 assertTrue(columnPrefixFilter 100 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnPrefixFilter)))); 101 } 102 103 @Test 104 public void testColumnRangeFilter() throws Exception { 105 // null columns 106 ColumnRangeFilter columnRangeFilter = new ColumnRangeFilter(null, true, null, false); 107 assertTrue(columnRangeFilter 108 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnRangeFilter)))); 109 110 // non-null columns 111 columnRangeFilter = new ColumnRangeFilter(Bytes.toBytes("a"), false, Bytes.toBytes("b"), true); 112 assertTrue(columnRangeFilter 113 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnRangeFilter)))); 114 } 115 116 @Test 117 public void testDependentColumnFilter() throws Exception { 118 // null column qualifier/family 119 DependentColumnFilter dependentColumnFilter = new DependentColumnFilter(null, null); 120 assertTrue(dependentColumnFilter.areSerializedFieldsEqual( 121 ProtobufUtil.toFilter(ProtobufUtil.toFilter(dependentColumnFilter)))); 122 123 // non-null column qualifier/family 124 dependentColumnFilter = new DependentColumnFilter(Bytes.toBytes("family"), 125 Bytes.toBytes("qual"), true, CompareOperator.GREATER_OR_EQUAL, 126 new BitComparator(Bytes.toBytes("bitComparator"), BitComparator.BitwiseOp.OR)); 127 assertTrue(dependentColumnFilter.areSerializedFieldsEqual( 128 ProtobufUtil.toFilter(ProtobufUtil.toFilter(dependentColumnFilter)))); 129 } 130 131 @Test 132 public void testFamilyFilter() throws Exception { 133 FamilyFilter familyFilter = new FamilyFilter(CompareOperator.EQUAL, 134 new BinaryPrefixComparator(Bytes.toBytes("testValueOne"))); 135 assertTrue(familyFilter 136 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(familyFilter)))); 137 } 138 139 @Test 140 public void testFilterList() throws Exception { 141 // empty filter list 142 FilterList filterList = new FilterList(new LinkedList<>()); 143 assertTrue(filterList 144 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(filterList)))); 145 146 // non-empty filter list 147 LinkedList<Filter> list = new LinkedList<>(); 148 list.add(new ColumnCountGetFilter(1)); 149 list.add(new RowFilter(CompareOperator.EQUAL, new SubstringComparator("testFilterList"))); 150 assertTrue(filterList 151 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(filterList)))); 152 } 153 154 @Test 155 public void testFilterWrapper() throws Exception { 156 FilterWrapper filterWrapper = 157 new FilterWrapper(new ColumnRangeFilter(Bytes.toBytes("e"), false, Bytes.toBytes("f"), true)); 158 assertTrue(filterWrapper 159 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(filterWrapper)))); 160 } 161 162 @Test 163 public void testFirstKeyOnlyFilter() throws Exception { 164 FirstKeyOnlyFilter firstKeyOnlyFilter = new FirstKeyOnlyFilter(); 165 assertTrue(firstKeyOnlyFilter 166 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(firstKeyOnlyFilter)))); 167 } 168 169 @Test 170 public void testFuzzyRowFilter() throws Exception { 171 LinkedList<Pair<byte[], byte[]>> fuzzyList = new LinkedList<>(); 172 fuzzyList.add(new Pair<>(Bytes.toBytes("999"), new byte[] { 0, 0, 1 })); 173 fuzzyList.add(new Pair<>(Bytes.toBytes("abcd"), new byte[] { 1, 0, 1, 1 })); 174 FuzzyRowFilter fuzzyRowFilter = new FuzzyRowFilter(fuzzyList); 175 assertTrue(fuzzyRowFilter 176 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(fuzzyRowFilter)))); 177 } 178 179 @Test 180 public void testInclusiveStopFilter() throws Exception { 181 // InclusveStopFilter with null stopRowKey 182 InclusiveStopFilter inclusiveStopFilter = new InclusiveStopFilter(null); 183 assertTrue(inclusiveStopFilter 184 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(inclusiveStopFilter)))); 185 186 // InclusveStopFilter with non-null stopRowKey 187 inclusiveStopFilter = new InclusiveStopFilter(Bytes.toBytes("inclusiveStopFilter")); 188 assertTrue(inclusiveStopFilter 189 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(inclusiveStopFilter)))); 190 } 191 192 @Test 193 public void testKeyOnlyFilter() throws Exception { 194 // KeyOnlyFilter with lenAsVal 195 KeyOnlyFilter keyOnlyFilter = new KeyOnlyFilter(true); 196 assertTrue(keyOnlyFilter 197 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(keyOnlyFilter)))); 198 199 // KeyOnlyFilter without lenAsVal 200 keyOnlyFilter = new KeyOnlyFilter(); 201 assertTrue(keyOnlyFilter 202 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(keyOnlyFilter)))); 203 } 204 205 @Test 206 public void testMultipleColumnPrefixFilter() throws Exception { 207 // empty array 208 byte[][] prefixes = null; 209 MultipleColumnPrefixFilter multipleColumnPrefixFilter = 210 new MultipleColumnPrefixFilter(prefixes); 211 assertTrue(multipleColumnPrefixFilter.areSerializedFieldsEqual( 212 ProtobufUtil.toFilter(ProtobufUtil.toFilter(multipleColumnPrefixFilter)))); 213 214 // non-empty array 215 prefixes = new byte[2][]; 216 prefixes[0] = Bytes.toBytes("a"); 217 prefixes[1] = Bytes.toBytes(""); 218 multipleColumnPrefixFilter = new MultipleColumnPrefixFilter(prefixes); 219 assertTrue(multipleColumnPrefixFilter.areSerializedFieldsEqual( 220 ProtobufUtil.toFilter(ProtobufUtil.toFilter(multipleColumnPrefixFilter)))); 221 } 222 223 @Test 224 public void testPageFilter() throws Exception { 225 PageFilter pageFilter = new PageFilter(6); 226 assertTrue(pageFilter 227 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(pageFilter)))); 228 } 229 230 @Test 231 public void testPrefixFilter() throws Exception { 232 // null prefix 233 PrefixFilter prefixFilter = new PrefixFilter(null); 234 assertTrue(prefixFilter 235 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(prefixFilter)))); 236 237 // non-null prefix 238 prefixFilter = new PrefixFilter(Bytes.toBytes("abc")); 239 assertTrue(prefixFilter 240 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(prefixFilter)))); 241 } 242 243 @Test 244 public void testQualifierFilter() throws Exception { 245 QualifierFilter qualifierFilter = 246 new QualifierFilter(CompareOperator.EQUAL, new NullComparator()); 247 assertTrue(qualifierFilter 248 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(qualifierFilter)))); 249 } 250 251 @Test 252 public void testRandomRowFilter() throws Exception { 253 RandomRowFilter randomRowFilter = new RandomRowFilter((float) 0.1); 254 assertTrue(randomRowFilter 255 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(randomRowFilter)))); 256 } 257 258 @Test 259 public void testRowFilter() throws Exception { 260 RowFilter rowFilter = 261 new RowFilter(CompareOperator.EQUAL, new SubstringComparator("testRowFilter")); 262 assertTrue( 263 rowFilter.areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(rowFilter)))); 264 } 265 266 @Test 267 public void testSingleColumnValueExcludeFilter() throws Exception { 268 // null family/column SingleColumnValueExcludeFilter 269 SingleColumnValueExcludeFilter singleColumnValueExcludeFilter = 270 new SingleColumnValueExcludeFilter(null, null, CompareOperator.GREATER_OR_EQUAL, 271 Bytes.toBytes("value")); 272 assertTrue(singleColumnValueExcludeFilter.areSerializedFieldsEqual( 273 ProtobufUtil.toFilter(ProtobufUtil.toFilter(singleColumnValueExcludeFilter)))); 274 275 // non-null family/column SingleColumnValueFilter 276 singleColumnValueExcludeFilter = new SingleColumnValueExcludeFilter(Bytes.toBytes("fam"), 277 Bytes.toBytes("qual"), CompareOperator.LESS_OR_EQUAL, new NullComparator(), false, false); 278 assertTrue(singleColumnValueExcludeFilter.areSerializedFieldsEqual( 279 ProtobufUtil.toFilter(ProtobufUtil.toFilter(singleColumnValueExcludeFilter)))); 280 } 281 282 @Test 283 public void testSingleColumnValueFilter() throws Exception { 284 // null family/column SingleColumnValueFilter 285 SingleColumnValueFilter singleColumnValueFilter = 286 new SingleColumnValueFilter(null, null, CompareOperator.LESS, Bytes.toBytes("value")); 287 assertTrue(singleColumnValueFilter.areSerializedFieldsEqual( 288 ProtobufUtil.toFilter(ProtobufUtil.toFilter(singleColumnValueFilter)))); 289 290 // non-null family/column SingleColumnValueFilter 291 singleColumnValueFilter = new SingleColumnValueFilter(Bytes.toBytes("family"), 292 Bytes.toBytes("qualifier"), CompareOperator.NOT_EQUAL, new NullComparator(), true, true); 293 assertTrue(singleColumnValueFilter.areSerializedFieldsEqual( 294 ProtobufUtil.toFilter(ProtobufUtil.toFilter(singleColumnValueFilter)))); 295 } 296 297 @Test 298 public void testSkipFilter() throws Exception { 299 SkipFilter skipFilter = new SkipFilter(new PageFilter(6)); 300 assertTrue(skipFilter 301 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(skipFilter)))); 302 } 303 304 @Test 305 public void testTimestampsFilter() throws Exception { 306 // Empty timestamp list 307 TimestampsFilter timestampsFilter = new TimestampsFilter(new LinkedList<>()); 308 assertTrue(timestampsFilter 309 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(timestampsFilter)))); 310 311 // Non-empty timestamp list 312 LinkedList<Long> list = new LinkedList<>(); 313 list.add(EnvironmentEdgeManager.currentTime()); 314 list.add(EnvironmentEdgeManager.currentTime()); 315 timestampsFilter = new TimestampsFilter(list); 316 assertTrue(timestampsFilter 317 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(timestampsFilter)))); 318 } 319 320 @Test 321 public void testValueFilter() throws Exception { 322 ValueFilter valueFilter = 323 new ValueFilter(CompareOperator.NO_OP, new BinaryComparator(Bytes.toBytes("testValueOne"))); 324 assertTrue(valueFilter 325 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(valueFilter)))); 326 } 327 328 @Test 329 public void testWhileMatchFilter() throws Exception { 330 WhileMatchFilter whileMatchFilter = new WhileMatchFilter( 331 new ColumnRangeFilter(Bytes.toBytes("c"), false, Bytes.toBytes("d"), true)); 332 assertTrue(whileMatchFilter 333 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(whileMatchFilter)))); 334 } 335 336 @Test 337 public void testMultiRowRangeFilter() throws Exception { 338 List<RowRange> ranges = new ArrayList<>(); 339 ranges.add(new RowRange(Bytes.toBytes(30), true, Bytes.toBytes(40), false)); 340 ranges.add(new RowRange(Bytes.toBytes(10), true, Bytes.toBytes(20), false)); 341 ranges.add(new RowRange(Bytes.toBytes(60), true, Bytes.toBytes(70), false)); 342 343 MultiRowRangeFilter multiRowRangeFilter = new MultiRowRangeFilter(ranges); 344 assertTrue(multiRowRangeFilter 345 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(multiRowRangeFilter)))); 346 } 347 348 @Test 349 public void testColumnValueFilter() throws Exception { 350 ColumnValueFilter columnValueFilter = new ColumnValueFilter(Bytes.toBytes("family"), 351 Bytes.toBytes("qualifier"), CompareOperator.EQUAL, Bytes.toBytes("value")); 352 assertTrue(columnValueFilter 353 .areSerializedFieldsEqual(ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnValueFilter)))); 354 } 355 356 /** 357 * Test that we can load and deserialize custom filters. Good to have generally, but also proves 358 * that this still works after HBASE-27276 despite not going through our fast function caches. 359 */ 360 @Test 361 public void testCustomFilter() throws Exception { 362 Filter baseFilter = new PrefixFilter("foo".getBytes()); 363 FilterProtos.Filter filterProto = ProtobufUtil.toFilter(baseFilter); 364 String suffix = "" + System.currentTimeMillis() + allowFastReflectionFallthrough; 365 String className = "CustomLoadedFilter" + suffix; 366 filterProto = filterProto.toBuilder().setName(className).build(); 367 368 Configuration conf = HBaseConfiguration.create(); 369 HBaseTestingUtil testUtil = new HBaseTestingUtil(); 370 String dataTestDir = testUtil.getDataTestDir().toString(); 371 372 // First make sure the test bed is clean, delete any pre-existing class. 373 // Below toComparator call is expected to fail because the comparator is not loaded now 374 ClassLoaderTestHelper.deleteClass(className, dataTestDir, conf); 375 try { 376 Filter filter = ProtobufUtil.toFilter(filterProto); 377 fail("expected to fail"); 378 } catch (DoNotRetryIOException e) { 379 // do nothing, this is expected 380 } 381 382 // Write a jar to be loaded into the classloader 383 String code = StringSubstitutor 384 .replace(IOUtils.toString(getClass().getResourceAsStream("/CustomLoadedFilter.java.template"), 385 Charset.defaultCharset()), Collections.singletonMap("suffix", suffix)); 386 ClassLoaderTestHelper.buildJar(dataTestDir, className, code, 387 ClassLoaderTestHelper.localDirPath(conf)); 388 389 // Disallow fallthrough at first. We expect below to fail because the custom filter is not 390 // available at initialization so not in the cache. 391 ProtobufUtil.setAllowFastReflectionFallthrough(false); 392 try { 393 ProtobufUtil.toFilter(filterProto); 394 fail("expected to fail"); 395 } catch (DoNotRetryIOException e) { 396 // do nothing, this is expected 397 } 398 399 // Now the deserialization should pass with fallthrough enabled. This proves that custom 400 // filters can work despite not being supported by cache. 401 ProtobufUtil.setAllowFastReflectionFallthrough(true); 402 ProtobufUtil.toFilter(filterProto); 403 404 } 405 406}