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 */
018
019 package org.apache.hadoop.mapred.lib;
020
021 import java.io.IOException;
022 import java.util.List;
023
024 import org.apache.hadoop.classification.InterfaceAudience;
025 import org.apache.hadoop.classification.InterfaceStability;
026 import org.apache.hadoop.fs.FileStatus;
027 import org.apache.hadoop.fs.PathFilter;
028 import org.apache.hadoop.mapred.InputFormat;
029 import org.apache.hadoop.mapred.InputSplit;
030 import org.apache.hadoop.mapred.JobConf;
031 import org.apache.hadoop.mapred.RecordReader;
032 import org.apache.hadoop.mapred.Reporter;
033 import org.apache.hadoop.mapreduce.Job;
034 import org.apache.hadoop.mapreduce.TaskAttemptContext;
035
036 /**
037 * An abstract {@link org.apache.hadoop.mapred.InputFormat} that returns {@link CombineFileSplit}'s
038 * in {@link org.apache.hadoop.mapred.InputFormat#getSplits(JobConf, int)} method.
039 * Splits are constructed from the files under the input paths.
040 * A split cannot have files from different pools.
041 * Each split returned may contain blocks from different files.
042 * If a maxSplitSize is specified, then blocks on the same node are
043 * combined to form a single split. Blocks that are left over are
044 * then combined with other blocks in the same rack.
045 * If maxSplitSize is not specified, then blocks from the same rack
046 * are combined in a single split; no attempt is made to create
047 * node-local splits.
048 * If the maxSplitSize is equal to the block size, then this class
049 * is similar to the default spliting behaviour in Hadoop: each
050 * block is a locally processed split.
051 * Subclasses implement {@link org.apache.hadoop.mapred.InputFormat#getRecordReader(InputSplit, JobConf, Reporter)}
052 * to construct <code>RecordReader</code>'s for <code>CombineFileSplit</code>'s.
053 * @see CombineFileSplit
054 */
055 @InterfaceAudience.Public
056 @InterfaceStability.Stable
057 public abstract class CombineFileInputFormat<K, V>
058 extends org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat<K, V>
059 implements InputFormat<K, V>{
060
061 /**
062 * default constructor
063 */
064 public CombineFileInputFormat() {
065 }
066
067 public InputSplit[] getSplits(JobConf job, int numSplits)
068 throws IOException {
069 List<org.apache.hadoop.mapreduce.InputSplit> newStyleSplits =
070 super.getSplits(new Job(job));
071 InputSplit[] ret = new InputSplit[newStyleSplits.size()];
072 for(int pos = 0; pos < newStyleSplits.size(); ++pos) {
073 org.apache.hadoop.mapreduce.lib.input.CombineFileSplit newStyleSplit =
074 (org.apache.hadoop.mapreduce.lib.input.CombineFileSplit) newStyleSplits.get(pos);
075 ret[pos] = new CombineFileSplit(job, newStyleSplit.getPaths(),
076 newStyleSplit.getStartOffsets(), newStyleSplit.getLengths(),
077 newStyleSplit.getLocations());
078 }
079 return ret;
080 }
081
082 /**
083 * Create a new pool and add the filters to it.
084 * A split cannot have files from different pools.
085 * @deprecated Use {@link #createPool(List)}.
086 */
087 @Deprecated
088 protected void createPool(JobConf conf, List<PathFilter> filters) {
089 createPool(filters);
090 }
091
092 /**
093 * Create a new pool and add the filters to it.
094 * A pathname can satisfy any one of the specified filters.
095 * A split cannot have files from different pools.
096 * @deprecated Use {@link #createPool(PathFilter...)}.
097 */
098 @Deprecated
099 protected void createPool(JobConf conf, PathFilter... filters) {
100 createPool(filters);
101 }
102
103 /**
104 * This is not implemented yet.
105 */
106 public abstract RecordReader<K, V> getRecordReader(InputSplit split,
107 JobConf job, Reporter reporter)
108 throws IOException;
109
110 // abstract method from super class implemented to return null
111 public org.apache.hadoop.mapreduce.RecordReader<K, V> createRecordReader(
112 org.apache.hadoop.mapreduce.InputSplit split,
113 TaskAttemptContext context) throws IOException {
114 return null;
115 }
116
117 /** List input directories.
118 * Subclasses may override to, e.g., select only files matching a regular
119 * expression.
120 *
121 * @param job the job to list input paths for
122 * @return array of FileStatus objects
123 * @throws IOException if zero items.
124 */
125 protected FileStatus[] listStatus(JobConf job) throws IOException {
126 List<FileStatus> result = super.listStatus(new Job(job));
127 return result.toArray(new FileStatus[result.size()]);
128 }
129
130 }