View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.replication;
20  
21  import java.io.IOException;
22  import java.util.ArrayList;
23  import java.util.List;
24  import java.util.Map;
25  import java.util.UUID;
26  import java.util.concurrent.ConcurrentHashMap;
27  import java.util.concurrent.atomic.AtomicBoolean;
28  import java.util.concurrent.atomic.AtomicInteger;
29  import java.util.concurrent.atomic.AtomicReference;
30  
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.hbase.Cell;
34  import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
35  import org.apache.hadoop.hbase.testclassification.MediumTests;
36  import org.apache.hadoop.hbase.HRegionInfo;
37  import org.apache.hadoop.hbase.KeyValue;
38  import org.apache.hadoop.hbase.TableName;
39  import org.apache.hadoop.hbase.Waiter;
40  import org.apache.hadoop.hbase.client.Connection;
41  import org.apache.hadoop.hbase.client.ConnectionFactory;
42  import org.apache.hadoop.hbase.client.Put;
43  import org.apache.hadoop.hbase.client.Table;
44  import org.apache.hadoop.hbase.wal.WAL.Entry;
45  import org.apache.hadoop.hbase.wal.WALKey;
46  import org.apache.hadoop.hbase.regionserver.HRegion;
47  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
48  import org.apache.hadoop.hbase.replication.regionserver.*;
49  import org.apache.hadoop.hbase.util.Bytes;
50  import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
51  import org.apache.hadoop.hbase.util.Pair;
52  import org.apache.hadoop.hbase.util.Threads;
53  import org.apache.hadoop.hbase.zookeeper.ZKConfig;
54  import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
55  import org.junit.AfterClass;
56  import org.junit.Assert;
57  import org.junit.Before;
58  import org.junit.BeforeClass;
59  import org.junit.Test;
60  import org.junit.experimental.categories.Category;
61  
62  import static org.mockito.Mockito.doNothing;
63  import static org.mockito.Mockito.mock;
64  import static org.mockito.Mockito.spy;
65  import static org.mockito.Mockito.verify;
66  import static org.mockito.Mockito.when;
67  
68  /**
69   * Tests ReplicationSource and ReplicationEndpoint interactions
70   */
71  @Category(MediumTests.class)
72  public class TestReplicationEndpoint extends TestReplicationBase {
73    private static final Log LOG = LogFactory.getLog(TestReplicationEndpoint.class);
74  
75    static int numRegionServers;
76  
77    @BeforeClass
78    public static void setUpBeforeClass() throws Exception {
79      TestReplicationBase.setUpBeforeClass();
80      admin.removePeer("2");
81      numRegionServers = utility1.getHBaseCluster().getRegionServerThreads().size();
82    }
83  
84    @AfterClass
85    public static void tearDownAfterClass() throws Exception {
86      TestReplicationBase.tearDownAfterClass();
87      // check stop is called
88      Assert.assertTrue(ReplicationEndpointForTest.stoppedCount.get() > 0);
89    }
90  
91    @Before
92    public void setup() throws Exception {
93      ReplicationEndpointForTest.contructedCount.set(0);
94      ReplicationEndpointForTest.startedCount.set(0);
95      ReplicationEndpointForTest.replicateCount.set(0);
96      ReplicationEndpointReturningFalse.replicated.set(false);
97      ReplicationEndpointForTest.lastEntries = null;
98      final List<RegionServerThread> rsThreads =
99          utility1.getMiniHBaseCluster().getRegionServerThreads();
100     for (RegionServerThread rs : rsThreads) {
101       utility1.getHBaseAdmin().rollWALWriter(rs.getRegionServer().getServerName());
102     }
103     // Wait for  all log roll to finish
104     utility1.waitFor(3000, new Waiter.ExplainingPredicate<Exception>() {
105       @Override
106       public boolean evaluate() throws Exception {
107         for (RegionServerThread rs : rsThreads) {
108           if (!rs.getRegionServer().walRollRequestFinished()) {
109             return false;
110           }
111         }
112         return true;
113       }
114 
115       @Override
116       public String explainFailure() throws Exception {
117         List<String> logRollInProgressRsList = new ArrayList<String>();
118         for (RegionServerThread rs : rsThreads) {
119           if (!rs.getRegionServer().walRollRequestFinished()) {
120             logRollInProgressRsList.add(rs.getRegionServer().toString());
121           }
122         }
123         return "Still waiting for log roll on regionservers: " + logRollInProgressRsList;
124       }
125     });
126   }
127 
128   @Test (timeout=120000)
129   public void testCustomReplicationEndpoint() throws Exception {
130     // test installing a custom replication endpoint other than the default one.
131     admin.addPeer("testCustomReplicationEndpoint",
132         new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(conf1))
133             .setReplicationEndpointImpl(ReplicationEndpointForTest.class.getName()), null);
134 
135     // check whether the class has been constructed and started
136     Waiter.waitFor(conf1, 60000, new Waiter.Predicate<Exception>() {
137       @Override
138       public boolean evaluate() throws Exception {
139         return ReplicationEndpointForTest.contructedCount.get() >= numRegionServers;
140       }
141     });
142 
143     Waiter.waitFor(conf1, 60000, new Waiter.Predicate<Exception>() {
144       @Override
145       public boolean evaluate() throws Exception {
146         return ReplicationEndpointForTest.startedCount.get() >= numRegionServers;
147       }
148     });
149 
150     Assert.assertEquals(0, ReplicationEndpointForTest.replicateCount.get());
151 
152     // now replicate some data.
153     doPut(Bytes.toBytes("row42"));
154 
155     Waiter.waitFor(conf1, 60000, new Waiter.Predicate<Exception>() {
156       @Override
157       public boolean evaluate() throws Exception {
158         return ReplicationEndpointForTest.replicateCount.get() >= 1;
159       }
160     });
161 
162     doAssert(Bytes.toBytes("row42"));
163 
164     admin.removePeer("testCustomReplicationEndpoint");
165   }
166 
167   @Test (timeout=120000)
168   public void testReplicationEndpointReturnsFalseOnReplicate() throws Exception {
169     Assert.assertEquals(0, ReplicationEndpointForTest.replicateCount.get());
170     Assert.assertTrue(!ReplicationEndpointReturningFalse.replicated.get());
171     int peerCount = admin.getPeersCount();
172     final String id = "testReplicationEndpointReturnsFalseOnReplicate";
173     admin.addPeer(id,
174       new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(conf1))
175         .setReplicationEndpointImpl(ReplicationEndpointReturningFalse.class.getName()), null);
176     // This test is flakey and then there is so much stuff flying around in here its, hard to
177     // debug.  Peer needs to be up for the edit to make it across. This wait on
178     // peer count seems to be a hack that has us not progress till peer is up.
179     if (admin.getPeersCount() <= peerCount) {
180       LOG.info("Waiting on peercount to go up from " + peerCount);
181       Threads.sleep(100);
182     }
183     // now replicate some data
184     doPut(row);
185 
186     Waiter.waitFor(conf1, 60000, new Waiter.Predicate<Exception>() {
187       @Override
188       public boolean evaluate() throws Exception {
189         // Looks like replication endpoint returns false unless we put more than 10 edits. We
190         // only send over one edit.
191         int count = ReplicationEndpointForTest.replicateCount.get();
192         LOG.info("count=" + count);
193         return ReplicationEndpointReturningFalse.replicated.get();
194       }
195     });
196     if (ReplicationEndpointReturningFalse.ex.get() != null) {
197       throw ReplicationEndpointReturningFalse.ex.get();
198     }
199 
200     admin.removePeer("testReplicationEndpointReturnsFalseOnReplicate");
201   }
202 
203   @Test (timeout=120000)
204   public void testInterClusterReplication() throws Exception {
205     final String id = "testInterClusterReplication";
206 
207     List<HRegion> regions = utility1.getHBaseCluster().getRegions(tableName);
208     int totEdits = 0;
209 
210     // Make sure edits are spread across regions because we do region based batching
211     // before shipping edits.
212     for(HRegion region: regions) {
213       HRegionInfo hri = region.getRegionInfo();
214       byte[] row = hri.getStartKey();
215       for (int i = 0; i < 100; i++) {
216         if (row.length > 0) {
217           Put put = new Put(row);
218           put.addColumn(famName, row, row);
219           region.put(put);
220           totEdits++;
221         }
222       }
223     }
224 
225     admin.addPeer(id,
226         new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(conf2))
227             .setReplicationEndpointImpl(InterClusterReplicationEndpointForTest.class.getName()),
228         null);
229 
230     final int numEdits = totEdits;
231     Waiter.waitFor(conf1, 30000, new Waiter.ExplainingPredicate<Exception>() {
232       @Override
233       public boolean evaluate() throws Exception {
234         return InterClusterReplicationEndpointForTest.replicateCount.get() == numEdits;
235       }
236       @Override
237       public String explainFailure() throws Exception {
238         String failure = "Failed to replicate all edits, expected = " + numEdits
239             + " replicated = " + InterClusterReplicationEndpointForTest.replicateCount.get();
240         return failure;
241       }
242     });
243 
244     admin.removePeer("testInterClusterReplication");
245     utility1.deleteTableData(tableName);
246   }
247 
248   @Test (timeout=120000)
249   public void testWALEntryFilterFromReplicationEndpoint() throws Exception {
250     ReplicationPeerConfig rpc =  new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(conf1))
251             .setReplicationEndpointImpl(ReplicationEndpointWithWALEntryFilter.class.getName());
252     //test that we can create mutliple WALFilters reflectively
253     rpc.getConfiguration().put(ReplicationAdmin.REPLICATION_WALENTRYFILTER_CONFIG_KEY,
254         EverythingPassesWALEntryFilter.class.getName() + ","
255             + EverythingPassesWALEntryFilterSubclass.class.getName());
256     admin.addPeer("testWALEntryFilterFromReplicationEndpoint", rpc);
257     // now replicate some data.
258     try (Connection connection = ConnectionFactory.createConnection(conf1)) {
259       doPut(connection, Bytes.toBytes("row1"));
260       doPut(connection, row);
261       doPut(connection, Bytes.toBytes("row2"));
262     }
263 
264     Waiter.waitFor(conf1, 60000, new Waiter.Predicate<Exception>() {
265       @Override
266       public boolean evaluate() throws Exception {
267         return ReplicationEndpointForTest.replicateCount.get() >= 1;
268       }
269     });
270 
271     Assert.assertNull(ReplicationEndpointWithWALEntryFilter.ex.get());
272     //make sure our reflectively created filter is in the filter chain
273     Assert.assertTrue(EverythingPassesWALEntryFilter.hasPassedAnEntry());
274     admin.removePeer("testWALEntryFilterFromReplicationEndpoint");
275   }
276 
277   @Test (timeout=120000, expected=ReplicationException.class)
278   public void testWALEntryFilterAddValidation() throws Exception {
279     ReplicationPeerConfig rpc =  new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(conf1))
280             .setReplicationEndpointImpl(ReplicationEndpointWithWALEntryFilter.class.getName());
281     //test that we can create mutliple WALFilters reflectively
282         rpc.getConfiguration().put(ReplicationAdmin.REPLICATION_WALENTRYFILTER_CONFIG_KEY,
283                 "IAmNotARealWalEntryFilter");
284     admin.addPeer("testWALEntryFilterAddValidation", rpc);
285   }
286 
287       @Test (timeout=120000, expected=ReplicationException.class)
288   public void testWALEntryFilterUpdateValidation() throws Exception {
289     ReplicationPeerConfig rpc =  new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(conf1))
290             .setReplicationEndpointImpl(ReplicationEndpointWithWALEntryFilter.class.getName());
291     //test that we can create mutliple WALFilters reflectively
292         rpc.getConfiguration().put(ReplicationAdmin.REPLICATION_WALENTRYFILTER_CONFIG_KEY,
293                 "IAmNotARealWalEntryFilter");
294     admin.updatePeerConfig("testWALEntryFilterUpdateValidation", rpc);
295   }
296 
297 
298   @Test
299   public void testMetricsSourceBaseSourcePassThrough() {
300     /*
301      * The replication MetricsSource wraps a MetricsReplicationTableSourceImpl,
302      * MetricsReplicationSourceSourceImpl and a MetricsReplicationGlobalSourceSource,
303      * so that metrics get written to both namespaces. Both of those classes wrap a
304      * MetricsReplicationSourceImpl that implements BaseSource, which allows
305      * for custom JMX metrics. This test checks to make sure the BaseSource decorator logic on
306      * MetricsSource actually calls down through the two layers of wrapping to the actual
307      * BaseSource.
308      */
309     String id = "id";
310     DynamicMetricsRegistry mockRegistry = mock(DynamicMetricsRegistry.class);
311     MetricsReplicationSourceImpl singleRms = mock(MetricsReplicationSourceImpl.class);
312     when(singleRms.getMetricsRegistry()).thenReturn(mockRegistry);
313     MetricsReplicationSourceImpl globalRms = mock(MetricsReplicationSourceImpl.class);
314     when(globalRms.getMetricsRegistry()).thenReturn(mockRegistry);
315 
316     MetricsReplicationSourceSource singleSourceSource = new MetricsReplicationSourceSourceImpl(singleRms, id);
317     MetricsReplicationSourceSource globalSourceSource = new MetricsReplicationGlobalSourceSource(globalRms);
318     MetricsReplicationSourceSource spyglobalSourceSource = spy(globalSourceSource);
319     doNothing().when(spyglobalSourceSource).incrFailedRecoveryQueue();
320 
321     Map<String, MetricsReplicationTableSource> singleSourceSourceByTable = new ConcurrentHashMap<>();
322     MetricsSource source = new MetricsSource(id, singleSourceSource,
323       spyglobalSourceSource, singleSourceSourceByTable);
324 
325     String gaugeName = "gauge";
326     String singleGaugeName = "source.id." + gaugeName;
327     long delta = 1;
328     String counterName = "counter";
329     String singleCounterName = "source.id." + counterName;
330     long count = 2;
331     source.decGauge(gaugeName, delta);
332     source.getMetricsContext();
333     source.getMetricsDescription();
334     source.getMetricsJmxContext();
335     source.getMetricsName();
336     source.incCounters(counterName, count);
337     source.incGauge(gaugeName, delta);
338     source.init();
339     source.removeMetric(gaugeName);
340     source.setGauge(gaugeName, delta);
341     source.updateHistogram(counterName, count);
342     source.incrFailedRecoveryQueue();
343 
344     verify(singleRms).decGauge(singleGaugeName, delta);
345     verify(globalRms).decGauge(gaugeName, delta);
346     verify(globalRms).getMetricsContext();
347     verify(globalRms).getMetricsJmxContext();
348     verify(globalRms).getMetricsName();
349     verify(singleRms).incCounters(singleCounterName, count);
350     verify(globalRms).incCounters(counterName, count);
351     verify(singleRms).incGauge(singleGaugeName, delta);
352     verify(globalRms).incGauge(gaugeName, delta);
353     verify(globalRms).init();
354     verify(singleRms).removeMetric(singleGaugeName);
355     verify(globalRms).removeMetric(gaugeName);
356     verify(singleRms).setGauge(singleGaugeName, delta);
357     verify(globalRms).setGauge(gaugeName, delta);
358     verify(singleRms).updateHistogram(singleCounterName, count);
359     verify(globalRms).updateHistogram(counterName, count);
360     verify(spyglobalSourceSource).incrFailedRecoveryQueue();
361 
362     // check singleSourceSourceByTable metrics.
363     // singleSourceSourceByTable map entry will be created only
364     // after calling #setAgeOfLastShippedOpByTable
365     boolean containsRandomNewTable = source.getSingleSourceSourceByTable()
366         .containsKey("RandomNewTable");
367     Assert.assertEquals(false, containsRandomNewTable);
368     source.updateTableLevelMetrics(createWALEntriesWithSize("RandomNewTable"));
369     containsRandomNewTable = source.getSingleSourceSourceByTable()
370       .containsKey("RandomNewTable");
371     Assert.assertEquals(true, containsRandomNewTable);
372     MetricsReplicationTableSource msr = source.getSingleSourceSourceByTable()
373       .get("RandomNewTable");
374     // age should be greater than zero we created the entry with time in the past
375     Assert.assertTrue(msr.getLastShippedAge() > 0);
376     Assert.assertTrue(msr.getShippedBytes() > 0);
377   }
378 
379   private List<Pair<Entry, Long>> createWALEntriesWithSize(String tableName) {
380     List<Pair<Entry, Long>> walEntriesWithSize = new ArrayList<>();
381     byte[] a = new byte[] { 'a' };
382     Entry entry = createEntry(tableName, a);
383     walEntriesWithSize.add(new Pair<>(entry, 10L));
384     return walEntriesWithSize;
385   }
386 
387   private Entry createEntry(String tableName, byte[]... kvs) {
388     WALKey key1 = new WALKey(new byte[0], TableName.valueOf(tableName),
389       System.currentTimeMillis() - 1L);
390     WALEdit edit1 = new WALEdit();
391     for (byte[] kv : kvs) {
392       edit1.add(new KeyValue(kv, kv, kv));
393     }
394     return new Entry(key1, edit1);
395   }
396 
397   private void doPut(byte[] row) throws IOException {
398     try (Connection connection = ConnectionFactory.createConnection(conf1)) {
399       doPut(connection, row);
400     }
401   }
402 
403   private void doPut(final Connection connection, final byte [] row) throws IOException {
404     try (Table t = connection.getTable(tableName)) {
405       Put put = new Put(row);
406       put.add(famName, row, row);
407       t.put(put);
408     }
409   }
410 
411   private static void doAssert(byte[] row) throws Exception {
412     if (ReplicationEndpointForTest.lastEntries == null) {
413       return; // first call
414     }
415     Assert.assertEquals(1, ReplicationEndpointForTest.lastEntries.size());
416     List<Cell> cells = ReplicationEndpointForTest.lastEntries.get(0).getEdit().getCells();
417     Assert.assertEquals(1, cells.size());
418     Assert.assertTrue(Bytes.equals(cells.get(0).getRowArray(), cells.get(0).getRowOffset(),
419       cells.get(0).getRowLength(), row, 0, row.length));
420   }
421 
422   /**
423    * Bad Endpoint with failing connection to peer on demand.
424    */
425   public static class BadReplicationEndpoint extends ReplicationEndpointForTest {
426     static boolean failing = true;
427 
428     public BadReplicationEndpoint() {
429       super();
430     }
431 
432     @Override
433     public synchronized UUID getPeerUUID() {
434       return failing ? null : super.getPeerUUID();
435     }
436   }
437 
438   public static class ReplicationEndpointForTest extends BaseReplicationEndpoint {
439     static UUID uuid = UUID.randomUUID();
440     static AtomicInteger contructedCount = new AtomicInteger();
441     static AtomicInteger startedCount = new AtomicInteger();
442     static AtomicInteger stoppedCount = new AtomicInteger();
443     static AtomicInteger replicateCount = new AtomicInteger();
444     static volatile List<Entry> lastEntries = null;
445 
446     public ReplicationEndpointForTest() {
447       contructedCount.incrementAndGet();
448     }
449 
450     @Override
451     public UUID getPeerUUID() {
452       return uuid;
453     }
454 
455     @Override
456     public boolean replicate(ReplicateContext replicateContext) {
457       replicateCount.incrementAndGet();
458       lastEntries = new ArrayList<>(replicateContext.entries);
459       return true;
460     }
461 
462     @Override
463     protected void doStart() {
464       startedCount.incrementAndGet();
465       notifyStarted();
466     }
467 
468     @Override
469     protected void doStop() {
470       stoppedCount.incrementAndGet();
471       notifyStopped();
472     }
473   }
474 
475   public static class InterClusterReplicationEndpointForTest
476       extends HBaseInterClusterReplicationEndpoint {
477 
478     static AtomicInteger replicateCount = new AtomicInteger();
479     static boolean failedOnce;
480 
481     @Override
482     public boolean replicate(ReplicateContext replicateContext) {
483       boolean success = super.replicate(replicateContext);
484       if (success) {
485         replicateCount.addAndGet(replicateContext.entries.size());
486       }
487       return success;
488     }
489 
490     @Override
491     protected Replicator createReplicator(List<Entry> entries, int ordinal) {
492       // Fail only once, we don't want to slow down the test.
493       if (failedOnce) {
494         return new DummyReplicator(entries, ordinal);
495       } else {
496         failedOnce = true;
497         return new FailingDummyReplicator(entries, ordinal);
498       }
499     }
500 
501     protected class DummyReplicator extends Replicator {
502 
503       private int ordinal;
504 
505       public DummyReplicator(List<Entry> entries, int ordinal) {
506         super(entries, ordinal);
507         this.ordinal = ordinal;
508       }
509 
510       @Override
511       public Integer call() throws IOException {
512         return ordinal;
513       }
514     }
515 
516     protected class FailingDummyReplicator extends DummyReplicator {
517 
518       public FailingDummyReplicator(List<Entry> entries, int ordinal) {
519         super(entries, ordinal);
520       }
521 
522       @Override
523       public Integer call() throws IOException {
524         throw new IOException("Sample Exception: Failed to replicate.");
525       }
526     }
527   }
528 
529   public static class ReplicationEndpointReturningFalse extends ReplicationEndpointForTest {
530     static int COUNT = 10;
531     static AtomicReference<Exception> ex = new AtomicReference<Exception>(null);
532     static AtomicBoolean replicated = new AtomicBoolean(false);
533     @Override
534     public boolean replicate(ReplicateContext replicateContext) {
535       try {
536         // check row
537         doAssert(row);
538       } catch (Exception e) {
539         ex.set(e);
540       }
541 
542       super.replicate(replicateContext);
543       LOG.info("Replicated " + Bytes.toString(row) + ", count=" + replicateCount.get());
544 
545       replicated.set(replicateCount.get() > COUNT); // first 10 times, we return false
546       return replicated.get();
547     }
548   }
549 
550   // return a WALEntry filter which only accepts "row", but not other rows
551   public static class ReplicationEndpointWithWALEntryFilter extends ReplicationEndpointForTest {
552     static AtomicReference<Exception> ex = new AtomicReference<Exception>(null);
553 
554     @Override
555     public boolean replicate(ReplicateContext replicateContext) {
556       try {
557         super.replicate(replicateContext);
558         doAssert(row);
559       } catch (Exception e) {
560         ex.set(e);
561       }
562       return true;
563     }
564 
565     @Override
566     public WALEntryFilter getWALEntryfilter() {
567       return new ChainWALEntryFilter(super.getWALEntryfilter(), new WALEntryFilter() {
568         @Override
569         public Entry filter(Entry entry) {
570           ArrayList<Cell> cells = entry.getEdit().getCells();
571           int size = cells.size();
572           for (int i = size-1; i >= 0; i--) {
573             Cell cell = cells.get(i);
574             if (!Bytes.equals(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
575               row, 0, row.length)) {
576               cells.remove(i);
577             }
578           }
579           return entry;
580         }
581       });
582     }
583   }
584 
585   public static class EverythingPassesWALEntryFilter implements WALEntryFilter {
586     private static boolean passedEntry = false;
587     @Override
588     public Entry filter(Entry entry) {
589       passedEntry = true;
590       return entry;
591     }
592 
593     public static boolean hasPassedAnEntry(){
594         return passedEntry;
595     }
596   }
597 
598   public static class EverythingPassesWALEntryFilterSubclass extends EverythingPassesWALEntryFilter {
599 
600   }
601 }