1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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.UUID;
25 import java.util.concurrent.atomic.AtomicBoolean;
26 import java.util.concurrent.atomic.AtomicInteger;
27 import java.util.concurrent.atomic.AtomicReference;
28
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.hbase.Cell;
32 import org.apache.hadoop.hbase.testclassification.MediumTests;
33 import org.apache.hadoop.hbase.Waiter;
34 import org.apache.hadoop.hbase.client.Connection;
35 import org.apache.hadoop.hbase.client.ConnectionFactory;
36 import org.apache.hadoop.hbase.client.Put;
37 import org.apache.hadoop.hbase.client.Table;
38 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
39 import org.apache.hadoop.hbase.wal.WAL.Entry;
40 import org.apache.hadoop.hbase.util.Bytes;
41 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
42 import org.apache.hadoop.hbase.util.Threads;
43 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
44 import org.junit.AfterClass;
45 import org.junit.Assert;
46 import org.junit.Before;
47 import org.junit.BeforeClass;
48 import org.junit.Test;
49 import org.junit.experimental.categories.Category;
50
51
52
53
54 @Category(MediumTests.class)
55 public class TestReplicationEndpoint extends TestReplicationBase {
56 static final Log LOG = LogFactory.getLog(TestReplicationEndpoint.class);
57
58 static int numRegionServers;
59
60 @BeforeClass
61 public static void setUpBeforeClass() throws Exception {
62 TestReplicationBase.setUpBeforeClass();
63 utility2.shutdownMiniCluster();
64 admin.removePeer("2");
65 numRegionServers = utility1.getHBaseCluster().getRegionServerThreads().size();
66 }
67
68 @AfterClass
69 public static void tearDownAfterClass() throws Exception {
70 TestReplicationBase.tearDownAfterClass();
71
72 Assert.assertTrue(ReplicationEndpointForTest.stoppedCount.get() > 0);
73 }
74
75 @Before
76 public void setup() throws FailedLogCloseException, IOException {
77 ReplicationEndpointForTest.contructedCount.set(0);
78 ReplicationEndpointForTest.startedCount.set(0);
79 ReplicationEndpointForTest.replicateCount.set(0);
80 ReplicationEndpointReturningFalse.replicated.set(false);
81 ReplicationEndpointForTest.lastEntries = null;
82 for (RegionServerThread rs : utility1.getMiniHBaseCluster().getRegionServerThreads()) {
83 utility1.getHBaseAdmin().rollWALWriter(rs.getRegionServer().getServerName());
84 }
85 }
86
87 @Test (timeout=120000)
88 public void testCustomReplicationEndpoint() throws Exception {
89
90 admin.addPeer("testCustomReplicationEndpoint",
91 new ReplicationPeerConfig().setClusterKey(ZKUtil.getZooKeeperClusterKey(conf1))
92 .setReplicationEndpointImpl(ReplicationEndpointForTest.class.getName()), null);
93
94
95 Waiter.waitFor(conf1, 60000, new Waiter.Predicate<Exception>() {
96 @Override
97 public boolean evaluate() throws Exception {
98 return ReplicationEndpointForTest.contructedCount.get() >= numRegionServers;
99 }
100 });
101
102 Waiter.waitFor(conf1, 60000, new Waiter.Predicate<Exception>() {
103 @Override
104 public boolean evaluate() throws Exception {
105 return ReplicationEndpointForTest.startedCount.get() >= numRegionServers;
106 }
107 });
108
109 Assert.assertEquals(0, ReplicationEndpointForTest.replicateCount.get());
110
111
112 doPut(Bytes.toBytes("row42"));
113
114 Waiter.waitFor(conf1, 60000, new Waiter.Predicate<Exception>() {
115 @Override
116 public boolean evaluate() throws Exception {
117 return ReplicationEndpointForTest.replicateCount.get() >= 1;
118 }
119 });
120
121 doAssert(Bytes.toBytes("row42"));
122
123 admin.removePeer("testCustomReplicationEndpoint");
124 }
125
126 @Test (timeout=120000)
127 public void testReplicationEndpointReturnsFalseOnReplicate() throws Exception {
128 Assert.assertEquals(0, ReplicationEndpointForTest.replicateCount.get());
129 Assert.assertTrue(!ReplicationEndpointReturningFalse.replicated.get());
130 int peerCount = admin.getPeersCount();
131 final String id = "testReplicationEndpointReturnsFalseOnReplicate";
132 admin.addPeer(id,
133 new ReplicationPeerConfig().setClusterKey(ZKUtil.getZooKeeperClusterKey(conf1))
134 .setReplicationEndpointImpl(ReplicationEndpointReturningFalse.class.getName()), null);
135
136
137
138 if (admin.getPeersCount() <= peerCount) {
139 LOG.info("Waiting on peercount to go up from " + peerCount);
140 Threads.sleep(100);
141 }
142
143 doPut(row);
144
145 Waiter.waitFor(conf1, 60000, new Waiter.Predicate<Exception>() {
146 @Override
147 public boolean evaluate() throws Exception {
148
149
150 int count = ReplicationEndpointForTest.replicateCount.get();
151 LOG.info("count=" + count);
152 return ReplicationEndpointReturningFalse.replicated.get();
153 }
154 });
155 if (ReplicationEndpointReturningFalse.ex.get() != null) {
156 throw ReplicationEndpointReturningFalse.ex.get();
157 }
158
159 admin.removePeer("testReplicationEndpointReturnsFalseOnReplicate");
160 }
161
162 @Test (timeout=120000)
163 public void testWALEntryFilterFromReplicationEndpoint() throws Exception {
164 admin.addPeer("testWALEntryFilterFromReplicationEndpoint",
165 new ReplicationPeerConfig().setClusterKey(ZKUtil.getZooKeeperClusterKey(conf1))
166 .setReplicationEndpointImpl(ReplicationEndpointWithWALEntryFilter.class.getName()), null);
167
168 try (Connection connection = ConnectionFactory.createConnection(conf1)) {
169 doPut(connection, Bytes.toBytes("row1"));
170 doPut(connection, row);
171 doPut(connection, Bytes.toBytes("row2"));
172 }
173
174 Waiter.waitFor(conf1, 60000, new Waiter.Predicate<Exception>() {
175 @Override
176 public boolean evaluate() throws Exception {
177 return ReplicationEndpointForTest.replicateCount.get() >= 1;
178 }
179 });
180
181 Assert.assertNull(ReplicationEndpointWithWALEntryFilter.ex.get());
182 admin.removePeer("testWALEntryFilterFromReplicationEndpoint");
183 }
184
185
186 private void doPut(byte[] row) throws IOException {
187 try (Connection connection = ConnectionFactory.createConnection(conf1)) {
188 doPut(connection, row);
189 }
190 }
191
192 private void doPut(final Connection connection, final byte [] row) throws IOException {
193 try (Table t = connection.getTable(tableName)) {
194 Put put = new Put(row);
195 put.add(famName, row, row);
196 t.put(put);
197 }
198 }
199
200 private static void doAssert(byte[] row) throws Exception {
201 if (ReplicationEndpointForTest.lastEntries == null) {
202 return;
203 }
204 Assert.assertEquals(1, ReplicationEndpointForTest.lastEntries.size());
205 List<Cell> cells = ReplicationEndpointForTest.lastEntries.get(0).getEdit().getCells();
206 Assert.assertEquals(1, cells.size());
207 Assert.assertTrue(Bytes.equals(cells.get(0).getRowArray(), cells.get(0).getRowOffset(),
208 cells.get(0).getRowLength(), row, 0, row.length));
209 }
210
211 public static class ReplicationEndpointForTest extends BaseReplicationEndpoint {
212 static UUID uuid = UUID.randomUUID();
213 static AtomicInteger contructedCount = new AtomicInteger();
214 static AtomicInteger startedCount = new AtomicInteger();
215 static AtomicInteger stoppedCount = new AtomicInteger();
216 static AtomicInteger replicateCount = new AtomicInteger();
217 static volatile List<Entry> lastEntries = null;
218
219 public ReplicationEndpointForTest() {
220 contructedCount.incrementAndGet();
221 }
222
223 @Override
224 public UUID getPeerUUID() {
225 return uuid;
226 }
227
228 @Override
229 public boolean replicate(ReplicateContext replicateContext) {
230 replicateCount.incrementAndGet();
231 lastEntries = replicateContext.entries;
232 return true;
233 }
234
235 @Override
236 protected void doStart() {
237 startedCount.incrementAndGet();
238 notifyStarted();
239 }
240
241 @Override
242 protected void doStop() {
243 stoppedCount.incrementAndGet();
244 notifyStopped();
245 }
246 }
247
248 public static class ReplicationEndpointReturningFalse extends ReplicationEndpointForTest {
249 static int COUNT = 10;
250 static AtomicReference<Exception> ex = new AtomicReference<Exception>(null);
251 static AtomicBoolean replicated = new AtomicBoolean(false);
252 @Override
253 public boolean replicate(ReplicateContext replicateContext) {
254 try {
255
256 doAssert(row);
257 } catch (Exception e) {
258 ex.set(e);
259 }
260
261 super.replicate(replicateContext);
262 LOG.info("Replicated " + row + ", count=" + replicateCount.get());
263
264 replicated.set(replicateCount.get() > COUNT);
265 return replicated.get();
266 }
267 }
268
269
270 public static class ReplicationEndpointWithWALEntryFilter extends ReplicationEndpointForTest {
271 static AtomicReference<Exception> ex = new AtomicReference<Exception>(null);
272
273 @Override
274 public boolean replicate(ReplicateContext replicateContext) {
275 try {
276 super.replicate(replicateContext);
277 doAssert(row);
278 } catch (Exception e) {
279 ex.set(e);
280 }
281 return true;
282 }
283
284 @Override
285 public WALEntryFilter getWALEntryfilter() {
286 return new ChainWALEntryFilter(super.getWALEntryfilter(), new WALEntryFilter() {
287 @Override
288 public Entry filter(Entry entry) {
289 ArrayList<Cell> cells = entry.getEdit().getCells();
290 int size = cells.size();
291 for (int i = size-1; i >= 0; i--) {
292 Cell cell = cells.get(i);
293 if (!Bytes.equals(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
294 row, 0, row.length)) {
295 cells.remove(i);
296 }
297 }
298 return entry;
299 }
300 });
301 }
302 }
303 }