Skip to content

Commit

Permalink
NUTCH-3013 Employ commons-lang3's StopWatch to simplify timing logic (a…
Browse files Browse the repository at this point in the history
  • Loading branch information
lewismc authored Oct 21, 2023
1 parent d2c3e96 commit 8431dcf
Show file tree
Hide file tree
Showing 30 changed files with 234 additions and 225 deletions.
1 change: 0 additions & 1 deletion .github/workflows/master-build.yml
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ on:
branches: [ master ]
pull_request:
branches: [ master ]


jobs:
build:
Expand Down
1 change: 1 addition & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -27,3 +27,4 @@ naivebayes-model
csvindexwriter
lib/spotbugs-*
ivy/dependency-check-ant/*
.gradle*
19 changes: 10 additions & 9 deletions src/java/org/apache/nutch/crawl/CrawlDb.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,15 @@
import java.io.File;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.TimeUnit;

import org.apache.commons.lang3.time.StopWatch;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
Expand All @@ -49,7 +50,6 @@
import org.apache.nutch.util.NutchConfiguration;
import org.apache.nutch.util.NutchJob;
import org.apache.nutch.util.NutchTool;
import org.apache.nutch.util.TimingUtil;

/**
* This class takes the output of the fetcher and updates the crawldb
Expand Down Expand Up @@ -85,10 +85,11 @@ public void update(Path crawlDb, Path[] segments, boolean normalize,
public void update(Path crawlDb, Path[] segments, boolean normalize,
boolean filter, boolean additionsAllowed, boolean force)
throws IOException, InterruptedException, ClassNotFoundException {
Path lock = lock(getConf(), crawlDb, force);

SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
long start = System.currentTimeMillis();
StopWatch stopWatch = new StopWatch();
stopWatch.start();

Path lock = lock(getConf(), crawlDb, force);

Job job = CrawlDb.createJob(getConf(), crawlDb);
Configuration conf = job.getConfiguration();
Expand All @@ -98,7 +99,7 @@ public void update(Path crawlDb, Path[] segments, boolean normalize,

boolean url404Purging = conf.getBoolean(CRAWLDB_PURGE_404, false);

LOG.info("CrawlDb update: starting at {}", sdf.format(start));
LOG.info("CrawlDb update: starting");
LOG.info("CrawlDb update: db: {}", crawlDb);
LOG.info("CrawlDb update: segments: {}", Arrays.asList(segments));
LOG.info("CrawlDb update: additions allowed: {}", additionsAllowed);
Expand Down Expand Up @@ -151,9 +152,9 @@ public void update(Path crawlDb, Path[] segments, boolean normalize,
urlsFiltered);
}

long end = System.currentTimeMillis();
LOG.info("CrawlDb update: finished at " + sdf.format(end) + ", elapsed: "
+ TimingUtil.elapsedTime(start, end));
stopWatch.stop();
LOG.info("CrawlDb update: finished, elapsed: {} ms", stopWatch.getTime(
TimeUnit.MILLISECONDS));
}

/*
Expand Down
16 changes: 8 additions & 8 deletions src/java/org/apache/nutch/crawl/CrawlDbMerger.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,12 @@

import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Map.Entry;
import java.util.Random;
import java.util.concurrent.TimeUnit;

import org.apache.commons.lang3.time.StopWatch;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -44,7 +45,6 @@
import org.apache.hadoop.util.ToolRunner;
import org.apache.nutch.util.NutchConfiguration;
import org.apache.nutch.util.NutchJob;
import org.apache.nutch.util.TimingUtil;

/**
* This tool merges several CrawlDb-s into one, optionally filtering URLs
Expand Down Expand Up @@ -129,9 +129,9 @@ public void merge(Path output, Path[] dbs, boolean normalize, boolean filter)
throws Exception {
Path lock = CrawlDb.lock(getConf(), output, false);

SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
long start = System.currentTimeMillis();
LOG.info("CrawlDb merge: starting at {}", sdf.format(start));
StopWatch stopWatch = new StopWatch();
stopWatch.start();
LOG.info("CrawlDb merge: starting");

Job job = createMergeJob(getConf(), output, normalize, filter);
for (int i = 0; i < dbs.length; i++) {
Expand All @@ -155,9 +155,9 @@ public void merge(Path output, Path[] dbs, boolean normalize, boolean filter)
NutchJob.cleanupAfterFailure(outPath, lock, fs);
throw e;
}
long end = System.currentTimeMillis();
LOG.info("CrawlDb merge: finished at " + sdf.format(end) + ", elapsed: "
+ TimingUtil.elapsedTime(start, end));
stopWatch.stop();
LOG.info("CrawlDb merge: finished, elapsed: {}", stopWatch.getTime(
TimeUnit.MILLISECONDS));
}

public static Job createMergeJob(Configuration conf, Path output,
Expand Down
16 changes: 8 additions & 8 deletions src/java/org/apache/nutch/crawl/DeduplicationJob.java
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,12 @@
import java.lang.invoke.MethodHandles;
import java.net.URLDecoder;
import java.nio.charset.StandardCharsets;
import java.text.SimpleDateFormat;
import java.util.HashMap;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.TimeUnit;

import org.apache.commons.lang3.time.StopWatch;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
Expand All @@ -48,7 +49,6 @@
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.apache.nutch.util.URLUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -298,9 +298,9 @@ public int run(String[] args) throws IOException {
}
}

SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
long start = System.currentTimeMillis();
LOG.info("DeduplicationJob: starting at " + sdf.format(start));
StopWatch stopWatch = new StopWatch();
stopWatch.start();
LOG.info("DeduplicationJob: starting");

Path tempDir = new Path(crawlDb, "dedup-temp-"
+ Integer.toString(new Random().nextInt(Integer.MAX_VALUE)));
Expand Down Expand Up @@ -381,9 +381,9 @@ public int run(String[] args) throws IOException {
// clean up
fs.delete(tempDir, true);

long end = System.currentTimeMillis();
LOG.info("Deduplication finished at " + sdf.format(end) + ", elapsed: "
+ TimingUtil.elapsedTime(start, end));
stopWatch.stop();
LOG.info("Deduplication finished, elapsed: {} ms", stopWatch.getTime(
TimeUnit.MILLISECONDS));

return 0;
}
Expand Down
17 changes: 9 additions & 8 deletions src/java/org/apache/nutch/crawl/Generator.java
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,9 @@
import java.util.Locale;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.TimeUnit;

import org.apache.commons.lang3.time.StopWatch;
import org.apache.hadoop.conf.Configurable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -76,7 +78,6 @@
import org.apache.nutch.util.NutchJob;
import org.apache.nutch.util.NutchTool;
import org.apache.nutch.util.SegmentReaderUtil;
import org.apache.nutch.util.TimingUtil;
import org.apache.nutch.util.URLUtil;

/**
Expand Down Expand Up @@ -821,10 +822,10 @@ public Path[] generate(Path dbDir, Path segments, int numLists, long topN,

Path lock = CrawlDb.lock(getConf(), dbDir, force);

SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
long start = System.currentTimeMillis();
LOG.info("Generator: starting at " + sdf.format(start));
LOG.info("Generator: Selecting best-scoring urls due for fetch.");
StopWatch stopWatch = new StopWatch();
stopWatch.start();
LOG.info("Generator: starting");
LOG.info("Generator: selecting best-scoring urls due for fetch.");
LOG.info("Generator: filtering: " + filter);
LOG.info("Generator: normalizing: " + norm);
if (topN != Long.MAX_VALUE) {
Expand Down Expand Up @@ -982,9 +983,9 @@ public Path[] generate(Path dbDir, Path segments, int numLists, long topN,
}
fs.delete(tempDir, true);

long end = System.currentTimeMillis();
LOG.info("Generator: finished at " + sdf.format(end) + ", elapsed: "
+ TimingUtil.elapsedTime(start, end));
stopWatch.stop();
LOG.info("Generator: finished, elapsed: {} ms", stopWatch.getTime(
TimeUnit.MILLISECONDS));

Path[] patharray = new Path[generatedSegments.size()];
return generatedSegments.toArray(patharray);
Expand Down
16 changes: 8 additions & 8 deletions src/java/org/apache/nutch/crawl/Injector.java
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
*/
package org.apache.nutch.crawl;

import org.apache.commons.lang3.time.StopWatch;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
Expand Down Expand Up @@ -45,17 +46,16 @@
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;

import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.text.SimpleDateFormat;
import java.util.HashMap;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.TimeUnit;

/**
* Injector takes a flat text file of URLs (or a folder containing text files)
Expand Down Expand Up @@ -372,10 +372,11 @@ public void inject(Path crawlDb, Path urlDir, boolean overwrite,
boolean update, boolean normalize, boolean filter,
boolean filterNormalizeAll)
throws IOException, ClassNotFoundException, InterruptedException {
SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
long start = System.currentTimeMillis();

LOG.info("Injector: starting at {}", sdf.format(start));
StopWatch stopWatch = new StopWatch();
stopWatch.start();

LOG.info("Injector: starting");
LOG.info("Injector: crawlDb: {}", crawlDb);
LOG.info("Injector: urlDir: {}", urlDir);
LOG.info("Injector: Converting injected urls to crawl db entries.");
Expand Down Expand Up @@ -479,9 +480,8 @@ public void inject(Path crawlDb, Path urlDir, boolean overwrite,
urlsPurged404);
}

long end = System.currentTimeMillis();
LOG.info("Injector: finished at {}, elapsed: {}", sdf.format(end),
TimingUtil.elapsedTime(start, end));
stopWatch.stop();
LOG.info("Injector: finished, elapsed: {} ms", stopWatch.getTime(TimeUnit.MILLISECONDS));
}
} catch (IOException | InterruptedException | ClassNotFoundException | NullPointerException e) {
LOG.error("Injector job failed: {}", e.getMessage());
Expand Down
15 changes: 8 additions & 7 deletions src/java/org/apache/nutch/crawl/LinkDb.java
Original file line number Diff line number Diff line change
Expand Up @@ -21,13 +21,14 @@
import java.lang.invoke.MethodHandles;
import java.net.MalformedURLException;
import java.net.URL;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.TimeUnit;

import org.apache.commons.lang3.time.StopWatch;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
Expand All @@ -54,7 +55,6 @@
import org.apache.nutch.util.NutchConfiguration;
import org.apache.nutch.util.NutchJob;
import org.apache.nutch.util.NutchTool;
import org.apache.nutch.util.TimingUtil;

/** Maintains an inverted link map, listing incoming links for each url. */
public class LinkDb extends NutchTool implements Tool {
Expand Down Expand Up @@ -196,9 +196,9 @@ public void invert(Path linkDb, Path[] segments, boolean normalize,
Path currentLinkDb = new Path(linkDb, CURRENT_NAME);
Configuration conf = job.getConfiguration();

SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
long start = System.currentTimeMillis();
LOG.info("LinkDb: starting at {}", sdf.format(start));
StopWatch stopWatch = new StopWatch();
stopWatch.start();
LOG.info("LinkDb: starting");
LOG.info("LinkDb: linkdb: {}", linkDb);
LOG.info("LinkDb: URL normalize: {}", normalize);
LOG.info("LinkDb: URL filter: {}", filter);
Expand Down Expand Up @@ -260,8 +260,9 @@ public void invert(Path linkDb, Path[] segments, boolean normalize,
}
LinkDb.install(job, linkDb);

long end = System.currentTimeMillis();
LOG.info("LinkDb: finished at {}, elapsed: {}", sdf.format(end), TimingUtil.elapsedTime(start, end));
stopWatch.stop();
LOG.info("LinkDb: finished, elapsed: {} ms", stopWatch.getTime(
TimeUnit.MILLISECONDS));
}

private static Job createJob(Configuration config, Path linkDb,
Expand Down
16 changes: 8 additions & 8 deletions src/java/org/apache/nutch/crawl/LinkDbMerger.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,12 @@

import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.Random;
import java.util.concurrent.TimeUnit;

import org.apache.commons.lang3.time.StopWatch;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
Expand All @@ -41,7 +42,6 @@
import org.apache.hadoop.util.ToolRunner;
import org.apache.nutch.util.NutchConfiguration;
import org.apache.nutch.util.NutchJob;
import org.apache.nutch.util.TimingUtil;

/**
* This tool merges several LinkDb-s into one, optionally filtering URLs through
Expand Down Expand Up @@ -112,9 +112,9 @@ public void reduce(Text key, Iterable<Inlinks> values, Context context)

public void merge(Path output, Path[] dbs, boolean normalize, boolean filter)
throws Exception {
SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
long start = System.currentTimeMillis();
LOG.info("LinkDb merge: starting at " + sdf.format(start));
StopWatch stopWatch = new StopWatch();
stopWatch.start();
LOG.info("LinkDb merge: starting");

Job job = createMergeJob(getConf(), output, normalize, filter);
for (int i = 0; i < dbs.length; i++) {
Expand All @@ -137,9 +137,9 @@ public void merge(Path output, Path[] dbs, boolean normalize, boolean filter)
fs.rename(FileOutputFormat.getOutputPath(job), new Path(output,
LinkDb.CURRENT_NAME));

long end = System.currentTimeMillis();
LOG.info("LinkDb merge: finished at " + sdf.format(end) + ", elapsed: "
+ TimingUtil.elapsedTime(start, end));
stopWatch.stop();
LOG.info("LinkDb merge: finished, elapsed: {} ms" + stopWatch.getTime(
TimeUnit.MILLISECONDS));
}

public static Job createMergeJob(Configuration config, Path linkDb,
Expand Down
Loading

0 comments on commit 8431dcf

Please sign in to comment.