forked from apache/nutch
/
GeneratorReducer.java
108 lines (97 loc) · 3.66 KB
/
GeneratorReducer.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
/*******************************************************************************
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
******************************************************************************/
package org.apache.nutch.crawl;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.net.MalformedURLException;
import org.apache.avro.util.Utf8;
import org.apache.hadoop.conf.Configuration;
import org.apache.nutch.crawl.GeneratorJob.SelectorEntry;
import org.apache.nutch.fetcher.FetcherJob.FetcherMapper;
import org.apache.nutch.storage.Mark;
import org.apache.nutch.storage.WebPage;
import org.apache.nutch.util.TableUtil;
import org.apache.nutch.util.URLUtil;
import org.apache.gora.mapreduce.GoraReducer;
/** Reduce class for generate
*
* The #reduce() method write a random integer to all generated URLs. This random
* number is then used by {@link FetcherMapper}.
*
*/
public class GeneratorReducer
extends GoraReducer<SelectorEntry, WebPage, String, WebPage> {
private long limit;
private long maxCount;
private long count = 0;
private boolean byDomain = false;
private Map<String, Integer> hostCountMap = new HashMap<String, Integer>();
private Utf8 batchId;
@Override
protected void reduce(SelectorEntry key, Iterable<WebPage> values,
Context context) throws IOException, InterruptedException {
for (WebPage page : values) {
if (maxCount > 0) {
String hostordomain;
if (byDomain) {
hostordomain = URLUtil.getDomainName(key.url);
} else {
hostordomain = URLUtil.getHost(key.url);
}
Integer hostCount = hostCountMap.get(hostordomain);
if (hostCount == null) {
hostCountMap.put(hostordomain, 0);
hostCount = 0;
}
if (hostCount >= maxCount) {
return;
}
hostCountMap.put(hostordomain, hostCount + 1);
}
if (count >= limit) {
return;
}
Mark.GENERATE_MARK.putMark(page, batchId);
try {
context.write(TableUtil.reverseUrl(key.url), page);
} catch (MalformedURLException e) {
continue;
}
context.getCounter("Generator", "GENERATE_MARK").increment(1);
count++;
}
}
@Override
protected void setup(Context context)
throws IOException, InterruptedException {
Configuration conf = context.getConfiguration();
long totalLimit = conf.getLong(GeneratorJob.GENERATOR_TOP_N, Long.MAX_VALUE);
if (totalLimit == Long.MAX_VALUE) {
limit = Long.MAX_VALUE;
} else {
limit = totalLimit / context.getNumReduceTasks();
}
maxCount = conf.getLong(GeneratorJob.GENERATOR_MAX_COUNT, -2);
batchId = new Utf8(conf.get(GeneratorJob.BATCH_ID));
String countMode =
conf.get(GeneratorJob.GENERATOR_COUNT_MODE, GeneratorJob.GENERATOR_COUNT_VALUE_HOST);
if (countMode.equals(GeneratorJob.GENERATOR_COUNT_VALUE_DOMAIN)) {
byDomain = true;
}
}
}