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.replication;
019
020import static org.junit.Assert.assertNull;
021import static org.junit.Assert.assertTrue;
022import static org.mockito.Mockito.mock;
023import static org.mockito.Mockito.when;
024
025import java.util.ArrayList;
026import java.util.HashMap;
027import java.util.HashSet;
028import java.util.List;
029import java.util.Map;
030import java.util.Set;
031import java.util.TreeMap;
032import org.apache.hadoop.hbase.Cell;
033import org.apache.hadoop.hbase.CellComparatorImpl;
034import org.apache.hadoop.hbase.HBaseClassTestRule;
035import org.apache.hadoop.hbase.HConstants;
036import org.apache.hadoop.hbase.KeyValue;
037import org.apache.hadoop.hbase.TableName;
038import org.apache.hadoop.hbase.client.RegionInfoBuilder;
039import org.apache.hadoop.hbase.security.access.PermissionStorage;
040import org.apache.hadoop.hbase.security.visibility.VisibilityConstants;
041import org.apache.hadoop.hbase.testclassification.ReplicationTests;
042import org.apache.hadoop.hbase.testclassification.SmallTests;
043import org.apache.hadoop.hbase.util.Bytes;
044import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
045import org.apache.hadoop.hbase.wal.WAL.Entry;
046import org.apache.hadoop.hbase.wal.WALEdit;
047import org.apache.hadoop.hbase.wal.WALKeyImpl;
048import org.junit.Assert;
049import org.junit.ClassRule;
050import org.junit.Test;
051import org.junit.experimental.categories.Category;
052
053import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
054
055@Category({ ReplicationTests.class, SmallTests.class })
056public class TestReplicationWALEntryFilters {
057
058  @ClassRule
059  public static final HBaseClassTestRule CLASS_RULE =
060    HBaseClassTestRule.forClass(TestReplicationWALEntryFilters.class);
061
062  static byte[] a = new byte[] { 'a' };
063  static byte[] b = new byte[] { 'b' };
064  static byte[] c = new byte[] { 'c' };
065  static byte[] d = new byte[] { 'd' };
066
067  @Test
068  public void testSystemTableWALEntryFilter() {
069    SystemTableWALEntryFilter filter = new SystemTableWALEntryFilter();
070
071    // meta
072    WALKeyImpl key1 =
073      new WALKeyImpl(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
074        TableName.META_TABLE_NAME, EnvironmentEdgeManager.currentTime());
075    Entry metaEntry = new Entry(key1, null);
076
077    assertNull(filter.filter(metaEntry));
078
079    // user table
080    WALKeyImpl key3 =
081      new WALKeyImpl(new byte[0], TableName.valueOf("foo"), EnvironmentEdgeManager.currentTime());
082    Entry userEntry = new Entry(key3, null);
083
084    assertEquals(userEntry, filter.filter(userEntry));
085
086    // hbase:acl should be allowed through the filter
087    WALKeyImpl key4 =
088      new WALKeyImpl(new byte[0], PermissionStorage.ACL_TABLE_NAME, System.currentTimeMillis());
089    Entry aclEntry = new Entry(key4, null);
090    assertEquals(aclEntry, filter.filter(aclEntry));
091
092    // hbase:labels should be allowed through the filter
093    WALKeyImpl key5 = new WALKeyImpl(new byte[0], VisibilityConstants.LABELS_TABLE_NAME,
094      System.currentTimeMillis());
095    Entry labelsEntry = new Entry(key5, null);
096    assertEquals(labelsEntry, filter.filter(labelsEntry));
097  }
098
099  @Test
100  public void testScopeWALEntryFilter() {
101    WALEntryFilter filter = new ChainWALEntryFilter(new ScopeWALEntryFilter());
102
103    Entry userEntry = createEntry(null, a, b);
104    Entry userEntryA = createEntry(null, a);
105    Entry userEntryB = createEntry(null, b);
106    Entry userEntryEmpty = createEntry(null);
107
108    // no scopes
109    // now we will not filter out entries without a replication scope since serial replication still
110    // need the sequence id, but the cells will all be filtered out.
111    assertTrue(filter.filter(userEntry).getEdit().isEmpty());
112
113    // empty scopes
114    // ditto
115    TreeMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
116    userEntry = createEntry(scopes, a, b);
117    assertTrue(filter.filter(userEntry).getEdit().isEmpty());
118
119    // different scope
120    scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
121    scopes.put(c, HConstants.REPLICATION_SCOPE_GLOBAL);
122    userEntry = createEntry(scopes, a, b);
123    // all kvs should be filtered
124    assertEquals(userEntryEmpty, filter.filter(userEntry));
125
126    // local scope
127    scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
128    scopes.put(a, HConstants.REPLICATION_SCOPE_LOCAL);
129    userEntry = createEntry(scopes, a, b);
130    assertEquals(userEntryEmpty, filter.filter(userEntry));
131    scopes.put(b, HConstants.REPLICATION_SCOPE_LOCAL);
132    assertEquals(userEntryEmpty, filter.filter(userEntry));
133
134    // only scope a
135    scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
136    scopes.put(a, HConstants.REPLICATION_SCOPE_GLOBAL);
137    userEntry = createEntry(scopes, a, b);
138    assertEquals(userEntryA, filter.filter(userEntry));
139    scopes.put(b, HConstants.REPLICATION_SCOPE_LOCAL);
140    assertEquals(userEntryA, filter.filter(userEntry));
141
142    // only scope b
143    scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
144    scopes.put(b, HConstants.REPLICATION_SCOPE_GLOBAL);
145    userEntry = createEntry(scopes, a, b);
146    assertEquals(userEntryB, filter.filter(userEntry));
147    scopes.put(a, HConstants.REPLICATION_SCOPE_LOCAL);
148    assertEquals(userEntryB, filter.filter(userEntry));
149
150    // scope a and b
151    scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
152    scopes.put(b, HConstants.REPLICATION_SCOPE_GLOBAL);
153    userEntry = createEntry(scopes, a, b);
154    assertEquals(userEntryB, filter.filter(userEntry));
155    scopes.put(a, HConstants.REPLICATION_SCOPE_LOCAL);
156    assertEquals(userEntryB, filter.filter(userEntry));
157  }
158
159  WALEntryFilter nullFilter = new WALEntryFilter() {
160    @Override
161    public Entry filter(Entry entry) {
162      return null;
163    }
164  };
165
166  WALEntryFilter passFilter = new WALEntryFilter() {
167    @Override
168    public Entry filter(Entry entry) {
169      return entry;
170    }
171  };
172
173  public static class FilterSomeCellsWALCellFilter implements WALEntryFilter, WALCellFilter {
174    @Override
175    public Entry filter(Entry entry) {
176      return entry;
177    }
178
179    @Override
180    public Cell filterCell(Entry entry, Cell cell) {
181      if (
182        Bytes.toString(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()).equals("a")
183      ) {
184        return null;
185      } else {
186        return cell;
187      }
188    }
189  }
190
191  public static class FilterAllCellsWALCellFilter implements WALEntryFilter, WALCellFilter {
192    @Override
193    public Entry filter(Entry entry) {
194      return entry;
195    }
196
197    @Override
198    public Cell filterCell(Entry entry, Cell cell) {
199      return null;
200    }
201  }
202
203  @Test
204  public void testChainWALEntryWithCellFilter() {
205    Entry userEntry = createEntry(null, a, b, c);
206    ChainWALEntryFilter filterSomeCells =
207      new ChainWALEntryFilter(new FilterSomeCellsWALCellFilter());
208    // since WALCellFilter filter cells with rowkey 'a'
209    assertEquals(createEntry(null, b, c), filterSomeCells.filter(userEntry));
210
211    Entry userEntry2 = createEntry(null, b, c, d);
212    // since there is no cell to get filtered, nothing should get filtered
213    assertEquals(userEntry2, filterSomeCells.filter(userEntry2));
214
215    // since we filter all the cells, we should get empty entry
216    ChainWALEntryFilter filterAllCells = new ChainWALEntryFilter(new FilterAllCellsWALCellFilter());
217    assertEquals(createEntry(null), filterAllCells.filter(userEntry));
218  }
219
220  @Test
221  public void testChainWALEmptyEntryWithCellFilter() {
222    Entry userEntry = createEntry(null, a, b, c);
223    ChainWALEmptyEntryFilter filterSomeCells =
224      new ChainWALEmptyEntryFilter(new FilterSomeCellsWALCellFilter());
225    // since WALCellFilter filter cells with rowkey 'a'
226    assertEquals(createEntry(null, b, c), filterSomeCells.filter(userEntry));
227
228    Entry userEntry2 = createEntry(null, b, c, d);
229    // since there is no cell to get filtered, nothing should get filtered
230    assertEquals(userEntry2, filterSomeCells.filter(userEntry2));
231
232    ChainWALEmptyEntryFilter filterAllCells =
233      new ChainWALEmptyEntryFilter(new FilterAllCellsWALCellFilter());
234    assertEquals(createEntry(null), filterAllCells.filter(userEntry));
235    // let's set the filter empty entry flag to true now for the above case
236    filterAllCells.setFilterEmptyEntry(true);
237    // since WALCellFilter filter all cells, whole entry should be filtered
238    assertEquals(null, filterAllCells.filter(userEntry));
239  }
240
241  @Test
242  public void testChainWALEntryFilter() {
243    Entry userEntry = createEntry(null, a, b, c);
244
245    ChainWALEntryFilter filter = new ChainWALEntryFilter(passFilter);
246    assertEquals(createEntry(null, a, b, c), filter.filter(userEntry));
247
248    filter = new ChainWALEntryFilter(passFilter, passFilter);
249    assertEquals(createEntry(null, a, b, c), filter.filter(userEntry));
250
251    filter = new ChainWALEntryFilter(passFilter, passFilter, passFilter);
252    assertEquals(createEntry(null, a, b, c), filter.filter(userEntry));
253
254    filter = new ChainWALEntryFilter(nullFilter);
255    assertEquals(null, filter.filter(userEntry));
256
257    filter = new ChainWALEntryFilter(nullFilter, passFilter);
258    assertEquals(null, filter.filter(userEntry));
259
260    filter = new ChainWALEntryFilter(passFilter, nullFilter);
261    assertEquals(null, filter.filter(userEntry));
262
263    filter = new ChainWALEntryFilter(nullFilter, passFilter, nullFilter);
264    assertEquals(null, filter.filter(userEntry));
265
266    filter = new ChainWALEntryFilter(nullFilter, nullFilter);
267    assertEquals(null, filter.filter(userEntry));
268
269    // flatten
270    filter = new ChainWALEntryFilter(
271      new ChainWALEntryFilter(passFilter, new ChainWALEntryFilter(passFilter, passFilter),
272        new ChainWALEntryFilter(passFilter), new ChainWALEntryFilter(passFilter)),
273      new ChainWALEntryFilter(passFilter));
274    assertEquals(createEntry(null, a, b, c), filter.filter(userEntry));
275
276    filter = new ChainWALEntryFilter(
277      new ChainWALEntryFilter(passFilter,
278        new ChainWALEntryFilter(passFilter, new ChainWALEntryFilter(nullFilter))),
279      new ChainWALEntryFilter(passFilter));
280    assertEquals(null, filter.filter(userEntry));
281  }
282
283  @Test
284  public void testNamespaceTableCfWALEntryFilter() {
285    ReplicationPeer peer = mock(ReplicationPeer.class);
286    ReplicationPeerConfigBuilder peerConfigBuilder = ReplicationPeerConfig.newBuilder();
287
288    // 1. replicate_all flag is false, no namespaces and table-cfs config
289    peerConfigBuilder.setReplicateAllUserTables(false).setNamespaces(null).setTableCFsMap(null);
290    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
291    Entry userEntry = createEntry(null, a, b, c);
292    ChainWALEntryFilter filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
293    assertEquals(null, filter.filter(userEntry));
294
295    // 2. replicate_all flag is false, and only config table-cfs in peer
296    // empty map
297    userEntry = createEntry(null, a, b, c);
298    Map<TableName, List<String>> tableCfs = new HashMap<>();
299    peerConfigBuilder.setReplicateAllUserTables(false).setTableCFsMap(tableCfs);
300    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
301    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
302    assertEquals(null, filter.filter(userEntry));
303
304    // table bar
305    userEntry = createEntry(null, a, b, c);
306    tableCfs = new HashMap<>();
307    tableCfs.put(TableName.valueOf("bar"), null);
308    peerConfigBuilder.setReplicateAllUserTables(false).setTableCFsMap(tableCfs);
309    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
310    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
311    assertEquals(null, filter.filter(userEntry));
312
313    // table foo:a
314    userEntry = createEntry(null, a, b, c);
315    tableCfs = new HashMap<>();
316    tableCfs.put(TableName.valueOf("foo"), Lists.newArrayList("a"));
317    peerConfigBuilder.setReplicateAllUserTables(false).setTableCFsMap(tableCfs);
318    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
319    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
320    assertEquals(createEntry(null, a), filter.filter(userEntry));
321
322    // table foo:a,c
323    userEntry = createEntry(null, a, b, c, d);
324    tableCfs = new HashMap<>();
325    tableCfs.put(TableName.valueOf("foo"), Lists.newArrayList("a", "c"));
326    peerConfigBuilder.setReplicateAllUserTables(false).setTableCFsMap(tableCfs);
327    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
328    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
329    assertEquals(createEntry(null, a, c), filter.filter(userEntry));
330
331    // 3. replicate_all flag is false, and only config namespaces in peer
332    when(peer.getTableCFs()).thenReturn(null);
333    // empty set
334    Set<String> namespaces = new HashSet<>();
335    peerConfigBuilder.setReplicateAllUserTables(false).setNamespaces(namespaces)
336      .setTableCFsMap(null);
337    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
338    userEntry = createEntry(null, a, b, c);
339    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
340    assertEquals(null, filter.filter(userEntry));
341
342    // namespace default
343    namespaces.add("default");
344    peerConfigBuilder.setReplicateAllUserTables(false).setNamespaces(namespaces);
345    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
346    userEntry = createEntry(null, a, b, c);
347    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
348    assertEquals(createEntry(null, a, b, c), filter.filter(userEntry));
349
350    // namespace ns1
351    namespaces = new HashSet<>();
352    namespaces.add("ns1");
353    peerConfigBuilder.setReplicateAllUserTables(false).setNamespaces(namespaces);
354    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
355    userEntry = createEntry(null, a, b, c);
356    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
357    assertEquals(null, filter.filter(userEntry));
358
359    // 4. replicate_all flag is false, and config namespaces and table-cfs both
360    // Namespaces config should not confict with table-cfs config
361    namespaces = new HashSet<>();
362    tableCfs = new HashMap<>();
363    namespaces.add("ns1");
364    tableCfs.put(TableName.valueOf("foo"), Lists.newArrayList("a", "c"));
365    peerConfigBuilder.setReplicateAllUserTables(false).setNamespaces(namespaces)
366      .setTableCFsMap(tableCfs);
367    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
368    userEntry = createEntry(null, a, b, c);
369    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
370    assertEquals(createEntry(null, a, c), filter.filter(userEntry));
371
372    namespaces = new HashSet<>();
373    tableCfs = new HashMap<>();
374    namespaces.add("default");
375    tableCfs.put(TableName.valueOf("ns1:foo"), Lists.newArrayList("a", "c"));
376    peerConfigBuilder.setReplicateAllUserTables(false).setNamespaces(namespaces)
377      .setTableCFsMap(tableCfs);
378    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
379    userEntry = createEntry(null, a, b, c);
380    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
381    assertEquals(createEntry(null, a, b, c), filter.filter(userEntry));
382
383    namespaces = new HashSet<>();
384    tableCfs = new HashMap<>();
385    namespaces.add("ns1");
386    tableCfs.put(TableName.valueOf("bar"), null);
387    peerConfigBuilder.setReplicateAllUserTables(false).setNamespaces(namespaces)
388      .setTableCFsMap(tableCfs);
389    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
390    userEntry = createEntry(null, a, b, c);
391    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
392    assertEquals(null, filter.filter(userEntry));
393  }
394
395  @Test
396  public void testNamespaceTableCfWALEntryFilter2() {
397    ReplicationPeer peer = mock(ReplicationPeer.class);
398    ReplicationPeerConfigBuilder peerConfigBuilder = ReplicationPeerConfig.newBuilder();
399
400    // 1. replicate_all flag is true
401    // and no exclude namespaces and no exclude table-cfs config
402    peerConfigBuilder.setReplicateAllUserTables(true).setExcludeNamespaces(null)
403      .setExcludeTableCFsMap(null);
404    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
405    Entry userEntry = createEntry(null, a, b, c);
406    ChainWALEntryFilter filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
407    assertEquals(createEntry(null, a, b, c), filter.filter(userEntry));
408
409    // 2. replicate_all flag is true, and only config exclude namespaces
410    // empty set
411    Set<String> namespaces = new HashSet<String>();
412    peerConfigBuilder.setExcludeNamespaces(namespaces).setExcludeTableCFsMap(null);
413    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
414    userEntry = createEntry(null, a, b, c);
415    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
416    assertEquals(createEntry(null, a, b, c), filter.filter(userEntry));
417
418    // exclude namespace default
419    namespaces.add("default");
420    peerConfigBuilder.setExcludeNamespaces(namespaces).setExcludeTableCFsMap(null);
421    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
422    userEntry = createEntry(null, a, b, c);
423    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
424    assertEquals(null, filter.filter(userEntry));
425
426    // exclude namespace ns1
427    namespaces = new HashSet<String>();
428    namespaces.add("ns1");
429    peerConfigBuilder.setExcludeNamespaces(namespaces).setExcludeTableCFsMap(null);
430    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
431    userEntry = createEntry(null, a, b, c);
432    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
433    assertEquals(createEntry(null, a, b, c), filter.filter(userEntry));
434
435    // 3. replicate_all flag is true, and only config exclude table-cfs
436    // empty table-cfs map
437    Map<TableName, List<String>> tableCfs = new HashMap<TableName, List<String>>();
438    peerConfigBuilder.setExcludeNamespaces(null).setExcludeTableCFsMap(tableCfs);
439    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
440    userEntry = createEntry(null, a, b, c);
441    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
442    assertEquals(createEntry(null, a, b, c), filter.filter(userEntry));
443
444    // exclude table bar
445    tableCfs = new HashMap<TableName, List<String>>();
446    tableCfs.put(TableName.valueOf("bar"), null);
447    peerConfigBuilder.setExcludeNamespaces(null).setExcludeTableCFsMap(tableCfs);
448    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
449    userEntry = createEntry(null, a, b, c);
450    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
451    assertEquals(createEntry(null, a, b, c), filter.filter(userEntry));
452
453    // exclude table foo:a
454    tableCfs = new HashMap<TableName, List<String>>();
455    tableCfs.put(TableName.valueOf("foo"), Lists.newArrayList("a"));
456    peerConfigBuilder.setExcludeNamespaces(null).setExcludeTableCFsMap(tableCfs);
457    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
458    userEntry = createEntry(null, a, b, c);
459    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
460    assertEquals(createEntry(null, b, c), filter.filter(userEntry));
461
462    // 4. replicate_all flag is true, and config exclude namespaces and table-cfs both
463    // exclude ns1 and table foo:a,c
464    namespaces = new HashSet<String>();
465    tableCfs = new HashMap<TableName, List<String>>();
466    namespaces.add("ns1");
467    tableCfs.put(TableName.valueOf("foo"), Lists.newArrayList("a", "c"));
468    peerConfigBuilder.setExcludeNamespaces(namespaces).setExcludeTableCFsMap(tableCfs);
469    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
470    userEntry = createEntry(null, a, b, c);
471    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
472    assertEquals(createEntry(null, b), filter.filter(userEntry));
473
474    // exclude namespace default and table ns1:bar
475    namespaces = new HashSet<String>();
476    tableCfs = new HashMap<TableName, List<String>>();
477    namespaces.add("default");
478    tableCfs.put(TableName.valueOf("ns1:bar"), new ArrayList<String>());
479    peerConfigBuilder.setExcludeNamespaces(namespaces).setExcludeTableCFsMap(tableCfs);
480    when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build());
481    userEntry = createEntry(null, a, b, c);
482    filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer));
483    assertEquals(null, filter.filter(userEntry));
484  }
485
486  private Entry createEntry(TreeMap<byte[], Integer> scopes, byte[]... kvs) {
487    WALKeyImpl key1 = new WALKeyImpl(new byte[0], TableName.valueOf("foo"),
488      EnvironmentEdgeManager.currentTime(), scopes);
489    WALEdit edit1 = new WALEdit();
490
491    for (byte[] kv : kvs) {
492      edit1.add(new KeyValue(kv, kv, kv));
493    }
494    return new Entry(key1, edit1);
495  }
496
497  private void assertEquals(Entry e1, Entry e2) {
498    Assert.assertEquals(e1 == null, e2 == null);
499    if (e1 == null) {
500      return;
501    }
502
503    // do not compare WALKeys
504
505    // compare kvs
506    Assert.assertEquals(e1.getEdit() == null, e2.getEdit() == null);
507    if (e1.getEdit() == null) {
508      return;
509    }
510    List<Cell> cells1 = e1.getEdit().getCells();
511    List<Cell> cells2 = e2.getEdit().getCells();
512    Assert.assertEquals(cells1.size(), cells2.size());
513    for (int i = 0; i < cells1.size(); i++) {
514      CellComparatorImpl.COMPARATOR.compare(cells1.get(i), cells2.get(i));
515    }
516  }
517}