Skip to content

Commit 7dcc2f4

Browse files
NUTCH-2455 Use secondary sorting for memory-efficient HostDb integration in Generator
Make HostDatum in SelectorEntry optional for faster serialization
1 parent fe29303 commit 7dcc2f4

1 file changed

Lines changed: 12 additions & 2 deletions

File tree

src/java/org/apache/nutch/crawl/Generator.java

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -60,6 +60,7 @@
6060
import org.apache.hadoop.fs.FileStatus;
6161
import org.apache.hadoop.fs.FileSystem;
6262
import org.apache.hadoop.fs.Path;
63+
import org.apache.hadoop.io.BooleanWritable;
6364
import org.apache.hadoop.io.FloatWritable;
6465
import org.apache.hadoop.io.IntWritable;
6566
import org.apache.hadoop.io.LongWritable;
@@ -158,12 +159,14 @@ public static class SelectorEntry implements Writable {
158159
public Text url;
159160
public CrawlDatum datum;
160161
public IntWritable segnum;
162+
public BooleanWritable hasHostDatum;
161163
public HostDatum hostdatum;
162164

163165
public SelectorEntry() {
164166
url = new Text();
165167
datum = new CrawlDatum();
166168
segnum = new IntWritable(0);
169+
hasHostDatum = new BooleanWritable(false);
167170
hostdatum = new HostDatum();
168171
}
169172

@@ -172,15 +175,21 @@ public void readFields(DataInput in) throws IOException {
172175
url.readFields(in);
173176
datum.readFields(in);
174177
segnum.readFields(in);
175-
hostdatum.readFields(in);
178+
hasHostDatum.readFields(in);
179+
if (hasHostDatum.get()) {
180+
hostdatum.readFields(in);
181+
}
176182
}
177183

178184
@Override
179185
public void write(DataOutput out) throws IOException {
180186
url.write(out);
181187
datum.write(out);
182188
segnum.write(out);
183-
hostdatum.write(out);
189+
hasHostDatum.write(out);
190+
if (hasHostDatum.get()) {
191+
hostdatum.write(out);
192+
}
184193
}
185194

186195
@Override
@@ -324,6 +333,7 @@ public static class HostDbReaderMapper
324333
public void map(Text hostname, HostDatum value, Context context)
325334
throws IOException, InterruptedException {
326335
SelectorEntry hostDataSelector = new SelectorEntry();
336+
hostDataSelector.hasHostDatum.set(true);
327337
try {
328338
hostDataSelector.hostdatum = (HostDatum) value.clone();
329339
} catch (CloneNotSupportedException e) {

0 commit comments

Comments
 (0)