|
@@ -40,7 +40,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.io.SequenceFile;
|
|
|
-import org.apache.hadoop.io.UTF8;
|
|
|
+import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.io.Writable;
|
|
|
import org.apache.hadoop.io.WritableComparable;
|
|
|
import org.apache.hadoop.mapred.ClusterStatus;
|
|
@@ -265,12 +265,10 @@ public class CopyFiles extends ToolBase {
|
|
|
// turn off speculative execution, because DFS doesn't handle
|
|
|
// multiple writers to the same file.
|
|
|
jobConf.setSpeculativeExecution(false);
|
|
|
- jobConf.setInputKeyClass(UTF8.class);
|
|
|
- jobConf.setInputValueClass(UTF8.class);
|
|
|
jobConf.setInputFormat(SequenceFileInputFormat.class);
|
|
|
|
|
|
- jobConf.setOutputKeyClass(UTF8.class);
|
|
|
- jobConf.setOutputValueClass(UTF8.class);
|
|
|
+ jobConf.setOutputKeyClass(Text.class);
|
|
|
+ jobConf.setOutputValueClass(Text.class);
|
|
|
jobConf.setOutputFormat(SequenceFileOutputFormat.class);
|
|
|
|
|
|
jobConf.setMapperClass(DFSCopyFilesMapper.class);
|
|
@@ -332,10 +330,10 @@ public class CopyFiles extends ToolBase {
|
|
|
for(int idx=0; idx < numMaps; ++idx) {
|
|
|
Path file = new Path(inDir, "part"+idx);
|
|
|
SequenceFile.Writer writer =
|
|
|
- new SequenceFile.Writer(fileSys, file, UTF8.class, UTF8.class);
|
|
|
+ new SequenceFile.Writer(fileSys, file, Text.class, Text.class);
|
|
|
for (int ipath = idx; ipath < nFiles; ipath += numMaps) {
|
|
|
String path = (String) finalPathList.get(ipath);
|
|
|
- writer.append(new UTF8(path), new UTF8(""));
|
|
|
+ writer.append(new Text(path), new Text(""));
|
|
|
}
|
|
|
writer.close();
|
|
|
}
|
|
@@ -388,7 +386,7 @@ public class CopyFiles extends ToolBase {
|
|
|
Writable value,
|
|
|
OutputCollector out,
|
|
|
Reporter reporter) throws IOException {
|
|
|
- String src = ((UTF8) key).toString();
|
|
|
+ String src = ((Text) key).toString();
|
|
|
try {
|
|
|
copy(src, reporter);
|
|
|
} catch (IOException except) {
|
|
@@ -449,12 +447,10 @@ public class CopyFiles extends ToolBase {
|
|
|
//Setup the MR-job configuration
|
|
|
jobConf.setSpeculativeExecution(false);
|
|
|
|
|
|
- jobConf.setInputKeyClass(UTF8.class);
|
|
|
- jobConf.setInputValueClass(UTF8.class);
|
|
|
jobConf.setInputFormat(SequenceFileInputFormat.class);
|
|
|
|
|
|
- jobConf.setOutputKeyClass(UTF8.class);
|
|
|
- jobConf.setOutputValueClass(UTF8.class);
|
|
|
+ jobConf.setOutputKeyClass(Text.class);
|
|
|
+ jobConf.setOutputValueClass(Text.class);
|
|
|
jobConf.setOutputFormat(SequenceFileOutputFormat.class);
|
|
|
|
|
|
jobConf.setMapperClass(HTTPCopyFilesMapper.class);
|
|
@@ -492,8 +488,8 @@ public class CopyFiles extends ToolBase {
|
|
|
for(int i=0; i < srcPaths.length; ++i) {
|
|
|
Path ipFile = new Path(jobInputDir, "part" + i);
|
|
|
SequenceFile.Writer writer =
|
|
|
- new SequenceFile.Writer(fileSystem, ipFile, UTF8.class, UTF8.class);
|
|
|
- writer.append(new UTF8(srcPaths[i]), new UTF8(""));
|
|
|
+ new SequenceFile.Writer(fileSystem, ipFile, Text.class, Text.class);
|
|
|
+ writer.append(new Text(srcPaths[i]), new Text(""));
|
|
|
writer.close();
|
|
|
}
|
|
|
}
|
|
@@ -538,7 +534,7 @@ public class CopyFiles extends ToolBase {
|
|
|
{
|
|
|
//The url of the file
|
|
|
try {
|
|
|
- srcURI = new URI(((UTF8)key).toString());
|
|
|
+ srcURI = new URI(((Text)key).toString());
|
|
|
|
|
|
//Construct the complete destination path
|
|
|
File urlPath = new File(srcURI.getPath());
|
|
@@ -574,11 +570,11 @@ public class CopyFiles extends ToolBase {
|
|
|
" to: " + destinationPath.toString());
|
|
|
|
|
|
} catch(Exception e) {
|
|
|
- reporter.setStatus("Failed to copy from: " + (UTF8)key);
|
|
|
+ reporter.setStatus("Failed to copy from: " + (Text)key);
|
|
|
if(ignoreReadFailures) {
|
|
|
return;
|
|
|
} else {
|
|
|
- throw new IOException("Failed to copy from: " + (UTF8)key);
|
|
|
+ throw new IOException("Failed to copy from: " + (Text)key);
|
|
|
}
|
|
|
}
|
|
|
}
|