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.util.Base64;
24  import org.apache.hadoop.hbase.util.Pair;
25  import org.apache.hadoop.mapreduce.Mapper;
26  import org.apache.hadoop.mapreduce.Counter;
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.hbase.classification.InterfaceStability;
29  import org.apache.hadoop.conf.Configuration;
30  
31  import java.io.IOException;
32  
33  /**
34   * Write table content out to map output files.
35   */
36  @InterfaceAudience.Public
37  @InterfaceStability.Evolving
38  public class TsvImporterTextMapper
39  extends Mapper<LongWritable, Text, ImmutableBytesWritable, Text>
40  {
41  
42    /** Column seperator */
43    private String separator;
44  
45    /** Should skip bad lines */
46    private boolean skipBadLines;
47    private Counter badLineCount;
48    private boolean logBadLines;
49  
50    private ImportTsv.TsvParser parser;
51  
52    public boolean getSkipBadLines() {
53      return skipBadLines;
54    }
55  
56    public Counter getBadLineCount() {
57      return badLineCount;
58    }
59  
60    public void incrementBadLineCount(int count) {
61      this.badLineCount.increment(count);
62    }
63  
64    /**
65     * Handles initializing this class with objects specific to it (i.e., the parser).
66     * Common initialization that might be leveraged by a subsclass is done in
67     * <code>doSetup</code>. Hence a subclass may choose to override this method
68     * and call <code>doSetup</code> as well before handling it's own custom params.
69     *
70     * @param context
71     */
72    @Override
73    protected void setup(Context context) {
74      doSetup(context);
75  
76      Configuration conf = context.getConfiguration();
77  
78      parser = new ImportTsv.TsvParser(conf.get(ImportTsv.COLUMNS_CONF_KEY), separator);
79      if (parser.getRowKeyColumnIndex() == -1) {
80        throw new RuntimeException("No row key column specified");
81      }
82    }
83  
84    /**
85     * Handles common parameter initialization that a subclass might want to leverage.
86     * @param context
87     */
88    protected void doSetup(Context context) {
89      Configuration conf = context.getConfiguration();
90  
91      // If a custom separator has been used,
92      // decode it back from Base64 encoding.
93      separator = conf.get(ImportTsv.SEPARATOR_CONF_KEY);
94      if (separator == null) {
95        separator = ImportTsv.DEFAULT_SEPARATOR;
96      } else {
97        separator = new String(Base64.decode(separator));
98      }
99  
100     skipBadLines = context.getConfiguration().getBoolean(ImportTsv.SKIP_LINES_CONF_KEY, true);
101     logBadLines = context.getConfiguration().getBoolean(ImportTsv.LOG_BAD_LINES_CONF_KEY, false);
102     badLineCount = context.getCounter("ImportTsv", "Bad Lines");
103   }
104 
105   /**
106    * Convert a line of TSV text into an HBase table row.
107    */
108   @Override
109   public void map(LongWritable offset, Text value, Context context) throws IOException {
110     try {
111       Pair<Integer,Integer> rowKeyOffests = parser.parseRowKey(value.getBytes(), value.getLength());
112       ImmutableBytesWritable rowKey = new ImmutableBytesWritable(
113           value.getBytes(), rowKeyOffests.getFirst(), rowKeyOffests.getSecond());
114       context.write(rowKey, value);
115     } catch (ImportTsv.TsvParser.BadTsvLineException|IllegalArgumentException badLine) {
116       if (logBadLines) {
117         System.err.println(value);
118       }
119       System.err.println("Bad line at offset: " + offset.get() + ":\n" + badLine.getMessage());
120       if (skipBadLines) {
121         incrementBadLineCount(1);
122         return;
123       } 
124       throw new IOException(badLine);
125     } catch (InterruptedException e) {
126       e.printStackTrace();
127       Thread.currentThread().interrupt();
128     } 
129   }
130 }