This is an automated email from the ASF dual-hosted git repository. snagel pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nutch.git
commit 89b16ce29f3bf6618ec2bf9df0807b24c1e40339 Author: Sebastian Nagel <sna...@apache.org> AuthorDate: Mon Oct 15 13:44:20 2018 +0200 NUTCH-2652 Fetcher launches more fetch tasks than fetch lists - properly override method getSplits(...) of FileInputFormat --- src/java/org/apache/nutch/fetcher/Fetcher.java | 37 +++++++++++++------------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/src/java/org/apache/nutch/fetcher/Fetcher.java b/src/java/org/apache/nutch/fetcher/Fetcher.java index f6584c5..fe9e71e 100644 --- a/src/java/org/apache/nutch/fetcher/Fetcher.java +++ b/src/java/org/apache/nutch/fetcher/Fetcher.java @@ -23,28 +23,24 @@ import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; -import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.Mapper; -import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.input.FileSplit; +import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; @@ -55,6 +51,8 @@ import org.apache.nutch.util.NutchConfiguration; import org.apache.nutch.util.NutchJob; import org.apache.nutch.util.NutchTool; import org.apache.nutch.util.TimingUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A queue-based fetcher. @@ -105,19 +103,20 @@ public class Fetcher extends NutchTool implements Tool { private static final Logger LOG = LoggerFactory .getLogger(MethodHandles.lookup().lookupClass()); - public static class InputFormat extends - SequenceFileInputFormat<Text, CrawlDatum> { - /** Don't split inputs, to keep things polite. */ - public InputSplit[] getSplits(JobContext job, int nSplits) throws IOException { + public static class InputFormat + extends SequenceFileInputFormat<Text, CrawlDatum> { + /** + * Don't split inputs to keep things polite - a single fetch list must be + * processed in one fetcher task. Do not split a fetch lists and assigning + * the splits to multiple parallel tasks. + */ + @Override + public List<InputSplit> getSplits(JobContext job) throws IOException { List<FileStatus> files = listStatus(job); - FileSplit[] splits = new FileSplit[files.size()]; - Iterator<FileStatus> iterator= files.listIterator(); - int index = 0; - while(iterator.hasNext()) { - index++; - FileStatus cur = iterator.next(); - splits[index] = new FileSplit(cur.getPath(), 0, cur.getLen(), - (String[]) null); + List<InputSplit> splits = new ArrayList<>(); + for (FileStatus cur : files) { + splits.add( + new FileSplit(cur.getPath(), 0, cur.getLen(), (String[]) null)); } return splits; }