View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.mapreduce;
20  
21  import java.io.IOException;
22  import java.util.HashMap;
23  import java.util.Map;
24  import java.util.Random;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.conf.Configured;
30  import org.apache.hadoop.fs.FileSystem;
31  import org.apache.hadoop.fs.Path;
32  import org.apache.hadoop.hbase.HBaseConfiguration;
33  import org.apache.hadoop.hbase.HConstants;
34  import org.apache.hadoop.hbase.TableName;
35  import org.apache.hadoop.hbase.classification.InterfaceAudience;
36  import org.apache.hadoop.hbase.classification.InterfaceStability;
37  import org.apache.hadoop.hbase.client.Admin;
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.util.Bytes;
42  import org.apache.hadoop.mapreduce.Job;
43  import org.apache.hadoop.util.Tool;
44  import org.apache.hadoop.util.ToolRunner;
45  
46  /**
47   * Tool used to copy a table to another one which can be on a different setup.
48   * It is also configurable with a start and time as well as a specification
49   * of the region server implementation if different from the local cluster.
50   */
51  @InterfaceAudience.Public
52  @InterfaceStability.Stable
53  public class CopyTable extends Configured implements Tool {
54    private static final Log LOG = LogFactory.getLog(CopyTable.class);
55  
56    final static String NAME = "copytable";
57    long startTime = 0;
58    long endTime = 0;
59    int versions = -1;
60    String tableName = null;
61    String startRow = null;
62    String stopRow = null;
63    String dstTableName = null;
64    String peerAddress = null;
65    String families = null;
66    boolean allCells = false;
67    static boolean shuffle = false;
68  
69    boolean bulkload = false;
70    Path bulkloadDir = null;
71  
72    private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
73  
74    /**
75     * Sets up the actual job.
76     *
77     * @param args  The command line parameters.
78     * @return The newly created job.
79     * @throws IOException When setting up the job fails.
80     */
81    public Job createSubmittableJob(String[] args)
82    throws IOException {
83      if (!doCommandLine(args)) {
84        return null;
85      }
86  
87      Job job = Job.getInstance(getConf(), getConf().get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));
88      job.setJarByClass(CopyTable.class);
89      Scan scan = new Scan();
90      scan.setCacheBlocks(false);
91      if (startTime != 0) {
92        scan.setTimeRange(startTime,
93            endTime == 0 ? HConstants.LATEST_TIMESTAMP : endTime);
94      }
95      if (allCells) {
96        scan.setRaw(true);
97      }
98      if (shuffle) {
99        job.getConfiguration().set(TableInputFormat.SHUFFLE_MAPS, "true");
100     }
101     if (versions >= 0) {
102       scan.setMaxVersions(versions);
103     }
104 
105     if (startRow != null) {
106       scan.setStartRow(Bytes.toBytes(startRow));
107     }
108 
109     if (stopRow != null) {
110       scan.setStopRow(Bytes.toBytes(stopRow));
111     }
112 
113     if(families != null) {
114       String[] fams = families.split(",");
115       Map<String,String> cfRenameMap = new HashMap<String,String>();
116       for(String fam : fams) {
117         String sourceCf;
118         if(fam.contains(":")) {
119             // fam looks like "sourceCfName:destCfName"
120             String[] srcAndDest = fam.split(":", 2);
121             sourceCf = srcAndDest[0];
122             String destCf = srcAndDest[1];
123             cfRenameMap.put(sourceCf, destCf);
124         } else {
125             // fam is just "sourceCf"
126             sourceCf = fam;
127         }
128         scan.addFamily(Bytes.toBytes(sourceCf));
129       }
130       Import.configureCfRenaming(job.getConfiguration(), cfRenameMap);
131     }
132     job.setNumReduceTasks(0);
133 
134     if (bulkload) {
135       TableMapReduceUtil.initTableMapperJob(tableName, scan, Import.KeyValueImporter.class, null,
136         null, job);
137 
138       // We need to split the inputs by destination tables so that output of Map can be bulk-loaded.
139       TableInputFormat.configureSplitTable(job, TableName.valueOf(dstTableName));
140 
141       FileSystem fs = FileSystem.get(getConf());
142       Random rand = new Random();
143       Path root = new Path(fs.getWorkingDirectory(), "copytable");
144       fs.mkdirs(root);
145       while (true) {
146         bulkloadDir = new Path(root, "" + rand.nextLong());
147         if (!fs.exists(bulkloadDir)) {
148           break;
149         }
150       }
151 
152       System.out.println("HFiles will be stored at " + this.bulkloadDir);
153       HFileOutputFormat2.setOutputPath(job, bulkloadDir);
154       try (Connection conn = ConnectionFactory.createConnection(getConf());
155           Admin admin = conn.getAdmin()) {
156         HFileOutputFormat2.configureIncrementalLoadMap(job,
157             admin.getTableDescriptor((TableName.valueOf(dstTableName))));
158       }
159     } else {
160       TableMapReduceUtil.initTableMapperJob(tableName, scan,
161         Import.Importer.class, null, null, job);
162 
163       TableMapReduceUtil.initTableReducerJob(dstTableName, null, job, null, peerAddress, null,
164         null);
165     }
166 
167     return job;
168   }
169 
170   /*
171    * @param errorMsg Error message.  Can be null.
172    */
173   private static void printUsage(final String errorMsg) {
174     if (errorMsg != null && errorMsg.length() > 0) {
175       System.err.println("ERROR: " + errorMsg);
176     }
177     System.err.println("Usage: CopyTable [general options] [--starttime=X] [--endtime=Y] " +
178         "[--new.name=NEW] [--peer.adr=ADR] <tablename>");
179     System.err.println();
180     System.err.println("Options:");
181     System.err.println(" rs.class     hbase.regionserver.class of the peer cluster");
182     System.err.println("              specify if different from current cluster");
183     System.err.println(" rs.impl      hbase.regionserver.impl of the peer cluster");
184     System.err.println(" startrow     the start row");
185     System.err.println(" stoprow      the stop row");
186     System.err.println(" starttime    beginning of the time range (unixtime in millis)");
187     System.err.println("              without endtime means from starttime to forever");
188     System.err.println(" endtime      end of the time range.  Ignored if no starttime specified.");
189     System.err.println(" versions     number of cell versions to copy");
190     System.err.println(" new.name     new table's name");
191     System.err.println(" peer.adr     Address of the peer cluster given in the format");
192     System.err.println("              hbase.zookeeper.quorum:hbase.zookeeper.client"
193         + ".port:zookeeper.znode.parent");
194     System.err.println(" families     comma-separated list of families to copy");
195     System.err.println("              To copy from cf1 to cf2, give sourceCfName:destCfName. ");
196     System.err.println("              To keep the same name, just give \"cfName\"");
197     System.err.println(" all.cells    also copy delete markers and deleted cells");
198     System.err.println(" bulkload     Write input into HFiles and bulk load to the destination "
199         + "table");
200     System.err.println();
201     System.err.println("Args:");
202     System.err.println(" tablename    Name of the table to copy");
203     System.err.println();
204     System.err.println("Examples:");
205     System.err.println(" To copy 'TestTable' to a cluster that uses replication for a 1 hour window:");
206     System.err.println(" $ bin/hbase " +
207         "org.apache.hadoop.hbase.mapreduce.CopyTable --starttime=1265875194289 --endtime=1265878794289 " +
208         "--peer.adr=server1,server2,server3:2181:/hbase --families=myOldCf:myNewCf,cf2,cf3 TestTable ");
209     System.err.println("For performance consider the following general option:\n"
210         + "  It is recommended that you set the following to >=100. A higher value uses more memory but\n"
211         + "  decreases the round trip time to the server and may increase performance.\n"
212         + "    -Dhbase.client.scanner.caching=100\n"
213         + "  The following should always be set to false, to prevent writing data twice, which may produce \n"
214         + "  inaccurate results.\n"
215         + "    -Dmapreduce.map.speculative=false");
216   }
217 
218   private boolean doCommandLine(final String[] args) {
219     // Process command-line args. TODO: Better cmd-line processing
220     // (but hopefully something not as painful as cli options).
221     if (args.length < 1) {
222       printUsage(null);
223       return false;
224     }
225     try {
226       for (int i = 0; i < args.length; i++) {
227         String cmd = args[i];
228         if (cmd.equals("-h") || cmd.startsWith("--h")) {
229           printUsage(null);
230           return false;
231         }
232 
233         final String startRowArgKey = "--startrow=";
234         if (cmd.startsWith(startRowArgKey)) {
235           startRow = cmd.substring(startRowArgKey.length());
236           continue;
237         }
238 
239         final String stopRowArgKey = "--stoprow=";
240         if (cmd.startsWith(stopRowArgKey)) {
241           stopRow = cmd.substring(stopRowArgKey.length());
242           continue;
243         }
244 
245         final String startTimeArgKey = "--starttime=";
246         if (cmd.startsWith(startTimeArgKey)) {
247           startTime = Long.parseLong(cmd.substring(startTimeArgKey.length()));
248           continue;
249         }
250 
251         final String endTimeArgKey = "--endtime=";
252         if (cmd.startsWith(endTimeArgKey)) {
253           endTime = Long.parseLong(cmd.substring(endTimeArgKey.length()));
254           continue;
255         }
256 
257         final String versionsArgKey = "--versions=";
258         if (cmd.startsWith(versionsArgKey)) {
259           versions = Integer.parseInt(cmd.substring(versionsArgKey.length()));
260           continue;
261         }
262 
263         final String newNameArgKey = "--new.name=";
264         if (cmd.startsWith(newNameArgKey)) {
265           dstTableName = cmd.substring(newNameArgKey.length());
266           continue;
267         }
268 
269         final String peerAdrArgKey = "--peer.adr=";
270         if (cmd.startsWith(peerAdrArgKey)) {
271           peerAddress = cmd.substring(peerAdrArgKey.length());
272           continue;
273         }
274 
275         final String familiesArgKey = "--families=";
276         if (cmd.startsWith(familiesArgKey)) {
277           families = cmd.substring(familiesArgKey.length());
278           continue;
279         }
280 
281         if (cmd.startsWith("--all.cells")) {
282           allCells = true;
283           continue;
284         }
285 
286         if (cmd.startsWith("--bulkload")) {
287           bulkload = true;
288           continue;
289         }
290 
291         if (cmd.startsWith("--shuffle")) {
292           shuffle = true;
293           continue;
294         }
295 
296         if (i == args.length-1) {
297           tableName = cmd;
298         } else {
299           printUsage("Invalid argument '" + cmd + "'");
300           return false;
301         }
302       }
303       if (dstTableName == null && peerAddress == null) {
304         printUsage("At least a new table name or a " +
305             "peer address must be specified");
306         return false;
307       }
308       if ((endTime != 0) && (startTime > endTime)) {
309         printUsage("Invalid time range filter: starttime=" + startTime + " >  endtime=" + endTime);
310         return false;
311       }
312 
313       if (bulkload && peerAddress != null) {
314         printUsage("Remote bulkload is not supported!");
315         return false;
316       }
317 
318       // set dstTableName if necessary
319       if (dstTableName == null) {
320         dstTableName = tableName;
321       }
322     } catch (Exception e) {
323       e.printStackTrace();
324       printUsage("Can't start because " + e.getMessage());
325       return false;
326     }
327     return true;
328   }
329 
330   /**
331    * Main entry point.
332    *
333    * @param args  The command line parameters.
334    * @throws Exception When running the job fails.
335    */
336   public static void main(String[] args) throws Exception {
337     int ret = ToolRunner.run(HBaseConfiguration.create(), new CopyTable(), args);
338     System.exit(ret);
339   }
340 
341   @Override
342   public int run(String[] args) throws Exception {
343     Job job = createSubmittableJob(args);
344     if (job == null) return 1;
345     if (!job.waitForCompletion(true)) {
346       LOG.info("Map-reduce job failed!");
347       if (bulkload) {
348         LOG.info("Files are not bulkloaded!");
349       }
350       return 1;
351     }
352     int code = 0;
353     if (bulkload) {
354       code = new LoadIncrementalHFiles(this.getConf()).run(new String[]{this.bulkloadDir.toString(),
355           this.dstTableName});
356       if (code == 0) {
357         // bulkloadDir is deleted only LoadIncrementalHFiles was successful so that one can rerun
358         // LoadIncrementalHFiles.
359         FileSystem fs = FileSystem.get(this.getConf());
360         if (!fs.delete(this.bulkloadDir, true)) {
361           LOG.error("Deleting folder " + bulkloadDir + " failed!");
362           code = 1;
363         }
364       }
365     }
366     return code;
367   }
368 }