PageRenderTime 19ms CodeModel.GetById 9ms app.highlight 7ms RepoModel.GetById 1ms app.codeStats 1ms

/tags/release-0.1-rc2/hive/external/ql/src/java/org/apache/hadoop/hive/ql/index/compact/HiveCompactIndexInputFormat.java

#
Java | 150 lines | 112 code | 17 blank | 21 comment | 15 complexity | d03661ba878ce5ab50d562398b7dd908 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 */
 18package org.apache.hadoop.hive.ql.index.compact;
 19
 20import java.io.IOException;
 21import java.util.ArrayList;
 22import java.util.Iterator;
 23import java.util.Set;
 24
 25import org.apache.commons.logging.Log;
 26import org.apache.commons.logging.LogFactory;
 27import org.apache.hadoop.fs.Path;
 28import org.apache.hadoop.hive.ql.exec.Utilities;
 29import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
 30import org.apache.hadoop.hive.ql.io.HiveInputFormat;
 31import org.apache.hadoop.hive.ql.io.IOPrepareCache;
 32import org.apache.hadoop.hive.ql.metadata.HiveException;
 33import org.apache.hadoop.hive.ql.plan.PartitionDesc;
 34import org.apache.hadoop.io.SequenceFile;
 35import org.apache.hadoop.mapred.FileInputFormat;
 36import org.apache.hadoop.mapred.FileSplit;
 37import org.apache.hadoop.mapred.InputFormat;
 38import org.apache.hadoop.mapred.InputSplit;
 39import org.apache.hadoop.mapred.JobConf;
 40
 41public class HiveCompactIndexInputFormat extends HiveInputFormat {
 42
 43  public static final Log l4j = LogFactory.getLog("HiveIndexInputFormat");
 44
 45  public HiveCompactIndexInputFormat() {
 46    super();
 47  }
 48
 49  public InputSplit[] doGetSplits(JobConf job, int numSplits) throws IOException {
 50
 51    super.init(job);
 52
 53    Path[] dirs = FileInputFormat.getInputPaths(job);
 54    if (dirs.length == 0) {
 55      throw new IOException("No input paths specified in job");
 56    }
 57    JobConf newjob = new JobConf(job);
 58    ArrayList<InputSplit> result = new ArrayList<InputSplit>();
 59
 60    // for each dir, get the InputFormat, and do getSplits.
 61    for (Path dir : dirs) {
 62      PartitionDesc part = HiveFileFormatUtils
 63          .getPartitionDescFromPathRecursively(pathToPartitionInfo, dir,
 64              IOPrepareCache.get().allocatePartitionDescMap(), true);
 65      // create a new InputFormat instance if this is the first time to see this
 66      // class
 67      Class inputFormatClass = part.getInputFileFormatClass();
 68      InputFormat inputFormat = getInputFormatFromCache(inputFormatClass, job);
 69      Utilities.copyTableJobPropertiesToConf(part.getTableDesc(), newjob);
 70
 71      FileInputFormat.setInputPaths(newjob, dir);
 72      newjob.setInputFormat(inputFormat.getClass());
 73      InputSplit[] iss = inputFormat.getSplits(newjob, numSplits / dirs.length);
 74      for (InputSplit is : iss) {
 75        result.add(new HiveInputSplit(is, inputFormatClass.getName()));
 76      }
 77    }
 78    return result.toArray(new HiveInputSplit[result.size()]);
 79  }
 80  
 81  @Override
 82  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
 83    String indexFileStr = job.get("hive.index.compact.file");
 84    l4j.info("index_file is " + indexFileStr);
 85
 86    HiveCompactIndexResult hiveIndexResult = null;
 87    if (indexFileStr != null) {
 88      try {
 89        hiveIndexResult = new HiveCompactIndexResult(indexFileStr, job);
 90      } catch (HiveException e) {
 91        l4j.error("Unable to read index..");
 92        throw new IOException(e);
 93      }
 94
 95      Set<String> inputFiles = hiveIndexResult.buckets.keySet();
 96      Iterator<String> iter = inputFiles.iterator();
 97      boolean first = true;
 98      StringBuilder newInputPaths = new StringBuilder();
 99      while(iter.hasNext()) {
100        String path = iter.next();
101        if (path.trim().equalsIgnoreCase(""))
102          continue;
103        if (!first) {
104          newInputPaths.append(",");
105        } else {
106          first = false;
107        }
108        newInputPaths.append(path);
109      }
110
111      FileInputFormat.setInputPaths(job, newInputPaths.toString());
112    } else {
113      return super.getSplits(job, numSplits);
114    }
115    
116    HiveInputSplit[] splits = (HiveInputSplit[]) this.doGetSplits(job, numSplits);
117
118    ArrayList<HiveInputSplit> newSplits = new ArrayList<HiveInputSplit>(
119        numSplits);
120    for (HiveInputSplit split : splits) {
121      l4j.info("split start : " + split.getStart());
122      l4j.info("split end : " + (split.getStart() + split.getLength()));
123
124      try {
125        if (hiveIndexResult.contains(split)) {
126          // we may miss a sync here
127          HiveInputSplit newSplit = split;
128          if (split.inputFormatClassName().contains("RCFile")
129              || split.inputFormatClassName().contains("SequenceFile")) {
130            if (split.getStart() > SequenceFile.SYNC_INTERVAL) {
131              newSplit = new HiveInputSplit(new FileSplit(split.getPath(), split
132                  .getStart()
133                  - SequenceFile.SYNC_INTERVAL, split.getLength()
134                  + SequenceFile.SYNC_INTERVAL, split.getLocations()), split
135                  .inputFormatClassName());
136            }
137          }
138          newSplits.add(newSplit);
139        }
140      } catch (HiveException e) {
141        throw new RuntimeException(
142            "Unable to get metadata for input table split" + split.getPath());
143      }
144    }
145    InputSplit retA[] = newSplits.toArray((new FileSplit[newSplits.size()]));
146    l4j.info("Number of input splits: " + splits.length + " new input splits: "
147        + retA.length);
148    return retA;
149  }
150}