View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.backup.util;
20  
21  import java.io.FileNotFoundException;
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.HashMap;
25  import java.util.TreeMap;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.fs.FileStatus;
31  import org.apache.hadoop.fs.FileSystem;
32  import org.apache.hadoop.fs.FileUtil;
33  import org.apache.hadoop.fs.Path;
34  import org.apache.hadoop.hbase.HConstants;
35  import org.apache.hadoop.hbase.HTableDescriptor;
36  import org.apache.hadoop.hbase.TableName;
37  import org.apache.hadoop.hbase.backup.BackupRestoreServerFactory;
38  import org.apache.hadoop.hbase.backup.HBackupFileSystem;
39  import org.apache.hadoop.hbase.backup.IncrementalRestoreService;
40  import org.apache.hadoop.hbase.classification.InterfaceAudience;
41  import org.apache.hadoop.hbase.classification.InterfaceStability;
42  import org.apache.hadoop.hbase.client.Admin;
43  import org.apache.hadoop.hbase.client.Connection;
44  import org.apache.hadoop.hbase.client.ConnectionFactory;
45  import org.apache.hadoop.hbase.client.HBaseAdmin;
46  import org.apache.hadoop.hbase.io.HFileLink;
47  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
48  import org.apache.hadoop.hbase.io.hfile.HFile;
49  import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
50  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
51  import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
52  import org.apache.hadoop.hbase.regionserver.HStore;
53  import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
54  import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
55  import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
56  import org.apache.hadoop.hbase.util.Bytes;
57  
58  /**
59   * A collection for methods used by multiple classes to restore HBase tables.
60   */
61  @InterfaceAudience.Private
62  @InterfaceStability.Evolving
63  public class RestoreServerUtil {
64  
65    public static final Log LOG = LogFactory.getLog(RestoreServerUtil.class);
66  
67    private final String[] ignoreDirs = { "recovered.edits" };
68  
69    protected Configuration conf = null;
70  
71    protected Path backupRootPath;
72  
73    protected String backupId;
74  
75    protected FileSystem fs;
76    private final String RESTORE_TMP_PATH = "/tmp";
77    private final Path restoreTmpPath;
78  
79    // store table name and snapshot dir mapping
80    private final HashMap<TableName, Path> snapshotMap = new HashMap<>();
81  
82    public RestoreServerUtil(Configuration conf, final Path backupRootPath, final String backupId)
83        throws IOException {
84      this.conf = conf;
85      this.backupRootPath = backupRootPath;
86      this.backupId = backupId;
87      this.fs = backupRootPath.getFileSystem(conf);
88      this.restoreTmpPath = new Path(conf.get("hbase.fs.tmp.dir") != null?
89          conf.get("hbase.fs.tmp.dir"): RESTORE_TMP_PATH,
90        "restore");
91    }
92  
93    /**
94     * return value represent path for:
95     * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/archive/data/default/t1_dn"
96     * @param tabelName table name
97     * @return path to table archive
98     * @throws IOException exception
99     */
100   Path getTableArchivePath(TableName tableName)
101       throws IOException {
102     Path baseDir = new Path(HBackupFileSystem.getTableBackupPath(tableName, backupRootPath, 
103       backupId), HConstants.HFILE_ARCHIVE_DIRECTORY);
104     Path dataDir = new Path(baseDir, HConstants.BASE_NAMESPACE_DIR);
105     Path archivePath = new Path(dataDir, tableName.getNamespaceAsString());
106     Path tableArchivePath =
107         new Path(archivePath, tableName.getQualifierAsString());
108     if (!fs.exists(tableArchivePath) || !fs.getFileStatus(tableArchivePath).isDirectory()) {
109       LOG.debug("Folder tableArchivePath: " + tableArchivePath.toString() + " does not exists");
110       tableArchivePath = null; // empty table has no archive
111     }
112     return tableArchivePath;
113   }
114 
115   /**
116    * Gets region list
117    * @param tableName table name
118    * @return RegionList region list
119    * @throws FileNotFoundException exception
120    * @throws IOException exception
121    */
122   ArrayList<Path> getRegionList(TableName tableName)
123       throws FileNotFoundException, IOException {
124     Path tableArchivePath = this.getTableArchivePath(tableName);
125     ArrayList<Path> regionDirList = new ArrayList<Path>();
126     FileStatus[] children = fs.listStatus(tableArchivePath);
127     for (FileStatus childStatus : children) {
128       // here child refer to each region(Name)
129       Path child = childStatus.getPath();
130       regionDirList.add(child);
131     }
132     return regionDirList;
133   }
134 
135   /**
136    * During incremental backup operation. Call WalPlayer to replay WAL in backup image Currently
137    * tableNames and newTablesNames only contain single table, will be expanded to multiple tables in
138    * the future
139    * @param logDir : incremental backup folders, which contains WAL
140    * @param tableNames : source tableNames(table names were backuped)
141    * @param newTableNames : target tableNames(table names to be restored to)
142    * @throws IOException exception
143    */
144   public void incrementalRestoreTable(Path[] logDirs,
145       TableName[] tableNames, TableName[] newTableNames) throws IOException {
146 
147     if (tableNames.length != newTableNames.length) {
148       throw new IOException("Number of source tables and target tables does not match!");
149     }
150 
151     // for incremental backup image, expect the table already created either by user or previous
152     // full backup. Here, check that all new tables exists
153     try (Connection conn = ConnectionFactory.createConnection(conf);
154         Admin admin = conn.getAdmin()) {
155       for (TableName tableName : newTableNames) {
156         if (!admin.tableExists(tableName)) {
157           admin.close();
158           throw new IOException("HBase table " + tableName
159             + " does not exist. Create the table first, e.g. by restoring a full backup.");
160         }
161       }
162       IncrementalRestoreService restoreService =
163           BackupRestoreServerFactory.getIncrementalRestoreService(conf);
164 
165       restoreService.run(logDirs, tableNames, newTableNames);
166     }
167   }
168 
169   public void fullRestoreTable(Path tableBackupPath, TableName tableName, TableName newTableName,
170       boolean converted, boolean truncateIfExists) throws IOException {
171     restoreTableAndCreate(tableName, newTableName, tableBackupPath, converted, truncateIfExists);
172   }
173 
174   /**
175    * return value represent path for:
176    * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/.hbase-snapshot"
177    * @param backupRootPath backup root path
178    * @param tableName table name
179    * @param backupId backup Id
180    * @return path for snapshot
181    */
182   static Path getTableSnapshotPath(Path backupRootPath, TableName tableName,
183       String backupId) {
184     return new Path(HBackupFileSystem.getTableBackupPath(tableName, backupRootPath, backupId),
185       HConstants.SNAPSHOT_DIR_NAME);
186   }
187 
188   /**
189    * return value represent path for:
190    * "..../default/t1_dn/backup_1396650096738/.hbase-snapshot/snapshot_1396650097621_default_t1_dn"
191    * this path contains .snapshotinfo, .tabledesc (0.96 and 0.98) this path contains .snapshotinfo,
192    * .data.manifest (trunk)
193    * @param tableName table name
194    * @return path to table info
195    * @throws FileNotFoundException exception
196    * @throws IOException exception
197    */
198   Path getTableInfoPath(TableName tableName)
199       throws FileNotFoundException, IOException {
200     Path tableSnapShotPath = getTableSnapshotPath(backupRootPath, tableName, backupId);
201     Path tableInfoPath = null;
202 
203     // can't build the path directly as the timestamp values are different
204     FileStatus[] snapshots = fs.listStatus(tableSnapShotPath);
205     for (FileStatus snapshot : snapshots) {
206       tableInfoPath = snapshot.getPath();
207       // SnapshotManifest.DATA_MANIFEST_NAME = "data.manifest";
208       if (tableInfoPath.getName().endsWith("data.manifest")) {
209         break;
210       }
211     }
212     return tableInfoPath;
213   }
214 
215   /**
216    * @param tableName is the table backed up
217    * @return {@link HTableDescriptor} saved in backup image of the table
218    */
219   HTableDescriptor getTableDesc(TableName tableName)
220       throws FileNotFoundException, IOException {
221     Path tableInfoPath = this.getTableInfoPath(tableName);
222     SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, tableInfoPath);
223     SnapshotManifest manifest = SnapshotManifest.open(conf, fs, tableInfoPath, desc);
224     HTableDescriptor tableDescriptor = manifest.getTableDescriptor();
225     if (!tableDescriptor.getTableName().equals(tableName)) {
226       LOG.error("couldn't find Table Desc for table: " + tableName + " under tableInfoPath: "
227           + tableInfoPath.toString());
228       LOG.error("tableDescriptor.getNameAsString() = " + tableDescriptor.getNameAsString());
229       throw new FileNotFoundException("couldn't find Table Desc for table: " + tableName + 
230         " under tableInfoPath: " + tableInfoPath.toString());
231     }
232     return tableDescriptor;
233   }
234 
235   /**
236    * Duplicate the backup image if it's on local cluster
237    * @see HStore#bulkLoadHFile(String, long)
238    * @see HRegionFileSystem#bulkLoadStoreFile(String familyName, Path srcPath, long seqNum)
239    * @param tableArchivePath archive path
240    * @return the new tableArchivePath
241    * @throws IOException exception
242    */
243   Path checkLocalAndBackup(Path tableArchivePath) throws IOException {
244     // Move the file if it's on local cluster
245     boolean isCopyNeeded = false;
246 
247     FileSystem srcFs = tableArchivePath.getFileSystem(conf);
248     FileSystem desFs = FileSystem.get(conf);
249     if (tableArchivePath.getName().startsWith("/")) {
250       isCopyNeeded = true;
251     } else {
252       // This should match what is done in @see HRegionFileSystem#bulkLoadStoreFile(String, Path,
253       // long)
254       if (srcFs.getUri().equals(desFs.getUri())) {
255         LOG.debug("cluster hold the backup image: " + srcFs.getUri() + "; local cluster node: "
256             + desFs.getUri());
257         isCopyNeeded = true;
258       }
259     }
260     if (isCopyNeeded) {
261       LOG.debug("File " + tableArchivePath + " on local cluster, back it up before restore");
262       if (desFs.exists(restoreTmpPath)) {
263         try {
264           desFs.delete(restoreTmpPath, true);
265         } catch (IOException e) {
266           LOG.debug("Failed to delete path: " + restoreTmpPath
267             + ", need to check whether restore target DFS cluster is healthy");
268         }
269       }
270       FileUtil.copy(srcFs, tableArchivePath, desFs, restoreTmpPath, false, conf);
271       LOG.debug("Copied to temporary path on local cluster: " + restoreTmpPath);
272       tableArchivePath = restoreTmpPath;
273     }
274     return tableArchivePath;
275   }
276 
277   private void restoreTableAndCreate(TableName tableName, TableName newTableName,
278       Path tableBackupPath, boolean converted, boolean truncateIfExists) throws IOException {
279     if (newTableName == null || newTableName.equals("")) {
280       newTableName = tableName;
281     }
282 
283     FileSystem fileSys = tableBackupPath.getFileSystem(this.conf);
284 
285     // get table descriptor first
286     HTableDescriptor tableDescriptor = null;
287 
288     Path tableSnapshotPath = getTableSnapshotPath(backupRootPath, tableName, backupId);
289 
290     if (fileSys.exists(tableSnapshotPath)) {
291       // snapshot path exist means the backup path is in HDFS
292       // check whether snapshot dir already recorded for target table
293       if (snapshotMap.get(tableName) != null) {
294         SnapshotDescription desc =
295             SnapshotDescriptionUtils.readSnapshotInfo(fileSys, tableSnapshotPath);
296         SnapshotManifest manifest = SnapshotManifest.open(conf, fileSys, tableSnapshotPath, desc);
297         tableDescriptor = manifest.getTableDescriptor();
298       } else {
299         tableDescriptor = getTableDesc(tableName);
300         snapshotMap.put(tableName, getTableInfoPath(tableName));
301       }
302       if (tableDescriptor == null) {
303         LOG.debug("Found no table descriptor in the snapshot dir, previous schema would be lost");
304       }
305     } else if (converted) {
306       // first check if this is a converted backup image
307       LOG.error("convert will be supported in a future jira");
308     }
309 
310     Path tableArchivePath = getTableArchivePath(tableName);
311     if (tableArchivePath == null) {
312       if (tableDescriptor != null) {
313         // find table descriptor but no archive dir means the table is empty, create table and exit
314         if(LOG.isDebugEnabled()) {
315           LOG.debug("find table descriptor but no archive dir for table " + tableName
316             + ", will only create table");
317         }
318         tableDescriptor.setName(newTableName);
319         checkAndCreateTable(tableBackupPath, tableName, newTableName, null, 
320           tableDescriptor, truncateIfExists);
321         return;
322       } else {
323         throw new IllegalStateException("Cannot restore hbase table because directory '"
324             + " tableArchivePath is null.");
325       }
326     }
327 
328     if (tableDescriptor == null) {
329       tableDescriptor = new HTableDescriptor(newTableName);
330     } else {
331       tableDescriptor.setName(newTableName);
332     }
333 
334     if (!converted) {
335       // record all region dirs:
336       // load all files in dir
337       try {
338         ArrayList<Path> regionPathList = getRegionList(tableName);
339 
340         // should only try to create the table with all region informations, so we could pre-split
341         // the regions in fine grain
342         checkAndCreateTable(tableBackupPath, tableName, newTableName, regionPathList,
343           tableDescriptor, truncateIfExists);
344         if (tableArchivePath != null) {
345           // start real restore through bulkload
346           // if the backup target is on local cluster, special action needed
347           Path tempTableArchivePath = checkLocalAndBackup(tableArchivePath);
348           if (tempTableArchivePath.equals(tableArchivePath)) {
349             if(LOG.isDebugEnabled()) {
350               LOG.debug("TableArchivePath for bulkload using existPath: " + tableArchivePath);
351             }
352           } else {
353             regionPathList = getRegionList(tempTableArchivePath); // point to the tempDir
354             if(LOG.isDebugEnabled()) {
355               LOG.debug("TableArchivePath for bulkload using tempPath: " + tempTableArchivePath);
356             }
357           }
358 
359           LoadIncrementalHFiles loader = createLoader(tempTableArchivePath, false);
360           for (Path regionPath : regionPathList) {
361             String regionName = regionPath.toString();
362             if(LOG.isDebugEnabled()) {
363               LOG.debug("Restoring HFiles from directory " + regionName);
364             }
365             String[] args = { regionName, newTableName.getNameAsString()};
366             loader.run(args);
367           }
368         }
369         // we do not recovered edits
370       } catch (Exception e) {
371         throw new IllegalStateException("Cannot restore hbase table", e);
372       }
373     } else {
374       LOG.debug("convert will be supported in a future jira");
375     }
376   }
377 
378   /**
379    * Gets region list
380    * @param tableArchivePath table archive path
381    * @return RegionList region list
382    * @throws FileNotFoundException exception
383    * @throws IOException exception
384    */
385   ArrayList<Path> getRegionList(Path tableArchivePath) throws FileNotFoundException,
386   IOException {
387     ArrayList<Path> regionDirList = new ArrayList<Path>();
388     FileStatus[] children = fs.listStatus(tableArchivePath);
389     for (FileStatus childStatus : children) {
390       // here child refer to each region(Name)
391       Path child = childStatus.getPath();
392       regionDirList.add(child);
393     }
394     return regionDirList;
395   }
396 
397   /**
398    * Counts the number of files in all subdirectories of an HBase table, i.e. HFiles.
399    * @param regionPath Path to an HBase table directory
400    * @return the number of files all directories
401    * @throws IOException exception
402    */
403   int getNumberOfFilesInDir(Path regionPath) throws IOException {
404     int result = 0;
405 
406     if (!fs.exists(regionPath) || !fs.getFileStatus(regionPath).isDirectory()) {
407       throw new IllegalStateException("Cannot restore hbase table because directory '"
408           + regionPath.toString() + "' is not a directory.");
409     }
410 
411     FileStatus[] tableDirContent = fs.listStatus(regionPath);
412     for (FileStatus subDirStatus : tableDirContent) {
413       FileStatus[] colFamilies = fs.listStatus(subDirStatus.getPath());
414       for (FileStatus colFamilyStatus : colFamilies) {
415         FileStatus[] colFamilyContent = fs.listStatus(colFamilyStatus.getPath());
416         result += colFamilyContent.length;
417       }
418     }
419     return result;
420   }
421 
422   /**
423    * Counts the number of files in all subdirectories of an HBase tables, i.e. HFiles. And finds the
424    * maximum number of files in one HBase table.
425    * @param tableArchivePath archive path
426    * @return the maximum number of files found in 1 HBase table
427    * @throws IOException exception
428    */
429   int getMaxNumberOfFilesInSubDir(Path tableArchivePath) throws IOException {
430     int result = 1;
431     ArrayList<Path> regionPathList = getRegionList(tableArchivePath);
432     // tableArchivePath = this.getTableArchivePath(tableName);
433 
434     if (regionPathList == null || regionPathList.size() == 0) {
435       throw new IllegalStateException("Cannot restore hbase table because directory '"
436           + tableArchivePath + "' is not a directory.");
437     }
438 
439     for (Path regionPath : regionPathList) {
440       result = Math.max(result, getNumberOfFilesInDir(regionPath));
441     }
442     return result;
443   }
444 
445   /**
446    * Create a {@link LoadIncrementalHFiles} instance to be used to restore the HFiles of a full
447    * backup.
448    * @return the {@link LoadIncrementalHFiles} instance
449    * @throws IOException exception
450    */
451   private LoadIncrementalHFiles createLoader(Path tableArchivePath, boolean multipleTables)
452       throws IOException {
453     // set configuration for restore:
454     // LoadIncrementalHFile needs more time
455     // <name>hbase.rpc.timeout</name> <value>600000</value>
456     // calculates
457     Integer milliSecInMin = 60000;
458     Integer previousMillis = this.conf.getInt("hbase.rpc.timeout", 0);
459     Integer numberOfFilesInDir =
460         multipleTables ? getMaxNumberOfFilesInSubDir(tableArchivePath) :
461             getNumberOfFilesInDir(tableArchivePath);
462     Integer calculatedMillis = numberOfFilesInDir * milliSecInMin; // 1 minute per file
463     Integer resultMillis = Math.max(calculatedMillis, previousMillis);
464     if (resultMillis > previousMillis) {
465       LOG.info("Setting configuration for restore with LoadIncrementalHFile: "
466           + "hbase.rpc.timeout to " + calculatedMillis / milliSecInMin
467           + " minutes, to handle the number of files in backup " + tableArchivePath);
468       this.conf.setInt("hbase.rpc.timeout", resultMillis);
469     }
470 
471     // By default, it is 32 and loader will fail if # of files in any region exceed this
472     // limit. Bad for snapshot restore.
473     this.conf.setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE);
474     LoadIncrementalHFiles loader = null;
475     try {
476       loader = new LoadIncrementalHFiles(this.conf);
477     } catch (Exception e1) {
478       throw new IOException(e1);
479     }
480     return loader;
481   }
482 
483   /**
484    * Calculate region boundaries and add all the column families to the table descriptor
485    * @param regionDirList region dir list
486    * @return a set of keys to store the boundaries
487    */
488   byte[][] generateBoundaryKeys(ArrayList<Path> regionDirList)
489       throws FileNotFoundException, IOException {
490     TreeMap<byte[], Integer> map = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
491     // Build a set of keys to store the boundaries
492     byte[][] keys = null;
493     // calculate region boundaries and add all the column families to the table descriptor
494     for (Path regionDir : regionDirList) {
495       LOG.debug("Parsing region dir: " + regionDir);
496       Path hfofDir = regionDir;
497 
498       if (!fs.exists(hfofDir)) {
499         LOG.warn("HFileOutputFormat dir " + hfofDir + " not found");
500       }
501 
502       FileStatus[] familyDirStatuses = fs.listStatus(hfofDir);
503       if (familyDirStatuses == null) {
504         throw new IOException("No families found in " + hfofDir);
505       }
506 
507       for (FileStatus stat : familyDirStatuses) {
508         if (!stat.isDirectory()) {
509           LOG.warn("Skipping non-directory " + stat.getPath());
510           continue;
511         }
512         boolean isIgnore = false;
513         String pathName = stat.getPath().getName();
514         for (String ignore : ignoreDirs) {
515           if (pathName.contains(ignore)) {
516             LOG.warn("Skipping non-family directory" + pathName);
517             isIgnore = true;
518             break;
519           }
520         }
521         if (isIgnore) {
522           continue;
523         }
524         Path familyDir = stat.getPath();
525         LOG.debug("Parsing family dir [" + familyDir.toString() + " in region [" + regionDir + "]");
526         // Skip _logs, etc
527         if (familyDir.getName().startsWith("_") || familyDir.getName().startsWith(".")) {
528           continue;
529         }
530 
531         // start to parse hfile inside one family dir
532         Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
533         for (Path hfile : hfiles) {
534           if (hfile.getName().startsWith("_") || hfile.getName().startsWith(".")
535               || StoreFileInfo.isReference(hfile.getName())
536               || HFileLink.isHFileLink(hfile.getName())) {
537             continue;
538           }
539           HFile.Reader reader = HFile.createReader(fs, hfile, new CacheConfig(conf), conf);
540           final byte[] first, last;
541           try {
542             reader.loadFileInfo();
543             first = reader.getFirstRowKey();
544             last = reader.getLastRowKey();
545             LOG.debug("Trying to figure out region boundaries hfile=" + hfile + " first="
546                 + Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last));
547 
548             // To eventually infer start key-end key boundaries
549             Integer value = map.containsKey(first) ? (Integer) map.get(first) : 0;
550             map.put(first, value + 1);
551             value = map.containsKey(last) ? (Integer) map.get(last) : 0;
552             map.put(last, value - 1);
553           } finally {
554             reader.close();
555           }
556         }
557       }
558     }
559     keys = LoadIncrementalHFiles.inferBoundaries(map);
560     return keys;
561   }
562 
563   /**
564    * Prepare the table for bulkload, most codes copied from
565    * {@link LoadIncrementalHFiles#createTable(String, String)}
566    * @param tableBackupPath path
567    * @param tableName table name
568    * @param targetTableName target table name
569    * @param regionDirList region directory list
570    * @param htd table descriptor
571    * @throws IOException exception
572    */
573   private void checkAndCreateTable(Path tableBackupPath, TableName tableName,
574       TableName targetTableName, ArrayList<Path> regionDirList, 
575       HTableDescriptor htd, boolean truncateIfExists)
576           throws IOException {
577     HBaseAdmin hbadmin = null;
578     Connection conn = null;
579     try {
580       conn = ConnectionFactory.createConnection(conf);
581       hbadmin = (HBaseAdmin) conn.getAdmin();
582       boolean createNew = false;
583       if (hbadmin.tableExists(targetTableName)) {
584         if(truncateIfExists) {
585           LOG.info("Truncating exising target table '" + targetTableName +
586             "', preserving region splits");
587           hbadmin.disableTable(targetTableName);
588           hbadmin.truncateTable(targetTableName, true);
589         } else{
590           LOG.info("Using exising target table '" + targetTableName + "'");
591         }
592       } else {
593         createNew = true;
594       }      
595       if(createNew){
596         LOG.info("Creating target table '" + targetTableName + "'");
597         // if no region directory given, create the table and return
598         if (regionDirList == null || regionDirList.size() == 0) {
599           hbadmin.createTable(htd);
600           return;
601         }
602         byte[][] keys = generateBoundaryKeys(regionDirList);
603         // create table using table descriptor and region boundaries
604         hbadmin.createTable(htd, keys);
605       }
606     } catch (Exception e) {
607       throw new IOException(e);
608     } finally {
609       if (hbadmin != null) {
610         hbadmin.close();
611       }
612       if(conn != null){
613         conn.close();
614       }
615     }
616   }
617 
618 }