|
@@ -23,9 +23,8 @@ import java.io.IOException;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
-import org.apache.hadoop.fs.LocalDirAllocator;
|
|
|
|
|
|
+import org.apache.hadoop.conf.Configurable;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
-import org.apache.hadoop.mapreduce.MRConfig;
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
* Manipulate the working area for the transient store for maps and reduces.
|
|
* Manipulate the working area for the transient store for maps and reduces.
|
|
@@ -38,164 +37,132 @@ import org.apache.hadoop.mapreduce.MRConfig;
|
|
*/
|
|
*/
|
|
@InterfaceAudience.Private
|
|
@InterfaceAudience.Private
|
|
@InterfaceStability.Unstable
|
|
@InterfaceStability.Unstable
|
|
-public class MapOutputFile {
|
|
|
|
|
|
+public abstract class MapOutputFile implements Configurable {
|
|
|
|
|
|
- private JobConf conf;
|
|
|
|
|
|
+ private Configuration conf;
|
|
|
|
|
|
|
|
+ static final String MAP_OUTPUT_FILENAME_STRING = "file.out";
|
|
|
|
+ static final String MAP_OUTPUT_INDEX_SUFFIX_STRING = ".index";
|
|
static final String REDUCE_INPUT_FILE_FORMAT_STRING = "%s/map_%d.out";
|
|
static final String REDUCE_INPUT_FILE_FORMAT_STRING = "%s/map_%d.out";
|
|
|
|
|
|
public MapOutputFile() {
|
|
public MapOutputFile() {
|
|
}
|
|
}
|
|
|
|
|
|
- private LocalDirAllocator lDirAlloc =
|
|
|
|
- new LocalDirAllocator(MRConfig.LOCAL_DIR);
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Return the path to local map output file created earlier
|
|
* Return the path to local map output file created earlier
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @return path
|
|
* @return path
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public Path getOutputFile()
|
|
|
|
- throws IOException {
|
|
|
|
- return lDirAlloc.getLocalPathToRead(TaskTracker.OUTPUT + Path.SEPARATOR
|
|
|
|
- + "file.out", conf);
|
|
|
|
- }
|
|
|
|
|
|
+ public abstract Path getOutputFile() throws IOException;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Create a local map output file name.
|
|
* Create a local map output file name.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @param size the size of the file
|
|
* @param size the size of the file
|
|
* @return path
|
|
* @return path
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public Path getOutputFileForWrite(long size)
|
|
|
|
- throws IOException {
|
|
|
|
- return lDirAlloc.getLocalPathForWrite(TaskTracker.OUTPUT + Path.SEPARATOR
|
|
|
|
- + "file.out", size, conf);
|
|
|
|
- }
|
|
|
|
|
|
+ public abstract Path getOutputFileForWrite(long size) throws IOException;
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Create a local map output file name on the same volume.
|
|
|
|
+ */
|
|
|
|
+ public abstract Path getOutputFileForWriteInVolume(Path existing);
|
|
|
|
|
|
/**
|
|
/**
|
|
* Return the path to a local map output index file created earlier
|
|
* Return the path to a local map output index file created earlier
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @return path
|
|
* @return path
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public Path getOutputIndexFile()
|
|
|
|
- throws IOException {
|
|
|
|
- return lDirAlloc.getLocalPathToRead(TaskTracker.OUTPUT + Path.SEPARATOR
|
|
|
|
- + "file.out.index", conf);
|
|
|
|
- }
|
|
|
|
|
|
+ public abstract Path getOutputIndexFile() throws IOException;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Create a local map output index file name.
|
|
* Create a local map output index file name.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @param size the size of the file
|
|
* @param size the size of the file
|
|
* @return path
|
|
* @return path
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public Path getOutputIndexFileForWrite(long size)
|
|
|
|
- throws IOException {
|
|
|
|
- return lDirAlloc.getLocalPathForWrite(TaskTracker.OUTPUT + Path.SEPARATOR
|
|
|
|
- + "file.out.index", size, conf);
|
|
|
|
- }
|
|
|
|
|
|
+ public abstract Path getOutputIndexFileForWrite(long size) throws IOException;
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Create a local map output index file name on the same volume.
|
|
|
|
+ */
|
|
|
|
+ public abstract Path getOutputIndexFileForWriteInVolume(Path existing);
|
|
|
|
|
|
/**
|
|
/**
|
|
* Return a local map spill file created earlier.
|
|
* Return a local map spill file created earlier.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @param spillNumber the number
|
|
* @param spillNumber the number
|
|
* @return path
|
|
* @return path
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public Path getSpillFile(int spillNumber)
|
|
|
|
- throws IOException {
|
|
|
|
- return lDirAlloc.getLocalPathToRead(TaskTracker.OUTPUT + "/spill"
|
|
|
|
- + spillNumber + ".out", conf);
|
|
|
|
- }
|
|
|
|
|
|
+ public abstract Path getSpillFile(int spillNumber) throws IOException;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Create a local map spill file name.
|
|
* Create a local map spill file name.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @param spillNumber the number
|
|
* @param spillNumber the number
|
|
* @param size the size of the file
|
|
* @param size the size of the file
|
|
* @return path
|
|
* @return path
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public Path getSpillFileForWrite(int spillNumber, long size)
|
|
|
|
- throws IOException {
|
|
|
|
- return lDirAlloc.getLocalPathForWrite(TaskTracker.OUTPUT + "/spill"
|
|
|
|
- + spillNumber + ".out", size, conf);
|
|
|
|
- }
|
|
|
|
|
|
+ public abstract Path getSpillFileForWrite(int spillNumber, long size)
|
|
|
|
+ throws IOException;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Return a local map spill index file created earlier
|
|
* Return a local map spill index file created earlier
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @param spillNumber the number
|
|
* @param spillNumber the number
|
|
* @return path
|
|
* @return path
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public Path getSpillIndexFile(int spillNumber)
|
|
|
|
- throws IOException {
|
|
|
|
- return lDirAlloc.getLocalPathToRead(TaskTracker.OUTPUT + "/spill"
|
|
|
|
- + spillNumber + ".out.index", conf);
|
|
|
|
- }
|
|
|
|
|
|
+ public abstract Path getSpillIndexFile(int spillNumber) throws IOException;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Create a local map spill index file name.
|
|
* Create a local map spill index file name.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @param spillNumber the number
|
|
* @param spillNumber the number
|
|
* @param size the size of the file
|
|
* @param size the size of the file
|
|
* @return path
|
|
* @return path
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public Path getSpillIndexFileForWrite(int spillNumber, long size)
|
|
|
|
- throws IOException {
|
|
|
|
- return lDirAlloc.getLocalPathForWrite(TaskTracker.OUTPUT + "/spill"
|
|
|
|
- + spillNumber + ".out.index", size, conf);
|
|
|
|
- }
|
|
|
|
|
|
+ public abstract Path getSpillIndexFileForWrite(int spillNumber, long size)
|
|
|
|
+ throws IOException;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Return a local reduce input file created earlier
|
|
* Return a local reduce input file created earlier
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @param mapId a map task id
|
|
* @param mapId a map task id
|
|
* @return path
|
|
* @return path
|
|
- * @throws IOException
|
|
|
|
|
|
+ * @throws IOException
|
|
*/
|
|
*/
|
|
- public Path getInputFile(int mapId)
|
|
|
|
- throws IOException {
|
|
|
|
- return lDirAlloc.getLocalPathToRead(String.format(
|
|
|
|
- REDUCE_INPUT_FILE_FORMAT_STRING, TaskTracker.OUTPUT, Integer
|
|
|
|
- .valueOf(mapId)), conf);
|
|
|
|
- }
|
|
|
|
|
|
+ public abstract Path getInputFile(int mapId) throws IOException;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Create a local reduce input file name.
|
|
* Create a local reduce input file name.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @param mapId a map task id
|
|
* @param mapId a map task id
|
|
* @param size the size of the file
|
|
* @param size the size of the file
|
|
* @return path
|
|
* @return path
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public Path getInputFileForWrite(org.apache.hadoop.mapreduce.TaskID mapId,
|
|
|
|
- long size)
|
|
|
|
- throws IOException {
|
|
|
|
- return lDirAlloc.getLocalPathForWrite(String.format(
|
|
|
|
- REDUCE_INPUT_FILE_FORMAT_STRING, TaskTracker.OUTPUT, mapId.getId()),
|
|
|
|
- size, conf);
|
|
|
|
- }
|
|
|
|
|
|
+ public abstract Path getInputFileForWrite(
|
|
|
|
+ org.apache.hadoop.mapreduce.TaskID mapId, long size) throws IOException;
|
|
|
|
|
|
/** Removes all of the files related to a task. */
|
|
/** Removes all of the files related to a task. */
|
|
- public void removeAll()
|
|
|
|
- throws IOException {
|
|
|
|
- conf.deleteLocalFiles(TaskTracker.OUTPUT);
|
|
|
|
- }
|
|
|
|
|
|
+ public abstract void removeAll() throws IOException;
|
|
|
|
|
|
|
|
+ @Override
|
|
public void setConf(Configuration conf) {
|
|
public void setConf(Configuration conf) {
|
|
- if (conf instanceof JobConf) {
|
|
|
|
- this.conf = (JobConf) conf;
|
|
|
|
- } else {
|
|
|
|
- this.conf = new JobConf(conf);
|
|
|
|
- }
|
|
|
|
|
|
+ this.conf = conf;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public Configuration getConf() {
|
|
|
|
+ return conf;
|
|
|
|
+ }
|
|
|
|
+
|
|
}
|
|
}
|