You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@nutch.apache.org by lewismc <gi...@git.apache.org> on 2016/03/02 05:23:45 UTC

[GitHub] nutch pull request: NUTCH-2184 Enable IndexingJob to function with...

GitHub user lewismc opened a pull request:

    https://github.com/apache/nutch/pull/95

    NUTCH-2184 Enable IndexingJob to function with no crawldb

    OK folks, this issue addresses https://issues.apache.org/jira/browse/NUTCH-2184 by
     * rebasing the [NUTCH-2184v2.patch](https://issues.apache.org/jira/secure/attachment/12784260/NUTCH-2184v2.patch) against master branch
     * making the IndexerMapReduceMapper and IndexerMapReduceReducer in IndexerMapReduce code explicit so that these functions can be tested
     * adding in some mrunit tests for testing the IndexerMapReduceMapper and IndexerMapReduceReducer
     * removing some trivial imports which are unsed
     * formatting ivy.xml which has somehow (again) become a dogs dinner
     * adding default constructor to NutchIndexAction()
    
    Any questions, then please let me know. I would really appreciate if people could pull this code and try it out within your test or local environment.
    Thanks, also thanks Markus for the original suggestions for tests, etc.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/lewismc/nutch NUTCH-2184

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nutch/pull/95.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #95
    
----
commit c4429eb7e4a33fc619cea5e5d6c26f54969e4f55
Author: Lewis John McGibbney <le...@jpl.nasa.gov>
Date:   2016-03-02T04:21:52Z

    NUTCH-2184 Enable IndexingJob to function with no crawldb

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nutch pull request: NUTCH-2184 Enable IndexingJob to function with...

Posted by sebastian-nagel <gi...@git.apache.org>.
Github user sebastian-nagel commented on a diff in the pull request:

    https://github.com/apache/nutch/pull/95#discussion_r54779042
  
    --- Diff: src/java/org/apache/nutch/indexer/IndexerMapReduce.java ---
    @@ -166,235 +145,310 @@ private String filterUrl(String url) {
         return url;
       }
     
    -  public void map(Text key, Writable value,
    -      OutputCollector<Text, NutchWritable> output, Reporter reporter)
    -          throws IOException {
    +  /**
    +   * Implementation of {@link org.apache.hadoop.mapred.Mapper}
    +   * which optionally normalizes then filters a URL before simply
    +   * collecting key and values with the keys being URLs (manifested
    +   * as {@link org.apache.hadoop.io.Text}) and the 
    +   * values as {@link org.apache.nutch.crawl.NutchWritable} instances
    +   * of {@link org.apache.nutch.crawl.CrawlDatum}.
    +   */
    +  public static class IndexerMapReduceMapper implements Mapper<Text, Writable, Text, NutchWritable> {
    +
    +    @Override
    +    public void configure(JobConf job) {
    +    }
    +
    +    public void map(Text key, Writable value,
    +        OutputCollector<Text, NutchWritable> output, Reporter reporter)
    +            throws IOException {
    +
    +      String urlString = filterUrl(normalizeUrl(key.toString()));
    +      if (urlString == null) {
    +        return;
    +      } else {
    +        key.set(urlString);
    +      }
    +
    +      output.collect(key, new NutchWritable(value));
    +    }
     
    -    String urlString = filterUrl(normalizeUrl(key.toString()));
    -    if (urlString == null) {
    -      return;
    -    } else {
    -      key.set(urlString);
    +    @Override
    +    public void close() throws IOException {
         }
     
    -    output.collect(key, new NutchWritable(value));
       }
     
    -  public void reduce(Text key, Iterator<NutchWritable> values,
    -      OutputCollector<Text, NutchIndexAction> output, Reporter reporter)
    -          throws IOException {
    -    Inlinks inlinks = null;
    -    CrawlDatum dbDatum = null;
    -    CrawlDatum fetchDatum = null;
    -    Content content = null;
    -    ParseData parseData = null;
    -    ParseText parseText = null;
    -
    -    while (values.hasNext()) {
    -      final Writable value = values.next().get(); // unwrap
    -      if (value instanceof Inlinks) {
    -        inlinks = (Inlinks) value;
    -      } else if (value instanceof CrawlDatum) {
    -        final CrawlDatum datum = (CrawlDatum) value;
    -        if (CrawlDatum.hasDbStatus(datum)) {
    -          dbDatum = datum;
    -        } else if (CrawlDatum.hasFetchStatus(datum)) {
    -          // don't index unmodified (empty) pages
    -          if (datum.getStatus() != CrawlDatum.STATUS_FETCH_NOTMODIFIED) {
    -            fetchDatum = datum;
    +  /**
    +   * Implementation of {@link org.apache.hadoop.mapred.Reducer}
    +   * which generates {@link org.apache.nutch.indexer.NutchIndexAction}'s
    +   * from combinations of various Nutch data structures. Essentially 
    +   * teh result is a key representing a URL and a value representing a
    --- End diff --
    
    typo teh -> the


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nutch pull request: NUTCH-2184 Enable IndexingJob to function with...

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/nutch/pull/95#discussion_r54788893
  
    --- Diff: src/java/org/apache/nutch/indexer/IndexerMapReduce.java ---
    @@ -52,14 +52,34 @@
     import org.apache.nutch.protocol.Content;
     import org.apache.nutch.scoring.ScoringFilterException;
     import org.apache.nutch.scoring.ScoringFilters;
    -
    -public class IndexerMapReduce extends Configured implements
    -    Mapper<Text, Writable, Text, NutchWritable>,
    -    Reducer<Text, NutchWritable, Text, NutchIndexAction> {
    +import org.apache.nutch.util.NutchConfiguration;
    +
    +/**
    + * <p>This class is typically invoked from within 
    + * {@link org.apache.nutch.indexer.IndexingJob}
    + * and handles all MapReduce functionality required
    + * when undertaking indexing.</p>
    + * <p>This is a consequence of one or more indexing plugins 
    + * being invoked which extend 
    + * {@link org.apache.nutch.indexer.IndexWriter}.</p>
    + * <p>See 
    + * {@link org.apache.nutch.indexer.IndexerMapReduce#initMRJob(Path, Path, Collection, JobConf, boolean)}
    + * for details on the specific data structures and parameters required for indexing.</p>
    + *
    + */
    +public class IndexerMapReduce {
     
       public static final Logger LOG = LoggerFactory
           .getLogger(IndexerMapReduce.class);
     
    +  // using normalizers and/or filters
    +  private static boolean normalize = false;
    +  private static boolean filter = false;
    +
    +  // url normalizers, filters and job configuration
    +  private static URLNormalizers urlNormalizers;
    +  private static URLFilters urlFilters;
    --- End diff --
    
    Thanks @sebastian-nagel, you suggest we create the variables within the mapper and reducer respectively?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nutch pull request: NUTCH-2184 Enable IndexingJob to function with...

Posted by naegelejd <gi...@git.apache.org>.
Github user naegelejd commented on a diff in the pull request:

    https://github.com/apache/nutch/pull/95#discussion_r64957448
  
    --- Diff: src/java/org/apache/nutch/indexer/IndexingJob.java ---
    @@ -155,43 +161,146 @@ public void index(Path crawlDb, Path linkDb, List<Path> segments,
                 counter.getName());
           }
           long end = System.currentTimeMillis();
    -      LOG.info("Indexer: finished at " + sdf.format(end) + ", elapsed: "
    -          + TimingUtil.elapsedTime(start, end));
    +      LOG.info("Indexer: finished at {}, elapsed: {}", sdf.format(end),
    +          TimingUtil.elapsedTime(start, end));
         } finally {
           FileSystem.get(job).delete(tmp, true);
         }
       }
     
       public int run(String[] args) throws Exception {
    -    if (args.length < 2) {
    -      System.err
    -      //.println("Usage: Indexer <crawldb> [-linkdb <linkdb>] [-params k1=v1&k2=v2...] (<segment> ... | -dir <segments>) [-noCommit] [-deleteGone] [-filter] [-normalize]");
    -      .println("Usage: Indexer <crawldb> [-linkdb <linkdb>] [-params k1=v1&k2=v2...] (<segment> ... | -dir <segments>) [-noCommit] [-deleteGone] [-filter] [-normalize] [-addBinaryContent] [-base64]");
    -      IndexWriters writers = new IndexWriters(getConf());
    -      System.err.println(writers.describe());
    -      return -1;
    -    }
    -
    -    final Path crawlDb = new Path(args[0]);
    -    Path linkDb = null;
    -
    -    final List<Path> segments = new ArrayList<Path>();
    -    String params = null;
    -
    -    boolean noCommit = false;
    -    boolean deleteGone = false;
    -    boolean filter = false;
    -    boolean normalize = false;
    -    boolean addBinaryContent = false;
    -    boolean base64 = false;
    +    // boolean options
    +    Option helpOpt = new Option("h", "help", false, "show this help message");
    +    // argument options
    +    @SuppressWarnings("static-access")
    +    Option crawldbOpt = OptionBuilder
    +    .withArgName("crawldb")
    +    .hasArg()
    +    .withDescription(
    +        "a crawldb directory to use with this tool (optional)")
    +    .create("crawldb");
    +    @SuppressWarnings("static-access")
    +    Option linkdbOpt = OptionBuilder
    +    .withArgName("linkdb")
    +    .hasArg()
    +    .withDescription(
    +        "a linkdb directory to use with this tool (optional)")
    +    .create("linkdb");
    +    @SuppressWarnings("static-access")
    +    Option paramsOpt = OptionBuilder
    +    .withArgName("params")
    +    .hasArg()
    +    .withDescription(
    +        "key value parameters to be used with this tool e.g. k1=v1&k2=v2... (optional)")
    +    .create("params");
    +    @SuppressWarnings("static-access")
    +    Option segOpt = OptionBuilder
    +    .withArgName("segment")
    +    .hasArgs()
    +    .withDescription("the segment(s) to use (either this or --segmentDir is mandatory)")
    +    .create("segment");
    +    @SuppressWarnings("static-access")
    +    Option segmentDirOpt = OptionBuilder
    +    .withArgName("segmentDir")
    +    .hasArg()
    +    .withDescription(
    +        "directory containing one or more segments to be used with this tool "
    +            + "(either this or --segment is mandatory)")
    +    .create("segmentDir");
    +    @SuppressWarnings("static-access")
    +    Option noCommitOpt = OptionBuilder
    +    .withArgName("noCommit")
    +    .withDescription(
    +        "do the commits once and for all the reducers in one go (optional)")
    --- End diff --
    
    This description is backward: the "-noCommit" option tells the Indexer *not* to do a final commit after the job finishes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nutch pull request: NUTCH-2184 Enable IndexingJob to function with...

Posted by sebastian-nagel <gi...@git.apache.org>.
Github user sebastian-nagel commented on a diff in the pull request:

    https://github.com/apache/nutch/pull/95#discussion_r54782218
  
    --- Diff: src/java/org/apache/nutch/indexer/IndexerMapReduce.java ---
    @@ -166,235 +145,310 @@ private String filterUrl(String url) {
         return url;
       }
     
    -  public void map(Text key, Writable value,
    -      OutputCollector<Text, NutchWritable> output, Reporter reporter)
    -          throws IOException {
    +  /**
    +   * Implementation of {@link org.apache.hadoop.mapred.Mapper}
    +   * which optionally normalizes then filters a URL before simply
    +   * collecting key and values with the keys being URLs (manifested
    +   * as {@link org.apache.hadoop.io.Text}) and the 
    +   * values as {@link org.apache.nutch.crawl.NutchWritable} instances
    +   * of {@link org.apache.nutch.crawl.CrawlDatum}.
    +   */
    +  public static class IndexerMapReduceMapper implements Mapper<Text, Writable, Text, NutchWritable> {
    +
    +    @Override
    +    public void configure(JobConf job) {
    +    }
    +
    +    public void map(Text key, Writable value,
    +        OutputCollector<Text, NutchWritable> output, Reporter reporter)
    +            throws IOException {
    +
    +      String urlString = filterUrl(normalizeUrl(key.toString()));
    +      if (urlString == null) {
    +        return;
    +      } else {
    +        key.set(urlString);
    +      }
    +
    +      output.collect(key, new NutchWritable(value));
    +    }
     
    -    String urlString = filterUrl(normalizeUrl(key.toString()));
    -    if (urlString == null) {
    -      return;
    -    } else {
    -      key.set(urlString);
    +    @Override
    +    public void close() throws IOException {
         }
     
    -    output.collect(key, new NutchWritable(value));
       }
     
    -  public void reduce(Text key, Iterator<NutchWritable> values,
    -      OutputCollector<Text, NutchIndexAction> output, Reporter reporter)
    -          throws IOException {
    -    Inlinks inlinks = null;
    -    CrawlDatum dbDatum = null;
    -    CrawlDatum fetchDatum = null;
    -    Content content = null;
    -    ParseData parseData = null;
    -    ParseText parseText = null;
    -
    -    while (values.hasNext()) {
    -      final Writable value = values.next().get(); // unwrap
    -      if (value instanceof Inlinks) {
    -        inlinks = (Inlinks) value;
    -      } else if (value instanceof CrawlDatum) {
    -        final CrawlDatum datum = (CrawlDatum) value;
    -        if (CrawlDatum.hasDbStatus(datum)) {
    -          dbDatum = datum;
    -        } else if (CrawlDatum.hasFetchStatus(datum)) {
    -          // don't index unmodified (empty) pages
    -          if (datum.getStatus() != CrawlDatum.STATUS_FETCH_NOTMODIFIED) {
    -            fetchDatum = datum;
    +  /**
    +   * Implementation of {@link org.apache.hadoop.mapred.Reducer}
    +   * which generates {@link org.apache.nutch.indexer.NutchIndexAction}'s
    +   * from combinations of various Nutch data structures. Essentially 
    +   * teh result is a key representing a URL and a value representing a
    +   * unit of indexing holding the document and action information.
    +   */
    +  public static class IndexerMapReduceReducer implements Reducer<Text, NutchWritable, Text, NutchIndexAction> {
    +
    +    private boolean skip = false;
    +    private boolean delete = false;
    +    private boolean deleteRobotsNoIndex = false;
    +    private boolean deleteSkippedByIndexingFilter = false;
    +    private boolean base64 = false;
    +    private IndexingFilters filters;
    +    private ScoringFilters scfilters;
    +
    +    @Override
    +    public void configure(JobConf job) {
    +      Configuration conf = NutchConfiguration.create();
    --- End diff --
    
    JobConf extends Configuration, there should be no need to create a new Configuration object.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nutch pull request: NUTCH-2184 Enable IndexingJob to function with...

Posted by sebastian-nagel <gi...@git.apache.org>.
Github user sebastian-nagel commented on a diff in the pull request:

    https://github.com/apache/nutch/pull/95#discussion_r54781452
  
    --- Diff: src/java/org/apache/nutch/indexer/IndexerMapReduce.java ---
    @@ -52,14 +52,34 @@
     import org.apache.nutch.protocol.Content;
     import org.apache.nutch.scoring.ScoringFilterException;
     import org.apache.nutch.scoring.ScoringFilters;
    -
    -public class IndexerMapReduce extends Configured implements
    -    Mapper<Text, Writable, Text, NutchWritable>,
    -    Reducer<Text, NutchWritable, Text, NutchIndexAction> {
    +import org.apache.nutch.util.NutchConfiguration;
    +
    +/**
    + * <p>This class is typically invoked from within 
    + * {@link org.apache.nutch.indexer.IndexingJob}
    + * and handles all MapReduce functionality required
    + * when undertaking indexing.</p>
    + * <p>This is a consequence of one or more indexing plugins 
    + * being invoked which extend 
    + * {@link org.apache.nutch.indexer.IndexWriter}.</p>
    + * <p>See 
    + * {@link org.apache.nutch.indexer.IndexerMapReduce#initMRJob(Path, Path, Collection, JobConf, boolean)}
    + * for details on the specific data structures and parameters required for indexing.</p>
    + *
    + */
    +public class IndexerMapReduce {
     
       public static final Logger LOG = LoggerFactory
           .getLogger(IndexerMapReduce.class);
     
    +  // using normalizers and/or filters
    +  private static boolean normalize = false;
    +  private static boolean filter = false;
    +
    +  // url normalizers, filters and job configuration
    +  private static URLNormalizers urlNormalizers;
    +  private static URLFilters urlFilters;
    --- End diff --
    
    Why are these 4 member variables now static?
    Also, it looks weird if a static variable of the outer class is initialized in a non-static method of one inner class (IndexerMapReduceReducer.config()). The mapper class cannot be used without instantiating the reducer.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nutch pull request: NUTCH-2184 Enable IndexingJob to function with...

Posted by sebastian-nagel <gi...@git.apache.org>.
Github user sebastian-nagel commented on a diff in the pull request:

    https://github.com/apache/nutch/pull/95#discussion_r54792239
  
    --- Diff: src/java/org/apache/nutch/indexer/IndexerMapReduce.java ---
    @@ -52,14 +52,34 @@
     import org.apache.nutch.protocol.Content;
     import org.apache.nutch.scoring.ScoringFilterException;
     import org.apache.nutch.scoring.ScoringFilters;
    -
    -public class IndexerMapReduce extends Configured implements
    -    Mapper<Text, Writable, Text, NutchWritable>,
    -    Reducer<Text, NutchWritable, Text, NutchIndexAction> {
    +import org.apache.nutch.util.NutchConfiguration;
    +
    +/**
    + * <p>This class is typically invoked from within 
    + * {@link org.apache.nutch.indexer.IndexingJob}
    + * and handles all MapReduce functionality required
    + * when undertaking indexing.</p>
    + * <p>This is a consequence of one or more indexing plugins 
    + * being invoked which extend 
    + * {@link org.apache.nutch.indexer.IndexWriter}.</p>
    + * <p>See 
    + * {@link org.apache.nutch.indexer.IndexerMapReduce#initMRJob(Path, Path, Collection, JobConf, boolean)}
    + * for details on the specific data structures and parameters required for indexing.</p>
    + *
    + */
    +public class IndexerMapReduce {
     
       public static final Logger LOG = LoggerFactory
           .getLogger(IndexerMapReduce.class);
     
    +  // using normalizers and/or filters
    +  private static boolean normalize = false;
    +  private static boolean filter = false;
    +
    +  // url normalizers, filters and job configuration
    +  private static URLNormalizers urlNormalizers;
    +  private static URLFilters urlFilters;
    --- End diff --
    
    No, this would duplicate the variables. Maybe pull out the nested static classes, see this trial https://github.com/sebastian-nagel/nutch/tree/NUTCH-2184


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---