You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by ro...@apache.org on 2012/06/08 15:55:57 UTC

svn commit: r1348073 - in /mahout/trunk: bin/mahout core/src/main/java/org/apache/mahout/vectorizer/EncodedVectorsFromSequenceFiles.java integration/src/main/java/org/apache/mahout/text/SequenceFilesFromMailArchives.java

Author: robinanil
Date: Fri Jun  8 13:55:57 2012
New Revision: 1348073

URL: http://svn.apache.org/viewvc?rev=1348073&view=rev
Log:
Correcting the encoder job to set the right configuration, changing mail archives job to use abstractjob as base

Modified:
    mahout/trunk/bin/mahout
    mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/EncodedVectorsFromSequenceFiles.java
    mahout/trunk/integration/src/main/java/org/apache/mahout/text/SequenceFilesFromMailArchives.java

Modified: mahout/trunk/bin/mahout
URL: http://svn.apache.org/viewvc/mahout/trunk/bin/mahout?rev=1348073&r1=1348072&r2=1348073&view=diff
==============================================================================
--- mahout/trunk/bin/mahout (original)
+++ mahout/trunk/bin/mahout Fri Jun  8 13:55:57 2012
@@ -1,15 +1,15 @@
 #!/bin/bash
-# 
+#
 # The Mahout command script
 #
 # Environment Variables
 #
 #   MAHOUT_JAVA_HOME   The java implementation to use.  Overrides JAVA_HOME.
 #
-#   MAHOUT_HEAPSIZE    The maximum amount of heap to use, in MB. 
+#   MAHOUT_HEAPSIZE    The maximum amount of heap to use, in MB.
 #                      Default is 1000.
 #
-#   HADOOP_CONF_DIR  The location of a hadoop config directory 
+#   HADOOP_CONF_DIR  The location of a hadoop config directory
 #
 #   MAHOUT_OPTS        Extra Java runtime options.
 #
@@ -88,14 +88,14 @@ if [ "$MAHOUT_JAVA_HOME" != "" ]; then
   #echo "run java in $MAHOUT_JAVA_HOME"
   JAVA_HOME=$MAHOUT_JAVA_HOME
 fi
-  
+
 if [ "$JAVA_HOME" = "" ]; then
   echo "Error: JAVA_HOME is not set."
   exit 1
 fi
 
 JAVA=$JAVA_HOME/bin/java
-JAVA_HEAP_MAX=-Xmx3g 
+JAVA_HEAP_MAX=-Xmx3g
 
 # check envvars which might override default args
 if [ "$MAHOUT_HEAPSIZE" != "" ]; then
@@ -131,7 +131,7 @@ CLASSPATH=${CLASSPATH}:$JAVA_HOME/lib/to
 # so that filenames w/ spaces are handled correctly in loops below
 IFS=
 
-if [ $IS_CORE == 0 ] 
+if [ $IS_CORE == 0 ]
 then
   # add release dependencies to CLASSPATH
   for f in $MAHOUT_HOME/mahout-*.jar; do
@@ -215,6 +215,7 @@ if [ -x "$HADOOP_BINARY" ] ; then
 fi
 
 if [ ! -x "$HADOOP_BINARY" ] || [ "$MAHOUT_LOCAL" != "" ] ; then
+  set -x
   if [ ! -x "$HADOOP_BINARY" ] ; then
     echo "hadoop binary is not in PATH,HADOOP_HOME/bin,HADOOP_PREFIX/bin, running locally"
   elif [ "$MAHOUT_LOCAL" != "" ] ; then
@@ -231,7 +232,7 @@ if [ ! -x "$HADOOP_BINARY" ] || [ "$MAHO
     esac
 else
   echo "Running on hadoop, using $HADOOP_BINARY and HADOOP_CONF_DIR=$HADOOP_CONF_DIR"
-	
+
   if [ "$MAHOUT_JOB" = "" ] ; then
     echo "ERROR: Could not find mahout-examples-*.job in $MAHOUT_HOME or $MAHOUT_HOME/examples/target, please run 'mvn install' to create the .job file"
     exit 1

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/EncodedVectorsFromSequenceFiles.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/EncodedVectorsFromSequenceFiles.java?rev=1348073&r1=1348072&r2=1348073&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/EncodedVectorsFromSequenceFiles.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/EncodedVectorsFromSequenceFiles.java Fri Jun  8 13:55:57 2012
@@ -37,7 +37,7 @@ public final class EncodedVectorsFromSeq
   private static final Logger log = LoggerFactory.getLogger(EncodedVectorsFromSequenceFiles.class);
 
   public static void main(String[] args) throws Exception {
-    ToolRunner.run(new EncodedVectorsFromSequenceFiles(), args);
+    ToolRunner.run(new Configuration(), new EncodedVectorsFromSequenceFiles(), args);
   }
 
   @Override

Modified: mahout/trunk/integration/src/main/java/org/apache/mahout/text/SequenceFilesFromMailArchives.java
URL: http://svn.apache.org/viewvc/mahout/trunk/integration/src/main/java/org/apache/mahout/text/SequenceFilesFromMailArchives.java?rev=1348073&r1=1348072&r2=1348073&view=diff
==============================================================================
--- mahout/trunk/integration/src/main/java/org/apache/mahout/text/SequenceFilesFromMailArchives.java (original)
+++ mahout/trunk/integration/src/main/java/org/apache/mahout/text/SequenceFilesFromMailArchives.java Fri Jun  8 13:55:57 2012
@@ -17,22 +17,18 @@
 package org.apache.mahout.text;
 
 import com.google.common.io.Closeables;
-import org.apache.commons.cli2.CommandLine;
-import org.apache.commons.cli2.Group;
-import org.apache.commons.cli2.Option;
-import org.apache.commons.cli2.OptionException;
+
 import org.apache.commons.cli2.builder.ArgumentBuilder;
 import org.apache.commons.cli2.builder.DefaultOptionBuilder;
 import org.apache.commons.cli2.builder.GroupBuilder;
-import org.apache.commons.cli2.commandline.Parser;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.SequenceFile;
-import org.apache.mahout.common.CommandLineUtil;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.mahout.common.AbstractJob;
 import org.apache.mahout.common.commandline.DefaultOptionCreator;
-
-import org.apache.mahout.utils.email.MailProcessor;
 import org.apache.mahout.utils.email.MailOptions;
+import org.apache.mahout.utils.email.MailProcessor;
 import org.apache.mahout.utils.io.ChunkedWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,29 +44,34 @@ import java.util.Map;
 import java.util.regex.Pattern;
 
 /**
- * Converts a directory of gzipped mail archives into SequenceFiles of specified chunkSize.
- * This class is similar to {@link SequenceFilesFromDirectory} except it uses block-compressed
- * {@link SequenceFile}s and parses out the subject and body text of each mail message into
- * a separate key/value pair.
+ * Converts a directory of gzipped mail archives into SequenceFiles of specified
+ * chunkSize. This class is similar to {@link SequenceFilesFromDirectory} except
+ * it uses block-compressed {@link SequenceFile}s and parses out the subject and
+ * body text of each mail message into a separate key/value pair.
  */
-public final class SequenceFilesFromMailArchives {
+public final class SequenceFilesFromMailArchives extends AbstractJob {
 
-  private static final Logger log = LoggerFactory.getLogger(SequenceFilesFromMailArchives.class);
+  private static final Logger log = LoggerFactory.getLogger(
+      SequenceFilesFromMailArchives.class);
 
   public void createSequenceFiles(MailOptions options) throws IOException {
-    ChunkedWriter writer = new ChunkedWriter(new Configuration(), options.getChunkSize(), new Path(options.getOutputDir()));
-    MailProcessor processor = new MailProcessor(options, options.getPrefix(), writer);
+    ChunkedWriter writer = new ChunkedWriter(
+        getConf(), options.getChunkSize(), new Path(options.getOutputDir()));
+    MailProcessor processor = new MailProcessor(
+        options, options.getPrefix(), writer);
     try {
       if (options.getInput().isDirectory()) {
-        PrefixAdditionFilter filter = new PrefixAdditionFilter(processor, writer);
+        PrefixAdditionFilter filter = new PrefixAdditionFilter(
+            processor, writer);
         options.getInput().listFiles(filter);
-        log.info("Parsed {} messages from {}", filter.getMessageCount(), options.getInput().getAbsolutePath());
+        log.info("Parsed {} messages from {}", filter.getMessageCount(),
+            options.getInput().getAbsolutePath());
       } else {
         long start = System.currentTimeMillis();
         long cnt = processor.parseMboxLineByLine(options.getInput());
         long finish = System.currentTimeMillis();
-        log.info("Parsed {} messages from {} in time: {}",
-                 new Object[] { cnt, options.getInput().getAbsolutePath(), (finish - start) });
+        log.info("Parsed {} messages from {} in time: {}", new Object[] {
+            cnt, options.getInput().getAbsolutePath(), finish - start});
       }
     } finally {
       Closeables.closeQuietly(writer);
@@ -96,11 +97,13 @@ public final class SequenceFilesFromMail
     public boolean accept(File current) {
       if (current.isDirectory()) {
         log.info("At {}", current.getAbsolutePath());
-        PrefixAdditionFilter nested = new PrefixAdditionFilter(new MailProcessor(
-            processor.getOptions(), processor.getPrefix() + File.separator + current.getName(), writer), writer);
+        PrefixAdditionFilter nested = new PrefixAdditionFilter(
+            new MailProcessor(processor.getOptions(), processor.getPrefix()
+                + File.separator + current.getName(), writer), writer);
         current.listFiles(nested);
         long dirCount = nested.getMessageCount();
-        log.info("Parsed {} messages from directory {}", dirCount, current.getAbsolutePath());
+        log.info("Parsed {} messages from directory {}", dirCount,
+            current.getAbsolutePath());
         messageCount += dirCount;
       } else {
         try {
@@ -114,130 +117,140 @@ public final class SequenceFilesFromMail
   }
 
   public static void main(String[] args) throws Exception {
+    ToolRunner.run(new Configuration(), new SequenceFilesFromMailArchives(), args);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
     DefaultOptionBuilder obuilder = new DefaultOptionBuilder();
     ArgumentBuilder abuilder = new ArgumentBuilder();
     GroupBuilder gbuilder = new GroupBuilder();
 
-    Option inputOpt = DefaultOptionCreator.inputOption().create();
-
-    Option outputDirOpt = DefaultOptionCreator.outputOption().create();
-
-    Option chunkSizeOpt = obuilder.withLongName("chunkSize").withArgument(
-            abuilder.withName("chunkSize").withMinimum(1).withMaximum(1).create()).withDescription(
-            "The chunkSize in MegaBytes. Defaults to 64").withShortName("chunk").create();
-
-    Option keyPrefixOpt = obuilder.withLongName("keyPrefix").withArgument(
-            abuilder.withName("keyPrefix").withMinimum(1).withMaximum(1).create()).withDescription(
-            "The prefix to be prepended to the key").withShortName("prefix").create();
-    Option charsetOpt = obuilder.withLongName("charset").withRequired(true).withArgument(
-            abuilder.withName("charset").withMinimum(1).withMaximum(1).create()).withDescription(
-            "The name of the character encoding of the input files").withShortName("c").create();
-    Option subjectOpt = obuilder.withLongName("subject").withRequired(false).
-            withDescription("Include the Mail subject as part of the text.  Default is false").withShortName("s").create();
-    Option toOpt = obuilder.withLongName("to").withRequired(false).
-            withDescription("Include the to field in the text.  Default is false").withShortName("to").create();
-    Option fromOpt = obuilder.withLongName("from").withRequired(false).
-            withDescription("Include the from field in the text.  Default is false").withShortName("from").create();
-    Option refsOpt = obuilder.withLongName("references").withRequired(false).
-            withDescription("Include the references field in the text.  Default is false").withShortName("refs").create();
-    Option bodyOpt = obuilder.withLongName("body").withRequired(false).
-            withDescription("Include the body in the output.  Default is false").withShortName("b").create();
-    Option quotedOpt = obuilder.withLongName("stripQuoted").withRequired(false).
-            withDescription("Strip (remove) quoted email text in the body.  Default is false").withShortName("q").create();
-    Option quotedRegexOpt = obuilder.withLongName("quotedRegex").withRequired(false).withArgument(abuilder.withName("regex").withMinimum(1).withMaximum(1).create())
-            .withDescription("Specify the regex that identifies quoted text.  Default is to look for > or | at the beginning of the line.").withShortName("q").create();
-    Option separatorOpt = obuilder.withLongName("separator").withRequired(false).withArgument(
-            abuilder.withName("separator").withMinimum(1).withMaximum(1).create()).
-            withDescription("The separator to use between metadata items (to, from, etc.).  Default is \\n").withShortName("sep").create();
-
-    Option bodySeparatorOpt = obuilder.withLongName("bodySeparator").withRequired(false).withArgument(
-            abuilder.withName("bodySeparator").withMinimum(1).withMaximum(1).create()).
-            withDescription("The separator to use between lines in the body.  Default is \\n.  Useful to change if you wish to have the message be on one line").withShortName("bodySep").create();
-    Option helpOpt = DefaultOptionCreator.helpOption();
-
-    Group group = gbuilder.withName("Options").withOption(keyPrefixOpt).withOption(chunkSizeOpt).withOption(
-            charsetOpt).withOption(outputDirOpt).withOption(helpOpt).withOption(inputOpt).withOption(subjectOpt).withOption(toOpt)
-            .withOption(fromOpt).withOption(bodyOpt).withOption(quotedOpt).withOption(refsOpt).withOption(bodySeparatorOpt)
-            .withOption(quotedRegexOpt)
-            .withOption(separatorOpt).create();
+    addInputOption();
+    addOutputOption();
 
-    try {
-      Parser parser = new Parser();
-      parser.setGroup(group);
-      parser.setHelpOption(helpOpt);
-      CommandLine cmdLine = parser.parse(args);
-      if (cmdLine.hasOption(helpOpt)) {
-        CommandLineUtil.printHelp(group);
-        return;
-      }
-      File input = new File((String) cmdLine.getValue(inputOpt));
-      String outputDir = (String) cmdLine.getValue(outputDirOpt);
+    addOption(obuilder.withLongName("chunkSize").withArgument(
+        abuilder.withName("chunkSize").withMinimum(1).withMaximum(1).create())
+        .withDescription("The chunkSize in MegaBytes. Defaults to 64")
+        .withShortName("chunk").create());
+
+    addOption(obuilder.withLongName("keyPrefix").withArgument(
+        abuilder.withName("keyPrefix").withMinimum(1).withMaximum(1).create())
+        .withDescription("The prefix to be prepended to the key")
+        .withShortName("prefix").create());
+    addOption(obuilder.withLongName("charset")
+        .withRequired(true).withArgument(abuilder.withName("charset")
+            .withMinimum(1).withMaximum(1).create()).withDescription(
+            "The name of the character encoding of the input files")
+        .withShortName("c").create());
+    addOption(obuilder.withLongName("subject")
+        .withRequired(false).withDescription(
+            "Include the Mail subject as part of the text.  Default is false")
+        .withShortName("s").create());
+    addOption(obuilder.withLongName("to").withRequired(false)
+        .withDescription("Include the to field in the text.  Default is false")
+        .withShortName("to").create());
+    addOption(obuilder.withLongName("from").withRequired(false).withDescription(
+        "Include the from field in the text.  Default is false")
+        .withShortName("from").create());
+    addOption(obuilder.withLongName("references")
+        .withRequired(false).withDescription(
+            "Include the references field in the text.  Default is false")
+        .withShortName("refs").create());
+    addOption(obuilder.withLongName("body").withRequired(false)
+        .withDescription("Include the body in the output.  Default is false")
+        .withShortName("b").create());
+    addOption(obuilder.withLongName("stripQuoted")
+        .withRequired(false).withDescription(
+            "Strip (remove) quoted email text in the body.  Default is false")
+        .withShortName("q").create());
+    addOption(
+        obuilder.withLongName("quotedRegex")
+            .withRequired(false).withArgument(abuilder.withName("regex")
+                .withMinimum(1).withMaximum(1).create()).withDescription(
+                "Specify the regex that identifies quoted text.  Default is to look for > or | at the beginning of the line.")
+            .withShortName("q").create());
+    addOption(
+        obuilder.withLongName("separator")
+            .withRequired(false).withArgument(abuilder.withName("separator")
+                .withMinimum(1).withMaximum(1).create()).withDescription(
+                "The separator to use between metadata items (to, from, etc.).  Default is \\n")
+            .withShortName("sep").create());
+
+    addOption(
+        obuilder.withLongName("bodySeparator")
+            .withRequired(false).withArgument(abuilder.withName("bodySeparator")
+                .withMinimum(1).withMaximum(1).create()).withDescription(
+                "The separator to use between lines in the body.  Default is \\n.  Useful to change if you wish to have the message be on one line")
+            .withShortName("bodySep").create());
+    addOption(DefaultOptionCreator.helpOption());
+    Map<String, List<String>> parsedArgs = parseArguments(args);
+    if (parsedArgs == null) {
+      return -1;
+    }
+    File input = getInputFile();
+    String outputDir = getOutputPath().toString();
 
-      int chunkSize = 64;
-      if (cmdLine.hasOption(chunkSizeOpt)) {
-        chunkSize = Integer.parseInt((String) cmdLine.getValue(chunkSizeOpt));
-      }
+    int chunkSize = 64;
+    if (hasOption("chunkSize")) {
+      chunkSize = Integer.parseInt(getOption("chunkSize"));
+    }
 
-      String prefix = "";
-      if (cmdLine.hasOption(keyPrefixOpt)) {
-        prefix = (String) cmdLine.getValue(keyPrefixOpt);
-      }
+    String prefix = "";
+    if (hasOption("keyPrefix")) {
+      prefix = getOption("keyPrefix");
+    }
 
-      Charset charset = Charset.forName((String) cmdLine.getValue(charsetOpt));
-      SequenceFilesFromMailArchives dir = new SequenceFilesFromMailArchives();
-      MailOptions options = new MailOptions();
-      options.setInput(input);
-      options.setOutputDir(outputDir);
-      options.setPrefix(prefix);
-      options.setChunkSize(chunkSize);
-      options.setCharset(charset);
-
-
-      List<Pattern> patterns = new ArrayList<Pattern>(5);
-      // patternOrder is used downstream so that we can know what order the text is in instead 
-      // of encoding it in the string, which
-      // would require more processing later to remove it pre feature selection.
-      Map<String, Integer> patternOrder = new HashMap<String, Integer>();
-      int order = 0;
-      if (cmdLine.hasOption(fromOpt)) {
-        patterns.add(MailProcessor.FROM_PREFIX);
-        patternOrder.put(MailOptions.FROM, order++);
-      }
-      if (cmdLine.hasOption(toOpt)) {
-        patterns.add(MailProcessor.TO_PREFIX);
-        patternOrder.put(MailOptions.TO, order++);
-      }
-      if (cmdLine.hasOption(refsOpt)) {
-        patterns.add(MailProcessor.REFS_PREFIX);
-        patternOrder.put(MailOptions.REFS, order++);
-      }
-      if (cmdLine.hasOption(subjectOpt)) {
-        patterns.add(MailProcessor.SUBJECT_PREFIX);
-        patternOrder.put(MailOptions.SUBJECT, order++);
-      }
-      options.setStripQuotedText(cmdLine.hasOption(quotedOpt));
+    Charset charset = Charset.forName(getOption("charset"));
+    MailOptions options = new MailOptions();
+    options.setInput(input);
+    options.setOutputDir(outputDir);
+    options.setPrefix(prefix);
+    options.setChunkSize(chunkSize);
+    options.setCharset(charset);
+
+    List<Pattern> patterns = new ArrayList<Pattern>(5);
+    // patternOrder is used downstream so that we can know what order the text
+    // is in instead
+    // of encoding it in the string, which
+    // would require more processing later to remove it pre feature selection.
+    Map<String,Integer> patternOrder = new HashMap<String,Integer>();
+    int order = 0;
+    if (hasOption("from")) {
+      patterns.add(MailProcessor.FROM_PREFIX);
+      patternOrder.put(MailOptions.FROM, order++);
+    }
+    if (hasOption("to")) {
+      patterns.add(MailProcessor.TO_PREFIX);
+      patternOrder.put(MailOptions.TO, order++);
+    }
+    if (hasOption("references")) {
+      patterns.add(MailProcessor.REFS_PREFIX);
+      patternOrder.put(MailOptions.REFS, order++);
+    }
+    if (hasOption("subject")) {
+      patterns.add(MailProcessor.SUBJECT_PREFIX);
+      patternOrder.put(MailOptions.SUBJECT, order++);
+    }
+    options.setStripQuotedText(hasOption("stripQuoted"));
 
-      options.setPatternsToMatch(patterns.toArray(new Pattern[patterns.size()]));
-      options.setPatternOrder(patternOrder);
-      options.setIncludeBody(cmdLine.hasOption(bodyOpt));
-      options.setSeparator("\n");
-      if (cmdLine.hasOption(separatorOpt)) {
-        options.setSeparator(cmdLine.getValue(separatorOpt).toString());
-      }
-      if (cmdLine.hasOption(bodySeparatorOpt)) {
-        options.setBodySeparator(cmdLine.getValue(bodySeparatorOpt).toString());
-      }
-      if (cmdLine.hasOption(quotedRegexOpt)) {
-        options.setQuotedTextPattern(Pattern.compile(cmdLine.getValue(quotedRegexOpt).toString()));
-      }
-      long start = System.currentTimeMillis();
-      dir.createSequenceFiles(options);
-      long finish = System.currentTimeMillis();
-      log.info("Conversion took {}ms", finish - start);
-    } catch (OptionException e) {
-      log.error("Exception", e);
-      CommandLineUtil.printHelp(group);
+    options.setPatternsToMatch(patterns.toArray(new Pattern[patterns.size()]));
+    options.setPatternOrder(patternOrder);
+    options.setIncludeBody(hasOption("body"));
+    options.setSeparator("\n");
+    if (hasOption("separator")) {
+      options.setSeparator(getOption("separator"));
     }
+    if (hasOption("bodySeparator")) {
+      options.setBodySeparator(getOption("bodySeparator"));
+    }
+    if (hasOption("quotedRegex")) {
+      options.setQuotedTextPattern(Pattern.compile(getOption("quotedRegex")));
+    }
+    long start = System.currentTimeMillis();
+    createSequenceFiles(options);
+    long finish = System.currentTimeMillis();
+    log.info("Conversion took {}ms", finish - start);
+    return 0;
   }
-
 }