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.Connection;
38  import org.apache.hadoop.hbase.client.ConnectionFactory;
39  import org.apache.hadoop.hbase.client.Scan;
40  import org.apache.hadoop.hbase.client.Table;
41  import org.apache.hadoop.hbase.util.Bytes;
42  import org.apache.hadoop.mapreduce.Job;
43  import org.apache.hadoop.util.GenericOptionsParser;
44  import org.apache.hadoop.util.Tool;
45  import org.apache.hadoop.util.ToolRunner;
46  
47  /**
48   * Tool used to copy a table to another one which can be on a different setup.
49   * It is also configurable with a start and time as well as a specification
50   * of the region server implementation if different from the local cluster.
51   */
52  @InterfaceAudience.Public
53  @InterfaceStability.Stable
54  public class CopyTable extends Configured implements Tool {
55    private static final Log LOG = LogFactory.getLog(CopyTable.class);
56  
57    final static String NAME = "copytable";
58    long startTime = 0;
59    long endTime = 0;
60    int versions = -1;
61    String tableName = null;
62    String startRow = null;
63    String stopRow = null;
64    String dstTableName = null;
65    String peerAddress = null;
66    String families = null;
67    boolean allCells = 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    public CopyTable(Configuration conf) {
75      super(conf);
76    }
77    /**
78     * Sets up the actual job.
79     *
80     * @param args  The command line parameters.
81     * @return The newly created job.
82     * @throws IOException When setting up the job fails.
83     */
84    public Job createSubmittableJob(String[] args)
85    throws IOException {
86      if (!doCommandLine(args)) {
87        return null;
88      }
89      
90      Job job = Job.getInstance(getConf(), getConf().get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));
91      job.setJarByClass(CopyTable.class);
92      Scan scan = new Scan();
93      scan.setCacheBlocks(false);
94      if (startTime != 0) {
95        scan.setTimeRange(startTime,
96            endTime == 0 ? HConstants.LATEST_TIMESTAMP : endTime);
97      }
98      if (allCells) {
99        scan.setRaw(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           Table htable = conn.getTable(TableName.valueOf(dstTableName))) {
156         HFileOutputFormat2.configureIncrementalLoadMap(job, htable);
157       }
158     } else {
159       TableMapReduceUtil.initTableMapperJob(tableName, scan,
160         Import.Importer.class, null, null, job);
161       
162       TableMapReduceUtil.initTableReducerJob(dstTableName, null, job, null, peerAddress, null,
163         null);
164     }
165     
166     return job;
167   }
168 
169   /*
170    * @param errorMsg Error message.  Can be null.
171    */
172   private static void printUsage(final String errorMsg) {
173     if (errorMsg != null && errorMsg.length() > 0) {
174       System.err.println("ERROR: " + errorMsg);
175     }
176     System.err.println("Usage: CopyTable [general options] [--starttime=X] [--endtime=Y] " +
177         "[--new.name=NEW] [--peer.adr=ADR] <tablename>");
178     System.err.println();
179     System.err.println("Options:");
180     System.err.println(" rs.class     hbase.regionserver.class of the peer cluster");
181     System.err.println("              specify if different from current cluster");
182     System.err.println(" rs.impl      hbase.regionserver.impl of the peer cluster");
183     System.err.println(" startrow     the start row");
184     System.err.println(" stoprow      the stop row");
185     System.err.println(" starttime    beginning of the time range (unixtime in millis)");
186     System.err.println("              without endtime means from starttime to forever");
187     System.err.println(" endtime      end of the time range.  Ignored if no starttime specified.");
188     System.err.println(" versions     number of cell versions to copy");
189     System.err.println(" new.name     new table's name");
190     System.err.println(" peer.adr     Address of the peer cluster given in the format");
191     System.err.println("              hbase.zookeeer.quorum:hbase.zookeeper.client.port:zookeeper.znode.parent");
192     System.err.println(" families     comma-separated list of families to copy");
193     System.err.println("              To copy from cf1 to cf2, give sourceCfName:destCfName. ");
194     System.err.println("              To keep the same name, just give \"cfName\"");
195     System.err.println(" all.cells    also copy delete markers and deleted cells");
196     System.err.println(" bulkload     Write input into HFiles and bulk load to the destination "
197         + "table");
198     System.err.println();
199     System.err.println("Args:");
200     System.err.println(" tablename    Name of the table to copy");
201     System.err.println();
202     System.err.println("Examples:");
203     System.err.println(" To copy 'TestTable' to a cluster that uses replication for a 1 hour window:");
204     System.err.println(" $ bin/hbase " +
205         "org.apache.hadoop.hbase.mapreduce.CopyTable --starttime=1265875194289 --endtime=1265878794289 " +
206         "--peer.adr=server1,server2,server3:2181:/hbase --families=myOldCf:myNewCf,cf2,cf3 TestTable ");
207     System.err.println("For performance consider the following general option:\n"
208         + "  It is recommended that you set the following to >=100. A higher value uses more memory but\n"
209         + "  decreases the round trip time to the server and may increase performance.\n"
210         + "    -Dhbase.client.scanner.caching=100\n"
211         + "  The following should always be set to false, to prevent writing data twice, which may produce \n"
212         + "  inaccurate results.\n"
213         + "    -Dmapreduce.map.speculative=false");
214   }
215 
216   private boolean doCommandLine(final String[] args) {
217     // Process command-line args. TODO: Better cmd-line processing
218     // (but hopefully something not as painful as cli options).
219     if (args.length < 1) {
220       printUsage(null);
221       return false;
222     }
223     try {
224       for (int i = 0; i < args.length; i++) {
225         String cmd = args[i];
226         if (cmd.equals("-h") || cmd.startsWith("--h")) {
227           printUsage(null);
228           return false;
229         }
230         
231         final String startRowArgKey = "--startrow=";
232         if (cmd.startsWith(startRowArgKey)) {
233           startRow = cmd.substring(startRowArgKey.length());
234           continue;
235         }
236         
237         final String stopRowArgKey = "--stoprow=";
238         if (cmd.startsWith(stopRowArgKey)) {
239           stopRow = cmd.substring(stopRowArgKey.length());
240           continue;
241         }
242         
243         final String startTimeArgKey = "--starttime=";
244         if (cmd.startsWith(startTimeArgKey)) {
245           startTime = Long.parseLong(cmd.substring(startTimeArgKey.length()));
246           continue;
247         }
248 
249         final String endTimeArgKey = "--endtime=";
250         if (cmd.startsWith(endTimeArgKey)) {
251           endTime = Long.parseLong(cmd.substring(endTimeArgKey.length()));
252           continue;
253         }
254 
255         final String versionsArgKey = "--versions=";
256         if (cmd.startsWith(versionsArgKey)) {
257           versions = Integer.parseInt(cmd.substring(versionsArgKey.length()));
258           continue;
259         }
260 
261         final String newNameArgKey = "--new.name=";
262         if (cmd.startsWith(newNameArgKey)) {
263           dstTableName = cmd.substring(newNameArgKey.length());
264           continue;
265         }
266 
267         final String peerAdrArgKey = "--peer.adr=";
268         if (cmd.startsWith(peerAdrArgKey)) {
269           peerAddress = cmd.substring(peerAdrArgKey.length());
270           continue;
271         }
272 
273         final String familiesArgKey = "--families=";
274         if (cmd.startsWith(familiesArgKey)) {
275           families = cmd.substring(familiesArgKey.length());
276           continue;
277         }
278 
279         if (cmd.startsWith("--all.cells")) {
280           allCells = true;
281           continue;
282         }
283         
284         if (cmd.startsWith("--bulkload")) {
285           bulkload = true;
286           continue;
287         }
288 
289         if (i == args.length-1) {
290           tableName = cmd;
291         } else {
292           printUsage("Invalid argument '" + cmd + "'" );
293           return false;
294         }
295       }
296       if (dstTableName == null && peerAddress == null) {
297         printUsage("At least a new table name or a " +
298             "peer address must be specified");
299         return false;
300       }
301       if ((endTime != 0) && (startTime > endTime)) {
302         printUsage("Invalid time range filter: starttime=" + startTime + " >  endtime=" + endTime);
303         return false;
304       }
305       
306       if (bulkload && peerAddress != null) {
307         printUsage("Remote bulkload is not supported!");
308         return false;
309       }
310       
311       // set dstTableName if necessary
312       if (dstTableName == null) {
313         dstTableName = tableName;
314       }
315     } catch (Exception e) {
316       e.printStackTrace();
317       printUsage("Can't start because " + e.getMessage());
318       return false;
319     }
320     return true;
321   }
322 
323   /**
324    * Main entry point.
325    *
326    * @param args  The command line parameters.
327    * @throws Exception When running the job fails.
328    */
329   public static void main(String[] args) throws Exception {
330     int ret = ToolRunner.run(new CopyTable(HBaseConfiguration.create()), args);
331     System.exit(ret);
332   }
333 
334   @Override
335   public int run(String[] args) throws Exception {
336     String[] otherArgs = new GenericOptionsParser(getConf(), args).getRemainingArgs();
337     Job job = createSubmittableJob(otherArgs);
338     if (job == null) return 1;
339     if (!job.waitForCompletion(true)) {
340       LOG.info("Map-reduce job failed!");
341       if (bulkload) {
342         LOG.info("Files are not bulkloaded!");
343       }
344       return 1;
345     }
346     int code = 0;
347     if (bulkload) {
348       code = new LoadIncrementalHFiles(this.getConf()).run(new String[]{this.bulkloadDir.toString(),
349           this.dstTableName});
350       if (code == 0) {
351         // bulkloadDir is deleted only LoadIncrementalHFiles was successful so that one can rerun
352         // LoadIncrementalHFiles.
353         FileSystem fs = FileSystem.get(this.getConf());
354         if (!fs.delete(this.bulkloadDir, true)) {
355           LOG.error("Deleting folder " + bulkloadDir + " failed!");
356           code = 1;
357         }
358       }
359     }
360     return code;
361   }
362 }