|
@@ -22,6 +22,7 @@ import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.net.URI;
|
|
|
import java.net.URISyntaxException;
|
|
|
+import java.net.URLEncoder;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Iterator;
|
|
@@ -549,7 +550,7 @@ public class StreamJob {
|
|
|
jobConf_.setMapperClass(c);
|
|
|
} else {
|
|
|
jobConf_.setMapperClass(PipeMapper.class);
|
|
|
- jobConf_.set("stream.map.streamprocessor", mapCmd_);
|
|
|
+ jobConf_.set("stream.map.streamprocessor", URLEncoder.encode(mapCmd_, "UTF-8"));
|
|
|
}
|
|
|
|
|
|
if (comCmd_ != null) {
|
|
@@ -558,7 +559,7 @@ public class StreamJob {
|
|
|
jobConf_.setCombinerClass(c);
|
|
|
} else {
|
|
|
jobConf_.setCombinerClass(PipeCombiner.class);
|
|
|
- jobConf_.set("stream.combine.streamprocessor", comCmd_);
|
|
|
+ jobConf_.set("stream.combine.streamprocessor", URLEncoder.encode(comCmd_, "UTF-8"));
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -570,7 +571,7 @@ public class StreamJob {
|
|
|
jobConf_.setReducerClass(c);
|
|
|
} else {
|
|
|
jobConf_.setReducerClass(PipeReducer.class);
|
|
|
- jobConf_.set("stream.reduce.streamprocessor", redCmd_);
|
|
|
+ jobConf_.set("stream.reduce.streamprocessor", URLEncoder.encode(redCmd_, "UTF-8"));
|
|
|
}
|
|
|
}
|
|
|
|