001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.mapreduce; 019 020import java.io.IOException; 021import org.apache.hadoop.conf.Configuration; 022import org.apache.hadoop.conf.Configured; 023import org.apache.hadoop.fs.Path; 024import org.apache.hadoop.hbase.HBaseConfiguration; 025import org.apache.hadoop.hbase.client.Put; 026import org.apache.hadoop.hbase.io.ImmutableBytesWritable; 027import org.apache.hadoop.hbase.util.Bytes; 028import org.apache.hadoop.io.LongWritable; 029import org.apache.hadoop.io.Text; 030import org.apache.hadoop.mapreduce.Job; 031import org.apache.hadoop.mapreduce.Mapper; 032import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; 033import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; 034import org.apache.hadoop.util.Tool; 035import org.apache.hadoop.util.ToolRunner; 036import org.apache.yetus.audience.InterfaceAudience; 037import org.slf4j.Logger; 038import org.slf4j.LoggerFactory; 039 040/** 041 * Sample Uploader MapReduce 042 * <p> 043 * This is EXAMPLE code. You will need to change it to work for your context. 044 * <p> 045 * Uses {@link TableReducer} to put the data into HBase. Change the InputFormat to suit your data. 046 * In this example, we are importing a CSV file. 047 * <p> 048 * 049 * <pre> 050 * row,family,qualifier,value 051 * </pre> 052 * <p> 053 * The table and columnfamily we're to insert into must preexist. 054 * <p> 055 * There is no reducer in this example as it is not necessary and adds significant overhead. If you 056 * need to do any massaging of data before inserting into HBase, you can do this in the map as well. 057 * <p> 058 * Do the following to start the MR job: 059 * 060 * <pre> 061 * ./bin/hadoop org.apache.hadoop.hbase.mapreduce.SampleUploader /tmp/input.csv TABLE_NAME 062 * </pre> 063 * <p> 064 * This code was written against HBase 0.21 trunk. 065 */ 066@InterfaceAudience.Private 067public class SampleUploader extends Configured implements Tool { 068 private static final Logger LOG = LoggerFactory.getLogger(SampleUploader.class); 069 070 private static final String NAME = "SampleUploader"; 071 072 static class Uploader extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put> { 073 private long checkpoint = 100; 074 private long count = 0; 075 076 @Override 077 public void map(LongWritable key, Text line, Context context) throws IOException { 078 // Input is a CSV file 079 // Each map() is a single line, where the key is the line number 080 // Each line is comma-delimited; row,family,qualifier,value 081 082 // Split CSV line 083 String[] values = line.toString().split(","); 084 if (values.length != 4) { 085 return; 086 } 087 088 // Extract each value 089 byte[] row = Bytes.toBytes(values[0]); 090 byte[] family = Bytes.toBytes(values[1]); 091 byte[] qualifier = Bytes.toBytes(values[2]); 092 byte[] value = Bytes.toBytes(values[3]); 093 094 // Create Put 095 Put put = new Put(row); 096 put.addColumn(family, qualifier, value); 097 098 // Uncomment below to disable WAL. This will improve performance but means 099 // you will experience data loss in the case of a RegionServer crash. 100 // put.setWriteToWAL(false); 101 102 try { 103 context.write(new ImmutableBytesWritable(row), put); 104 } catch (InterruptedException e) { 105 LOG.error("Interrupted emitting put", e); 106 Thread.currentThread().interrupt(); 107 } 108 109 // Set status every checkpoint lines 110 if (++count % checkpoint == 0) { 111 context.setStatus("Emitting Put " + count); 112 } 113 } 114 } 115 116 /** 117 * Job configuration. 118 */ 119 public static Job configureJob(Configuration conf, String[] args) throws IOException { 120 Path inputPath = new Path(args[0]); 121 String tableName = args[1]; 122 Job job = new Job(conf, NAME + "_" + tableName); 123 job.setJarByClass(Uploader.class); 124 FileInputFormat.setInputPaths(job, inputPath); 125 job.setInputFormatClass(SequenceFileInputFormat.class); 126 job.setMapperClass(Uploader.class); 127 // No reducers. Just write straight to table. Call initTableReducerJob 128 // because it sets up the TableOutputFormat. 129 TableMapReduceUtil.initTableReducerJob(tableName, null, job); 130 job.setNumReduceTasks(0); 131 return job; 132 } 133 134 /** 135 * Main entry point. 136 * @param otherArgs The command line parameters after ToolRunner handles standard. 137 * @throws Exception When running the job fails. 138 */ 139 public int run(String[] otherArgs) throws Exception { 140 if (otherArgs.length != 2) { 141 System.err.println("Wrong number of arguments: " + otherArgs.length); 142 System.err.println("Usage: " + NAME + " <input> <tablename>"); 143 return -1; 144 } 145 Job job = configureJob(getConf(), otherArgs); 146 return (job.waitForCompletion(true) ? 0 : 1); 147 } 148 149 public static void main(String[] args) throws Exception { 150 int status = ToolRunner.run(HBaseConfiguration.create(), new SampleUploader(), args); 151 System.exit(status); 152 } 153}