1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.replication.regionserver;
19
20 import static org.junit.Assert.assertEquals;
21
22 import java.io.IOException;
23 import java.util.List;
24 import java.util.concurrent.atomic.AtomicBoolean;
25 import java.util.concurrent.atomic.AtomicInteger;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.fs.Path;
30 import org.apache.hadoop.hbase.HBaseTestingUtility;
31 import org.apache.hadoop.hbase.TableName;
32 import org.apache.hadoop.hbase.Waiter;
33 import org.apache.hadoop.hbase.client.HBaseAdmin;
34 import org.apache.hadoop.hbase.client.Put;
35 import org.apache.hadoop.hbase.ipc.RpcServer;
36 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
37 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.*;
38 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
39 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
40 import org.apache.hadoop.hbase.replication.TestReplicationBase;
41 import org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint;
42 import org.apache.hadoop.hbase.testclassification.MediumTests;
43 import org.apache.hadoop.hbase.wal.WAL.Entry;
44
45 import org.junit.AfterClass;
46 import org.junit.BeforeClass;
47 import org.junit.Test;
48 import org.junit.Ignore;
49 import org.junit.experimental.categories.Category;
50
51 import com.google.protobuf.RpcController;
52 import com.google.protobuf.ServiceException;
53
54 @Category(MediumTests.class)
55 @Ignore("Flaky, needs to be rewritten, see HBASE-19125")
56 public class TestReplicator extends TestReplicationBase {
57
58 static final Log LOG = LogFactory.getLog(TestReplicator.class);
59 static final int NUM_ROWS = 10;
60
61 @BeforeClass
62 public static void setUpBeforeClass() throws Exception {
63
64 conf1.setInt(RpcServer.MAX_REQUEST_SIZE, 1024 * 10);
65 TestReplicationBase.setUpBeforeClass();
66 admin.removePeer("2");
67 }
68
69 @Test
70 public void testReplicatorBatching() throws Exception {
71
72 truncateTable(utility1, tableName);
73 truncateTable(utility2, tableName);
74
75
76 admin.addPeer("testReplicatorBatching",
77 new ReplicationPeerConfig().setClusterKey(utility2.getClusterKey())
78 .setReplicationEndpointImpl(ReplicationEndpointForTest.class.getName()), null);
79
80 ReplicationEndpointForTest.setBatchCount(0);
81 ReplicationEndpointForTest.setEntriesCount(0);
82 try {
83 ReplicationEndpointForTest.pause();
84 try {
85
86
87 final byte[] valueBytes = new byte[8 *1024];
88 for (int i = 0; i < NUM_ROWS; i++) {
89 htable1.put(new Put(("row"+Integer.toString(i)).getBytes())
90 .addColumn(famName, null, valueBytes)
91 );
92 }
93 } finally {
94 ReplicationEndpointForTest.resume();
95 }
96
97
98
99 Waiter.waitFor(conf1, 60000, new Waiter.ExplainingPredicate<Exception>() {
100 @Override
101 public boolean evaluate() throws Exception {
102 LOG.info("Count=" + ReplicationEndpointForTest.getBatchCount());
103 return ReplicationEndpointForTest.getBatchCount() >= NUM_ROWS;
104 }
105
106 @Override
107 public String explainFailure() throws Exception {
108 return "We waited too long for expected replication of " + NUM_ROWS + " entries";
109 }
110 });
111
112 assertEquals("We sent an incorrect number of batches", NUM_ROWS,
113 ReplicationEndpointForTest.getBatchCount());
114 assertEquals("We did not replicate enough rows", NUM_ROWS,
115 utility2.countRows(htable2));
116 } finally {
117 admin.removePeer("testReplicatorBatching");
118 }
119 }
120
121 @Test
122 public void testReplicatorWithErrors() throws Exception {
123
124 truncateTable(utility1, tableName);
125 truncateTable(utility2, tableName);
126
127
128 admin.addPeer("testReplicatorWithErrors",
129 new ReplicationPeerConfig().setClusterKey(utility2.getClusterKey())
130 .setReplicationEndpointImpl(FailureInjectingReplicationEndpointForTest.class.getName()),
131 null);
132
133 FailureInjectingReplicationEndpointForTest.setBatchCount(0);
134 FailureInjectingReplicationEndpointForTest.setEntriesCount(0);
135 try {
136 FailureInjectingReplicationEndpointForTest.pause();
137 try {
138
139
140 final byte[] valueBytes = new byte[8 *1024];
141 for (int i = 0; i < NUM_ROWS; i++) {
142 htable1.put(new Put(("row"+Integer.toString(i)).getBytes())
143 .addColumn(famName, null, valueBytes)
144 );
145 }
146 } finally {
147 FailureInjectingReplicationEndpointForTest.resume();
148 }
149
150
151
152 Waiter.waitFor(conf1, 60000, new Waiter.ExplainingPredicate<Exception>() {
153 @Override
154 public boolean evaluate() throws Exception {
155 return FailureInjectingReplicationEndpointForTest.getEntriesCount() >= NUM_ROWS;
156 }
157
158 @Override
159 public String explainFailure() throws Exception {
160 return "We waited too long for expected replication of " + NUM_ROWS + " entries";
161 }
162 });
163
164 assertEquals("We did not replicate enough rows", NUM_ROWS,
165 utility2.countRows(htable2));
166 } finally {
167 admin.removePeer("testReplicatorWithErrors");
168 }
169 }
170
171 @AfterClass
172 public static void tearDownAfterClass() throws Exception {
173 TestReplicationBase.tearDownAfterClass();
174 }
175
176 private void truncateTable(HBaseTestingUtility util, TableName tablename) throws IOException {
177 HBaseAdmin admin = util.getHBaseAdmin();
178 admin.disableTable(tableName);
179 admin.truncateTable(tablename, false);
180 }
181
182 public static class ReplicationEndpointForTest extends HBaseInterClusterReplicationEndpoint {
183
184 private static AtomicInteger batchCount = new AtomicInteger(0);
185 private static int entriesCount;
186 private static final Object latch = new Object();
187 private static AtomicBoolean useLatch = new AtomicBoolean(false);
188
189 public static void resume() {
190 useLatch.set(false);
191 synchronized (latch) {
192 latch.notifyAll();
193 }
194 }
195
196 public static void pause() {
197 useLatch.set(true);
198 }
199
200 public static void await() throws InterruptedException {
201 if (useLatch.get()) {
202 LOG.info("Waiting on latch");
203 synchronized(latch) {
204 latch.wait();
205 }
206 LOG.info("Waited on latch, now proceeding");
207 }
208 }
209
210 public static int getBatchCount() {
211 return batchCount.get();
212 }
213
214 public static void setBatchCount(int i) {
215 LOG.info("SetBatchCount=" + i + ", old=" + getBatchCount());
216 batchCount.set(i);
217 }
218
219 public static int getEntriesCount() {
220 return entriesCount;
221 }
222
223 public static void setEntriesCount(int i) {
224 LOG.info("SetEntriesCount=" + i);
225 entriesCount = i;
226 }
227
228 public class ReplicatorForTest extends Replicator {
229
230 public ReplicatorForTest(List<Entry> entries, int ordinal) {
231 super(entries, ordinal);
232 }
233
234 @Override
235 protected void replicateEntries(BlockingInterface rrs, final List<Entry> entries,
236 String replicationClusterId, Path baseNamespaceDir, Path hfileArchiveDir)
237 throws IOException {
238 try {
239 long size = 0;
240 for (Entry e: entries) {
241 size += e.getKey().estimatedSerializedSizeOf();
242 size += e.getEdit().estimatedSerializedSizeOf();
243 }
244 LOG.info("Replicating batch " + System.identityHashCode(entries) + " of " +
245 entries.size() + " entries with total size " + size + " bytes to " +
246 replicationClusterId);
247 super.replicateEntries(rrs, entries, replicationClusterId, baseNamespaceDir,
248 hfileArchiveDir);
249 entriesCount += entries.size();
250 int count = batchCount.incrementAndGet();
251 LOG.info("Completed replicating batch " + System.identityHashCode(entries) +
252 " count=" + count);
253 } catch (IOException e) {
254 LOG.info("Failed to replicate batch " + System.identityHashCode(entries), e);
255 throw e;
256 }
257 }
258 }
259
260 @Override
261 public boolean replicate(ReplicateContext replicateContext) {
262 try {
263 await();
264 } catch (InterruptedException e) {
265 LOG.warn("Interrupted waiting for latch", e);
266 }
267 return super.replicate(replicateContext);
268 }
269
270 @Override
271 protected Replicator createReplicator(List<Entry> entries, int ordinal) {
272 return new ReplicatorForTest(entries, ordinal);
273 }
274 }
275
276 public static class FailureInjectingReplicationEndpointForTest
277 extends ReplicationEndpointForTest {
278
279 static class FailureInjectingBlockingInterface implements BlockingInterface {
280
281 private final BlockingInterface delegate;
282 private volatile boolean failNext;
283
284 public FailureInjectingBlockingInterface(BlockingInterface delegate) {
285 this.delegate = delegate;
286 }
287
288 @Override
289 public ReplicateWALEntryResponse replicateWALEntry(RpcController controller,
290 ReplicateWALEntryRequest request) throws ServiceException {
291 if (!failNext) {
292 failNext = true;
293 return delegate.replicateWALEntry(controller, request);
294 } else {
295 failNext = false;
296 throw new ServiceException("Injected failure");
297 }
298 }
299
300 @Override
301 public GetRegionInfoResponse getRegionInfo(RpcController controller,
302 GetRegionInfoRequest request) throws ServiceException {
303 return delegate.getRegionInfo(controller, request);
304 }
305
306 @Override
307 public GetStoreFileResponse getStoreFile(RpcController controller,
308 GetStoreFileRequest request) throws ServiceException {
309 return delegate.getStoreFile(controller, request);
310 }
311
312 @Override
313 public GetOnlineRegionResponse getOnlineRegion(RpcController controller,
314 GetOnlineRegionRequest request) throws ServiceException {
315 return delegate.getOnlineRegion(controller, request);
316 }
317
318 @Override
319 public OpenRegionResponse openRegion(RpcController controller, OpenRegionRequest request)
320 throws ServiceException {
321 return delegate.openRegion(controller, request);
322 }
323
324 @Override
325 public WarmupRegionResponse warmupRegion(RpcController controller,
326 WarmupRegionRequest request) throws ServiceException {
327 return delegate.warmupRegion(controller, request);
328 }
329
330 @Override
331 public CloseRegionResponse closeRegion(RpcController controller, CloseRegionRequest request)
332 throws ServiceException {
333 return delegate.closeRegion(controller, request);
334 }
335
336 @Override
337 public FlushRegionResponse flushRegion(RpcController controller, FlushRegionRequest request)
338 throws ServiceException {
339 return delegate.flushRegion(controller, request);
340 }
341
342 @Override
343 public SplitRegionResponse splitRegion(RpcController controller, SplitRegionRequest request)
344 throws ServiceException {
345 return delegate.splitRegion(controller, request);
346 }
347
348 @Override
349 public CompactionSwitchResponse compactionSwitch(RpcController controller,
350 CompactionSwitchRequest request) throws ServiceException {
351 return null;
352 }
353
354 @Override
355 public CompactRegionResponse compactRegion(RpcController controller,
356 CompactRegionRequest request) throws ServiceException {
357 return delegate.compactRegion(controller, request);
358 }
359
360 @Override
361 public MergeRegionsResponse mergeRegions(RpcController controller,
362 MergeRegionsRequest request) throws ServiceException {
363 return delegate.mergeRegions(controller, request);
364 }
365
366 @Override
367 public ReplicateWALEntryResponse replay(RpcController controller,
368 ReplicateWALEntryRequest request) throws ServiceException {
369 return delegate.replay(controller, request);
370 }
371
372 @Override
373 public RollWALWriterResponse rollWALWriter(RpcController controller,
374 RollWALWriterRequest request) throws ServiceException {
375 return delegate.rollWALWriter(controller, request);
376 }
377
378 @Override
379 public GetServerInfoResponse getServerInfo(RpcController controller,
380 GetServerInfoRequest request) throws ServiceException {
381 return delegate.getServerInfo(controller, request);
382 }
383
384 @Override
385 public StopServerResponse stopServer(RpcController controller, StopServerRequest request)
386 throws ServiceException {
387 return delegate.stopServer(controller, request);
388 }
389
390 @Override
391 public UpdateFavoredNodesResponse updateFavoredNodes(RpcController controller,
392 UpdateFavoredNodesRequest request) throws ServiceException {
393 return delegate.updateFavoredNodes(controller, request);
394 }
395
396 @Override
397 public UpdateConfigurationResponse updateConfiguration(RpcController controller,
398 UpdateConfigurationRequest request) throws ServiceException {
399 return delegate.updateConfiguration(controller, request);
400 }
401
402 @Override
403 public ClearSlowLogResponses clearSlowLogsResponses(RpcController controller,
404 ClearSlowLogResponseRequest request) throws ServiceException {
405 return delegate.clearSlowLogsResponses(controller, request);
406 }
407
408 @Override
409 public HBaseProtos.LogEntry getLogEntries(RpcController controller,
410 HBaseProtos.LogRequest request) throws ServiceException {
411 return delegate.getLogEntries(controller, request);
412 }
413 }
414
415 public class FailureInjectingReplicatorForTest extends ReplicatorForTest {
416
417 public FailureInjectingReplicatorForTest(List<Entry> entries, int ordinal) {
418 super(entries, ordinal);
419 }
420
421 @Override
422 protected void replicateEntries(BlockingInterface rrs, List<Entry> entries,
423 String replicationClusterId, Path baseNamespaceDir, Path hfileArchiveDir)
424 throws IOException {
425 super.replicateEntries(new FailureInjectingBlockingInterface(rrs), entries,
426 replicationClusterId, baseNamespaceDir, hfileArchiveDir);
427 }
428 }
429
430 @Override
431 protected Replicator createReplicator(List<Entry> entries, int ordinal) {
432 return new FailureInjectingReplicatorForTest(entries, ordinal);
433 }
434 }
435
436 }