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 java.util.Base64;
022
023import org.apache.hadoop.io.LongWritable;
024import org.apache.hadoop.io.Text;
025import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
026import org.apache.hadoop.hbase.util.Pair;
027import org.apache.hadoop.mapreduce.Mapper;
028import org.apache.hadoop.mapreduce.Counter;
029import org.apache.yetus.audience.InterfaceAudience;
030import org.apache.hadoop.conf.Configuration;
031
032/**
033 * Write table content out to map output files.
034 */
035@InterfaceAudience.Public
036public class TsvImporterTextMapper
037extends Mapper<LongWritable, Text, ImmutableBytesWritable, Text>
038{
039
040  /** Column seperator */
041  private String separator;
042
043  /** Should skip bad lines */
044  private boolean skipBadLines;
045  private Counter badLineCount;
046  private boolean logBadLines;
047
048  private ImportTsv.TsvParser parser;
049
050  public boolean getSkipBadLines() {
051    return skipBadLines;
052  }
053
054  public Counter getBadLineCount() {
055    return badLineCount;
056  }
057
058  public void incrementBadLineCount(int count) {
059    this.badLineCount.increment(count);
060  }
061
062  /**
063   * Handles initializing this class with objects specific to it (i.e., the parser).
064   * Common initialization that might be leveraged by a subclass is done in
065   * <code>doSetup</code>. Hence a subclass may choose to override this method
066   * and call <code>doSetup</code> as well before handling it's own custom params.
067   *
068   * @param context
069   */
070  @Override
071  protected void setup(Context context) {
072    doSetup(context);
073
074    Configuration conf = context.getConfiguration();
075
076    parser = new ImportTsv.TsvParser(conf.get(ImportTsv.COLUMNS_CONF_KEY), separator);
077    if (parser.getRowKeyColumnIndex() == -1) {
078      throw new RuntimeException("No row key column specified");
079    }
080  }
081
082  /**
083   * Handles common parameter initialization that a subclass might want to leverage.
084   * @param context
085   */
086  protected void doSetup(Context context) {
087    Configuration conf = context.getConfiguration();
088
089    // If a custom separator has been used,
090    // decode it back from Base64 encoding.
091    separator = conf.get(ImportTsv.SEPARATOR_CONF_KEY);
092    if (separator == null) {
093      separator = ImportTsv.DEFAULT_SEPARATOR;
094    } else {
095      separator = new String(Base64.getDecoder().decode(separator));
096    }
097
098    skipBadLines = context.getConfiguration().getBoolean(ImportTsv.SKIP_LINES_CONF_KEY, true);
099    logBadLines = context.getConfiguration().getBoolean(ImportTsv.LOG_BAD_LINES_CONF_KEY, false);
100    badLineCount = context.getCounter("ImportTsv", "Bad Lines");
101  }
102
103  /**
104   * Convert a line of TSV text into an HBase table row.
105   */
106  @Override
107  public void map(LongWritable offset, Text value, Context context) throws IOException {
108    try {
109      Pair<Integer,Integer> rowKeyOffests = parser.parseRowKey(value.getBytes(), value.getLength());
110      ImmutableBytesWritable rowKey = new ImmutableBytesWritable(
111          value.getBytes(), rowKeyOffests.getFirst(), rowKeyOffests.getSecond());
112      context.write(rowKey, value);
113    } catch (ImportTsv.TsvParser.BadTsvLineException|IllegalArgumentException badLine) {
114      if (logBadLines) {
115        System.err.println(value);
116      }
117      System.err.println("Bad line at offset: " + offset.get() + ":\n" + badLine.getMessage());
118      if (skipBadLines) {
119        incrementBadLineCount(1);
120        return;
121      }
122      throw new IOException(badLine);
123    } catch (InterruptedException e) {
124      e.printStackTrace();
125      Thread.currentThread().interrupt();
126    }
127  }
128}