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.regionserver;
20
21 import static org.junit.Assert.assertEquals;
22 import static org.junit.Assert.assertTrue;
23
24 import java.util.ArrayList;
25 import java.util.List;
26 import java.util.concurrent.atomic.AtomicBoolean;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.hbase.*;
32 import org.apache.hadoop.hbase.client.Get;
33 import org.apache.hadoop.hbase.client.HTable;
34 import org.apache.hadoop.hbase.client.Result;
35 import org.apache.hadoop.hbase.client.ResultScanner;
36 import org.apache.hadoop.hbase.client.Scan;
37 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
38 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.UUID;
39 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey;
40 import org.apache.hadoop.hbase.util.Bytes;
41 import org.junit.AfterClass;
42 import org.junit.Before;
43 import org.junit.BeforeClass;
44 import org.junit.Test;
45 import org.junit.experimental.categories.Category;
46
47 import com.google.protobuf.ByteString;
48
49 @Category(MediumTests.class)
50 public class TestReplicationSink {
51 private static final Log LOG = LogFactory.getLog(TestReplicationSink.class);
52 private static final int BATCH_SIZE = 10;
53
54 private final static HBaseTestingUtility TEST_UTIL =
55 new HBaseTestingUtility();
56
57 private static ReplicationSink SINK;
58
59 private static final byte[] TABLE_NAME1 =
60 Bytes.toBytes("table1");
61 private static final byte[] TABLE_NAME2 =
62 Bytes.toBytes("table2");
63
64 private static final byte[] FAM_NAME1 = Bytes.toBytes("info1");
65 private static final byte[] FAM_NAME2 = Bytes.toBytes("info2");
66
67 private static HTable table1;
68 private static Stoppable STOPPABLE = new Stoppable() {
69 final AtomicBoolean stop = new AtomicBoolean(false);
70
71 @Override
72 public boolean isStopped() {
73 return this.stop.get();
74 }
75
76 @Override
77 public void stop(String why) {
78 LOG.info("STOPPING BECAUSE: " + why);
79 this.stop.set(true);
80 }
81
82 };
83
84 private static HTable table2;
85
86
87
88
89 @BeforeClass
90 public static void setUpBeforeClass() throws Exception {
91 TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
92 TEST_UTIL.getConfiguration().setBoolean(HConstants.REPLICATION_ENABLE_KEY, true);
93 TEST_UTIL.startMiniCluster(3);
94 SINK =
95 new ReplicationSink(new Configuration(TEST_UTIL.getConfiguration()), STOPPABLE);
96 table1 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAME1);
97 table2 = TEST_UTIL.createTable(TABLE_NAME2, FAM_NAME2);
98 }
99
100
101
102
103 @AfterClass
104 public static void tearDownAfterClass() throws Exception {
105 STOPPABLE.stop("Shutting down");
106 TEST_UTIL.shutdownMiniCluster();
107 }
108
109
110
111
112 @Before
113 public void setUp() throws Exception {
114 table1 = TEST_UTIL.truncateTable(TABLE_NAME1);
115 table2 = TEST_UTIL.truncateTable(TABLE_NAME2);
116 }
117
118
119
120
121
122 @Test
123 public void testBatchSink() throws Exception {
124 List<WALEntry> entries = new ArrayList<WALEntry>(BATCH_SIZE);
125 List<Cell> cells = new ArrayList<Cell>();
126 for(int i = 0; i < BATCH_SIZE; i++) {
127 entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
128 }
129 SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
130 Scan scan = new Scan();
131 ResultScanner scanRes = table1.getScanner(scan);
132 assertEquals(BATCH_SIZE, scanRes.next(BATCH_SIZE).length);
133 }
134
135
136
137
138
139 @Test
140 public void testMixedPutDelete() throws Exception {
141 List<WALEntry> entries = new ArrayList<WALEntry>(BATCH_SIZE/2);
142 List<Cell> cells = new ArrayList<Cell>();
143 for(int i = 0; i < BATCH_SIZE/2; i++) {
144 entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
145 }
146 SINK.replicateEntries(entries, CellUtil.createCellScanner(cells));
147
148 entries = new ArrayList<WALEntry>(BATCH_SIZE);
149 cells = new ArrayList<Cell>();
150 for(int i = 0; i < BATCH_SIZE; i++) {
151 entries.add(createEntry(TABLE_NAME1, i,
152 i % 2 != 0 ? KeyValue.Type.Put: KeyValue.Type.DeleteColumn, cells));
153 }
154
155 SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
156 Scan scan = new Scan();
157 ResultScanner scanRes = table1.getScanner(scan);
158 assertEquals(BATCH_SIZE/2, scanRes.next(BATCH_SIZE).length);
159 }
160
161
162
163
164
165 @Test
166 public void testMixedPutTables() throws Exception {
167 List<WALEntry> entries = new ArrayList<WALEntry>(BATCH_SIZE/2);
168 List<Cell> cells = new ArrayList<Cell>();
169 for(int i = 0; i < BATCH_SIZE; i++) {
170 entries.add(createEntry( i % 2 == 0 ? TABLE_NAME2 : TABLE_NAME1,
171 i, KeyValue.Type.Put, cells));
172 }
173
174 SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
175 Scan scan = new Scan();
176 ResultScanner scanRes = table2.getScanner(scan);
177 for(Result res : scanRes) {
178 assertTrue(Bytes.toInt(res.getRow()) % 2 == 0);
179 }
180 }
181
182
183
184
185
186 @Test
187 public void testMixedDeletes() throws Exception {
188 List<WALEntry> entries = new ArrayList<WALEntry>(3);
189 List<Cell> cells = new ArrayList<Cell>();
190 for(int i = 0; i < 3; i++) {
191 entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
192 }
193 SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
194 entries = new ArrayList<WALEntry>(3);
195 cells = new ArrayList<Cell>();
196 entries.add(createEntry(TABLE_NAME1, 0, KeyValue.Type.DeleteColumn, cells));
197 entries.add(createEntry(TABLE_NAME1, 1, KeyValue.Type.DeleteFamily, cells));
198 entries.add(createEntry(TABLE_NAME1, 2, KeyValue.Type.DeleteColumn, cells));
199
200 SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
201
202 Scan scan = new Scan();
203 ResultScanner scanRes = table1.getScanner(scan);
204 assertEquals(0, scanRes.next(3).length);
205 }
206
207
208
209
210
211
212 @Test
213 public void testApplyDeleteBeforePut() throws Exception {
214 List<WALEntry> entries = new ArrayList<WALEntry>(5);
215 List<Cell> cells = new ArrayList<Cell>();
216 for(int i = 0; i < 2; i++) {
217 entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
218 }
219 entries.add(createEntry(TABLE_NAME1, 1, KeyValue.Type.DeleteFamily, cells));
220 for(int i = 3; i < 5; i++) {
221 entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
222 }
223 SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
224 Get get = new Get(Bytes.toBytes(1));
225 Result res = table1.get(get);
226 assertEquals(0, res.size());
227 }
228
229 private WALEntry createEntry(byte [] table, int row, KeyValue.Type type, List<Cell> cells) {
230 byte[] fam = Bytes.equals(table, TABLE_NAME1) ? FAM_NAME1 : FAM_NAME2;
231 byte[] rowBytes = Bytes.toBytes(row);
232
233
234 try {
235 Thread.sleep(1);
236 } catch (InterruptedException e) {
237 LOG.info("Was interrupted while sleep, meh", e);
238 }
239 final long now = System.currentTimeMillis();
240 KeyValue kv = null;
241 if(type.getCode() == KeyValue.Type.Put.getCode()) {
242 kv = new KeyValue(rowBytes, fam, fam, now,
243 KeyValue.Type.Put, Bytes.toBytes(row));
244 } else if (type.getCode() == KeyValue.Type.DeleteColumn.getCode()) {
245 kv = new KeyValue(rowBytes, fam, fam,
246 now, KeyValue.Type.DeleteColumn);
247 } else if (type.getCode() == KeyValue.Type.DeleteFamily.getCode()) {
248 kv = new KeyValue(rowBytes, fam, null,
249 now, KeyValue.Type.DeleteFamily);
250 }
251 WALEntry.Builder builder = WALEntry.newBuilder();
252 builder.setAssociatedCellCount(1);
253 WALKey.Builder keyBuilder = WALKey.newBuilder();
254 UUID.Builder uuidBuilder = UUID.newBuilder();
255 uuidBuilder.setLeastSigBits(HConstants.DEFAULT_CLUSTER_ID.getLeastSignificantBits());
256 uuidBuilder.setMostSigBits(HConstants.DEFAULT_CLUSTER_ID.getMostSignificantBits());
257 keyBuilder.setClusterId(uuidBuilder.build());
258 keyBuilder.setTableName(ByteString.copyFrom(table));
259 keyBuilder.setWriteTime(now);
260 keyBuilder.setEncodedRegionName(ByteString.copyFrom(HConstants.EMPTY_BYTE_ARRAY));
261 keyBuilder.setLogSequenceNumber(-1);
262 builder.setKey(keyBuilder.build());
263 cells.add(kv);
264
265 return builder.build();
266 }
267
268 }