[ 
https://issues.apache.org/jira/browse/NUTCH-2375?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16166488#comment-16166488
 ] 

ASF GitHub Bot commented on NUTCH-2375:
---------------------------------------

sebastian-nagel commented on a change in pull request #221: NUTCH-2375 
Upgrading nutch to use org.apache.hadoop.mapreduce
URL: https://github.com/apache/nutch/pull/221#discussion_r138919091
 
 

 ##########
 File path: src/java/org/apache/nutch/crawl/Generator.java
 ##########
 @@ -114,81 +122,34 @@ public String toString() {
   }
 
   /** Selects entries due for fetch. */
-  public static class Selector implements
-      Mapper<Text, CrawlDatum, FloatWritable, SelectorEntry>,
-      Partitioner<FloatWritable, Writable>,
-      Reducer<FloatWritable, SelectorEntry, FloatWritable, SelectorEntry> {
-    private LongWritable genTime = new 
LongWritable(System.currentTimeMillis());
-    private long curTime;
-    private long limit;
-    private long count;
-    private HashMap<String, int[]> hostCounts = new HashMap<>();
-    private int segCounts[];
-    private int maxCount;
-    private boolean byDomain = false;
-    private Partitioner<Text, Writable> partitioner = new URLPartitioner();
-    private URLFilters filters;
-    private URLNormalizers normalizers;
-    private ScoringFilters scfilters;
-    private SelectorEntry entry = new SelectorEntry();
-    private FloatWritable sortValue = new FloatWritable();
-    private boolean filter;
-    private boolean normalise;
-    private long genDelay;
-    private FetchSchedule schedule;
-    private float scoreThreshold = 0f;
-    private int intervalThreshold = -1;
-    private String restrictStatus = null;
-    private int maxNumSegments = 1;
-    private Expression expr = null;
-    private int currentsegmentnum = 1;
-    private SequenceFile.Reader[] hostdbReaders = null;
-    private Expression maxCountExpr = null;
-    private Expression fetchDelayExpr = null;
-    private JobConf conf = null;
-    
-    public void configure(JobConf job) {
-      this.conf = job;
-      curTime = job.getLong(GENERATOR_CUR_TIME, System.currentTimeMillis());
-      limit = job.getLong(GENERATOR_TOP_N, Long.MAX_VALUE)
-          / job.getNumReduceTasks();
-      maxCount = job.getInt(GENERATOR_MAX_COUNT, -1);
-      if (maxCount == -1) {
-        byDomain = false;
-      }
-      if (GENERATOR_COUNT_VALUE_DOMAIN.equals(job.get(GENERATOR_COUNT_MODE)))
-        byDomain = true;
-      filters = new URLFilters(job);
-      normalise = job.getBoolean(GENERATOR_NORMALISE, true);
-      if (normalise)
-        normalizers = new URLNormalizers(job,
-            URLNormalizers.SCOPE_GENERATE_HOST_COUNT);
-      scfilters = new ScoringFilters(job);
-      partitioner.configure(job);
-      filter = job.getBoolean(GENERATOR_FILTER, true);
-      genDelay = job.getLong(GENERATOR_DELAY, 7L) * 3600L * 24L * 1000L;
-      long time = job.getLong(Nutch.GENERATE_TIME_KEY, 0L);
-      if (time > 0)
-        genTime.set(time);
-      schedule = FetchScheduleFactory.getFetchSchedule(job);
-      scoreThreshold = job.getFloat(GENERATOR_MIN_SCORE, Float.NaN);
-      intervalThreshold = job.getInt(GENERATOR_MIN_INTERVAL, -1);
-      restrictStatus = job.get(GENERATOR_RESTRICT_STATUS, null);
-      expr = JexlUtil.parseExpression(job.get(GENERATOR_EXPR, null));
-      maxNumSegments = job.getInt(GENERATOR_MAX_NUM_SEGMENTS, 1);
-      segCounts = new int[maxNumSegments];
-      
-      if (job.get(GENERATOR_HOSTDB) != null) {
-        try {
-          Path path = new Path(job.get(GENERATOR_HOSTDB), "current");
-          hostdbReaders = SequenceFileOutputFormat.getReaders(job, path);
-          maxCountExpr = 
JexlUtil.parseExpression(job.get(GENERATOR_MAX_COUNT_EXPR, null));
-          fetchDelayExpr = 
JexlUtil.parseExpression(job.get(GENERATOR_FETCH_DELAY_EXPR, null));
-        } catch (IOException e) {
-          LOG.error("Error reading HostDB because {}", e.getMessage());
-        }
-      }
-    }
+  public static class Selector extends
+      Partitioner<FloatWritable, Writable> {
+    private static LongWritable genTime = new 
LongWritable(System.currentTimeMillis());
+    private static long curTime;
+    private static long limit;
+    private static int segCounts[];
+    private static int maxCount;
+    private static boolean byDomain = false;
+    private static URLFilters filters;
+    private static URLNormalizers normalizers;
+    private static ScoringFilters scfilters;
+    private static SelectorEntry entry = new SelectorEntry();
+    private static FloatWritable sortValue = new FloatWritable();
+    private static boolean filter;
+    private static boolean normalise;
+    private static long genDelay;
+    private static FetchSchedule schedule;
+    private static float scoreThreshold = 0f;
+    private static int intervalThreshold = -1;
+    private static String restrictStatus = null;
+    private static int maxNumSegments = 1;
+    private static Expression expr = null;
+    private static MapFile.Reader[] hostdbReaders = null;
+    private static Expression maxCountExpr = null;
+    private static Expression fetchDelayExpr = null;
+    private static Configuration config;
+
 
 Review comment:
   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 is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> Upgrade the code base from org.apache.hadoop.mapred to 
> org.apache.hadoop.mapreduce
> ----------------------------------------------------------------------------------
>
>                 Key: NUTCH-2375
>                 URL: https://issues.apache.org/jira/browse/NUTCH-2375
>             Project: Nutch
>          Issue Type: Improvement
>          Components: deployment
>            Reporter: Omkar Reddy
>
> Nutch is still using the deprecated org.apache.hadoop.mapred dependency which 
> has been deprecated. It need to be updated to org.apache.hadoop.mapreduce 
> dependency. 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to