1
2
3
4
5
6
7
8
9
10
11 package org.apache.hadoop.hbase.replication.regionserver;
12
13 import com.google.common.util.concurrent.ThreadFactoryBuilder;
14
15 import java.io.FileNotFoundException;
16 import java.io.IOException;
17 import java.io.InterruptedIOException;
18 import java.math.BigInteger;
19 import java.security.SecureRandom;
20 import java.util.ArrayList;
21 import java.util.Deque;
22 import java.util.HashMap;
23 import java.util.LinkedList;
24 import java.util.List;
25 import java.util.Map;
26 import java.util.Map.Entry;
27 import java.util.concurrent.Callable;
28 import java.util.concurrent.ExecutionException;
29 import java.util.concurrent.Future;
30 import java.util.concurrent.LinkedBlockingQueue;
31 import java.util.concurrent.ThreadPoolExecutor;
32 import java.util.concurrent.TimeUnit;
33
34 import org.apache.commons.logging.Log;
35 import org.apache.commons.logging.LogFactory;
36 import org.apache.hadoop.conf.Configuration;
37 import org.apache.hadoop.fs.FileSystem;
38 import org.apache.hadoop.fs.FileUtil;
39 import org.apache.hadoop.fs.Path;
40 import org.apache.hadoop.fs.permission.FsPermission;
41 import org.apache.hadoop.hbase.HBaseConfiguration;
42 import org.apache.hadoop.hbase.HConstants;
43 import org.apache.hadoop.hbase.TableName;
44 import org.apache.hadoop.hbase.classification.InterfaceAudience;
45 import org.apache.hadoop.hbase.client.Connection;
46 import org.apache.hadoop.hbase.client.RegionLocator;
47 import org.apache.hadoop.hbase.client.Table;
48 import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
49 import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles.LoadQueueItem;
50 import org.apache.hadoop.hbase.security.User;
51 import org.apache.hadoop.hbase.security.UserProvider;
52 import org.apache.hadoop.hbase.security.token.FsDelegationToken;
53 import org.apache.hadoop.hbase.util.Bytes;
54 import org.apache.hadoop.hbase.util.Pair;
55
56
57
58
59
60
61 @InterfaceAudience.Private
62 public class HFileReplicator {
63
64 public static final String REPLICATION_BULKLOAD_COPY_MAXTHREADS_KEY =
65 "hbase.replication.bulkload.copy.maxthreads";
66 public static final int REPLICATION_BULKLOAD_COPY_MAXTHREADS_DEFAULT = 10;
67
68 public static final String REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_KEY =
69 "hbase.replication.bulkload.copy.hfiles.perthread";
70 public static final int REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_DEFAULT = 10;
71
72 private static final Log LOG = LogFactory.getLog(HFileReplicator.class);
73 private final String UNDERSCORE = "_";
74 private final static FsPermission PERM_ALL_ACCESS = FsPermission.valueOf("-rwxrwxrwx");
75
76 private Configuration sourceClusterConf;
77 private String sourceBaseNamespaceDirPath;
78 private String sourceHFileArchiveDirPath;
79 private Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap;
80 private FileSystem sinkFs;
81 private FsDelegationToken fsDelegationToken;
82 private UserProvider userProvider;
83 private Configuration conf;
84 private Connection connection;
85 private String hbaseStagingDir;
86 private ThreadPoolExecutor exec;
87 private int maxCopyThreads;
88 private int copiesPerThread;
89 private List<String> sourceClusterIds;
90
91 public HFileReplicator(Configuration sourceClusterConf,
92 String sourceBaseNamespaceDirPath, String sourceHFileArchiveDirPath,
93 Map<String, List<Pair<byte[], List<String>>>> tableQueueMap, Configuration conf,
94 Connection connection, List<String> sourceClusterIds) throws IOException {
95 this.sourceClusterConf = sourceClusterConf;
96 this.sourceBaseNamespaceDirPath = sourceBaseNamespaceDirPath;
97 this.sourceHFileArchiveDirPath = sourceHFileArchiveDirPath;
98 this.bulkLoadHFileMap = tableQueueMap;
99 this.conf = conf;
100 this.connection = connection;
101 this.sourceClusterIds = sourceClusterIds;
102
103 userProvider = UserProvider.instantiate(conf);
104 fsDelegationToken = new FsDelegationToken(userProvider, "renewer");
105 this.hbaseStagingDir = conf.get("hbase.bulkload.staging.dir");
106 this.maxCopyThreads =
107 this.conf.getInt(REPLICATION_BULKLOAD_COPY_MAXTHREADS_KEY,
108 REPLICATION_BULKLOAD_COPY_MAXTHREADS_DEFAULT);
109 ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
110 builder.setNameFormat("HFileReplicationCallable-%1$d");
111 this.exec =
112 new ThreadPoolExecutor(1, maxCopyThreads, 60, TimeUnit.SECONDS,
113 new LinkedBlockingQueue<Runnable>(), builder.build());
114 this.exec.allowCoreThreadTimeOut(true);
115 this.copiesPerThread =
116 conf.getInt(REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_KEY,
117 REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_DEFAULT);
118
119 sinkFs = FileSystem.get(conf);
120 }
121
122 public Void replicate() throws IOException {
123
124 Map<String, Path> tableStagingDirsMap = copyHFilesToStagingDir();
125
126 int maxRetries = conf.getInt(HConstants.BULKLOAD_MAX_RETRIES_NUMBER, 10);
127
128 for (Entry<String, Path> tableStagingDir : tableStagingDirsMap.entrySet()) {
129 String tableNameString = tableStagingDir.getKey();
130 Path stagingDir = tableStagingDir.getValue();
131
132 LoadIncrementalHFiles loadHFiles = null;
133 try {
134 loadHFiles = new LoadIncrementalHFiles(conf);
135 loadHFiles.setClusterIds(sourceClusterIds);
136 } catch (Exception e) {
137 LOG.error("Failed to initialize LoadIncrementalHFiles for replicating bulk loaded"
138 + " data.", e);
139 throw new IOException(e);
140 }
141 Configuration newConf = HBaseConfiguration.create(conf);
142 newConf.set(LoadIncrementalHFiles.CREATE_TABLE_CONF_KEY, "no");
143 loadHFiles.setConf(newConf);
144
145 TableName tableName = TableName.valueOf(tableNameString);
146 Table table = this.connection.getTable(tableName);
147
148
149 Deque<LoadQueueItem> queue = new LinkedList<LoadQueueItem>();
150 loadHFiles.prepareHFileQueue(stagingDir, table, queue, false);
151
152 if (queue.isEmpty()) {
153 LOG.warn("Replication process did not find any files to replicate in directory "
154 + stagingDir.toUri());
155 return null;
156 }
157
158 try (RegionLocator locator = connection.getRegionLocator(tableName)) {
159
160 fsDelegationToken.acquireDelegationToken(sinkFs);
161
162
163
164 loadHFiles.setBulkToken(stagingDir.toString());
165
166 doBulkLoad(loadHFiles, table, queue, locator, maxRetries);
167 } finally {
168 cleanup(stagingDir.toString(), table);
169 }
170 }
171 return null;
172 }
173
174 private void doBulkLoad(LoadIncrementalHFiles loadHFiles, Table table,
175 Deque<LoadQueueItem> queue, RegionLocator locator, int maxRetries) throws IOException {
176 int count = 0;
177 Pair<byte[][], byte[][]> startEndKeys;
178 while (!queue.isEmpty()) {
179
180 startEndKeys = locator.getStartEndKeys();
181 if (count != 0) {
182 LOG.warn("Error occurred while replicating HFiles, retry attempt " + count + " with "
183 + queue.size() + " files still remaining to replicate.");
184 }
185
186 if (maxRetries != 0 && count >= maxRetries) {
187 throw new IOException("Retry attempted " + count
188 + " times without completing, bailing out.");
189 }
190 count++;
191
192
193 loadHFiles.loadHFileQueue(table, connection, queue, startEndKeys);
194 }
195 }
196
197 private void cleanup(String stagingDir, Table table) {
198
199 fsDelegationToken.releaseDelegationToken();
200
201 if (stagingDir != null) {
202 try {
203 sinkFs.delete(new Path(stagingDir), true);
204 } catch (IOException e) {
205 LOG.warn("Failed to delete the staging directory " + stagingDir, e);
206 }
207 }
208
209
210
211
212
213
214
215
216 if (table != null) {
217 try {
218 table.close();
219 } catch (IOException e) {
220 LOG.warn("Failed to close the table.", e);
221 }
222 }
223 }
224
225 private Map<String, Path> copyHFilesToStagingDir() throws IOException {
226 Map<String, Path> mapOfCopiedHFiles = new HashMap<String, Path>();
227 Pair<byte[], List<String>> familyHFilePathsPair;
228 List<String> hfilePaths;
229 byte[] family;
230 Path familyStagingDir;
231 int familyHFilePathsPairsListSize;
232 int totalNoOfHFiles;
233 List<Pair<byte[], List<String>>> familyHFilePathsPairsList;
234 FileSystem sourceFs = null;
235
236 try {
237 Path sourceClusterPath = new Path(sourceBaseNamespaceDirPath);
238
239
240
241
242
243
244 String sourceScheme = sourceClusterPath.toUri().getScheme();
245 String disableCacheName =
246 String.format("fs.%s.impl.disable.cache", new Object[] { sourceScheme });
247 sourceClusterConf.setBoolean(disableCacheName, true);
248
249 sourceFs = sourceClusterPath.getFileSystem(sourceClusterConf);
250
251 User user = userProvider.getCurrent();
252
253 for (Entry<String, List<Pair<byte[], List<String>>>> tableEntry : bulkLoadHFileMap
254 .entrySet()) {
255 String tableName = tableEntry.getKey();
256
257
258 Path stagingDir =
259 createStagingDir(new Path(hbaseStagingDir), user, TableName.valueOf(tableName));
260
261 familyHFilePathsPairsList = tableEntry.getValue();
262 familyHFilePathsPairsListSize = familyHFilePathsPairsList.size();
263
264
265 for (int i = 0; i < familyHFilePathsPairsListSize; i++) {
266 familyHFilePathsPair = familyHFilePathsPairsList.get(i);
267
268 family = familyHFilePathsPair.getFirst();
269 hfilePaths = familyHFilePathsPair.getSecond();
270
271 familyStagingDir = new Path(stagingDir, Bytes.toString(family));
272 totalNoOfHFiles = hfilePaths.size();
273
274
275 List<Future<Void>> futures = new ArrayList<Future<Void>>();
276 Callable<Void> c;
277 Future<Void> future;
278 int currentCopied = 0;
279
280 while (totalNoOfHFiles > currentCopied + this.copiesPerThread) {
281 c =
282 new Copier(sourceFs, familyStagingDir, hfilePaths.subList(currentCopied,
283 currentCopied + this.copiesPerThread));
284 future = exec.submit(c);
285 futures.add(future);
286 currentCopied += this.copiesPerThread;
287 }
288
289 int remaining = totalNoOfHFiles - currentCopied;
290 if (remaining > 0) {
291 c =
292 new Copier(sourceFs, familyStagingDir, hfilePaths.subList(currentCopied,
293 currentCopied + remaining));
294 future = exec.submit(c);
295 futures.add(future);
296 }
297
298 for (Future<Void> f : futures) {
299 try {
300 f.get();
301 } catch (InterruptedException e) {
302 InterruptedIOException iioe =
303 new InterruptedIOException(
304 "Failed to copy HFiles to local file system. This will be retried again "
305 + "by the source cluster.");
306 iioe.initCause(e);
307 throw iioe;
308 } catch (ExecutionException e) {
309 throw new IOException("Failed to copy HFiles to local file system. This will "
310 + "be retried again by the source cluster.", e);
311 }
312 }
313 }
314
315
316 mapOfCopiedHFiles.put(tableName, stagingDir);
317 }
318 return mapOfCopiedHFiles;
319 } finally {
320 if (sourceFs != null) {
321 sourceFs.close();
322 }
323 if(exec != null) {
324 exec.shutdown();
325 }
326 }
327 }
328
329 private Path createStagingDir(Path baseDir, User user, TableName tableName) throws IOException {
330 String tblName = tableName.getNameAsString().replace(":", UNDERSCORE);
331 int RANDOM_WIDTH = 320;
332 int RANDOM_RADIX = 32;
333 String doubleUnderScore = UNDERSCORE + UNDERSCORE;
334 String randomDir = user.getShortName() + doubleUnderScore + tblName + doubleUnderScore
335 + (new BigInteger(RANDOM_WIDTH, new SecureRandom()).toString(RANDOM_RADIX));
336 return createStagingDir(baseDir, user, randomDir);
337 }
338
339 private Path createStagingDir(Path baseDir, User user, String randomDir) throws IOException {
340 Path p = new Path(baseDir, randomDir);
341 sinkFs.mkdirs(p, PERM_ALL_ACCESS);
342 sinkFs.setPermission(p, PERM_ALL_ACCESS);
343 return p;
344 }
345
346
347
348
349
350 private class Copier implements Callable<Void> {
351 private FileSystem sourceFs;
352 private Path stagingDir;
353 private List<String> hfiles;
354
355 public Copier(FileSystem sourceFs, final Path stagingDir, final List<String> hfiles)
356 throws IOException {
357 this.sourceFs = sourceFs;
358 this.stagingDir = stagingDir;
359 this.hfiles = hfiles;
360 }
361
362 @Override
363 public Void call() throws IOException {
364 Path sourceHFilePath;
365 Path localHFilePath;
366 int totalHFiles = hfiles.size();
367 for (int i = 0; i < totalHFiles; i++) {
368 sourceHFilePath = new Path(sourceBaseNamespaceDirPath, hfiles.get(i));
369 localHFilePath = new Path(stagingDir, sourceHFilePath.getName());
370 try {
371 FileUtil.copy(sourceFs, sourceHFilePath, sinkFs, localHFilePath, false, conf);
372
373
374 } catch (FileNotFoundException e) {
375 LOG.info("Failed to copy hfile from " + sourceHFilePath + " to " + localHFilePath
376 + ". Trying to copy from hfile archive directory.",
377 e);
378 sourceHFilePath = new Path(sourceHFileArchiveDirPath, hfiles.get(i));
379
380 try {
381 FileUtil.copy(sourceFs, sourceHFilePath, sinkFs, localHFilePath, false, conf);
382 } catch (FileNotFoundException e1) {
383
384
385 LOG.debug("Failed to copy hfile from " + sourceHFilePath + " to " + localHFilePath
386 + ". Hence ignoring this hfile from replication..",
387 e1);
388 continue;
389 }
390 }
391 sinkFs.setPermission(localHFilePath, PERM_ALL_ACCESS);
392 }
393 return null;
394 }
395 }
396 }