View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.mapreduce;
19  
20  import org.apache.hadoop.io.LongWritable;
21  import org.apache.hadoop.io.Text;
22  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
23  import org.apache.hadoop.hbase.client.Put;
24  import org.apache.hadoop.hbase.util.Base64;
25  import org.apache.hadoop.hbase.KeyValue;
26  import org.apache.hadoop.mapreduce.Mapper;
27  import org.apache.hadoop.mapreduce.Counter;
28  import org.apache.hadoop.classification.InterfaceAudience;
29  import org.apache.hadoop.classification.InterfaceStability;
30  import org.apache.hadoop.conf.Configuration;
31  
32  import java.io.IOException;
33  
34  /**
35   * Write table content out to files in hdfs.
36   */
37  @InterfaceAudience.Public
38  @InterfaceStability.Stable
39  public class TsvImporterMapper
40  extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
41  {
42  
43    /** Timestamp for all inserted rows */
44    private long ts;
45  
46    /** Column seperator */
47    private String separator;
48  
49    /** Should skip bad lines */
50    private boolean skipBadLines;
51    private Counter badLineCount;
52  
53    private ImportTsv.TsvParser parser;
54  
55    public long getTs() {
56      return ts;
57    }
58  
59    public boolean getSkipBadLines() {
60      return skipBadLines;
61    }
62  
63    public Counter getBadLineCount() {
64      return badLineCount;
65    }
66  
67    public void incrementBadLineCount(int count) {
68      this.badLineCount.increment(count);
69    }
70  
71    /**
72     * Handles initializing this class with objects specific to it (i.e., the parser).
73     * Common initialization that might be leveraged by a subsclass is done in
74     * <code>doSetup</code>. Hence a subclass may choose to override this method
75     * and call <code>doSetup</code> as well before handling it's own custom params.
76     *
77     * @param context
78     */
79    @Override
80    protected void setup(Context context) {
81      doSetup(context);
82  
83      Configuration conf = context.getConfiguration();
84  
85      parser = new ImportTsv.TsvParser(conf.get(ImportTsv.COLUMNS_CONF_KEY),
86                             separator);
87      if (parser.getRowKeyColumnIndex() == -1) {
88        throw new RuntimeException("No row key column specified");
89      }
90    }
91  
92    /**
93     * Handles common parameter initialization that a subclass might want to leverage.
94     * @param context
95     */
96    protected void doSetup(Context context) {
97      Configuration conf = context.getConfiguration();
98  
99      // If a custom separator has been used,
100     // decode it back from Base64 encoding.
101     separator = conf.get(ImportTsv.SEPARATOR_CONF_KEY);
102     if (separator == null) {
103       separator = ImportTsv.DEFAULT_SEPARATOR;
104     } else {
105       separator = new String(Base64.decode(separator));
106     }
107 
108     // Should never get 0 as we are setting this to a valid value in job
109     // configuration.
110     ts = conf.getLong(ImportTsv.TIMESTAMP_CONF_KEY, 0);
111 
112     skipBadLines = context.getConfiguration().getBoolean(
113         ImportTsv.SKIP_LINES_CONF_KEY, true);
114     badLineCount = context.getCounter("ImportTsv", "Bad Lines");
115   }
116 
117   /**
118    * Convert a line of TSV text into an HBase table row.
119    */
120   @Override
121   public void map(LongWritable offset, Text value,
122     Context context)
123   throws IOException {
124     byte[] lineBytes = value.getBytes();
125 
126     try {
127       ImportTsv.TsvParser.ParsedLine parsed = parser.parse(
128           lineBytes, value.getLength());
129       ImmutableBytesWritable rowKey =
130         new ImmutableBytesWritable(lineBytes,
131             parsed.getRowKeyOffset(),
132             parsed.getRowKeyLength());
133       // Retrieve timestamp if exists
134       ts = parsed.getTimestamp(ts);
135 
136       Put put = new Put(rowKey.copyBytes());
137       for (int i = 0; i < parsed.getColumnCount(); i++) {
138         if (i == parser.getRowKeyColumnIndex()
139             || i == parser.getTimestampKeyColumnIndex()) {
140           continue;
141         }
142         KeyValue kv = new KeyValue(
143             lineBytes, parsed.getRowKeyOffset(), parsed.getRowKeyLength(),
144             parser.getFamily(i), 0, parser.getFamily(i).length,
145             parser.getQualifier(i), 0, parser.getQualifier(i).length,
146             ts,
147             KeyValue.Type.Put,
148             lineBytes, parsed.getColumnOffset(i), parsed.getColumnLength(i));
149         put.add(kv);
150       }
151       context.write(rowKey, put);
152     } catch (ImportTsv.TsvParser.BadTsvLineException badLine) {
153       if (skipBadLines) {
154         System.err.println(
155             "Bad line at offset: " + offset.get() + ":\n" +
156             badLine.getMessage());
157         incrementBadLineCount(1);
158         return;
159       } else {
160         throw new IOException(badLine);
161       }
162     } catch (IllegalArgumentException e) {
163       if (skipBadLines) {
164         System.err.println(
165             "Bad line at offset: " + offset.get() + ":\n" +
166             e.getMessage());
167         incrementBadLineCount(1);
168         return;
169       } else {
170         throw new IOException(e);
171       }
172     } catch (InterruptedException e) {
173       e.printStackTrace();
174     }
175   }
176 }