PageRenderTime 53ms CodeModel.GetById 10ms app.highlight 37ms RepoModel.GetById 1ms app.codeStats 1ms

/tags/release-0.1-rc2/hive/external/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormat.java

#
Java | 421 lines | 284 code | 63 blank | 74 comment | 35 complexity | ed9fd4d43a1d66f9eafa0d02b2d45eae MD5 | raw file
  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
 19package org.apache.hadoop.hive.hbase;
 20
 21import java.io.IOException;
 22import java.util.ArrayList;
 23import java.util.Arrays;
 24import java.util.List;
 25
 26import org.apache.commons.logging.Log;
 27import org.apache.commons.logging.LogFactory;
 28import org.apache.hadoop.fs.Path;
 29import org.apache.hadoop.hbase.HBaseConfiguration;
 30import org.apache.hadoop.hbase.client.HTable;
 31import org.apache.hadoop.hbase.client.Result;
 32import org.apache.hadoop.hbase.client.Scan;
 33import org.apache.hadoop.hbase.filter.BinaryComparator;
 34import org.apache.hadoop.hbase.filter.CompareFilter;
 35import org.apache.hadoop.hbase.filter.RowFilter;
 36import org.apache.hadoop.hbase.filter.WhileMatchFilter;
 37import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 38import org.apache.hadoop.hbase.mapreduce.TableInputFormatBase;
 39import org.apache.hadoop.hbase.mapreduce.TableSplit;
 40import org.apache.hadoop.hbase.util.Bytes;
 41import org.apache.hadoop.hbase.util.Writables;
 42import org.apache.hadoop.hive.ql.exec.ExprNodeConstantEvaluator;
 43import org.apache.hadoop.hive.ql.exec.Utilities;
 44import org.apache.hadoop.hive.ql.index.IndexPredicateAnalyzer;
 45import org.apache.hadoop.hive.ql.index.IndexSearchCondition;
 46import org.apache.hadoop.hive.ql.metadata.HiveException;
 47import org.apache.hadoop.hive.ql.metadata.HiveStoragePredicateHandler;
 48import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 49import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 50import org.apache.hadoop.hive.serde.Constants;
 51import org.apache.hadoop.hive.serde2.ByteStream;
 52import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 53import org.apache.hadoop.hive.serde2.SerDeException;
 54import org.apache.hadoop.hive.serde2.lazy.LazyUtils;
 55import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 56import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 57import org.apache.hadoop.io.Writable;
 58import org.apache.hadoop.mapred.InputFormat;
 59import org.apache.hadoop.mapred.InputSplit;
 60import org.apache.hadoop.mapred.JobConf;
 61import org.apache.hadoop.mapred.RecordReader;
 62import org.apache.hadoop.mapred.Reporter;
 63import org.apache.hadoop.mapreduce.Job;
 64import org.apache.hadoop.mapreduce.JobContext;
 65import org.apache.hadoop.mapreduce.TaskAttemptContext;
 66import org.apache.hadoop.mapreduce.TaskAttemptID;
 67import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 68
 69/**
 70 * HiveHBaseTableInputFormat implements InputFormat for HBase storage handler
 71 * tables, decorating an underlying HBase TableInputFormat with extra Hive logic
 72 * such as column pruning and filter pushdown.
 73 */
 74public class HiveHBaseTableInputFormat extends TableInputFormatBase
 75    implements InputFormat<ImmutableBytesWritable, Result> {
 76
 77  static final Log LOG = LogFactory.getLog(HiveHBaseTableInputFormat.class);
 78
 79  @Override
 80  public RecordReader<ImmutableBytesWritable, Result> getRecordReader(
 81    InputSplit split,
 82    JobConf jobConf,
 83    final Reporter reporter) throws IOException {
 84
 85    HBaseSplit hbaseSplit = (HBaseSplit) split;
 86    TableSplit tableSplit = hbaseSplit.getSplit();
 87    String hbaseTableName = jobConf.get(HBaseSerDe.HBASE_TABLE_NAME);
 88    setHTable(new HTable(new HBaseConfiguration(jobConf), Bytes.toBytes(hbaseTableName)));
 89    String hbaseColumnsMapping = jobConf.get(HBaseSerDe.HBASE_COLUMNS_MAPPING);
 90    List<String> hbaseColumnFamilies = new ArrayList<String>();
 91    List<String> hbaseColumnQualifiers = new ArrayList<String>();
 92    List<byte []> hbaseColumnFamiliesBytes = new ArrayList<byte []>();
 93    List<byte []> hbaseColumnQualifiersBytes = new ArrayList<byte []>();
 94
 95    int iKey;
 96    try {
 97      iKey = HBaseSerDe.parseColumnMapping(hbaseColumnsMapping, hbaseColumnFamilies,
 98          hbaseColumnFamiliesBytes, hbaseColumnQualifiers, hbaseColumnQualifiersBytes);
 99    } catch (SerDeException se) {
100      throw new IOException(se);
101    }
102    List<Integer> readColIDs = ColumnProjectionUtils.getReadColumnIDs(jobConf);
103
104    if (hbaseColumnFamilies.size() < readColIDs.size()) {
105      throw new IOException("Cannot read more columns than the given table contains.");
106    }
107
108    boolean addAll = (readColIDs.size() == 0);
109    Scan scan = new Scan();
110    boolean empty = true;
111
112    if (!addAll) {
113      for (int i : readColIDs) {
114        if (i == iKey) {
115          continue;
116        }
117
118        if (hbaseColumnQualifiers.get(i) == null) {
119          scan.addFamily(hbaseColumnFamiliesBytes.get(i));
120        } else {
121          scan.addColumn(hbaseColumnFamiliesBytes.get(i), hbaseColumnQualifiersBytes.get(i));
122        }
123
124        empty = false;
125      }
126    }
127
128    // The HBase table's row key maps to a Hive table column. In the corner case when only the
129    // row key column is selected in Hive, the HBase Scan will be empty i.e. no column family/
130    // column qualifier will have been added to the scan. We arbitrarily add at least one column
131    // to the HBase scan so that we can retrieve all of the row keys and return them as the Hive
132    // tables column projection.
133    if (empty) {
134      for (int i = 0; i < hbaseColumnFamilies.size(); i++) {
135        if (i == iKey) {
136          continue;
137        }
138
139        if (hbaseColumnQualifiers.get(i) == null) {
140          scan.addFamily(hbaseColumnFamiliesBytes.get(i));
141        } else {
142          scan.addColumn(hbaseColumnFamiliesBytes.get(i), hbaseColumnQualifiersBytes.get(i));
143        }
144
145        if (!addAll) {
146          break;
147        }
148      }
149    }
150
151    // If Hive's optimizer gave us a filter to process, convert it to the
152    // HBase scan form now.
153    tableSplit = convertFilter(jobConf, scan, tableSplit, iKey);
154
155    setScan(scan);
156
157    Job job = new Job(jobConf);
158    TaskAttemptContext tac =
159      new TaskAttemptContext(job.getConfiguration(), new TaskAttemptID()) {
160
161        @Override
162        public void progress() {
163          reporter.progress();
164        }
165      };
166
167    final org.apache.hadoop.mapreduce.RecordReader<ImmutableBytesWritable, Result>
168    recordReader = createRecordReader(tableSplit, tac);
169
170    return new RecordReader<ImmutableBytesWritable, Result>() {
171
172      @Override
173      public void close() throws IOException {
174        recordReader.close();
175      }
176
177      @Override
178      public ImmutableBytesWritable createKey() {
179        return new ImmutableBytesWritable();
180      }
181
182      @Override
183      public Result createValue() {
184        return new Result();
185      }
186
187      @Override
188      public long getPos() throws IOException {
189        return 0;
190      }
191
192      @Override
193      public float getProgress() throws IOException {
194        float progress = 0.0F;
195
196        try {
197          progress = recordReader.getProgress();
198        } catch (InterruptedException e) {
199          throw new IOException(e);
200        }
201
202        return progress;
203      }
204
205      @Override
206      public boolean next(ImmutableBytesWritable rowKey, Result value) throws IOException {
207
208        boolean next = false;
209
210        try {
211          next = recordReader.nextKeyValue();
212
213          if (next) {
214            rowKey.set(recordReader.getCurrentValue().getRow());
215            Writables.copyWritable(recordReader.getCurrentValue(), value);
216          }
217        } catch (InterruptedException e) {
218          throw new IOException(e);
219        }
220
221        return next;
222      }
223    };
224  }
225
226  /**
227   * Converts a filter (which has been pushed down from Hive's optimizer)
228   * into corresponding restrictions on the HBase scan.  The
229   * filter should already be in a form which can be fully converted.
230   *
231   * @param jobConf configuration for the scan
232   *
233   * @param scan the HBase scan object to restrict
234   *
235   * @param tableSplit the HBase table split to restrict, or null
236   * if calculating splits
237   *
238   * @param iKey 0-based offset of key column within Hive table
239   *
240   * @return converted table split if any
241   */
242  private TableSplit convertFilter(
243    JobConf jobConf,
244    Scan scan,
245    TableSplit tableSplit,
246    int iKey)
247    throws IOException {
248
249    String filterExprSerialized =
250      jobConf.get(TableScanDesc.FILTER_EXPR_CONF_STR);
251    if (filterExprSerialized == null) {
252      return tableSplit;
253    }
254    ExprNodeDesc filterExpr =
255      Utilities.deserializeExpression(filterExprSerialized, jobConf);
256
257    String columnNameProperty = jobConf.get(Constants.LIST_COLUMNS);
258    List<String> columnNames =
259      Arrays.asList(columnNameProperty.split(","));
260
261    IndexPredicateAnalyzer analyzer =
262      newIndexPredicateAnalyzer(columnNames.get(iKey));
263
264    List<IndexSearchCondition> searchConditions =
265      new ArrayList<IndexSearchCondition>();
266    ExprNodeDesc residualPredicate =
267      analyzer.analyzePredicate(filterExpr, searchConditions);
268
269    // There should be no residual since we already negotiated
270    // that earlier in HBaseStorageHandler.decomposePredicate.
271    if (residualPredicate != null) {
272      throw new RuntimeException(
273        "Unexpected residual predicate " + residualPredicate.getExprString());
274    }
275
276    // There should be exactly one predicate since we already
277    // negotiated that also.
278    if (searchConditions.size() != 1) {
279      throw new RuntimeException(
280        "Exactly one search condition expected in push down");
281    }
282
283    // Convert the search condition into a restriction on the HBase scan
284    IndexSearchCondition sc = searchConditions.get(0);
285    ExprNodeConstantEvaluator eval =
286      new ExprNodeConstantEvaluator(sc.getConstantDesc());
287    byte [] startRow;
288    try {
289      ObjectInspector objInspector = eval.initialize(null);
290      Object writable = eval.evaluate(null);
291      ByteStream.Output serializeStream = new ByteStream.Output();
292      LazyUtils.writePrimitiveUTF8(
293        serializeStream,
294        writable,
295        (PrimitiveObjectInspector) objInspector,
296        false,
297        (byte) 0,
298        null);
299      startRow = new byte[serializeStream.getCount()];
300      System.arraycopy(
301        serializeStream.getData(), 0,
302        startRow, 0, serializeStream.getCount());
303    } catch (HiveException ex) {
304      throw new IOException(ex);
305    }
306
307    // stopRow is exclusive, so pad it with a trailing 0 byte to
308    // make it compare as the very next value after startRow
309    byte [] stopRow = new byte[startRow.length + 1];
310    System.arraycopy(startRow, 0, stopRow, 0, startRow.length);
311
312    if (tableSplit != null) {
313      tableSplit = new TableSplit(
314        tableSplit.getTableName(),
315        startRow,
316        stopRow,
317        tableSplit.getRegionLocation());
318    }
319    scan.setStartRow(startRow);
320    scan.setStopRow(stopRow);
321
322    // Add a WhileMatchFilter to make the scan terminate as soon
323    // as we see a non-matching key.  This is probably redundant
324    // since the stopRow above should already take care of it for us.
325    scan.setFilter(
326      new WhileMatchFilter(
327        new RowFilter(
328          CompareFilter.CompareOp.EQUAL,
329          new BinaryComparator(startRow))));
330    return tableSplit;
331  }
332
333  /**
334   * Instantiates a new predicate analyzer suitable for
335   * determining how to push a filter down into the HBase scan,
336   * based on the rules for what kinds of pushdown we currently support.
337   *
338   * @param keyColumnName name of the Hive column mapped to the HBase row key
339   *
340   * @return preconfigured predicate analyzer
341   */
342  static IndexPredicateAnalyzer newIndexPredicateAnalyzer(
343    String keyColumnName) {
344
345    IndexPredicateAnalyzer analyzer = new IndexPredicateAnalyzer();
346
347    // for now, we only support equality comparisons
348    analyzer.addComparisonOp(
349      "org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual");
350
351    // and only on the key column
352    analyzer.clearAllowedColumnNames();
353    analyzer.allowColumnName(keyColumnName);
354
355    return analyzer;
356  }
357  
358  @Override
359  public InputSplit[] getSplits(JobConf jobConf, int numSplits) throws IOException {
360
361    String hbaseTableName = jobConf.get(HBaseSerDe.HBASE_TABLE_NAME);
362    setHTable(new HTable(new HBaseConfiguration(jobConf), Bytes.toBytes(hbaseTableName)));
363    String hbaseColumnsMapping = jobConf.get(HBaseSerDe.HBASE_COLUMNS_MAPPING);
364
365    if (hbaseColumnsMapping == null) {
366      throw new IOException("hbase.columns.mapping required for HBase Table.");
367    }
368
369    List<String> hbaseColumnFamilies = new ArrayList<String>();
370    List<String> hbaseColumnQualifiers = new ArrayList<String>();
371    List<byte []> hbaseColumnFamiliesBytes = new ArrayList<byte []>();
372    List<byte []> hbaseColumnQualifiersBytes = new ArrayList<byte []>();
373
374    int iKey;
375    try {
376      iKey = HBaseSerDe.parseColumnMapping(hbaseColumnsMapping, hbaseColumnFamilies,
377          hbaseColumnFamiliesBytes, hbaseColumnQualifiers, hbaseColumnQualifiersBytes);
378    } catch (SerDeException se) {
379      throw new IOException(se);
380    }
381
382    Scan scan = new Scan();
383
384    // Take filter pushdown into account while calculating splits; this
385    // allows us to prune off regions immediately.  Note that although
386    // the Javadoc for the superclass getSplits says that it returns one
387    // split per region, the implementation actually takes the scan
388    // definition into account and excludes regions which don't satisfy
389    // the start/stop row conditions (HBASE-1829).
390    convertFilter(jobConf, scan, null, iKey);
391
392    // REVIEW:  are we supposed to be applying the getReadColumnIDs
393    // same as in getRecordReader?
394    for (int i = 0; i < hbaseColumnFamilies.size(); i++) {
395      if (i == iKey) {
396        continue;
397      }
398
399      if (hbaseColumnQualifiers.get(i) == null) {
400        scan.addFamily(hbaseColumnFamiliesBytes.get(i));
401      } else {
402        scan.addColumn(hbaseColumnFamiliesBytes.get(i), hbaseColumnQualifiersBytes.get(i));
403      }
404    }
405
406    setScan(scan);
407    Job job = new Job(jobConf);
408    JobContext jobContext = new JobContext(job.getConfiguration(), job.getJobID());
409    Path [] tablePaths = FileInputFormat.getInputPaths(jobContext);
410
411    List<org.apache.hadoop.mapreduce.InputSplit> splits =
412      super.getSplits(jobContext);
413    InputSplit [] results = new InputSplit[splits.size()];
414
415    for (int i = 0; i < splits.size(); i++) {
416      results[i] = new HBaseSplit((TableSplit) splits.get(i), tablePaths[0]);
417    }
418
419    return results;
420  }
421}