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 java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.List;
23  
24  import org.apache.hadoop.conf.Configuration;
25  import org.apache.hadoop.hbase.Cell;
26  import org.apache.hadoop.hbase.KeyValue;
27  import org.apache.hadoop.hbase.Tag;
28  import org.apache.hadoop.hbase.TagType;
29  import org.apache.hadoop.hbase.classification.InterfaceAudience;
30  import org.apache.hadoop.hbase.classification.InterfaceStability;
31  import org.apache.hadoop.hbase.client.Put;
32  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
33  import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.BadTsvLineException;
34  import org.apache.hadoop.hbase.security.visibility.CellVisibility;
35  import org.apache.hadoop.hbase.security.visibility.InvalidLabelException;
36  import org.apache.hadoop.hbase.util.Base64;
37  import org.apache.hadoop.hbase.util.Bytes;
38  import org.apache.hadoop.io.LongWritable;
39  import org.apache.hadoop.io.Text;
40  import org.apache.hadoop.mapreduce.Counter;
41  import org.apache.hadoop.mapreduce.Mapper;
42  
43  /**
44   * Write table content out to files in hdfs.
45   */
46  @InterfaceAudience.Public
47  @InterfaceStability.Stable
48  public class TsvImporterMapper
49  extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
50  {
51  
52    /** Timestamp for all inserted rows */
53    protected long ts;
54  
55    /** Column seperator */
56    private String separator;
57  
58    /** Should skip bad lines */
59    private boolean skipBadLines;
60    /** Should skip empty columns*/
61    private boolean skipEmptyColumns;
62    private Counter badLineCount;
63    private boolean logBadLines;
64  
65    protected ImportTsv.TsvParser parser;
66  
67    protected Configuration conf;
68  
69    protected String cellVisibilityExpr;
70  
71    protected long ttl;
72  
73    protected CellCreator kvCreator;
74  
75    private String hfileOutPath;
76  
77    public long getTs() {
78      return ts;
79    }
80  
81    public boolean getSkipBadLines() {
82      return skipBadLines;
83    }
84  
85    public Counter getBadLineCount() {
86      return badLineCount;
87    }
88  
89    public void incrementBadLineCount(int count) {
90      this.badLineCount.increment(count);
91    }
92  
93    /**
94     * Handles initializing this class with objects specific to it (i.e., the parser).
95     * Common initialization that might be leveraged by a subsclass is done in
96     * <code>doSetup</code>. Hence a subclass may choose to override this method
97     * and call <code>doSetup</code> as well before handling it's own custom params.
98     *
99     * @param context
100    */
101   @Override
102   protected void setup(Context context) {
103     doSetup(context);
104 
105     conf = context.getConfiguration();
106     parser = new ImportTsv.TsvParser(conf.get(ImportTsv.COLUMNS_CONF_KEY),
107                            separator);
108     if (parser.getRowKeyColumnIndex() == -1) {
109       throw new RuntimeException("No row key column specified");
110     }
111     this.kvCreator = new CellCreator(conf);
112   }
113 
114   /**
115    * Handles common parameter initialization that a subclass might want to leverage.
116    * @param context
117    */
118   protected void doSetup(Context context) {
119     Configuration conf = context.getConfiguration();
120 
121     // If a custom separator has been used,
122     // decode it back from Base64 encoding.
123     separator = conf.get(ImportTsv.SEPARATOR_CONF_KEY);
124     if (separator == null) {
125       separator = ImportTsv.DEFAULT_SEPARATOR;
126     } else {
127       separator = new String(Base64.decode(separator));
128     }
129     // Should never get 0 as we are setting this to a valid value in job
130     // configuration.
131     ts = conf.getLong(ImportTsv.TIMESTAMP_CONF_KEY, 0);
132 
133     skipEmptyColumns = context.getConfiguration().getBoolean(
134         ImportTsv.SKIP_EMPTY_COLUMNS, false);
135     skipBadLines = context.getConfiguration().getBoolean(
136         ImportTsv.SKIP_LINES_CONF_KEY, true);
137     badLineCount = context.getCounter("ImportTsv", "Bad Lines");
138     logBadLines = context.getConfiguration().getBoolean(ImportTsv.LOG_BAD_LINES_CONF_KEY, false);
139     hfileOutPath = conf.get(ImportTsv.BULK_OUTPUT_CONF_KEY);
140   }
141 
142   /**
143    * Convert a line of TSV text into an HBase table row.
144    */
145   @Override
146   public void map(LongWritable offset, Text value,
147     Context context)
148   throws IOException {
149     byte[] lineBytes = value.getBytes();
150 
151     try {
152       ImportTsv.TsvParser.ParsedLine parsed = parser.parse(
153           lineBytes, value.getLength());
154       ImmutableBytesWritable rowKey =
155         new ImmutableBytesWritable(lineBytes,
156             parsed.getRowKeyOffset(),
157             parsed.getRowKeyLength());
158       // Retrieve timestamp if exists
159       ts = parsed.getTimestamp(ts);
160       cellVisibilityExpr = parsed.getCellVisibility();
161       ttl = parsed.getCellTTL();
162 
163       Put put = new Put(rowKey.copyBytes());
164       for (int i = 0; i < parsed.getColumnCount(); i++) {
165         if (i == parser.getRowKeyColumnIndex() || i == parser.getTimestampKeyColumnIndex()
166             || i == parser.getAttributesKeyColumnIndex() || i == parser.getCellVisibilityColumnIndex()
167             || i == parser.getCellTTLColumnIndex() || (skipEmptyColumns 
168             && parsed.getColumnLength(i) == 0)) {
169           continue;
170         }
171         populatePut(lineBytes, parsed, put, i);
172       }
173       context.write(rowKey, put);
174     } catch (ImportTsv.TsvParser.BadTsvLineException | IllegalArgumentException
175         | InvalidLabelException badLine) {
176       if (logBadLines) {
177         System.err.println(value);
178       }
179       System.err.println("Bad line at offset: " + offset.get() + ":\n" + badLine.getMessage());
180       if (skipBadLines) {
181         incrementBadLineCount(1);
182         return;
183       }
184       throw new IOException(badLine);
185     } catch (InterruptedException e) {
186       e.printStackTrace();
187     }
188   }
189 
190   protected void populatePut(byte[] lineBytes, ImportTsv.TsvParser.ParsedLine parsed, Put put,
191       int i) throws BadTsvLineException, IOException {
192     Cell cell = null;
193     if (hfileOutPath == null) {
194       cell = new KeyValue(lineBytes, parsed.getRowKeyOffset(), parsed.getRowKeyLength(),
195           parser.getFamily(i), 0, parser.getFamily(i).length, parser.getQualifier(i), 0,
196           parser.getQualifier(i).length, ts, KeyValue.Type.Put, lineBytes,
197           parsed.getColumnOffset(i), parsed.getColumnLength(i));
198       if (cellVisibilityExpr != null) {
199         // We won't be validating the expression here. The Visibility CP will do
200         // the validation
201         put.setCellVisibility(new CellVisibility(cellVisibilityExpr));
202       }
203       if (ttl > 0) {
204         put.setTTL(ttl);
205       }
206     } else {
207       // Creating the KV which needs to be directly written to HFiles. Using the Facade
208       // KVCreator for creation of kvs.
209       List<Tag> tags = new ArrayList<Tag>();
210       if (cellVisibilityExpr != null) {
211         tags.addAll(kvCreator.getVisibilityExpressionResolver()
212           .createVisibilityExpTags(cellVisibilityExpr));
213       }
214       // Add TTL directly to the KV so we can vary them when packing more than one KV
215       // into puts
216       if (ttl > 0) {
217         tags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl)));
218       }
219       cell = this.kvCreator.create(lineBytes, parsed.getRowKeyOffset(), parsed.getRowKeyLength(),
220           parser.getFamily(i), 0, parser.getFamily(i).length, parser.getQualifier(i), 0,
221           parser.getQualifier(i).length, ts, lineBytes, parsed.getColumnOffset(i),
222           parsed.getColumnLength(i), tags);
223     }
224     put.add(cell);
225   }
226 }