PageRenderTime 30ms CodeModel.GetById 14ms app.highlight 13ms RepoModel.GetById 1ms app.codeStats 0ms

/tags/release-0.0.0-rc0/hive/external/ql/src/java/org/apache/hadoop/hive/ql/io/BucketizedHiveInputFormat.java

#
Java | 147 lines | 99 code | 19 blank | 29 comment | 15 complexity | db0993db076a424f08e32fb99ab8ebb1 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.ql.io;
 20
 21import java.io.IOException;
 22import java.util.ArrayList;
 23import java.util.List;
 24
 25import org.apache.commons.logging.Log;
 26import org.apache.commons.logging.LogFactory;
 27import org.apache.hadoop.fs.FileStatus;
 28import org.apache.hadoop.fs.FileSystem;
 29import org.apache.hadoop.fs.Path;
 30import org.apache.hadoop.hive.common.FileUtils;
 31import org.apache.hadoop.hive.ql.plan.PartitionDesc;
 32import org.apache.hadoop.io.Writable;
 33import org.apache.hadoop.io.WritableComparable;
 34import org.apache.hadoop.mapred.FileInputFormat;
 35import org.apache.hadoop.mapred.InputFormat;
 36import org.apache.hadoop.mapred.InputSplit;
 37import org.apache.hadoop.mapred.InvalidInputException;
 38import org.apache.hadoop.mapred.JobConf;
 39import org.apache.hadoop.mapred.RecordReader;
 40import org.apache.hadoop.mapred.Reporter;
 41
 42/**
 43 * BucketizedHiveInputFormat serves the similar function as hiveInputFormat but
 44 * its getSplits() always group splits from one input file into one wrapper
 45 * split. It is useful for the applications that requires input files to fit in
 46 * one mapper.
 47 */
 48public class BucketizedHiveInputFormat<K extends WritableComparable, V extends Writable>
 49    extends HiveInputFormat<K, V> {
 50
 51  public static final Log LOG = LogFactory
 52      .getLog("org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat");
 53
 54  @Override
 55  public RecordReader getRecordReader(InputSplit split, JobConf job,
 56      Reporter reporter) throws IOException {
 57
 58    BucketizedHiveInputSplit hsplit = (BucketizedHiveInputSplit) split;
 59
 60    String inputFormatClassName = null;
 61    Class inputFormatClass = null;
 62    try {
 63      inputFormatClassName = hsplit.inputFormatClassName();
 64      inputFormatClass = job.getClassByName(inputFormatClassName);
 65    } catch (Exception e) {
 66      throw new IOException("cannot find class " + inputFormatClassName);
 67    }
 68
 69    // clone a jobConf for setting needed columns for reading
 70    JobConf cloneJobConf = new JobConf(job);
 71    pushProjectionsAndFilters(cloneJobConf, inputFormatClass, hsplit.getPath()
 72        .toString(), hsplit.getPath().toUri().getPath());
 73
 74    InputFormat inputFormat = getInputFormatFromCache(inputFormatClass,
 75        cloneJobConf);
 76    BucketizedHiveRecordReader<K, V> rr= new BucketizedHiveRecordReader(inputFormat, hsplit, cloneJobConf,
 77        reporter);
 78    rr.initIOContext(hsplit, cloneJobConf, inputFormatClass);
 79    return rr;
 80  }
 81
 82  protected FileStatus[] listStatus(JobConf job, Path path) throws IOException {
 83    ArrayList<FileStatus> result = new ArrayList<FileStatus>();
 84    List<IOException> errors = new ArrayList<IOException>();
 85
 86    FileSystem fs = path.getFileSystem(job);
 87    FileStatus[] matches = fs.globStatus(path);
 88    if (matches == null) {
 89      errors.add(new IOException("Input path does not exist: " + path));
 90    } else if (matches.length == 0) {
 91      errors.add(new IOException("Input Pattern " + path + " matches 0 files"));
 92    } else {
 93      for (FileStatus globStat : matches) {
 94        FileUtils.listStatusRecursively(fs, globStat, result);
 95      }
 96    }
 97
 98    if (!errors.isEmpty()) {
 99      throw new InvalidInputException(errors);
100    }
101    LOG.info("Total input paths to process : " + result.size());
102    return result.toArray(new FileStatus[result.size()]);
103
104  }
105
106  @Override
107  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
108    init(job);
109
110    Path[] dirs = FileInputFormat.getInputPaths(job);
111    if (dirs.length == 0) {
112      throw new IOException("No input paths specified in job");
113    }
114    JobConf newjob = new JobConf(job);
115    ArrayList<InputSplit> result = new ArrayList<InputSplit>();
116
117    int numOrigSplits = 0;
118    // for each dir, get all files under the dir, do getSplits to each
119    // individual file,
120    // and then create a BucketizedHiveInputSplit on it
121    for (Path dir : dirs) {
122      PartitionDesc part = getPartitionDescFromPath(pathToPartitionInfo, dir);
123      // create a new InputFormat instance if this is the first time to see this
124      // class
125      Class inputFormatClass = part.getInputFileFormatClass();
126      InputFormat inputFormat = getInputFormatFromCache(inputFormatClass, job);
127      newjob.setInputFormat(inputFormat.getClass());
128
129      FileStatus[] listStatus = listStatus(newjob, dir);
130
131      for (FileStatus status : listStatus) {
132        LOG.info("block size: " + status.getBlockSize());
133        LOG.info("file length: " + status.getLen());
134        FileInputFormat.setInputPaths(newjob, status.getPath());
135        InputSplit[] iss = inputFormat.getSplits(newjob, 0);
136        if (iss != null && iss.length > 0) {
137          numOrigSplits += iss.length;
138          result.add(new BucketizedHiveInputSplit(iss, inputFormatClass
139              .getName()));
140        }
141      }
142    }
143    LOG.info(result.size() + " bucketized splits generated from "
144        + numOrigSplits + " original splits.");
145    return result.toArray(new BucketizedHiveInputSplit[result.size()]);
146  }
147}