|
@@ -693,28 +693,33 @@ public class StreamJob {
|
|
|
fmt = TextInputFormat.class;
|
|
|
} else if (inputFormatSpec_ != null) {
|
|
|
if (inputFormatSpec_.equals(TextInputFormat.class.getName())
|
|
|
- || inputFormatSpec_.equals(TextInputFormat.class.getCanonicalName())) {
|
|
|
+ || inputFormatSpec_.equals(TextInputFormat.class.getCanonicalName())
|
|
|
+ || inputFormatSpec_.equals(TextInputFormat.class.getSimpleName())) {
|
|
|
fmt = TextInputFormat.class;
|
|
|
} else if (inputFormatSpec_.equals(KeyValueTextInputFormat.class
|
|
|
.getName())
|
|
|
|| inputFormatSpec_.equals(KeyValueTextInputFormat.class
|
|
|
- .getCanonicalName())) {
|
|
|
+ .getCanonicalName())
|
|
|
+ || inputFormatSpec_.equals(KeyValueTextInputFormat.class.getSimpleName())) {
|
|
|
} else if (inputFormatSpec_.equals(SequenceFileInputFormat.class
|
|
|
.getName())
|
|
|
|| inputFormatSpec_
|
|
|
.equals(org.apache.hadoop.mapred.SequenceFileInputFormat.class
|
|
|
- .getCanonicalName())) {
|
|
|
+ .getCanonicalName())
|
|
|
+ || inputFormatSpec_
|
|
|
+ .equals(org.apache.hadoop.mapred.SequenceFileInputFormat.class.getSimpleName())) {
|
|
|
} else if (inputFormatSpec_.equals(SequenceFileAsTextInputFormat.class
|
|
|
.getName())
|
|
|
|| inputFormatSpec_.equals(SequenceFileAsTextInputFormat.class
|
|
|
- .getCanonicalName())) {
|
|
|
+ .getCanonicalName())
|
|
|
+ || inputFormatSpec_.equals(SequenceFileAsTextInputFormat.class.getSimpleName())) {
|
|
|
fmt = SequenceFileAsTextInputFormat.class;
|
|
|
} else {
|
|
|
c = StreamUtil.goodClassOrNull(inputFormatSpec_, defaultPackage);
|
|
|
if (c != null) {
|
|
|
fmt = c;
|
|
|
} else {
|
|
|
-
|
|
|
+ fail("-inputformat : class not found : " + inputFormatSpec_);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -744,7 +749,9 @@ public class StreamJob {
|
|
|
c = StreamUtil.goodClassOrNull(comCmd_, defaultPackage);
|
|
|
if (c != null) {
|
|
|
jobConf_.setCombinerClass(c);
|
|
|
- }
|
|
|
+ } else {
|
|
|
+ fail("-combiner : class not found : " + comCmd_);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
boolean reducerNone_ = false;
|
|
@@ -791,7 +798,9 @@ public class StreamJob {
|
|
|
c = StreamUtil.goodClassOrNull(outputFormatSpec_, defaultPackage);
|
|
|
if (c != null) {
|
|
|
fmt = c;
|
|
|
- }
|
|
|
+ } else {
|
|
|
+ fail("-outputformat : class not found : " + outputFormatSpec_);
|
|
|
+ }
|
|
|
}
|
|
|
if (fmt == null) {
|
|
|
fmt = TextOutputFormat.class;
|
|
@@ -802,7 +811,9 @@ public class StreamJob {
|
|
|
c = StreamUtil.goodClassOrNull(partitionerSpec_, defaultPackage);
|
|
|
if (c != null) {
|
|
|
jobConf_.setPartitionerClass(c);
|
|
|
- }
|
|
|
+ } else {
|
|
|
+ fail("-partitioner : class not found : " + partitionerSpec_);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
if (numReduceTasksSpec_!= null) {
|