Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

NUTCH-2375 Upgrading nutch to use org.apache.hadoop.mapreduce #221

Merged
merged 9 commits into from Feb 27, 2018

Conversation

Projects
None yet
6 participants
@Omkar20895
Copy link
Contributor

commented Sep 11, 2017

  • This code has been tested locally by me.
  • Some of the discussions regarding this code can be found here.
@sebastian-nagel
Copy link
Contributor

left a comment

Hi @Omkar20895 ,

I've tried to run a local test crawl which failed while doing the first CrawlDb update due to significant changes in the directory structure of a segment. Please, fix this first and try to verify it by running a test crawl.

There are also issues reading the CrawlDb (also HostDb and Segments).

If everything is fixed I'll continue testing, including on a Hadoop cluster.

Sebastian


job.setMapperClass(CrawlDbStatMapper.class);
job.setCombinerClass(CrawlDbStatCombiner.class);
job.setReducerClass(CrawlDbStatReducer.class);

FileOutputFormat.setOutputPath(job, tmpFolder);
job.setOutputFormat(SequenceFileOutputFormat.class);
job.setOutputFormatClass(SequenceFileOutputFormat.class);

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 11, 2017

Contributor

If the job writes it's output to a sequence file ...

// reading the result
FileSystem fileSystem = tmpFolder.getFileSystem(config);
SequenceFile.Reader[] readers = SequenceFileOutputFormat.getReaders(config,
tmpFolder);
MapFile.Reader[] readers = MapFileOutputFormat.getReaders(tmpFolder, config);

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 11, 2017

Contributor

... a SequenceFile.Reader is required to read it. Otherwise reading fails with

Exception in thread "main" java.io.FileNotFoundException: File file:.../stat_tmp1505114561240/part-r-00000/data does not exist
        at org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:609)
        at org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:822)
        at org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:599)
        at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:421)
        at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1820)
        at org.apache.hadoop.io.MapFile$Reader.createDataFileReader(MapFile.java:456)
        at org.apache.hadoop.io.MapFile$Reader.open(MapFile.java:429)
        at org.apache.hadoop.io.MapFile$Reader.<init>(MapFile.java:399)
        at org.apache.hadoop.io.MapFile$Reader.<init>(MapFile.java:408)
        at org.apache.hadoop.mapreduce.lib.output.MapFileOutputFormat.getReaders(MapFileOutputFormat.java:98)
        at org.apache.nutch.crawl.CrawlDbReader.processStatJobHelper(CrawlDbReader.java:402)
        at org.apache.nutch.crawl.CrawlDbReader.processStatJob(CrawlDbReader.java:444)
        at org.apache.nutch.crawl.CrawlDbReader.run(CrawlDbReader.java:740)
        at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:70)
        at org.apache.nutch.crawl.CrawlDbReader.main(CrawlDbReader.java:792)

This comment has been minimized.

Copy link
@Omkar20895

Omkar20895 Sep 11, 2017

Author Contributor

@sebastian-nagel SequenceFileOutputFormat does not have the sub-routine in the upgrade(new API). One of the things that I can do is replicate the SequenceFileOutputFormat.getReaders(of the old API) in a separate util file in org/apache/nutch/util/ please let me know your thoughts in it.

The implementation of the old API getReaders can be found here. Thanks.

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 11, 2017

Contributor

Yes, reimplementing the getReaders method seems the only way, at least, I didn't find another one.

@@ -205,7 +205,7 @@ private void readHostDb(Path hostDb, Path output, boolean dumpHomepages, boolean

private void getHostDbRecord(Path hostDb, String host) throws Exception {
Configuration conf = getConf();
SequenceFile.Reader[] readers = SequenceFileOutputFormat.getReaders(conf, hostDb);
MapFile.Reader[] readers = MapFileOutputFormat.getReaders(hostDb, conf);

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 11, 2017

Contributor

If HostDb isn't changed to be stored as MapFile must still use a SequenceFile reader, see comments in CrawlDbReader.

SequenceFile.Reader[] readers = SequenceFileOutputFormat.getReaders(
getConf(), new Path(segment, CrawlDatum.GENERATE_DIR_NAME));
MapFile.Reader[] readers = MapFileOutputFormat.getReaders(
new Path(segment, CrawlDatum.GENERATE_DIR_NAME), getConf());

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 11, 2017

Contributor

crawl_generate is stored as SequenceFile (and must be because it's not ordered by key) - must still use a SequenceFile reader.

String name = context.getJobName();//getTaskAttemptID().toString();
Path dir = FileOutputFormat.getOutputPath(context);
FileSystem fs = dir.getFileSystem(context.getConfiguration());
Path out = FileOutputFormat.getOutputPath(context);

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 11, 2017

Contributor

This will change the output folder structure and probably will cause collisions of output folders if run in distributed mode (on a Hadoop cluster). The directory tree of a segment should look as before:

crawl/segments/20170816093452/                                                                                                                                
|-- content                                                                                                                                                   
|   `-- part-00000
|       |-- data
|       `-- index
|-- crawl_fetch
|   `-- part-00000
|       |-- data
|       `-- index
|-- crawl_generate
|   `-- part-00000
|-- crawl_parse
|   `-- part-00000
|-- parse_data
|   `-- part-00000
|       |-- data
|       `-- index
`-- parse_text
    `-- part-00000
        |-- data
        `-- index

There will be changes due to the MapReduce upgrade (part-xxxxx -> part-r-xxxxx). The tree is now

crawl/segments/20170911103223/
|-- content
|   `-- FetchData
|       |-- data
|       `-- index
|-- crawl_fetch
|   `-- FetchData
|       |-- data
|       `-- index
|-- crawl_generate
|   `-- part-r-00000
|-- crawl_parse
|   `-- parse\ crawl
|       `-- segments
|           `-- 20170911103223
|-- parse_data
|   `-- parse\ crawl
|       `-- segments
|           `-- 20170911103223
|               |-- data
|               `-- index
`-- parse_text
    `-- parse\ crawl
        `-- segments
            `-- 20170911103223
                |-- data
                `-- index

which makes a crawl failing, e.g. with

CrawlDb update: java.io.FileNotFoundException: File file:.../crawl/segments/20170911103223/crawl_parse/parse crawl/data does not exist
        at org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:609)
        at org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:822)
        at org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:599)
        at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:421)
@Omkar20895

This comment has been minimized.

Copy link
Contributor Author

commented Sep 11, 2017

@sebastian-nagel please take a look at my latest commit. I have tested it locally and the build is running successfully. The directory hierarchy seems to be fixed with this commit. Thanks.

@sebastian-nagel

This comment has been minimized.

Copy link
Contributor

commented Sep 11, 2017

Thanks, @Omkar20895. Could you also add the class file SegmentReaderUtil.java, it's not included in the last commit.

@Omkar20895

This comment has been minimized.

Copy link
Contributor Author

commented Sep 11, 2017

@sebastian-nagel

This comment has been minimized.

Copy link
Contributor

commented Sep 11, 2017

Excellent! Continued testing:

  • unit tests pass
  • successfully run a small test crawl in local mode (only inject + few generate-fetch-parse-update cycles)
  • nutch readdb works now

I'll continue testing the next days (todo: indexer, linkdb, ...) and hope to test it also on a Hadoop cluster.

@lewismc

This comment has been minimized.

Copy link
Contributor

commented Sep 11, 2017

Very good @sebastian-nagel thank you for the updates

@Omkar20895

This comment has been minimized.

Copy link
Contributor Author

commented Sep 12, 2017

@sebastian-nagel Thank you very much. Do keep this thread posted with any results or errors that you face.

@Omkar20895

This comment has been minimized.

Copy link
Contributor Author

commented Sep 14, 2017

@sebastian-nagel thank you very much for the review.

@sebastian-nagel
Copy link
Contributor

left a comment

Hi @Omkar20895, regarding static variables:

  • if shared between mapper and reducer both must initialize them in distributed mode
  • concurrency may harm

Please, review the pull request completely. It's likely that I've overseen some potentially critical uses of static variables.

private static Expression maxCountExpr = null;
private static Expression fetchDelayExpr = null;
private static Configuration config;

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 14, 2017

Contributor

Sharing configuration values between mapper and reducer via static variables does not work in distributed mode because mapper and reducer running each in its own JavaVM. It's not enough to set a static variable in the mapper's setup method.
Maybe it's better get rid of the Selector class and split it into independent classes SelectorMapper, SelectorPartitioner and SelectorReducer. No shared static variables would also allow for easy checks whether a variable is properly set from the configuration.

This comment has been minimized.

Copy link
@Omkar20895

Omkar20895 Sep 14, 2017

Author Contributor

@sebastian-nagel I will look at the usage of each variable and declare them in their respective mapper and reducer classes as class variables, which are not obligated to be static. Does that sound good?

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 14, 2017

Contributor

Yes, that seems clear and safe. Of course, not to forget about setting the values from the configuration.

This comment has been minimized.

Copy link
@Omkar20895

Omkar20895 Sep 14, 2017

Author Contributor

Sharing configuration values between mapper and reducer via static variables does not work in distributed mode because mapper and reducer running each in its own JavaVM

@sebastian-nagel regarding this, if in distributed mode mapper and reducer are running on each of its own JVM how do they share the context object?

Of course, not to forget about setting the values from the configuration.

and regarding this, we set values in configuration object before running the job itself and then they are later on used by mapper and reducer to retrieve the values right? And moreover, if the mapper or reducer are setting any value in configuration object I will make sure that it is happening in setup() respectively. Please feel free to correct me if I am missing something.

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 14, 2017

Contributor

Strictly speaking, it's a different object but it represents the "same" "job context" - configuration, counters, output location, etc.

You can set properties in the configuration object in the run() method of a job. The changed configuration is then passed via setup(Context) to the mapper and reducer objects. But it does not work the other way: if the configuration is modified in the setup() method, this has no effects to other mapper or reducer instances. The setup(context) method is called only once at the beginning of a task which makes it the right place to initialize variables from the configuration, create and configure required objects and plugins.

private static boolean ignoreInternalLinks;
private static boolean ignoreExternalLinks;
private static URLFilters urlFilters;
private static URLNormalizers urlNormalizers;

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 14, 2017

Contributor

Similar to the situation in Generator: maybe it's better to make them instance variables in LinkDbMapper.

private static final Logger LOG = LoggerFactory
.getLogger(MethodHandles.lookup().lookupClass());

private int maxInlinks;
private static int maxInlinks;

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 14, 2017

Contributor

Similar to the situation in Generator: maybe it's clearer and safer to move the static variable to LinkDbMergeReducer as instance variable.

private URLNormalizers urlNormalizers;
private URLFilters urlFilters;
private static URLNormalizers urlNormalizers;
private static URLFilters urlFilters;

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 14, 2017

Contributor

Regarding static variables, see comments in Generator. Afaics, they're really shared between IndexerMapper and IndexerReducer. But then the code for initialization is also best shared to avoid duplicate code (cf. setup methods of mapper and reducer).

private static final Logger LOG = LoggerFactory
.getLogger(MethodHandles.lookup().lookupClass());
private URLFilters filters;
private URLExemptionFilters exemptionFilters;
private URLNormalizers normalizers;
private ScoringFilters scfilters;

private static final NumberFormat NUMBER_FORMAT = NumberFormat.getInstance();

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 14, 2017

Contributor

NumberFormat isn't thread-safe. Should definitely not be static!

private static URLNormalizers normalizers = null;
private static SegmentMergeFilters mergeFilters = null;
private static long sliceSize = -1;
private static long curCount = 0;

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 14, 2017

Contributor

See comments in Generator etc. regarding static variables shared between mapper and reducer classes.


private static final Logger LOG = LoggerFactory
.getLogger(MethodHandles.lookup().lookupClass());

long recNo = 0L;
static long recNo = 0L;

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 14, 2017

Contributor

See comments in Generator etc. regarding static variables shared between mapper and reducer classes.

Mapper<WritableComparable<?>, Writable, Text, NutchWritable> {
private Text newKey = new Text();
private static Text newKey = new Text();

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 14, 2017

Contributor

This will cause trouble when the mapper class is used by multiple threads (see MultithreadedMapper).

private static ScoringFilters scfilters;
private static CrawlDatum datum = new CrawlDatum();
private static Text url = new Text();
private static int defaultInterval = 0;

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 14, 2017

Contributor

See comments in Generator etc. regarding static variables shared between mapper and reducer classes.

private static ScoringFilters scfilters;
private static ParseUtil parseUtil;
private static URLNormalizers normalizers;
private static int interval;

This comment has been minimized.

Copy link
@sebastian-nagel

sebastian-nagel Sep 14, 2017

Contributor

See comments in Generator etc. regarding static variables shared between mapper and reducer classes.

@Omkar20895

This comment has been minimized.

Copy link
Contributor Author

commented Sep 18, 2017

@sebastian-nagel @lewismc please review my latest commit. Thanks.

@lewismc

This comment has been minimized.

Copy link
Contributor

commented Sep 18, 2017

@Omkar20895 thanks, did you update the pull request with the removal of all instances where wildcard's are used for imports?

@Omkar20895

This comment has been minimized.

Copy link
Contributor Author

commented Sep 20, 2017

Hi @lewismc I have opened a separate issue for this[0] and will update that separately since this patch is already a very large patch.

[0] https://issues.apache.org/jira/browse/NUTCH-2427

@Omkar20895

This comment has been minimized.

Copy link
Contributor Author

commented Sep 20, 2017

@sebastian-nagel can you please test this patch on a Hadoop cluster and let us know the results? Thanks in advance.

@sebastian-nagel

This comment has been minimized.

Copy link
Contributor

commented Sep 20, 2017

I'll do this for sure, but it may take still a couple of days. If you have time, please, test further components (linkdb, indexer Solr/ElasticSearch, webgraph) in local mode. Or try to pseudo-distributed mode (see https://wiki.apache.org/nutch/NutchHadoopSingleNodeTutorial). Sorting out issues earlier will speed-up everything. Thanks!

@Omkar20895

This comment has been minimized.

Copy link
Contributor Author

commented Sep 28, 2017

@sebastian-nagel Pardon me I could not get to this sooner, did you get a chance to test this in distributed mode? Thanks.

@lewismc

This comment has been minimized.

Copy link
Contributor

commented Sep 28, 2017

@Omkar20895 you also should be testing this in pseudo-distributed mode, if you have any issues then let me know offline

@sebastian-nagel

This comment has been minimized.

Copy link
Contributor

commented Sep 28, 2017

Hi @Omkar20895,

running a test crawl on a Hadoop cluster failed again. I've got two ClassNotFoundException-s, first in Generator, in the mapper of the last step "partitioning":

17/09/28 17:38:16 INFO crawl.Generator: Generator: Partitioning selected urls for politeness.

...

17/09/28 17:54:56 INFO mapreduce.Job: Task Id : attempt_1505293155476_0250_m_000000_98, Status : FAILED
Error: java.lang.RuntimeException: java.lang.ClassNotFoundException: Class org.apache.nutch.crawl.Generator$SelectorInverseMapper not found
        at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2203)
        at org.apache.hadoop.mapreduce.task.JobContextImpl.getMapperClass(JobContextImpl.java:196)
        at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:751)
        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:341)
        at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:164)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:422)
        at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1917)
        at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158)
Caused by: java.lang.ClassNotFoundException: Class org.apache.nutch.crawl.Generator$SelectorInverseMapper not found
        at org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2109)
        at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2201)
        ... 8 more

17/09/28 17:55:07 INFO mapreduce.Job:  map 100% reduce 100%
17/09/28 17:55:07 INFO mapreduce.Job: Job job_1505293155476_0250 failed with state FAILED due to: Task failed task_1505293155476_0250_m_000000
Job failed as tasks failed. failedMaps:1 failedReduces:0

17/09/28 17:55:07 INFO mapreduce.Job: Counters: 9
        Job Counters 
                Failed map tasks=100
                Killed reduce tasks=2
                Launched map tasks=100
                Other local map tasks=100
                Total time spent by all maps in occupied slots (ms)=2073900
                Total time spent by all reduces in occupied slots (ms)=0
                Total time spent by all map tasks (ms)=691300
                Total vcore-milliseconds taken by all map tasks=691300
                Total megabyte-milliseconds taken by all map tasks=2123673600
17/09/28 17:55:07 INFO crawl.Generator: Generator: finished at 2017-09-28 17:55:07, elapsed: 00:28:26

Second, in the mapper of the updatedb tool:

17/09/28 18:14:22 INFO crawl.CrawlDb: CrawlDb update: starting at 2017-09-28 18:14:22

...

17/09/28 18:29:06 INFO mapreduce.Job: Task Id : attempt_1505293155476_0253_m_000003_98, Status : FAILED
Error: java.lang.RuntimeException: java.lang.ClassNotFoundException: Class org.apache.nutch.crawl.CrawlDbFilter not found
        at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2203)
        at org.apache.hadoop.mapreduce.task.JobContextImpl.getMapperClass(JobContextImpl.java:196)
        at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:751)
        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:341)
        at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:164)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:422)
        at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1917)
        at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158)
Caused by: java.lang.ClassNotFoundException: Class org.apache.nutch.crawl.CrawlDbFilter not found
        at org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2109)
        at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2201)
        ... 8 more

17/09/28 18:29:07 INFO mapreduce.Job:  map 33% reduce 100%
17/09/28 18:29:08 INFO mapreduce.Job:  map 100% reduce 100%
17/09/28 18:29:08 INFO mapreduce.Job: Job job_1505293155476_0253 failed with state FAILED due to: Task failed task_1505293155476_0253_m_000001
Job failed as tasks failed. failedMaps:1 failedReduces:0

...

17/09/28 18:29:08 INFO crawl.CrawlDb: CrawlDb update: finished at 2017-09-28 18:29:08, elapsed: 00:14:45

I have no glue why, the classes are in the job file. It could be because of some incompatibilities when running on Cloudera CDH 5.12.1 (Hadoop 2.6). I'll try to investigate this problem.

Meanwhile, please, have a look at another issue uncovered: although a job failed (note: a MapReduce job can be just one of multiple steps), the generate or updatedb "job" (here: one run of a tool) signalized "success" and the crawl script just continued as if there wasn't any problem. Please, always check the return value of job.waitForCompletion(...) and if it returns false:

  • perform the necessary cleanups: delete temporary data, etc.
  • make the main routine return 1

I can only second Lewis: please, try to run tests independently in local and pseudo-distributed mode. One iteration (commit/PR, test, analyze and report error) takes too long otherwise. Thanks!

@Omkar20895

This comment has been minimized.

Copy link
Contributor Author

commented Sep 29, 2017

@lewismc @sebastian-nagel you are right, I will start testing it in pseudo-distributed mode first. Thanks.

@Omkar20895

This comment has been minimized.

Copy link
Contributor Author

commented Nov 8, 2017

@sebastian-nagel I am getting the same error as mentioned above, Generator$SelectorInverseMapper class not found. I have been trying to find the reason for it. I have also verified that the class is in the job file. Any suggestions for the plausible reasons for this issue?

@sebastian-nagel

This comment has been minimized.

Copy link
Contributor

commented Nov 8, 2017

I haven't verified it, but probably that's the reason: job.setJarByClass(Generator.class) isn't specified in partitionSegment(...), cf. slides about upgrading to the new mapreduce API. That's one of the tiny little details easily overseen. Why not do this in NutchJob.getInstance()? The mapper and reducer classes should be in the same job/jar file where NutchJob is located.

@Omkar20895

This comment has been minimized.

Copy link
Contributor Author

commented Nov 9, 2017

@sebastian-nagel you are right. I overlooked the statement job.setJarByClass(<>) statement. The ClassNotFound exception is gone now. I have a couple of more errors in pseudo-distributed mode. Working on it! Thanks.

@lewismc

This comment has been minimized.

Copy link
Contributor

commented Dec 27, 2017

Hi @Omkar20895 can you please update this patch and provide a comment on what the current status is? Thank you

@Omkar20895

This comment has been minimized.

Copy link
Contributor Author

commented Jan 5, 2018

Hello, Apologies for not updating the branch I was on a vacation. I have updated the PR and now I am getting a NullPointerException and I can't quite get my head around it.

The Exception comes when I run the command : runtime/deploy/bin/nutch generate crawl/crawldb crawl/segments

The exception :
Error: java.lang.NullPointerException
at org.apache.nutch.crawl.URLPartitioner.getPartition(URLPartitioner.java:75)
at org.apache.nutch.crawl.URLPartitioner.getPartition(URLPartitioner.java:39)
at org.apache.hadoop.mapred.MapTask$NewOutputCollector.write(MapTask.java:715)
at org.apache.hadoop.mapreduce.task.TaskInputOutputContextImpl.write(TaskInputOutputContextImpl.java:89)
at org.apache.hadoop.mapreduce.lib.map.WrappedMapper$Context.write(WrappedMapper.java:112)
at org.apache.nutch.crawl.Generator$SelectorInverseMapper.map(Generator.java:538)
at org.apache.nutch.crawl.Generator$SelectorInverseMapper.map(Generator.java:531)
at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:146)
at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:787)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:341)
at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:164)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657)
at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158)

The exception occurs only in pseudo-distributed mode, the code looks fine and I do not understand the reason for this exception. I think this has something to do with execution in distributed mode of hadoop. @sebastian-nagel @lewismc please suggest me some reasons of why this might be happening so that I can work in it. Thanks.

@lewismc

This comment has been minimized.

Copy link
Contributor

commented Jan 10, 2018

We spoke offline @Omkar20895, you stated that crawls were running well with Hadoop 2.7.4 correct? If so, we should seriously consider testing this PR in the dev community.

@lewismc

This comment has been minimized.

Copy link
Contributor

commented Jan 10, 2018

Please squash the commits @Omkar20895 thanks

Omkar20895 added some commits Jan 26, 2018

@Omkar20895

This comment has been minimized.

Copy link
Contributor Author

commented Jan 26, 2018

@lewismc yes, the crawls were running fine with Hadoop-2.7.4. Everybody is welcome to test this PR out. Thanks.

Md Ashraful Islam and others added some commits Jan 11, 2018

Merge pull request #283 from smartive/NUTCH-2508
fix for NUTCH-2508 contributed by mfeltscher
Merge pull request #284 from YossiTamari/master
NUTCH-2489: Dependency collision with lucene-analyzers-common in scoring-similarity plugin
@lewismc

This comment has been minimized.

Copy link
Contributor

commented Feb 26, 2018

Folks, I would like to merge this into master as it has been sitting for quite some time. Are there any objections?

@sebastian-nagel

This comment has been minimized.

Copy link
Contributor

commented Feb 27, 2018

+1

  • afaics, everything looks fine (diff, tests pass)
  • there may be tiny bugs which appear in distributed mode only: we can fix them before the release of 1.15. I'll test the core components (inject, generate, fetch, parse, crawldb reader) during the next weeks in fully distributed mode. For me it's even easier to test the master, merging this huge branch isn't trivial.

@lewismc lewismc merged commit 54510e5 into apache:master Feb 27, 2018

@lewismc

This comment has been minimized.

Copy link
Contributor

commented Feb 27, 2018

OK folks, thank you for your work @Omkar20895 it was a pleasure working with you through the summer and I am glad that your work can be integrated and benefit the larger project community.
@sebastian-nagel we will iron out any subsequent bugs if and when they arise. Thank you for your comments..

@Omkar20895

This comment has been minimized.

Copy link
Contributor Author

commented Feb 28, 2018

Thank you @lewismc :)

@HansBrende

This comment has been minimized.

Copy link
Contributor

commented on src/java/org/apache/nutch/fetcher/FetcherThread.java in c93d908 Apr 7, 2018

Redirects are broken now, because you changed fit.url to url in all these places, but never assign fit.url back to url inside the redirect loop. Therefore, the fetcher will continue fetching the original url, not the redirect url until http.redirect.max is exceeded.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
You can’t perform that action at this time.