1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.mapreduce;
20
21 import org.apache.hadoop.hbase.classification.InterfaceAudience;
22 import org.apache.hadoop.hbase.classification.InterfaceStability;
23
24 import java.io.IOException;
25 import java.util.HashMap;
26 import java.util.Map;
27 import java.util.UUID;
28
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.conf.Configured;
33 import org.apache.hadoop.fs.FileSystem;
34 import org.apache.hadoop.fs.Path;
35 import org.apache.hadoop.hbase.HBaseConfiguration;
36 import org.apache.hadoop.hbase.HConstants;
37 import org.apache.hadoop.hbase.TableName;
38 import org.apache.hadoop.hbase.client.Connection;
39 import org.apache.hadoop.hbase.client.ConnectionFactory;
40 import org.apache.hadoop.hbase.client.Scan;
41 import org.apache.hadoop.hbase.client.Table;
42 import org.apache.hadoop.hbase.mapreduce.Import.KeyValueImporter;
43 import org.apache.hadoop.hbase.mapreduce.Import.Importer;
44 import org.apache.hadoop.hbase.util.Bytes;
45 import org.apache.hadoop.hbase.util.FSUtils;
46 import org.apache.hadoop.mapreduce.Job;
47 import org.apache.hadoop.util.GenericOptionsParser;
48 import org.apache.hadoop.util.Tool;
49 import org.apache.hadoop.util.ToolRunner;
50
51
52
53
54
55
56 @InterfaceAudience.Public
57 @InterfaceStability.Stable
58 public class CopyTable extends Configured implements Tool {
59 private static final Log LOG = LogFactory.getLog(CopyTable.class);
60
61 final static String NAME = "copytable";
62 long startTime = 0;
63 long endTime = HConstants.LATEST_TIMESTAMP;
64 int batch = Integer.MAX_VALUE;
65 int cacheRow = -1;
66 int versions = -1;
67 String tableName = null;
68 String startRow = null;
69 String stopRow = null;
70 String dstTableName = null;
71 String peerAddress = null;
72 String families = null;
73 boolean allCells = false;
74 static boolean shuffle = false;
75
76 boolean bulkload = false;
77 Path bulkloadDir = null;
78
79 boolean readingSnapshot = false;
80 String snapshot = null;
81
82 private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
83
84 public CopyTable(Configuration conf) {
85 super(conf);
86 }
87
88 private Path generateUniqTempDir(boolean withDirCreated) throws IOException {
89 FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
90 Path dir = new Path(getConf().get(HConstants.TEMPORARY_FS_DIRECTORY_KEY), NAME);
91 if (!fs.exists(dir)) {
92 fs.mkdirs(dir);
93 }
94 Path newDir = new Path(dir, UUID.randomUUID().toString());
95 if (withDirCreated) {
96 fs.mkdirs(newDir);
97 }
98 return newDir;
99 }
100
101 @SuppressWarnings({"rawtypes","unchecked"})
102 private void initCopyTableMapperReducerJob(Job job, Scan scan) throws IOException {
103 Class mapper = bulkload ? KeyValueImporter.class : Importer.class;
104 if (readingSnapshot) {
105 TableMapReduceUtil.initTableSnapshotMapperJob(snapshot, scan, mapper, null, null, job, true,
106 generateUniqTempDir(true));
107 } else {
108 TableMapReduceUtil.initTableMapperJob(tableName, scan, mapper, null, null, job);
109 }
110 }
111
112
113
114
115
116
117
118
119 public Job createSubmittableJob(String[] args) throws IOException {
120 if (!doCommandLine(args)) {
121 return null;
122 }
123
124 String jobName = NAME + "_" + (tableName == null ? snapshot : tableName);
125 Job job = Job.getInstance(getConf(), getConf().get(JOB_NAME_CONF_KEY, jobName));
126 job.setJarByClass(CopyTable.class);
127 Scan scan = new Scan();
128 scan.setBatch(batch);
129 scan.setCacheBlocks(false);
130
131 if (cacheRow > 0) {
132 scan.setCaching(cacheRow);
133 } else {
134 scan.setCaching(getConf().getInt(HConstants.HBASE_CLIENT_SCANNER_CACHING, 100));
135 }
136
137 scan.setTimeRange(startTime, endTime);
138 if (allCells) {
139 scan.setRaw(true);
140 }
141 if (shuffle) {
142 job.getConfiguration().set(TableInputFormat.SHUFFLE_MAPS, "true");
143 }
144 if (versions >= 0) {
145 scan.setMaxVersions(versions);
146 }
147
148 if (startRow != null) {
149 scan.setStartRow(Bytes.toBytesBinary(startRow));
150 }
151
152 if (stopRow != null) {
153 scan.setStopRow(Bytes.toBytesBinary(stopRow));
154 }
155
156 if(families != null) {
157 String[] fams = families.split(",");
158 Map<String,String> cfRenameMap = new HashMap<String,String>();
159 for(String fam : fams) {
160 String sourceCf;
161 if(fam.contains(":")) {
162
163 String[] srcAndDest = fam.split(":", 2);
164 sourceCf = srcAndDest[0];
165 String destCf = srcAndDest[1];
166 cfRenameMap.put(sourceCf, destCf);
167 } else {
168
169 sourceCf = fam;
170 }
171 scan.addFamily(Bytes.toBytes(sourceCf));
172 }
173 Import.configureCfRenaming(job.getConfiguration(), cfRenameMap);
174 }
175 job.setNumReduceTasks(0);
176
177 if (bulkload) {
178 initCopyTableMapperReducerJob(job, scan);
179
180
181 TableInputFormat.configureSplitTable(job, TableName.valueOf(dstTableName));
182
183 bulkloadDir = generateUniqTempDir(false);
184 LOG.info("HFiles will be stored at " + this.bulkloadDir);
185 HFileOutputFormat2.setOutputPath(job, bulkloadDir);
186 try (Connection conn = ConnectionFactory.createConnection(getConf());
187 Table htable = conn.getTable(TableName.valueOf(dstTableName))) {
188 HFileOutputFormat2.configureIncrementalLoadMap(job, htable);
189 }
190 } else {
191 initCopyTableMapperReducerJob(job, scan);
192 TableMapReduceUtil.initTableReducerJob(dstTableName, null, job, null, peerAddress, null,
193 null);
194 }
195
196 return job;
197 }
198
199
200
201
202 private static void printUsage(final String errorMsg) {
203 if (errorMsg != null && errorMsg.length() > 0) {
204 System.err.println("ERROR: " + errorMsg);
205 }
206 System.err.println("Usage: CopyTable [general options] [--starttime=X] [--endtime=Y] " +
207 "[--new.name=NEW] [--peer.adr=ADR] <tablename | snapshotName>");
208 System.err.println();
209 System.err.println("Options:");
210 System.err.println(" rs.class hbase.regionserver.class of the peer cluster");
211 System.err.println(" specify if different from current cluster");
212 System.err.println(" rs.impl hbase.regionserver.impl of the peer cluster");
213 System.err.println(" startrow the start row");
214 System.err.println(" stoprow the stop row");
215 System.err.println(" starttime beginning of the time range (unixtime in millis)");
216 System.err.println(" without endtime means from starttime to forever");
217 System.err.println(" endtime end of the time range. Ignored if no starttime specified.");
218 System.err.println(" versions number of cell versions to copy");
219 System.err.println(" new.name new table's name");
220 System.err.println(" peer.adr Address of the peer cluster given in the format");
221 System.err.println(" hbase.zookeeer.quorum:hbase.zookeeper.client.port:zookeeper.znode.parent");
222 System.err.println(" families comma-separated list of families to copy");
223 System.err.println(" To copy from cf1 to cf2, give sourceCfName:destCfName. ");
224 System.err.println(" To keep the same name, just give \"cfName\"");
225 System.err.println(" all.cells also copy delete markers and deleted cells");
226 System.err.println(" bulkload Write input into HFiles and bulk load to the destination "
227 + "table");
228 System.err.println(" snapshot Copy the data from snapshot to destination table.");
229 System.err.println();
230 System.err.println("Args:");
231 System.err.println(" tablename Name of the table to copy");
232 System.err.println();
233 System.err.println("Examples:");
234 System.err.println(" To copy 'TestTable' to a cluster that uses replication for a 1 hour window:");
235 System.err.println(" $ bin/hbase " +
236 "org.apache.hadoop.hbase.mapreduce.CopyTable --starttime=1265875194289 --endtime=1265878794289 " +
237 "--peer.adr=server1,server2,server3:2181:/hbase --families=myOldCf:myNewCf,cf2,cf3 TestTable ");
238 System.err.println(" To copy data from 'sourceTableSnapshot' to 'destTable': ");
239 System.err.println(" $ hbase org.apache.hadoop.hbase.mapreduce.CopyTable "
240 + "--snapshot --new.name=destTable sourceTableSnapshot");
241 System.err.println(" To copy data from 'sourceTableSnapshot' and bulk load to 'destTable': ");
242 System.err.println(" $ hbase org.apache.hadoop.hbase.mapreduce.CopyTable "
243 + "--new.name=destTable --snapshot --bulkload sourceTableSnapshot");
244 System.err.println("For performance consider the following general option:\n"
245 + " It is recommended that you set the following to >=100. A higher value uses more memory but\n"
246 + " decreases the round trip time to the server and may increase performance.\n"
247 + " -Dhbase.client.scanner.caching=100\n"
248 + " The following should always be set to false, to prevent writing data twice, which may produce \n"
249 + " inaccurate results.\n"
250 + " -Dmapreduce.map.speculative=false");
251 }
252
253 private boolean doCommandLine(final String[] args) {
254 if (args.length < 1) {
255 printUsage(null);
256 return false;
257 }
258 try {
259 for (int i = 0; i < args.length; i++) {
260 String cmd = args[i];
261 if (cmd.equals("-h") || cmd.startsWith("--h")) {
262 printUsage(null);
263 return false;
264 }
265
266 final String startRowArgKey = "--startrow=";
267 if (cmd.startsWith(startRowArgKey)) {
268 startRow = cmd.substring(startRowArgKey.length());
269 continue;
270 }
271
272 final String stopRowArgKey = "--stoprow=";
273 if (cmd.startsWith(stopRowArgKey)) {
274 stopRow = cmd.substring(stopRowArgKey.length());
275 continue;
276 }
277
278 final String startTimeArgKey = "--starttime=";
279 if (cmd.startsWith(startTimeArgKey)) {
280 startTime = Long.parseLong(cmd.substring(startTimeArgKey.length()));
281 continue;
282 }
283
284 final String endTimeArgKey = "--endtime=";
285 if (cmd.startsWith(endTimeArgKey)) {
286 endTime = Long.parseLong(cmd.substring(endTimeArgKey.length()));
287 continue;
288 }
289
290 final String batchArgKey = "--batch=";
291 if (cmd.startsWith(batchArgKey)) {
292 batch = Integer.parseInt(cmd.substring(batchArgKey.length()));
293 continue;
294 }
295
296 final String cacheRowArgKey = "--cacheRow=";
297 if (cmd.startsWith(cacheRowArgKey)) {
298 cacheRow = Integer.parseInt(cmd.substring(cacheRowArgKey.length()));
299 continue;
300 }
301
302 final String versionsArgKey = "--versions=";
303 if (cmd.startsWith(versionsArgKey)) {
304 versions = Integer.parseInt(cmd.substring(versionsArgKey.length()));
305 continue;
306 }
307
308 final String newNameArgKey = "--new.name=";
309 if (cmd.startsWith(newNameArgKey)) {
310 dstTableName = cmd.substring(newNameArgKey.length());
311 continue;
312 }
313
314 final String peerAdrArgKey = "--peer.adr=";
315 if (cmd.startsWith(peerAdrArgKey)) {
316 peerAddress = cmd.substring(peerAdrArgKey.length());
317 continue;
318 }
319
320 final String familiesArgKey = "--families=";
321 if (cmd.startsWith(familiesArgKey)) {
322 families = cmd.substring(familiesArgKey.length());
323 continue;
324 }
325
326 if (cmd.startsWith("--all.cells")) {
327 allCells = true;
328 continue;
329 }
330
331 if (cmd.startsWith("--bulkload")) {
332 bulkload = true;
333 continue;
334 }
335
336 if (cmd.startsWith("--shuffle")) {
337 shuffle = true;
338 continue;
339 }
340
341 if(cmd.startsWith("--snapshot")){
342 readingSnapshot = true;
343 continue;
344 }
345
346 if (i == args.length - 1) {
347 if (readingSnapshot) {
348 snapshot = cmd;
349 } else {
350 tableName = cmd;
351 }
352 } else {
353 printUsage("Invalid argument '" + cmd + "'" );
354 return false;
355 }
356 }
357 if (dstTableName == null && peerAddress == null) {
358 printUsage("At least a new table name or a peer address must be specified");
359 return false;
360 }
361 if ((endTime != 0) && (startTime > endTime)) {
362 printUsage("Invalid time range filter: starttime=" + startTime + " > endtime=" + endTime);
363 return false;
364 }
365
366 if (bulkload && peerAddress != null) {
367 printUsage("Remote bulkload is not supported!");
368 return false;
369 }
370
371 if (readingSnapshot && peerAddress != null) {
372 printUsage("Loading data from snapshot to remote peer cluster is not supported.");
373 return false;
374 }
375
376 if (readingSnapshot && dstTableName == null) {
377 printUsage("The --new.name=<table> for destination table should be "
378 + "provided when copying data from snapshot .");
379 return false;
380 }
381
382 if (readingSnapshot && snapshot == null) {
383 printUsage("Snapshot shouldn't be null when --snapshot is enabled.");
384 return false;
385 }
386
387
388 if (dstTableName == null) {
389 dstTableName = tableName;
390 }
391 } catch (Exception e) {
392 e.printStackTrace();
393 printUsage("Can't start because " + e.getMessage());
394 return false;
395 }
396 return true;
397 }
398
399
400
401
402
403
404
405 public static void main(String[] args) throws Exception {
406 int ret = ToolRunner.run(new CopyTable(HBaseConfiguration.create()), args);
407 System.exit(ret);
408 }
409
410 @Override
411 public int run(String[] args) throws Exception {
412 String[] otherArgs = new GenericOptionsParser(getConf(), args).getRemainingArgs();
413 Job job = createSubmittableJob(otherArgs);
414 if (job == null) return 1;
415 if (!job.waitForCompletion(true)) {
416 LOG.info("Map-reduce job failed!");
417 if (bulkload) {
418 LOG.info("Files are not bulkloaded!");
419 }
420 return 1;
421 }
422 int code = 0;
423 if (bulkload) {
424 LOG.info("Trying to bulk load data to destination table: " + dstTableName
425 + ", command: ./bin/hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles "
426 + bulkloadDir.toString() + " " + dstTableName);
427 code = new LoadIncrementalHFiles(this.getConf())
428 .run(new String[] { this.bulkloadDir.toString(), this.dstTableName });
429 if (code == 0) {
430
431
432 FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
433 if (!fs.delete(this.bulkloadDir, true)) {
434 LOG.error("Deleting folder " + bulkloadDir + " failed!");
435 code = 1;
436 }
437 }
438 }
439 return code;
440 }
441 }