|
| 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 | +package org.apache.hadoop.hbase.replication.regionserver; |
| 19 | + |
| 20 | +import static org.junit.Assert.assertEquals; |
| 21 | + |
| 22 | +import java.util.Collections; |
| 23 | +import java.util.UUID; |
| 24 | +import java.util.concurrent.atomic.AtomicInteger; |
| 25 | +import org.apache.hadoop.conf.Configuration; |
| 26 | +import org.apache.hadoop.hbase.HBaseConfiguration; |
| 27 | +import org.apache.hadoop.hbase.Waiter; |
| 28 | +import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint; |
| 29 | +import org.apache.hadoop.hbase.wal.WAL; |
| 30 | +import org.junit.Test; |
| 31 | +import org.mockito.Mockito; |
| 32 | + |
| 33 | +/** |
| 34 | + * Tests for staged WAL flush behavior in ReplicationSourceShipper. These tests validate that |
| 35 | + * beforePersistingReplicationOffset() is invoked only when there is staged WAL data, and is not |
| 36 | + * invoked for empty batches. |
| 37 | + */ |
| 38 | +public class TestReplicationSourceShipperBufferedFlush { |
| 39 | + |
| 40 | + /** |
| 41 | + * ReplicationEndpoint implementation used for testing. Counts how many times |
| 42 | + * beforePersistingReplicationOffset() is called. |
| 43 | + */ |
| 44 | + static class CountingReplicationEndpoint extends BaseReplicationEndpoint { |
| 45 | + |
| 46 | + private final AtomicInteger beforePersistCalls = new AtomicInteger(); |
| 47 | + |
| 48 | + @Override |
| 49 | + protected void doStart() { |
| 50 | + notifyStarted(); |
| 51 | + } |
| 52 | + |
| 53 | + @Override |
| 54 | + protected void doStop() { |
| 55 | + notifyStopped(); |
| 56 | + } |
| 57 | + |
| 58 | + @Override |
| 59 | + public UUID getPeerUUID() { |
| 60 | + return null; |
| 61 | + } |
| 62 | + |
| 63 | + @Override |
| 64 | + public boolean replicate(ReplicateContext ctx) { |
| 65 | + return true; |
| 66 | + } |
| 67 | + |
| 68 | + @Override |
| 69 | + public void start() { |
| 70 | + |
| 71 | + } |
| 72 | + |
| 73 | + @Override |
| 74 | + public void stop() { |
| 75 | + |
| 76 | + } |
| 77 | + |
| 78 | + @Override |
| 79 | + public void beforePersistingReplicationOffset() { |
| 80 | + beforePersistCalls.incrementAndGet(); |
| 81 | + } |
| 82 | + |
| 83 | + int getBeforePersistCalls() { |
| 84 | + return beforePersistCalls.get(); |
| 85 | + } |
| 86 | + |
| 87 | + @Override |
| 88 | + public long getMaxBufferSize() { |
| 89 | + // Force size-based flush after any non-empty batch |
| 90 | + return 1L; |
| 91 | + } |
| 92 | + |
| 93 | + @Override |
| 94 | + public long maxFlushInterval() { |
| 95 | + return Long.MAX_VALUE; |
| 96 | + } |
| 97 | + } |
| 98 | + |
| 99 | + @Test |
| 100 | + public void testBeforePersistNotCalledForEmptyBatch() throws Exception { |
| 101 | + Configuration conf = HBaseConfiguration.create(); |
| 102 | + |
| 103 | + CountingReplicationEndpoint endpoint = new CountingReplicationEndpoint(); |
| 104 | + |
| 105 | + ReplicationSource source = Mockito.mock(ReplicationSource.class); |
| 106 | + ReplicationSourceWALReader walReader = Mockito.mock(ReplicationSourceWALReader.class); |
| 107 | + |
| 108 | + WALEntryBatch emptyBatch = Mockito.mock(WALEntryBatch.class); |
| 109 | + Mockito.when(emptyBatch.getWalEntries()).thenReturn(Collections.emptyList()); |
| 110 | + |
| 111 | + Mockito.when(walReader.take()).thenReturn(emptyBatch).thenReturn(null); |
| 112 | + |
| 113 | + Mockito.when(source.isPeerEnabled()).thenReturn(true); |
| 114 | + Mockito.when(source.getReplicationEndpoint()).thenReturn(endpoint); |
| 115 | + Mockito.when(source.getPeerId()).thenReturn("1"); |
| 116 | + |
| 117 | + ReplicationSourceShipper shipper = |
| 118 | + new ReplicationSourceShipper(conf, "wal-group", source, walReader); |
| 119 | + |
| 120 | + shipper.start(); |
| 121 | + |
| 122 | + // Give the shipper thread time to process the empty batch |
| 123 | + Waiter.waitFor(conf, 3000, () -> true); |
| 124 | + |
| 125 | + shipper.interrupt(); |
| 126 | + shipper.join(); |
| 127 | + |
| 128 | + assertEquals("beforePersistingReplicationOffset should not be called for empty batch", 0, |
| 129 | + endpoint.getBeforePersistCalls()); |
| 130 | + } |
| 131 | + |
| 132 | + @Test |
| 133 | + public void testBeforePersistCalledForNonEmptyBatch() throws Exception { |
| 134 | + Configuration conf = HBaseConfiguration.create(); |
| 135 | + |
| 136 | + CountingReplicationEndpoint endpoint = new CountingReplicationEndpoint(); |
| 137 | + |
| 138 | + ReplicationSource source = Mockito.mock(ReplicationSource.class); |
| 139 | + ReplicationSourceWALReader walReader = Mockito.mock(ReplicationSourceWALReader.class); |
| 140 | + |
| 141 | + WALEntryBatch batch = Mockito.mock(WALEntryBatch.class); |
| 142 | + WAL.Entry entry = Mockito.mock(WAL.Entry.class); |
| 143 | + |
| 144 | + Mockito.when(batch.getWalEntries()).thenReturn(Collections.singletonList(entry)); |
| 145 | + Mockito.when(batch.getHeapSize()).thenReturn(10L); |
| 146 | + Mockito.when(batch.isEndOfFile()).thenReturn(false); |
| 147 | + Mockito.when(walReader.take()).thenReturn(batch).thenReturn(null); |
| 148 | + |
| 149 | + Mockito.when(source.isPeerEnabled()).thenReturn(true); |
| 150 | + Mockito.when(source.getReplicationEndpoint()).thenReturn(endpoint); |
| 151 | + Mockito.when(source.getPeerId()).thenReturn("1"); |
| 152 | + |
| 153 | + ReplicationSourceShipper shipper = |
| 154 | + new ReplicationSourceShipper(conf, "wal-group", source, walReader); |
| 155 | + |
| 156 | + shipper.start(); |
| 157 | + |
| 158 | + // Wait until beforePersistingReplicationOffset() is invoked once |
| 159 | + Waiter.waitFor(conf, 5000, () -> endpoint.getBeforePersistCalls() == 1); |
| 160 | + |
| 161 | + shipper.interrupt(); |
| 162 | + shipper.join(); |
| 163 | + |
| 164 | + assertEquals("beforePersistingReplicationOffset should be called exactly once", 1, |
| 165 | + endpoint.getBeforePersistCalls()); |
| 166 | + } |
| 167 | +} |
0 commit comments