package org.archive.hadoop.mapreduce;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.TaskID;
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
import org.apache.hadoop.util.ReflectionUtils;
import org.archive.hadoop.util.PartitionName;
/**
* Custom TextOutputFormat which produces output in the "zipnum" format. This is
* fairly specific to the needs of the Wayback Machine CDX "clusters".
* <p>
* Most of the heavy lifting is done by <code>ZipNumRecordWriter</code>, this
* class just sets-up the environment:
* <ul>
* <li>instantiate the Hadoop codec to compress the output</li>
* <li>create the output file</li>
* <li>create summary file</li>
* </ul>
*/
public class ZipNumOutputFormat extends TextOutputFormat<Text, Text> {
public static final int DEFAULT_ZIP_NUM_LINES = 3000;
public static final String ZIP_NUM_LINES_CONFIGURATION = "conf.zipnum.count";
public static final String ZIP_NUM_PART_MOD = "conf.zipnum.partmod";
public static final String DEFAULT_PART_MOD = "a-";
public static final String ZIP_USE_MAP_INPUT_FILENAME = "conf.zipnum.useMapInputFilename";
public static final String ZIP_SUMMARY_EXT = "conf.zipnum.summaryExt";
public static final String DEFAULT_SUMMARY_EXT = "-idx";
// public String partMod = "";
/**
* Construct a <code>ZipNumOutputFormat</code> with the default number of
* lines per compressed envelope.
*/
public ZipNumOutputFormat() {
}
/**
* Set the number of lines per compressed envelope.
*/
public static void setZipNumLineCount(Configuration conf, int count) {
conf.setInt(ZIP_NUM_LINES_CONFIGURATION, count);
}
public static int getZipNumLineCount(Configuration conf) {
return conf.getInt(ZIP_NUM_LINES_CONFIGURATION, DEFAULT_ZIP_NUM_LINES);
}
public static String getPartMod(Configuration conf) {
return conf.get(ZIP_NUM_PART_MOD, DEFAULT_PART_MOD);
}
public static String getSummaryExt(Configuration conf) {
return conf.get(ZIP_SUMMARY_EXT, DEFAULT_SUMMARY_EXT);
}
public static boolean isUseMapInputFilename(Configuration conf) {
return conf.getBoolean(ZIP_USE_MAP_INPUT_FILENAME, false);
}
/**
*
*/
@Override
public RecordWriter<Text, Text> getRecordWriter(TaskAttemptContext context)
throws IOException, InterruptedException {
Configuration conf = context.getConfiguration();
int count = getZipNumLineCount(conf);
String outputBaseName = null;
boolean useMapInputFilename = isUseMapInputFilename(conf);
if (useMapInputFilename) {
String inputFilename = conf.get("map.input.file");
if (inputFilename == null) {
throw new IOException(
"map.input.file is not set, but running with "
+ ZIP_USE_MAP_INPUT_FILENAME + " set to true");
}
outputBaseName = inputFilename;
int ext = inputFilename.lastIndexOf('.');
if (ext > 0) {
outputBaseName = inputFilename.substring(0, ext);
}
} else {
String partMod = getPartMod(conf);
outputBaseName = getPartitionName(context, partMod);
}
String summaryExt = getSummaryExt(conf);
// Obtain the compression codec from the Hadoop environment.
Class<? extends CompressionCodec> codecClass = getOutputCompressorClass(context, GzipCodec.class);
CompressionCodec codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, conf);
// System.err.println( "Using codec:" + codec.toString() );
// Use a file extension basd on the codec, don't hard-code it.
Path mainFile = getWorkFile(context, outputBaseName + codec.getDefaultExtension());
Path summaryFile = getWorkFile(context, outputBaseName + summaryExt);
FileSystem mainFs = mainFile.getFileSystem(conf);
FileSystem summaryFs = summaryFile.getFileSystem(conf);
FSDataOutputStream mainOut = mainFs.create(mainFile, false);
FSDataOutputStream summaryOut = summaryFs.create(summaryFile, false);
return new ZipNumRecordWriter(codec, mainOut, summaryOut, outputBaseName, count);
}
/**
* Get the path and filename for the output format.
*/
public Path getWorkFile(TaskAttemptContext context, String partWithExt)
throws IOException {
FileOutputCommitter committer = (FileOutputCommitter) getOutputCommitter(context);
return new Path(committer.getWorkPath(), partWithExt);
}
/**
* Retrieve partition name based on the current task ID and a custom
* "partMod".
*/
public String getPartitionName(TaskAttemptContext context, String partMod) {
TaskID taskId = context.getTaskAttemptID().getTaskID();
int partition = taskId.getId();
String basename = PartitionName.getPartitionOutputName(context.getConfiguration(), partition);
if (basename == null) {
// use default name:
basename = String.format("part-%s%05d", partMod, partition);
}
return basename;
}
}