1
2
3
4
5
6
7
8
9
10
11 package org.apache.hadoop.hbase.replication;
12
13 import static org.junit.Assert.assertEquals;
14
15 import java.io.IOException;
16 import java.util.ArrayList;
17 import java.util.Collections;
18 import java.util.HashSet;
19 import java.util.Iterator;
20 import java.util.List;
21 import java.util.Set;
22 import java.util.UUID;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.fs.FileSystem;
27 import org.apache.hadoop.fs.Path;
28 import org.apache.hadoop.hbase.HConstants;
29 import org.apache.hadoop.hbase.TableName;
30 import org.apache.hadoop.hbase.client.Table;
31 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
32 import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
33 import org.apache.hadoop.hbase.replication.regionserver.TestSourceFSConfigurationProvider;
34 import org.apache.hadoop.hbase.testclassification.LargeTests;
35 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
36 import org.apache.hadoop.hbase.util.Bytes;
37 import org.apache.hadoop.hbase.util.HFileTestUtil;
38 import org.junit.BeforeClass;
39 import org.junit.experimental.categories.Category;
40
41 @Category({ ReplicationTests.class, LargeTests.class })
42 public class TestReplicationSyncUpToolWithBulkLoadedData extends TestReplicationSyncUpTool {
43
44 private static final Log LOG = LogFactory
45 .getLog(TestReplicationSyncUpToolWithBulkLoadedData.class);
46
47 @BeforeClass
48 public static void setUpBeforeClass() throws Exception {
49 conf1.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
50 conf1.set(HConstants.REPLICATION_CLUSTER_ID, "12345");
51 conf1.set("hbase.replication.source.fs.conf.provider",
52 TestSourceFSConfigurationProvider.class.getCanonicalName());
53 String classes = conf1.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
54 if (!classes.contains("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint")) {
55 classes = classes + ",org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint";
56 conf1.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, classes);
57 }
58
59 TestReplicationBase.setUpBeforeClass();
60 }
61
62 @Override
63 public void testSyncUpTool() throws Exception {
64
65
66
67
68 setupReplication();
69
70
71
72
73 Iterator<String> randomHFileRangeListIterator = null;
74 Set<String> randomHFileRanges = new HashSet<String>(16);
75 for (int i = 0; i < 16; i++) {
76 randomHFileRanges.add(UUID.randomUUID().toString());
77 }
78 List<String> randomHFileRangeList = new ArrayList<>(randomHFileRanges);
79 Collections.sort(randomHFileRangeList);
80 randomHFileRangeListIterator = randomHFileRangeList.iterator();
81
82
83
84
85
86 loadAndReplicateHFiles(true, randomHFileRangeListIterator);
87
88
89
90
91
92
93
94
95
96
97 mimicSyncUpAfterBulkLoad(randomHFileRangeListIterator);
98
99 }
100
101 private void mimicSyncUpAfterBulkLoad(Iterator<String> randomHFileRangeListIterator)
102 throws Exception {
103 LOG.debug("mimicSyncUpAfterBulkLoad");
104 utility2.shutdownMiniHBaseCluster();
105
106 loadAndReplicateHFiles(false, randomHFileRangeListIterator);
107
108 int rowCount_ht1Source = utility1.countRows(ht1Source);
109 assertEquals("t1_syncup has 206 rows on source, after bulk load of another 103 hfiles", 206,
110 rowCount_ht1Source);
111
112 int rowCount_ht2Source = utility1.countRows(ht2Source);
113 assertEquals("t2_syncup has 406 rows on source, after bulk load of another 203 hfiles", 406,
114 rowCount_ht2Source);
115
116 utility1.shutdownMiniHBaseCluster();
117 utility2.restartHBaseCluster(1);
118
119 Thread.sleep(SLEEP_TIME);
120
121
122 int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
123 int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
124 assertEquals("@Peer1 t1_syncup should still have 100 rows", 100, rowCount_ht1TargetAtPeer1);
125 assertEquals("@Peer1 t2_syncup should still have 200 rows", 200, rowCount_ht2TargetAtPeer1);
126
127
128 syncUp(utility1);
129
130
131 for (int i = 0; i < NB_RETRIES; i++) {
132 syncUp(utility1);
133 rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
134 rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
135 if (i == NB_RETRIES - 1) {
136 if (rowCount_ht1TargetAtPeer1 != 200 || rowCount_ht2TargetAtPeer1 != 400) {
137
138 utility1.restartHBaseCluster(1);
139 rowCount_ht1Source = utility1.countRows(ht1Source);
140 LOG.debug("t1_syncup should have 206 rows at source, and it is " + rowCount_ht1Source);
141 rowCount_ht2Source = utility1.countRows(ht2Source);
142 LOG.debug("t2_syncup should have 406 rows at source, and it is " + rowCount_ht2Source);
143 }
144 assertEquals("@Peer1 t1_syncup should be sync up and have 200 rows", 200,
145 rowCount_ht1TargetAtPeer1);
146 assertEquals("@Peer1 t2_syncup should be sync up and have 400 rows", 400,
147 rowCount_ht2TargetAtPeer1);
148 }
149 if (rowCount_ht1TargetAtPeer1 == 200 && rowCount_ht2TargetAtPeer1 == 400) {
150 LOG.info("SyncUpAfterBulkLoad succeeded at retry = " + i);
151 break;
152 } else {
153 LOG.debug("SyncUpAfterBulkLoad failed at retry = " + i + ", with rowCount_ht1TargetPeer1 ="
154 + rowCount_ht1TargetAtPeer1 + " and rowCount_ht2TargetAtPeer1 ="
155 + rowCount_ht2TargetAtPeer1);
156 }
157 Thread.sleep(SLEEP_TIME);
158 }
159 }
160
161 private void loadAndReplicateHFiles(boolean verifyReplicationOnSlave,
162 Iterator<String> randomHFileRangeListIterator) throws Exception {
163 LOG.debug("loadAndReplicateHFiles");
164
165
166 byte[][][] hfileRanges =
167 new byte[][][] { new byte[][] { Bytes.toBytes(randomHFileRangeListIterator.next()),
168 Bytes.toBytes(randomHFileRangeListIterator.next()) } };
169 loadAndValidateHFileReplication("HFileReplication_1", row, famName, ht1Source, hfileRanges,
170 100);
171
172 hfileRanges =
173 new byte[][][] { new byte[][] { Bytes.toBytes(randomHFileRangeListIterator.next()),
174 Bytes.toBytes(randomHFileRangeListIterator.next()) } };
175 loadAndValidateHFileReplication("HFileReplication_1", row, noRepfamName, ht1Source,
176 hfileRanges, 3);
177
178
179 hfileRanges =
180 new byte[][][] { new byte[][] { Bytes.toBytes(randomHFileRangeListIterator.next()),
181 Bytes.toBytes(randomHFileRangeListIterator.next()) } };
182 loadAndValidateHFileReplication("HFileReplication_1", row, famName, ht2Source, hfileRanges,
183 200);
184
185 hfileRanges =
186 new byte[][][] { new byte[][] { Bytes.toBytes(randomHFileRangeListIterator.next()),
187 Bytes.toBytes(randomHFileRangeListIterator.next()) } };
188 loadAndValidateHFileReplication("HFileReplication_1", row, noRepfamName, ht2Source,
189 hfileRanges, 3);
190
191 if (verifyReplicationOnSlave) {
192
193 wait(ht1TargetAtPeer1, utility1.countRows(ht1Source) - 3,
194 "t1_syncup has 103 rows on source, and 100 on slave1");
195
196 wait(ht2TargetAtPeer1, utility1.countRows(ht2Source) - 3,
197 "t2_syncup has 203 rows on source, and 200 on slave1");
198 }
199 }
200
201 private void loadAndValidateHFileReplication(String testName, byte[] row, byte[] fam,
202 Table source, byte[][][] hfileRanges, int numOfRows) throws Exception {
203 Path dir = utility1.getDataTestDirOnTestFS(testName);
204 FileSystem fs = utility1.getTestFileSystem();
205 dir = dir.makeQualified(fs);
206 Path familyDir = new Path(dir, Bytes.toString(fam));
207
208 int hfileIdx = 0;
209 for (byte[][] range : hfileRanges) {
210 byte[] from = range[0];
211 byte[] to = range[1];
212 HFileTestUtil.createHFile(utility1.getConfiguration(), fs, new Path(familyDir, "hfile_"
213 + hfileIdx++), fam, row, from, to, numOfRows);
214 }
215
216 final TableName tableName = source.getName();
217 LoadIncrementalHFiles loader = new LoadIncrementalHFiles(utility1.getConfiguration());
218 String[] args = { dir.toString(), tableName.toString() };
219 loader.run(args);
220 }
221
222 private void wait(Table target, int expectedCount, String msg) throws IOException,
223 InterruptedException {
224 for (int i = 0; i < NB_RETRIES; i++) {
225 int rowCount_ht2TargetAtPeer1 = utility2.countRows(target);
226 if (i == NB_RETRIES - 1) {
227 assertEquals(msg, expectedCount, rowCount_ht2TargetAtPeer1);
228 }
229 if (expectedCount == rowCount_ht2TargetAtPeer1) {
230 break;
231 }
232 Thread.sleep(SLEEP_TIME);
233 }
234 }
235 }