View Javadoc

1   /**
2    * Copyright 2009 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.mapreduce;
21  
22  import java.io.IOException;
23  import java.util.Map;
24  import java.util.TreeMap;
25  
26  import org.apache.hadoop.conf.Configuration;
27  import org.apache.hadoop.fs.Path;
28  import org.apache.hadoop.hbase.HBaseConfiguration;
29  import org.apache.hadoop.hbase.KeyValue;
30  import org.apache.hadoop.hbase.client.Put;
31  import org.apache.hadoop.hbase.client.Result;
32  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
33  import org.apache.hadoop.hbase.util.Bytes;
34  import org.apache.hadoop.mapreduce.Job;
35  import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
36  import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
37  import org.apache.hadoop.util.GenericOptionsParser;
38  
39  /**
40   * Import data written by {@link Export}.
41   */
42  public class Import {
43    final static String NAME = "import";
44    public static final String CF_RENAME_PROP = "HBASE_IMPORTER_RENAME_CFS";
45  
46    /**
47     * Write table content out to files in hdfs.
48     */
49    static class Importer
50    extends TableMapper<ImmutableBytesWritable, Put> {
51      private Map<byte[], byte[]> cfRenameMap;
52        
53      /**
54       * @param row  The current table row key.
55       * @param value  The columns.
56       * @param context  The current context.
57       * @throws IOException When something is broken with the data.
58       * @see org.apache.hadoop.mapreduce.Mapper#map(KEYIN, VALUEIN,
59       *   org.apache.hadoop.mapreduce.Mapper.Context)
60       */
61      @Override
62      public void map(ImmutableBytesWritable row, Result value,
63        Context context)
64      throws IOException {
65        try {
66          context.write(row, resultToPut(row, value));
67        } catch (InterruptedException e) {
68          e.printStackTrace();
69        }
70      }
71  
72      private Put resultToPut(ImmutableBytesWritable key, Result result)
73      throws IOException {
74        Put put = new Put(key.get());
75        for (KeyValue kv : result.raw()) {
76          if(cfRenameMap != null) {
77              // If there's a rename mapping for this CF, create a new KeyValue
78              byte[] newCfName = cfRenameMap.get(kv.getFamily());
79              if(newCfName != null) {
80                  kv = new KeyValue(kv.getBuffer(), // row buffer 
81                          kv.getRowOffset(),        // row offset
82                          kv.getRowLength(),        // row length
83                          newCfName,                // CF buffer
84                          0,                        // CF offset 
85                          newCfName.length,         // CF length 
86                          kv.getBuffer(),           // qualifier buffer
87                          kv.getQualifierOffset(),  // qualifier offset
88                          kv.getQualifierLength(),  // qualifier length
89                          kv.getTimestamp(),        // timestamp
90                          KeyValue.Type.codeToType(kv.getType()), // KV Type
91                          kv.getBuffer(),           // value buffer 
92                          kv.getValueOffset(),      // value offset
93                          kv.getValueLength());     // value length
94              } 
95          }
96          put.add(kv);
97        }
98        return put;
99      }
100     
101     @Override
102     public void setup(Context context) {
103       // Make a map from sourceCfName to destCfName by parsing a config key
104       cfRenameMap = null;
105       String allMappingsPropVal = context.getConfiguration().get(CF_RENAME_PROP);
106       if(allMappingsPropVal != null) {
107         // The conf value format should be sourceCf1:destCf1,sourceCf2:destCf2,...
108         String[] allMappings = allMappingsPropVal.split(",");
109         for (String mapping: allMappings) {
110           if(cfRenameMap == null) {
111               cfRenameMap = new TreeMap<byte[],byte[]>(Bytes.BYTES_COMPARATOR);
112           }
113           String [] srcAndDest = mapping.split(":");
114           if(srcAndDest.length != 2) {
115               continue;
116           }
117           cfRenameMap.put(srcAndDest[0].getBytes(), srcAndDest[1].getBytes());
118         }
119       }
120     }
121   }
122   
123   /**
124    * <p>Sets a configuration property with key {@link #CF_RENAME_PROP} in conf that tells
125    * the mapper how to rename column families.
126    * 
127    * <p>Alternately, instead of calling this function, you could set the configuration key 
128    * {@link #CF_RENAME_PROP} yourself. The value should look like 
129    * <pre>srcCf1:destCf1,srcCf2:destCf2,....</pre>. This would have the same effect on
130    * the mapper behavior.
131    * 
132    * @param conf the Configuration in which the {@link #CF_RENAME_PROP} key will be
133    *  set
134    * @param renameMap a mapping from source CF names to destination CF names
135    */
136   static public void configureCfRenaming(Configuration conf, 
137           Map<String, String> renameMap) {
138     StringBuilder sb = new StringBuilder();
139     for(Map.Entry<String,String> entry: renameMap.entrySet()) {
140       String sourceCf = entry.getKey();
141       String destCf = entry.getValue();
142 
143       if(sourceCf.contains(":") || sourceCf.contains(",") || 
144               destCf.contains(":") || destCf.contains(",")) {
145         throw new IllegalArgumentException("Illegal character in CF names: " 
146               + sourceCf + ", " + destCf);
147       }
148 
149       if(sb.length() != 0) {
150         sb.append(",");
151       }
152       sb.append(sourceCf + ":" + destCf);
153     }
154     conf.set(CF_RENAME_PROP, sb.toString());
155   }
156   
157 
158   /**
159    * Sets up the actual job.
160    *
161    * @param conf  The current configuration.
162    * @param args  The command line parameters.
163    * @return The newly created job.
164    * @throws IOException When setting up the job fails.
165    */
166   public static Job createSubmittableJob(Configuration conf, String[] args)
167   throws IOException {
168     String tableName = args[0];
169     Path inputDir = new Path(args[1]);
170     Job job = new Job(conf, NAME + "_" + tableName);
171     job.setJarByClass(Importer.class);
172     FileInputFormat.setInputPaths(job, inputDir);
173     job.setInputFormatClass(SequenceFileInputFormat.class);
174     job.setMapperClass(Importer.class);
175     // No reducers.  Just write straight to table.  Call initTableReducerJob
176     // because it sets up the TableOutputFormat.
177     TableMapReduceUtil.initTableReducerJob(tableName, null, job);
178     job.setNumReduceTasks(0);
179     return job;
180   }
181 
182   /*
183    * @param errorMsg Error message.  Can be null.
184    */
185   private static void usage(final String errorMsg) {
186     if (errorMsg != null && errorMsg.length() > 0) {
187       System.err.println("ERROR: " + errorMsg);
188     }
189     System.err.println("Usage: Import <tablename> <inputdir>");
190   }
191 
192   /**
193    * Main entry point.
194    *
195    * @param args  The command line parameters.
196    * @throws Exception When running the job fails.
197    */
198   public static void main(String[] args) throws Exception {
199     Configuration conf = HBaseConfiguration.create();
200     String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
201     if (otherArgs.length < 2) {
202       usage("Wrong number of arguments: " + otherArgs.length);
203       System.exit(-1);
204     }
205     Job job = createSubmittableJob(conf, otherArgs);
206     System.exit(job.waitForCompletion(true) ? 0 : 1);
207   }
208 }