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 */
018package org.apache.hadoop.hbase.util.bulkdatagenerator;
019
020import java.io.DataInput;
021import java.io.DataOutput;
022import java.io.IOException;
023import java.util.ArrayList;
024import java.util.List;
025import org.apache.hadoop.io.NullWritable;
026import org.apache.hadoop.io.Text;
027import org.apache.hadoop.io.Writable;
028import org.apache.hadoop.mapreduce.InputFormat;
029import org.apache.hadoop.mapreduce.InputSplit;
030import org.apache.hadoop.mapreduce.JobContext;
031import org.apache.hadoop.mapreduce.RecordReader;
032import org.apache.hadoop.mapreduce.TaskAttemptContext;
033
034import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
035
036public class BulkDataGeneratorInputFormat extends InputFormat<Text, NullWritable> {
037
038  public static final String MAPPER_TASK_COUNT_KEY =
039    BulkDataGeneratorInputFormat.class.getName() + "mapper.task.count";
040
041  @Override
042  public List<InputSplit> getSplits(JobContext job) throws IOException {
043    // Get the number of mapper tasks configured
044    int mapperCount = job.getConfiguration().getInt(MAPPER_TASK_COUNT_KEY, -1);
045    Preconditions.checkArgument(mapperCount > 1, MAPPER_TASK_COUNT_KEY + " is not set.");
046
047    // Create a number of input splits equal to the number of mapper tasks
048    ArrayList<InputSplit> splits = new ArrayList<InputSplit>();
049    for (int i = 0; i < mapperCount; ++i) {
050      splits.add(new FakeInputSplit());
051    }
052    return splits;
053  }
054
055  @Override
056  public RecordReader<Text, NullWritable> createRecordReader(InputSplit split,
057    TaskAttemptContext context) throws IOException, InterruptedException {
058    BulkDataGeneratorRecordReader bulkDataGeneratorRecordReader =
059      new BulkDataGeneratorRecordReader();
060    bulkDataGeneratorRecordReader.initialize(split, context);
061    return bulkDataGeneratorRecordReader;
062  }
063
064  /**
065   * Dummy input split to be used by {@link BulkDataGeneratorRecordReader}
066   */
067  private static class FakeInputSplit extends InputSplit implements Writable {
068
069    @Override
070    public void readFields(DataInput arg0) throws IOException {
071    }
072
073    @Override
074    public void write(DataOutput arg0) throws IOException {
075    }
076
077    @Override
078    public long getLength() throws IOException, InterruptedException {
079      return 0;
080    }
081
082    @Override
083    public String[] getLocations() throws IOException, InterruptedException {
084      return new String[0];
085    }
086  }
087}