001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.replication.regionserver; 019 020import java.io.Closeable; 021import java.io.FileNotFoundException; 022import java.io.IOException; 023import java.io.InterruptedIOException; 024import java.math.BigInteger; 025import java.util.ArrayList; 026import java.util.Deque; 027import java.util.HashMap; 028import java.util.LinkedList; 029import java.util.List; 030import java.util.Map; 031import java.util.Map.Entry; 032import java.util.concurrent.Callable; 033import java.util.concurrent.ExecutionException; 034import java.util.concurrent.Future; 035import java.util.concurrent.ThreadLocalRandom; 036import java.util.concurrent.ThreadPoolExecutor; 037import java.util.concurrent.TimeUnit; 038import org.apache.hadoop.conf.Configuration; 039import org.apache.hadoop.fs.FileSystem; 040import org.apache.hadoop.fs.FileUtil; 041import org.apache.hadoop.fs.Path; 042import org.apache.hadoop.fs.permission.FsPermission; 043import org.apache.hadoop.hbase.HConstants; 044import org.apache.hadoop.hbase.TableName; 045import org.apache.hadoop.hbase.client.AsyncClusterConnection; 046import org.apache.hadoop.hbase.security.User; 047import org.apache.hadoop.hbase.security.UserProvider; 048import org.apache.hadoop.hbase.security.token.FsDelegationToken; 049import org.apache.hadoop.hbase.tool.BulkLoadHFiles; 050import org.apache.hadoop.hbase.tool.BulkLoadHFiles.LoadQueueItem; 051import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool; 052import org.apache.hadoop.hbase.util.Bytes; 053import org.apache.hadoop.hbase.util.CommonFSUtils; 054import org.apache.hadoop.hbase.util.Pair; 055import org.apache.hadoop.hbase.util.Threads; 056import org.apache.yetus.audience.InterfaceAudience; 057import org.slf4j.Logger; 058import org.slf4j.LoggerFactory; 059 060import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; 061 062/** 063 * It is used for replicating HFile entries. It will first copy parallely all the hfiles to a local 064 * staging directory and then it will use ({@link BulkLoadHFiles} to prepare a collection of 065 * {@link LoadQueueItem} which will finally be loaded(replicated) into the table of this cluster. 066 * Call {@link #close()} when done. 067 */ 068@InterfaceAudience.Private 069public class HFileReplicator implements Closeable { 070 /** Maximum number of threads to allow in pool to copy hfiles during replication */ 071 public static final String REPLICATION_BULKLOAD_COPY_MAXTHREADS_KEY = 072 "hbase.replication.bulkload.copy.maxthreads"; 073 public static final int REPLICATION_BULKLOAD_COPY_MAXTHREADS_DEFAULT = 10; 074 /** Number of hfiles to copy per thread during replication */ 075 public static final String REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_KEY = 076 "hbase.replication.bulkload.copy.hfiles.perthread"; 077 public static final int REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_DEFAULT = 10; 078 079 private static final Logger LOG = LoggerFactory.getLogger(HFileReplicator.class); 080 private static final String UNDERSCORE = "_"; 081 private final static FsPermission PERM_ALL_ACCESS = FsPermission.valueOf("-rwxrwxrwx"); 082 083 private Configuration sourceClusterConf; 084 private String sourceBaseNamespaceDirPath; 085 private String sourceHFileArchiveDirPath; 086 private Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap; 087 private FileSystem sinkFs; 088 private FsDelegationToken fsDelegationToken; 089 private UserProvider userProvider; 090 private Configuration conf; 091 private AsyncClusterConnection connection; 092 private Path hbaseStagingDir; 093 private ThreadPoolExecutor exec; 094 private int maxCopyThreads; 095 private int copiesPerThread; 096 private List<String> sourceClusterIds; 097 098 public HFileReplicator(Configuration sourceClusterConf, String sourceBaseNamespaceDirPath, 099 String sourceHFileArchiveDirPath, Map<String, List<Pair<byte[], List<String>>>> tableQueueMap, 100 Configuration conf, AsyncClusterConnection connection, List<String> sourceClusterIds) 101 throws IOException { 102 this.sourceClusterConf = sourceClusterConf; 103 this.sourceBaseNamespaceDirPath = sourceBaseNamespaceDirPath; 104 this.sourceHFileArchiveDirPath = sourceHFileArchiveDirPath; 105 this.bulkLoadHFileMap = tableQueueMap; 106 this.conf = conf; 107 this.connection = connection; 108 this.sourceClusterIds = sourceClusterIds; 109 110 userProvider = UserProvider.instantiate(conf); 111 fsDelegationToken = new FsDelegationToken(userProvider, "renewer"); 112 this.hbaseStagingDir = 113 new Path(CommonFSUtils.getRootDir(conf), HConstants.BULKLOAD_STAGING_DIR_NAME); 114 this.maxCopyThreads = this.conf.getInt(REPLICATION_BULKLOAD_COPY_MAXTHREADS_KEY, 115 REPLICATION_BULKLOAD_COPY_MAXTHREADS_DEFAULT); 116 this.exec = Threads.getBoundedCachedThreadPool(maxCopyThreads, 60, TimeUnit.SECONDS, 117 new ThreadFactoryBuilder().setDaemon(true) 118 .setNameFormat("HFileReplicationCopier-%1$d-" + this.sourceBaseNamespaceDirPath).build()); 119 this.copiesPerThread = conf.getInt(REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_KEY, 120 REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_DEFAULT); 121 122 sinkFs = FileSystem.get(conf); 123 } 124 125 @Override 126 public void close() throws IOException { 127 if (this.exec != null) { 128 this.exec.shutdown(); 129 } 130 } 131 132 public Void replicate() throws IOException { 133 // Copy all the hfiles to the local file system 134 Map<String, Path> tableStagingDirsMap = copyHFilesToStagingDir(); 135 136 int maxRetries = conf.getInt(HConstants.BULKLOAD_MAX_RETRIES_NUMBER, 10); 137 138 for (Entry<String, Path> tableStagingDir : tableStagingDirsMap.entrySet()) { 139 String tableNameString = tableStagingDir.getKey(); 140 Path stagingDir = tableStagingDir.getValue(); 141 TableName tableName = TableName.valueOf(tableNameString); 142 143 // Prepare collection of queue of hfiles to be loaded(replicated) 144 Deque<LoadQueueItem> queue = new LinkedList<>(); 145 BulkLoadHFilesTool.prepareHFileQueue(conf, connection, tableName, stagingDir, queue, false, 146 false); 147 148 if (queue.isEmpty()) { 149 LOG.warn("Did not find any files to replicate in directory {}", stagingDir.toUri()); 150 return null; 151 } 152 fsDelegationToken.acquireDelegationToken(sinkFs); 153 try { 154 doBulkLoad(conf, tableName, stagingDir, queue, maxRetries); 155 } finally { 156 cleanup(stagingDir); 157 } 158 } 159 return null; 160 } 161 162 private void doBulkLoad(Configuration conf, TableName tableName, Path stagingDir, 163 Deque<LoadQueueItem> queue, int maxRetries) throws IOException { 164 BulkLoadHFilesTool loader = new BulkLoadHFilesTool(conf); 165 // Set the staging directory which will be used by BulkLoadHFilesTool for loading the data 166 loader.setBulkToken(stagingDir.toString()); 167 // updating list of cluster ids where this bulkload event has already been processed 168 loader.setClusterIds(sourceClusterIds); 169 for (int count = 0; !queue.isEmpty(); count++) { 170 if (count != 0) { 171 LOG.warn("Error replicating HFiles; retry={} with {} remaining.", count, queue.size()); 172 } 173 174 if (maxRetries != 0 && count >= maxRetries) { 175 throw new IOException("Retry attempted " + count + " times without completing, bailing."); 176 } 177 178 // Try bulk load 179 loader.loadHFileQueue(connection, tableName, queue, false); 180 } 181 } 182 183 private void cleanup(Path stagingDir) { 184 // Release the file system delegation token 185 fsDelegationToken.releaseDelegationToken(); 186 // Delete the staging directory 187 if (stagingDir != null) { 188 try { 189 sinkFs.delete(stagingDir, true); 190 } catch (IOException e) { 191 LOG.warn("Failed to delete the staging directory " + stagingDir, e); 192 } 193 } 194 // Do not close the file system 195 } 196 197 private Map<String, Path> copyHFilesToStagingDir() throws IOException { 198 Map<String, Path> mapOfCopiedHFiles = new HashMap<>(); 199 Pair<byte[], List<String>> familyHFilePathsPair; 200 List<String> hfilePaths; 201 byte[] family; 202 Path familyStagingDir; 203 int familyHFilePathsPairsListSize; 204 int totalNoOfHFiles; 205 List<Pair<byte[], List<String>>> familyHFilePathsPairsList; 206 FileSystem sourceFs = null; 207 208 try { 209 Path sourceClusterPath = new Path(sourceBaseNamespaceDirPath); 210 /* 211 * Path#getFileSystem will by default get the FS from cache. If both source and sink cluster 212 * has same FS name service then it will return peer cluster FS. To avoid this we explicitly 213 * disable the loading of FS from cache, so that a new FS is created with source cluster 214 * configuration. 215 */ 216 String sourceScheme = sourceClusterPath.toUri().getScheme(); 217 String disableCacheName = 218 String.format("fs.%s.impl.disable.cache", new Object[] { sourceScheme }); 219 sourceClusterConf.setBoolean(disableCacheName, true); 220 221 sourceFs = sourceClusterPath.getFileSystem(sourceClusterConf); 222 223 User user = userProvider.getCurrent(); 224 // For each table name in the map 225 for (Entry<String, List<Pair<byte[], List<String>>>> tableEntry : bulkLoadHFileMap 226 .entrySet()) { 227 String tableName = tableEntry.getKey(); 228 229 // Create staging directory for each table 230 Path stagingDir = createStagingDir(hbaseStagingDir, user, TableName.valueOf(tableName)); 231 232 familyHFilePathsPairsList = tableEntry.getValue(); 233 familyHFilePathsPairsListSize = familyHFilePathsPairsList.size(); 234 235 // For each list of family hfile paths pair in the table 236 for (int i = 0; i < familyHFilePathsPairsListSize; i++) { 237 familyHFilePathsPair = familyHFilePathsPairsList.get(i); 238 239 family = familyHFilePathsPair.getFirst(); 240 hfilePaths = familyHFilePathsPair.getSecond(); 241 242 familyStagingDir = new Path(stagingDir, Bytes.toString(family)); 243 totalNoOfHFiles = hfilePaths.size(); 244 245 // For each list of hfile paths for the family 246 List<Future<Void>> futures = new ArrayList<>(); 247 Callable<Void> c; 248 Future<Void> future; 249 int currentCopied = 0; 250 // Copy the hfiles parallely 251 while (totalNoOfHFiles > currentCopied + this.copiesPerThread) { 252 c = new Copier(sourceFs, familyStagingDir, 253 hfilePaths.subList(currentCopied, currentCopied + this.copiesPerThread)); 254 future = exec.submit(c); 255 futures.add(future); 256 currentCopied += this.copiesPerThread; 257 } 258 259 int remaining = totalNoOfHFiles - currentCopied; 260 if (remaining > 0) { 261 c = new Copier(sourceFs, familyStagingDir, 262 hfilePaths.subList(currentCopied, currentCopied + remaining)); 263 future = exec.submit(c); 264 futures.add(future); 265 } 266 267 for (Future<Void> f : futures) { 268 try { 269 f.get(); 270 } catch (InterruptedException e) { 271 InterruptedIOException iioe = new InterruptedIOException( 272 "Failed to copy HFiles to local file system. This will be retried again " 273 + "by the source cluster."); 274 iioe.initCause(e); 275 throw iioe; 276 } catch (ExecutionException e) { 277 throw new IOException("Failed to copy HFiles to local file system. This will " 278 + "be retried again by the source cluster.", e); 279 } 280 } 281 } 282 // Add the staging directory to this table. Staging directory contains all the hfiles 283 // belonging to this table 284 mapOfCopiedHFiles.put(tableName, stagingDir); 285 } 286 return mapOfCopiedHFiles; 287 } finally { 288 if (sourceFs != null) { 289 sourceFs.close(); 290 } 291 if (exec != null) { 292 exec.shutdown(); 293 } 294 } 295 } 296 297 private Path createStagingDir(Path baseDir, User user, TableName tableName) throws IOException { 298 String tblName = tableName.getNameAsString().replace(":", UNDERSCORE); 299 int RANDOM_WIDTH = 320; 300 int RANDOM_RADIX = 32; 301 String doubleUnderScore = UNDERSCORE + UNDERSCORE; 302 String randomDir = user.getShortName() + doubleUnderScore + tblName + doubleUnderScore 303 + (new BigInteger(RANDOM_WIDTH, ThreadLocalRandom.current()).toString(RANDOM_RADIX)); 304 return createStagingDir(baseDir, user, randomDir); 305 } 306 307 private Path createStagingDir(Path baseDir, User user, String randomDir) throws IOException { 308 Path p = new Path(baseDir, randomDir); 309 sinkFs.mkdirs(p, PERM_ALL_ACCESS); 310 sinkFs.setPermission(p, PERM_ALL_ACCESS); 311 return p; 312 } 313 314 /** 315 * This class will copy the given hfiles from the given source file system to the given local file 316 * system staging directory. 317 */ 318 private class Copier implements Callable<Void> { 319 private FileSystem sourceFs; 320 private Path stagingDir; 321 private List<String> hfiles; 322 323 public Copier(FileSystem sourceFs, final Path stagingDir, final List<String> hfiles) 324 throws IOException { 325 this.sourceFs = sourceFs; 326 this.stagingDir = stagingDir; 327 this.hfiles = hfiles; 328 } 329 330 @Override 331 public Void call() throws IOException { 332 Path sourceHFilePath; 333 Path localHFilePath; 334 int totalHFiles = hfiles.size(); 335 for (int i = 0; i < totalHFiles; i++) { 336 sourceHFilePath = new Path(sourceBaseNamespaceDirPath, hfiles.get(i)); 337 localHFilePath = new Path(stagingDir, sourceHFilePath.getName()); 338 try { 339 FileUtil.copy(sourceFs, sourceHFilePath, sinkFs, localHFilePath, false, conf); 340 // If any other exception other than FNFE then we will fail the replication requests and 341 // source will retry to replicate these data. 342 } catch (FileNotFoundException e) { 343 LOG.info("Failed to copy hfile from " + sourceHFilePath + " to " + localHFilePath 344 + ". Trying to copy from hfile archive directory.", e); 345 sourceHFilePath = new Path(sourceHFileArchiveDirPath, hfiles.get(i)); 346 347 try { 348 FileUtil.copy(sourceFs, sourceHFilePath, sinkFs, localHFilePath, false, conf); 349 } catch (FileNotFoundException e1) { 350 // This will mean that the hfile does not exists any where in source cluster FS. So we 351 // cannot do anything here just log and continue. 352 LOG.debug("Failed to copy hfile from " + sourceHFilePath + " to " + localHFilePath 353 + ". Hence ignoring this hfile from replication..", e1); 354 continue; 355 } 356 } 357 sinkFs.setPermission(localHFilePath, PERM_ALL_ACCESS); 358 } 359 return null; 360 } 361 } 362}