|
@@ -20,14 +20,10 @@ package org.apache.hadoop.tools.mapred.lib;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
-import org.apache.hadoop.fs.FileStatus;
|
|
|
-import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.io.SequenceFile;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.tools.CopyListingFileStatus;
|
|
|
-import org.apache.hadoop.tools.DistCpConstants;
|
|
|
import org.apache.hadoop.tools.util.DistCpUtils;
|
|
|
import org.apache.hadoop.mapreduce.lib.input.SequenceFileRecordReader;
|
|
|
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
|
|
@@ -47,71 +43,27 @@ import java.io.IOException;
|
|
|
*/
|
|
|
class DynamicInputChunk<K, V> {
|
|
|
private static Log LOG = LogFactory.getLog(DynamicInputChunk.class);
|
|
|
-
|
|
|
- private static Configuration configuration;
|
|
|
- private static Path chunkRootPath;
|
|
|
- private static String chunkFilePrefix;
|
|
|
- private static int numChunksLeft = -1; // Un-initialized before 1st dir-scan.
|
|
|
- private static FileSystem fs;
|
|
|
-
|
|
|
private Path chunkFilePath;
|
|
|
private SequenceFileRecordReader<K, V> reader;
|
|
|
private SequenceFile.Writer writer;
|
|
|
+ private DynamicInputChunkContext chunkContext;
|
|
|
|
|
|
- private static void initializeChunkInvariants(Configuration config)
|
|
|
- throws IOException {
|
|
|
- configuration = config;
|
|
|
- Path listingFilePath = new Path(getListingFilePath(configuration));
|
|
|
- chunkRootPath = new Path(listingFilePath.getParent(), "chunkDir");
|
|
|
- fs = chunkRootPath.getFileSystem(configuration);
|
|
|
- chunkFilePrefix = listingFilePath.getName() + ".chunk.";
|
|
|
- }
|
|
|
-
|
|
|
- private static String getListingFilePath(Configuration configuration) {
|
|
|
- final String listingFileString = configuration.get(
|
|
|
- DistCpConstants.CONF_LABEL_LISTING_FILE_PATH, "");
|
|
|
- assert !listingFileString.equals("") : "Listing file not found.";
|
|
|
- return listingFileString;
|
|
|
- }
|
|
|
-
|
|
|
- private static boolean areInvariantsInitialized() {
|
|
|
- return chunkRootPath != null;
|
|
|
- }
|
|
|
-
|
|
|
- private DynamicInputChunk(String chunkId, Configuration configuration)
|
|
|
+ DynamicInputChunk(String chunkId, DynamicInputChunkContext chunkContext)
|
|
|
throws IOException {
|
|
|
- if (!areInvariantsInitialized())
|
|
|
- initializeChunkInvariants(configuration);
|
|
|
-
|
|
|
- chunkFilePath = new Path(chunkRootPath, chunkFilePrefix + chunkId);
|
|
|
+ this.chunkContext = chunkContext;
|
|
|
+ chunkFilePath = new Path(chunkContext.getChunkRootPath(),
|
|
|
+ chunkContext.getChunkFilePrefix() + chunkId);
|
|
|
openForWrite();
|
|
|
}
|
|
|
|
|
|
-
|
|
|
private void openForWrite() throws IOException {
|
|
|
writer = SequenceFile.createWriter(
|
|
|
- chunkFilePath.getFileSystem(configuration), configuration,
|
|
|
+ chunkContext.getFs(), chunkContext.getConfiguration(),
|
|
|
chunkFilePath, Text.class, CopyListingFileStatus.class,
|
|
|
SequenceFile.CompressionType.NONE);
|
|
|
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Factory method to create chunk-files for writing to.
|
|
|
- * (For instance, when the DynamicInputFormat splits the input-file into
|
|
|
- * chunks.)
|
|
|
- * @param chunkId String to identify the chunk.
|
|
|
- * @param configuration Configuration, describing the location of the listing-
|
|
|
- * file, file-system for the map-job, etc.
|
|
|
- * @return A DynamicInputChunk, corresponding to a chunk-file, with the name
|
|
|
- * incorporating the chunk-id.
|
|
|
- * @throws IOException Exception on failure to create the chunk.
|
|
|
- */
|
|
|
- public static DynamicInputChunk createChunkForWrite(String chunkId,
|
|
|
- Configuration configuration) throws IOException {
|
|
|
- return new DynamicInputChunk(chunkId, configuration);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Method to write records into a chunk.
|
|
|
* @param key Key from the listing file.
|
|
@@ -135,19 +87,19 @@ class DynamicInputChunk<K, V> {
|
|
|
* @throws IOException Exception on failure to reassign.
|
|
|
*/
|
|
|
public void assignTo(TaskID taskId) throws IOException {
|
|
|
- Path newPath = new Path(chunkRootPath, taskId.toString());
|
|
|
- if (!fs.rename(chunkFilePath, newPath)) {
|
|
|
+ Path newPath = new Path(chunkContext.getChunkRootPath(), taskId.toString());
|
|
|
+ if (!chunkContext.getFs().rename(chunkFilePath, newPath)) {
|
|
|
LOG.warn(chunkFilePath + " could not be assigned to " + taskId);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private DynamicInputChunk(Path chunkFilePath,
|
|
|
- TaskAttemptContext taskAttemptContext)
|
|
|
- throws IOException, InterruptedException {
|
|
|
- if (!areInvariantsInitialized())
|
|
|
- initializeChunkInvariants(taskAttemptContext.getConfiguration());
|
|
|
+ public DynamicInputChunk(Path chunkFilePath,
|
|
|
+ TaskAttemptContext taskAttemptContext,
|
|
|
+ DynamicInputChunkContext chunkContext) throws IOException,
|
|
|
+ InterruptedException {
|
|
|
|
|
|
this.chunkFilePath = chunkFilePath;
|
|
|
+ this.chunkContext = chunkContext;
|
|
|
openForRead(taskAttemptContext);
|
|
|
}
|
|
|
|
|
@@ -155,45 +107,8 @@ class DynamicInputChunk<K, V> {
|
|
|
throws IOException, InterruptedException {
|
|
|
reader = new SequenceFileRecordReader<K, V>();
|
|
|
reader.initialize(new FileSplit(chunkFilePath, 0,
|
|
|
- DistCpUtils.getFileSize(chunkFilePath, configuration), null),
|
|
|
- taskAttemptContext);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Factory method that
|
|
|
- * 1. acquires a chunk for the specified map-task attempt
|
|
|
- * 2. returns a DynamicInputChunk associated with the acquired chunk-file.
|
|
|
- * @param taskAttemptContext The attempt-context for the map task that's
|
|
|
- * trying to acquire a chunk.
|
|
|
- * @return The acquired dynamic-chunk. The chunk-file is renamed to the
|
|
|
- * attempt-id (from the attempt-context.)
|
|
|
- * @throws IOException Exception on failure.
|
|
|
- * @throws InterruptedException Exception on failure.
|
|
|
- */
|
|
|
- public static DynamicInputChunk acquire(TaskAttemptContext taskAttemptContext)
|
|
|
- throws IOException, InterruptedException {
|
|
|
- if (!areInvariantsInitialized())
|
|
|
- initializeChunkInvariants(taskAttemptContext.getConfiguration());
|
|
|
-
|
|
|
- String taskId
|
|
|
- = taskAttemptContext.getTaskAttemptID().getTaskID().toString();
|
|
|
- Path acquiredFilePath = new Path(chunkRootPath, taskId);
|
|
|
-
|
|
|
- if (fs.exists(acquiredFilePath)) {
|
|
|
- LOG.info("Acquiring pre-assigned chunk: " + acquiredFilePath);
|
|
|
- return new DynamicInputChunk(acquiredFilePath, taskAttemptContext);
|
|
|
- }
|
|
|
-
|
|
|
- for (FileStatus chunkFile : getListOfChunkFiles()) {
|
|
|
- if (fs.rename(chunkFile.getPath(), acquiredFilePath)) {
|
|
|
- LOG.info(taskId + " acquired " + chunkFile.getPath());
|
|
|
- return new DynamicInputChunk(acquiredFilePath, taskAttemptContext);
|
|
|
- }
|
|
|
- else
|
|
|
- LOG.warn(taskId + " could not acquire " + chunkFile.getPath());
|
|
|
- }
|
|
|
-
|
|
|
- return null;
|
|
|
+ DistCpUtils.getFileSize(chunkFilePath,
|
|
|
+ chunkContext.getConfiguration()), null), taskAttemptContext);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -204,19 +119,13 @@ class DynamicInputChunk<K, V> {
|
|
|
*/
|
|
|
public void release() throws IOException {
|
|
|
close();
|
|
|
- if (!fs.delete(chunkFilePath, false)) {
|
|
|
+ if (!chunkContext.getFs().delete(chunkFilePath, false)) {
|
|
|
LOG.error("Unable to release chunk at path: " + chunkFilePath);
|
|
|
- throw new IOException("Unable to release chunk at path: " + chunkFilePath);
|
|
|
+ throw new IOException("Unable to release chunk at path: " +
|
|
|
+ chunkFilePath);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- static FileStatus [] getListOfChunkFiles() throws IOException {
|
|
|
- Path chunkFilePattern = new Path(chunkRootPath, chunkFilePrefix + "*");
|
|
|
- FileStatus chunkFiles[] = fs.globStatus(chunkFilePattern);
|
|
|
- numChunksLeft = chunkFiles.length;
|
|
|
- return chunkFiles;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Getter for the chunk-file's path, on HDFS.
|
|
|
* @return The qualified path to the chunk-file.
|
|
@@ -234,14 +143,4 @@ class DynamicInputChunk<K, V> {
|
|
|
return reader;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Getter for the number of chunk-files left in the chunk-file directory.
|
|
|
- * Useful to determine how many chunks (and hence, records) are left to be
|
|
|
- * processed.
|
|
|
- * @return Before the first scan of the directory, the number returned is -1.
|
|
|
- * Otherwise, the number of chunk-files seen from the last scan is returned.
|
|
|
- */
|
|
|
- public static int getNumChunksLeft() {
|
|
|
- return numChunksLeft;
|
|
|
- }
|
|
|
}
|