You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@joshua.apache.org by mj...@apache.org on 2016/05/25 18:08:05 UTC

[01/15] incubator-joshua git commit: large commit moving in the direction of simplifying grammar reading and packing

Repository: incubator-joshua
Updated Branches:
  refs/heads/master d0f7b5308 -> 4d73c1784


large commit moving in the direction of simplifying grammar reading and packing

-  SAMT grammars are no longer supported
-  removed special PhraseRule, now just have Rule again; Thrax needs to be updated to add [X,1] to the source and target sides of rules
-  removed redundant functions in GrammarReader and Vocabulary dealing with identifying and parsing nonterminals (and using expensive regexs), now all moved to FormatUtils
-  cleaned up the GrammarReader interface
-  Moses phrase table packing updated
-


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/76eb9587
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/76eb9587
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/76eb9587

Branch: refs/heads/master
Commit: 76eb9587a9c321d45a9560481429e2dffd918e4e
Parents: d0f7b53
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 00:05:30 2016 +0200
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 00:05:30 2016 +0200

----------------------------------------------------------------------
 src/joshua/corpus/Vocabulary.java               |   6 +-
 src/joshua/decoder/ff/tm/GrammarReader.java     |  41 ----
 src/joshua/decoder/ff/tm/PhraseRule.java        |  94 ---------
 src/joshua/decoder/ff/tm/Rule.java              |  30 +--
 .../decoder/ff/tm/format/HieroFormatReader.java |  84 ++++----
 .../decoder/ff/tm/format/MosesFormatReader.java | 131 ++++++++++++
 .../ff/tm/format/PhraseFormatReader.java        | 128 ------------
 .../decoder/ff/tm/format/SamtFormatReader.java  | 136 ------------
 .../tm/hash_based/MemoryBasedBatchGrammar.java  |   9 +-
 .../decoder/ff/tm/packed/PackedGrammar.java     | 104 +---------
 .../GrammarBuilderWalkerFunction.java           |   4 +-
 src/joshua/util/CompareGrammars.java            | 207 -------------------
 src/joshua/util/FormatUtils.java                |   9 +
 13 files changed, 197 insertions(+), 786 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/76eb9587/src/joshua/corpus/Vocabulary.java
----------------------------------------------------------------------
diff --git a/src/joshua/corpus/Vocabulary.java b/src/joshua/corpus/Vocabulary.java
index d79170d..6f72ad8 100644
--- a/src/joshua/corpus/Vocabulary.java
+++ b/src/joshua/corpus/Vocabulary.java
@@ -155,7 +155,7 @@ public class Vocabulary {
       if (stringToId.containsKey(token)) {
         return stringToId.get(token);
       }
-      int id = idToString.size() * (nt(token) ? -1 : 1);
+      int id = idToString.size() * (FormatUtils.isNonterminal(token) ? -1 : 1);
 
       // register this (token,id) mapping with each language
       // model, so that they can map it to their own private
@@ -237,10 +237,6 @@ public class Vocabulary {
     return (id < 0);
   }
 
-  public static boolean nt(String word) {
-    return FormatUtils.isNonterminal(word);
-  }
-
   public static int size() {
     long lock_stamp = lock.readLock();
     try {

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/76eb9587/src/joshua/decoder/ff/tm/GrammarReader.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/GrammarReader.java b/src/joshua/decoder/ff/tm/GrammarReader.java
index f94a472..7edab7c 100644
--- a/src/joshua/decoder/ff/tm/GrammarReader.java
+++ b/src/joshua/decoder/ff/tm/GrammarReader.java
@@ -36,8 +36,6 @@ import joshua.util.io.LineReader;
 public abstract class GrammarReader<R extends Rule> implements Iterable<R>, Iterator<R> {
 
   protected static String fieldDelimiter;
-  protected static String nonTerminalRegEx;
-  protected static String nonTerminalCleanRegEx;
 
   protected static String description;
 
@@ -165,43 +163,4 @@ public abstract class GrammarReader<R extends Rule> implements Iterable<R>, Iter
   }
 
   protected abstract R parseLine(String line);
-
-  // TODO: keep these around or not?
-  public abstract String toWords(R rule);
-
-  public abstract String toWordsWithoutFeatureScores(R rule);
-
-  /**
-   * Removes square brackets (and index, if present) from nonterminal id 
-   * @param tokenID
-   * @return cleaned ID
-   */
-  public static int cleanNonTerminal(int tokenID) {
-    // cleans NT of any markup, e.g., [X,1] may becomes [X], depending
-    return Vocabulary.id(cleanNonTerminal(Vocabulary.word(tokenID)));
-  }
-
-  /**
-   * Removes square brackets (and index, if present) from nonterminal id 
-   * @param token
-   * @return cleaned token
-   */
-  public static String cleanNonTerminal(String token) {
-    // cleans NT of any markup, e.g., [X,1] may becomes [X], depending on nonTerminalCleanRegEx
-    return token.replaceAll(nonTerminalCleanRegEx, "");
-  }
-
-  public static boolean isNonTerminal(final String word) {
-    // checks if word matches NT regex
-    return word.matches(nonTerminalRegEx);
-  }
-
-  public String getNonTerminalRegEx() {
-    return nonTerminalRegEx;
-  }
-
-  public String getNonTerminalCleanRegEx() {
-    return nonTerminalCleanRegEx;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/76eb9587/src/joshua/decoder/ff/tm/PhraseRule.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/PhraseRule.java b/src/joshua/decoder/ff/tm/PhraseRule.java
deleted file mode 100644
index 8f5d249..0000000
--- a/src/joshua/decoder/ff/tm/PhraseRule.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package joshua.decoder.ff.tm;
-
-import com.google.common.base.Supplier;
-import com.google.common.base.Suppliers;
-
-/***
- * A class for reading in rules from a Moses phrase table. Most of the conversion work is done
- * in {@link joshua.decoder.ff.tm.format.PhraseFormatReader}. This includes prepending every
- * rule with a nonterminal, so that the phrase-based decoder can assume the same hypergraph
- * format as the hierarchical decoder (by pretending to be a strictly left-branching grammar and
- * dispensing with the notion of coverage spans). However, prepending the nonterminals means all
- * the alignments are off by 1. We do not want to fix those when reading in due to the expense,
- * so instead we use this rule which adjust the alignments on the fly.
- * 
- * Also, we only convert the Moses dense features on the fly, via this class.
- * 
- * TODO: this class should also be responsible for prepending the nonterminals.
- * 
- * @author Matt Post
- *
- */
-public class PhraseRule extends Rule {
-
-
-  private final String mosesFeatureString;
-  private final Supplier<byte[]> alignmentSupplier;
-  private final Supplier<String> sparseFeaturesStringSupplier;
-  
-  public PhraseRule(int lhs, int[] french, int[] english, String sparse_features, int arity,
-      String alignment) {
-    super(lhs, french, english, null, arity, alignment);
-    this.mosesFeatureString = sparse_features;
-    this.alignmentSupplier = initializeAlignmentSupplier();
-    this.sparseFeaturesStringSupplier = initializeSparseFeaturesStringSupplier();
-  }
-  
-  /** 
-   * Moses features are probabilities; we need to convert them here by taking the negative log prob.
-   * We do this only when the rule is used to amortize.
-   */
-  private Supplier<String> initializeSparseFeaturesStringSupplier() {
-    return Suppliers.memoize(() ->{
-      StringBuffer values = new StringBuffer();
-      for (String value: mosesFeatureString.split(" ")) {
-        float f = Float.parseFloat(value);
-        values.append(String.format("%f ", f <= 0.0 ? -100 : -Math.log(f)));
-      }
-      return values.toString().trim();
-    });
-  }
-
-  /**
-   * This is the exact same as the parent implementation, but we need to add 1 to each alignment
-   * point to account for the nonterminal [X] that was prepended to each rule. 
-   */
-  private Supplier<byte[]> initializeAlignmentSupplier(){
-    return Suppliers.memoize(() ->{
-      String[] tokens = getAlignmentString().split("[-\\s]+");
-      byte[] alignmentArray = new byte[tokens.length + 2];
-      alignmentArray[0] = alignmentArray[1] = 0;
-      for (int i = 0; i < tokens.length; i++)
-          alignmentArray[i + 2] = (byte) (Short.parseShort(tokens[i]) + 1);
-      return alignmentArray;
-    });
-  }
-
-  @Override
-  public String getFeatureString() {
-    return this.sparseFeaturesStringSupplier.get();
-  }
-  
-  @Override
-  public byte[] getAlignment() {
-    return this.alignmentSupplier.get();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/76eb9587/src/joshua/decoder/ff/tm/Rule.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/Rule.java b/src/joshua/decoder/ff/tm/Rule.java
index 9f1fb8f..89bb2a2 100644
--- a/src/joshua/decoder/ff/tm/Rule.java
+++ b/src/joshua/decoder/ff/tm/Rule.java
@@ -56,7 +56,7 @@ import joshua.decoder.segment_file.Sentence;
 public class Rule implements Comparator<Rule>, Comparable<Rule> {
 
   private int lhs; // tag of this rule
-  private int[] pFrench; // pointer to the RuleCollection, as all the rules under it share the same
+  private int[] source; // pointer to the RuleCollection, as all the rules under it share the same
                          // Source side
   protected int arity;
 
@@ -81,7 +81,7 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
 
   private float precomputableCost = Float.NEGATIVE_INFINITY;
 
-  private int[] english;
+  private int[] target;
 
   // The alignment string, e.g., 0-0 0-1 1-1 2-1
   private String alignmentString;
@@ -96,18 +96,18 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
    * Constructor used by other constructors below;
    * 
    * @param lhs Left-hand side of the rule.
-   * @param sourceRhs Source language right-hand side of the rule.
-   * @param targetRhs Target language right-hand side of the rule.
+   * @param source Source language right-hand side of the rule.
+   * @param target Target language right-hand side of the rule.
    * @param sparseFeatures Feature value scores for the rule.
    * @param arity Number of nonterminals in the source language right-hand side.
    * @param owner
    */
-  public Rule(int lhs, int[] sourceRhs, int[] targetRhs, String sparseFeatures, int arity, int owner) {
+  public Rule(int lhs, int[] source, int[] target, String sparseFeatures, int arity, int owner) {
     this.lhs = lhs;
-    this.pFrench = sourceRhs;
+    this.source = source;
     this.arity = arity;
     this.owner = owner;
-    this.english = targetRhs;
+    this.target = target;
     this.sparseFeatureStringSupplier = Suppliers.memoize(() -> { return sparseFeatures; });
     this.featuresSupplier = initializeFeatureSupplierFromString();
     this.alignmentSupplier = initializeAlignmentSupplier();
@@ -118,10 +118,10 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
    */
   public Rule(int lhs, int[] sourceRhs, int[] targetRhs, FeatureVector features, int arity, int owner) {
     this.lhs = lhs;
-    this.pFrench = sourceRhs;
+    this.source = sourceRhs;
     this.arity = arity;
     this.owner = owner;
-    this.english = targetRhs;
+    this.target = targetRhs;
     this.featuresSupplier = Suppliers.memoize(() -> { return features; });
     this.sparseFeatureStringSupplier = initializeSparseFeaturesStringSupplier();
     this.alignmentSupplier = initializeAlignmentSupplier();
@@ -199,11 +199,11 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
   // ===============================================================
 
   public void setEnglish(int[] eng) {
-    this.english = eng;
+    this.target = eng;
   }
 
   public int[] getEnglish() {
-    return this.english;
+    return this.target;
   }
 
   /**
@@ -224,7 +224,7 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
     if (!Arrays.equals(getFrench(), other.getFrench())) {
       return false;
     }
-    if (!Arrays.equals(english, other.getEnglish())) {
+    if (!Arrays.equals(target, other.getEnglish())) {
       return false;
     }
     return true;
@@ -234,7 +234,7 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
     // I just made this up. If two rules are equal they'll have the
     // same hashcode. Maybe someone else can do a better job though?
     int frHash = Arrays.hashCode(getFrench());
-    int enHash = Arrays.hashCode(english);
+    int enHash = Arrays.hashCode(target);
     return frHash ^ enHash ^ getLHS();
   }
 
@@ -267,11 +267,11 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
   }
 
   public void setFrench(int[] french) {
-    this.pFrench = french;
+    this.source = french;
   }
 
   public int[] getFrench() {
-    return this.pFrench;
+    return this.source;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/76eb9587/src/joshua/decoder/ff/tm/format/HieroFormatReader.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/format/HieroFormatReader.java b/src/joshua/decoder/ff/tm/format/HieroFormatReader.java
index a47813d..d2a01eb 100644
--- a/src/joshua/decoder/ff/tm/format/HieroFormatReader.java
+++ b/src/joshua/decoder/ff/tm/format/HieroFormatReader.java
@@ -21,6 +21,7 @@ package joshua.decoder.ff.tm.format;
 import joshua.corpus.Vocabulary;
 import joshua.decoder.ff.tm.GrammarReader;
 import joshua.decoder.ff.tm.Rule;
+import joshua.util.FormatUtils;
 
 /**
  * This class implements reading files in the format defined by David Chiang for Hiero. 
@@ -33,10 +34,6 @@ public class HieroFormatReader extends GrammarReader<Rule> {
 
   static {
     fieldDelimiter = "\\s\\|{3}\\s";
-    nonTerminalRegEx = "^\\[[^\\s]+\\,[0-9]*\\]$";
-    nonTerminalCleanRegEx = ",[0-9\\s]+";
-    // nonTerminalRegEx = "^\\[[A-Z]+\\,[0-9]*\\]$";
-    // nonTerminalCleanRegEx = "[\\[\\]\\,0-9\\s]+";
     description = "Original Hiero format";
   }
 
@@ -55,69 +52,58 @@ public class HieroFormatReader extends GrammarReader<Rule> {
       throw new RuntimeException(String.format("Rule '%s' does not have four fields", line));
     }
 
-    int lhs = Vocabulary.id(cleanNonTerminal(fields[0]));
+    int lhs = Vocabulary.id(FormatUtils.stripNonTerminalIndex(fields[0]));
 
+    /**
+     * On the foreign side, we map nonterminals to negative IDs, and terminals to positive IDs.
+     */
     int arity = 0;
-    // foreign side
-    String[] foreignWords = fields[1].split("\\s+");
-    int[] french = new int[foreignWords.length];
-    for (int i = 0; i < foreignWords.length; i++) {
-      french[i] = Vocabulary.id(foreignWords[i]);
-      if (Vocabulary.nt(french[i])) {
+    String[] sourceWords = fields[1].split("\\s+");
+    int[] sourceIDs = new int[sourceWords.length];
+    for (int i = 0; i < sourceWords.length; i++) {
+      if (FormatUtils.isNonterminal(sourceWords[i])) {
+        Vocabulary.id(sourceWords[i]);
+        sourceIDs[i] = Vocabulary.id(FormatUtils.stripNonTerminalIndex(sourceWords[i]));
         arity++;
-        french[i] = cleanNonTerminal(french[i]);
+        
+        // TODO: the arity here (after incrementing) should match the rule index. Should
+        // check that arity == FormatUtils.getNonterminalIndex(foreignWords[i]), throw runtime
+        // error if not
+      } else {
+        sourceIDs[i] = Vocabulary.id(sourceWords[i]);
       }
     }
 
-    // English side
-    String[] englishWords = fields[2].split("\\s+");
-    int[] english = new int[englishWords.length];
-    for (int i = 0; i < englishWords.length; i++) {
-      english[i] = Vocabulary.id(englishWords[i]);
-      if (Vocabulary.nt(english[i])) {
-        english[i] = -Vocabulary.getTargetNonterminalIndex(english[i]);
+    /**
+     * The English side maps terminal symbols to positive IDs. Nonterminal symbols are linked
+     * to the index of the source-side nonterminal they are linked to. So for a rule
+     * 
+     * [X] ||| [X,1] [X,2] [X,3] ||| [X,2] [X,1] [X,3] ||| ...
+     * 
+     * the English side nonterminals will be -2, -1, -3. This assumes that the source side of
+     * the rule is always listed monotonically.
+     */
+    String[] targetWords = fields[2].split("\\s+");
+    int[] targetIDs = new int[targetWords.length];
+    for (int i = 0; i < targetWords.length; i++) {
+      if (FormatUtils.isNonterminal(targetWords[i])) {
+        targetIDs[i] = -FormatUtils.getNonterminalIndex(targetWords[i]);
+      } else {
+        targetIDs[i] = Vocabulary.id(targetWords[i]);
       }
     }
 
     String sparse_features = (fields.length > 3 ? fields[3] : "");
     String alignment = (fields.length > 4) ? fields[4] : null;
 
-    return new Rule(lhs, french, english, sparse_features, arity, alignment);
+    return new Rule(lhs, sourceIDs, targetIDs, sparse_features, arity, alignment);
   }
 
-  @Override
-  public String toWords(Rule rule) {
-    StringBuffer sb = new StringBuffer("");
-    sb.append(Vocabulary.word(rule.getLHS()));
-    sb.append(" ||| ");
-    sb.append(Vocabulary.getWords(rule.getFrench()));
-    sb.append(" ||| ");
-    sb.append(Vocabulary.getWords(rule.getEnglish()));
-    sb.append(" |||");
-    sb.append(" " + rule.getFeatureVector());
-
-    return sb.toString();
-  }
-
-  @Override
-  public String toWordsWithoutFeatureScores(Rule rule) {
-    StringBuffer sb = new StringBuffer();
-    sb.append(rule.getLHS());
-    sb.append(" ||| ");
-    sb.append(Vocabulary.getWords(rule.getFrench()));
-    sb.append(" ||| ");
-    sb.append(Vocabulary.getWords(rule.getEnglish()));
-    sb.append(" |||");
-
-    return sb.toString();
-  }
-
-
   public static String getFieldDelimiter() {
     return fieldDelimiter;
   }
 
   public static boolean isNonTerminal(final String word) {
-    return GrammarReader.isNonTerminal(word);
+    return FormatUtils.isNonterminal(word);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/76eb9587/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/format/MosesFormatReader.java b/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
new file mode 100644
index 0000000..0b33ba1
--- /dev/null
+++ b/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package joshua.decoder.ff.tm.format;
+
+import joshua.corpus.Vocabulary;
+import joshua.decoder.ff.tm.Rule;
+import joshua.util.io.LineReader;
+
+/***
+ * This class reads in the Moses phrase table format, with support for the source and target side,
+ * list of features, and word alignments. It works by
+ * 
+ * - casting the phrase-based rules to left-branching hierarchical rules and passing them on \
+ *   to its parent class, {@HieroFormatReader}.
+ * - converting the probabilities to -log probabilities
+ * 
+ * There is also a tool to convert the grammars directly, so that they can be suitably packed. Usage:
+ * 
+ * <pre>
+ *     cat PHRASE_TABLE | java -cp $JOSHUA/class joshua.decoder.ff.tm.format.MosesFormatReader > grammar
+ * </pre>
+ * 
+ * @author Matt Post <po...@cs.jhu.edu>
+ *
+ */
+
+public class MosesFormatReader extends HieroFormatReader {
+
+  private int lhs;
+  
+  public MosesFormatReader(String grammarFile) {
+    super(grammarFile);
+    this.lhs = Vocabulary.id("[X]");
+  }
+  
+  public MosesFormatReader() {
+    super();
+    this.lhs = Vocabulary.id("[X]");
+  }
+  
+  /**
+   * When dealing with Moses format, this munges a Moses-style phrase table into a grammar.
+   * 
+   *    mots francaises ||| French words ||| 1 2 3 ||| 0-1 1-0
+   *    
+   * becomes
+   * 
+   *    [X] ||| [X,1] mots francaises ||| [X,1] French words ||| 1 2 3  ||| 0-1 1-0
+   *    
+   * For thrax-extracted phrasal grammars, it transforms
+   * 
+   *    [X] ||| mots francaises ||| French words ||| 1 2 3 ||| 0-1 1-0
+   *
+   * into
+   * 
+   *    [X] ||| [X,1] mots francaises ||| [X,1] French words ||| 1 2 3 ||| 0-1 1-0
+   */
+  @Override
+  public Rule parseLine(String line) {
+    String[] fields = line.split(fieldDelimiter);
+
+    int arity = 1;
+    int fieldIndex = 0;
+    
+    // foreign side
+    String[] foreignWords = fields[fieldIndex].split("\\s+");
+    int[] french = new int[foreignWords.length + 1];
+    french[0] = lhs; 
+    for (int i = 0; i < foreignWords.length; i++) {
+      french[i+1] = Vocabulary.id(foreignWords[i]);
+    }
+
+    // English side
+    fieldIndex++;
+    String[] englishWords = fields[fieldIndex].split("\\s+");
+    int[] english = new int[englishWords.length + 1];
+    english[0] = -1;
+    for (int i = 0; i < englishWords.length; i++) {
+      english[i+1] = Vocabulary.id(englishWords[i]);
+    }
+
+    // transform feature values
+    fieldIndex++;
+    
+    String mosesFeatureString = fields[fieldIndex];
+    StringBuffer values = new StringBuffer();
+    for (String value: mosesFeatureString.split(" ")) {
+      float f = Float.parseFloat(value);
+      values.append(String.format("%f ", f <= 0.0 ? -100 : -Math.log(f)));
+    }
+
+    String sparse_features = values.toString().trim();
+
+//    System.out.println(String.format("parseLine: %s\n  ->%s", line, sparse_features));
+
+    // alignments
+    fieldIndex++;
+    String alignment = (fields.length > fieldIndex) ? fields[fieldIndex] : null;
+
+    return new Rule(lhs, french, english, sparse_features, arity, alignment);
+  }
+  
+  /**
+   * Converts a Moses phrase table to a Joshua grammar. 
+   * 
+   * @param args
+   */
+  public static void main(String[] args) {
+    MosesFormatReader reader = new MosesFormatReader();
+    for (String line: new LineReader(System.in)) {
+      Rule rule = reader.parseLine(line);
+      System.out.println(rule.textFormat());
+    }    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/76eb9587/src/joshua/decoder/ff/tm/format/PhraseFormatReader.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/format/PhraseFormatReader.java b/src/joshua/decoder/ff/tm/format/PhraseFormatReader.java
deleted file mode 100644
index be4d522..0000000
--- a/src/joshua/decoder/ff/tm/format/PhraseFormatReader.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package joshua.decoder.ff.tm.format;
-
-import joshua.corpus.Vocabulary;
-import joshua.decoder.ff.tm.PhraseRule;
-import joshua.util.io.LineReader;
-
-/***
- * This class reads in the Moses phrase table format, with support for the source and target side,
- * list of features, and word alignments. It works by simply casting the phrase-based rules to
- * left-branching hierarchical rules and passing them on to its parent class, {@HieroFormatReader}.
- * 
- * There is also a tool to convert the grammars directly, so that they can be suitably packed. Usage:
- * 
- * <pre>
- *     cat PHRASE_TABLE | java -cp $JOSHUA/class joshua.decoder.ff.tm.format.PhraseFormatReader > grammar
- * </pre>
- * 
- * @author Matt Post <po...@cs.jhu.edu>
- *
- */
-
-public class PhraseFormatReader extends HieroFormatReader {
-
-  private int lhs;
-  
-  /* Whether we are reading a Moses phrase table or Thrax phrase table */
-  private boolean moses_format = false;
-
-  public PhraseFormatReader(String grammarFile, boolean is_moses) {
-    super(grammarFile);
-    this.lhs = Vocabulary.id("[X]");
-    this.moses_format = is_moses;
-  }
-  
-  public PhraseFormatReader() {
-    super();
-    this.lhs = Vocabulary.id("[X]");
-  }
-  
-  /**
-   * When dealing with Moses format, this munges a Moses-style phrase table into a grammar.
-   * 
-   *    mots francaises ||| French words ||| 1 2 3 ||| 0-1 1-0
-   *    
-   * becomes
-   * 
-   *    [X] ||| [X,1] mots francaises ||| [X,1] French words ||| 1 2 3  ||| 0-1 1-0
-   *    
-   * For thrax-extracted phrasal grammars, it transforms
-   * 
-   *    [X] ||| mots francaises ||| French words ||| 1 2 3 ||| 0-1 1-0
-   *
-   * into
-   * 
-   *    [X] ||| [X,1] mots francaises ||| [X,1] French words ||| 1 2 3 ||| 0-1 1-0
-   */
-  @Override
-  public PhraseRule parseLine(String line) {
-    String[] fields = line.split(fieldDelimiter);
-
-    int arity = 1;
-    
-    /* For Thrax phrase-based grammars, skip over the beginning nonterminal */
-    int fieldIndex = 0;
-    if (! moses_format)
-      fieldIndex++;
-    
-    // foreign side
-    String[] foreignWords = fields[fieldIndex].split("\\s+");
-    int[] french = new int[foreignWords.length + 1];
-    french[0] = lhs; 
-    for (int i = 0; i < foreignWords.length; i++) {
-      french[i+1] = Vocabulary.id(foreignWords[i]);
-    }
-
-    // English side
-    fieldIndex++;
-    String[] englishWords = fields[fieldIndex].split("\\s+");
-    int[] english = new int[englishWords.length + 1];
-    english[0] = -1;
-    for (int i = 0; i < englishWords.length; i++) {
-      english[i+1] = Vocabulary.id(englishWords[i]);
-    }
-
-    // transform feature values
-    fieldIndex++;
-    String sparse_features = fields[fieldIndex];
-
-//    System.out.println(String.format("parseLine: %s\n  ->%s", line, sparse_features));
-
-    // alignments
-    fieldIndex++;
-    String alignment = (fields.length > fieldIndex) ? fields[fieldIndex] : null;
-
-    return new PhraseRule(lhs, french, english, sparse_features, arity, alignment);
-  }
-  
-  /**
-   * Converts a Moses phrase table to a Joshua grammar. 
-   * 
-   * @param args
-   */
-  public static void main(String[] args) {
-    PhraseFormatReader reader = new PhraseFormatReader();
-    for (String line: new LineReader(System.in)) {
-      PhraseRule rule = reader.parseLine(line);
-      System.out.println(rule.textFormat());
-    }    
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/76eb9587/src/joshua/decoder/ff/tm/format/SamtFormatReader.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/format/SamtFormatReader.java b/src/joshua/decoder/ff/tm/format/SamtFormatReader.java
deleted file mode 100644
index 6539d38..0000000
--- a/src/joshua/decoder/ff/tm/format/SamtFormatReader.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package joshua.decoder.ff.tm.format;
-
-import java.util.logging.Logger;
-
-import joshua.corpus.Vocabulary;
-import joshua.decoder.ff.tm.Rule;
-import joshua.decoder.ff.tm.GrammarReader;
-
-public class SamtFormatReader extends GrammarReader<Rule> {
-
-  private static final Logger logger = Logger.getLogger(SamtFormatReader.class.getName());
-
-  private static final String samtNonTerminalMarkup;
-
-  static {
-    fieldDelimiter = "#";
-    nonTerminalRegEx = "^@[^\\s]+";
-    nonTerminalCleanRegEx = ",[0-9\\s]+";
-
-    samtNonTerminalMarkup = "@";
-
-    description = "Original SAMT format";
-  }
-
-  public SamtFormatReader(String grammarFile) {
-    super(grammarFile);
-  }
-
-  // Format example:
-  // @VZ-HD @APPR-DA+ART-DA minutes#@2 protokoll @1#@PP-MO+VZ-HD#0 1 1 -0 0.5 -0
-
-  @Override
-  protected Rule parseLine(String line) {
-    String[] fields = line.split(fieldDelimiter);
-    if (fields.length != 4) {
-      logger.severe("Rule line does not have four fields: " + line);
-      logger.severe("Skipped.");
-      return null;
-    }
-
-    int lhs = Vocabulary.id(adaptNonTerminalMarkup(fields[2]));
-
-    int arity = 0;
-
-    // foreign side
-    String[] foreignWords = fields[0].split("\\s+");
-    int[] french = new int[foreignWords.length];
-    for (int i = 0; i < foreignWords.length; i++) {
-      if (isNonTerminal(foreignWords[i])) {
-        arity++;
-        french[i] = Vocabulary.id(adaptNonTerminalMarkup(foreignWords[i], arity));
-      } else {
-        french[i] = Vocabulary.id(foreignWords[i]);
-      }
-    }
-
-    // english side
-    String[] englishWords = fields[1].split("\\s+");
-    int[] english = new int[englishWords.length];
-    for (int i = 0; i < englishWords.length; i++) {
-      if (isNonTerminal(englishWords[i])) {
-        english[i] = -Integer.parseInt(cleanSamtNonTerminal(englishWords[i]));
-      } else {
-        english[i] = Vocabulary.id(englishWords[i]);
-      }
-    }
-
-    // feature scores
-    String sparseFeatures = fields[3];
-
-    return new Rule(lhs, french, english, sparseFeatures, arity);
-  }
-
-  protected String cleanSamtNonTerminal(String word) {
-    // changes SAMT markup to Hiero-style
-    return word.replaceAll(samtNonTerminalMarkup, "");
-  }
-
-  protected String adaptNonTerminalMarkup(String word) {
-    // changes SAMT markup to Hiero-style
-    return "["
-        + word.replaceAll(",", "_COMMA_").replaceAll("\\$", "_DOLLAR_")
-            .replaceAll(samtNonTerminalMarkup, "") + "]";
-  }
-
-  protected String adaptNonTerminalMarkup(String word, int ntIndex) {
-    // changes SAMT markup to Hiero-style
-    return "["
-        + word.replaceAll(",", "_COMMA_").replaceAll("\\$", "_DOLLAR_")
-            .replaceAll(samtNonTerminalMarkup, "") + "," + ntIndex + "]";
-  }
-
-  @Override
-  public String toWords(Rule rule) {
-    StringBuffer sb = new StringBuffer();
-    sb.append(Vocabulary.word(rule.getLHS()));
-    sb.append(" ||| ");
-    sb.append(Vocabulary.getWords(rule.getFrench()));
-    sb.append(" ||| ");
-    sb.append(Vocabulary.getWords(rule.getEnglish()));
-    sb.append(" ||| " + rule.getFeatureString());
-
-    return sb.toString();
-  }
-
-  @Override
-  public String toWordsWithoutFeatureScores(Rule rule) {
-    StringBuffer sb = new StringBuffer();
-    sb.append(Vocabulary.word(rule.getLHS()));
-    sb.append(" ||| ");
-    sb.append(Vocabulary.getWords(rule.getFrench()));
-    sb.append(" ||| ");
-    sb.append(Vocabulary.getWords(rule.getEnglish()));
-    sb.append(" |||");
-
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/76eb9587/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java b/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
index 4ba514a..e85ce09 100644
--- a/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
+++ b/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
@@ -33,8 +33,7 @@ import joshua.decoder.ff.tm.Rule;
 import joshua.decoder.ff.tm.GrammarReader;
 import joshua.decoder.ff.tm.Trie;
 import joshua.decoder.ff.tm.format.HieroFormatReader;
-import joshua.decoder.ff.tm.format.PhraseFormatReader;
-import joshua.decoder.ff.tm.format.SamtFormatReader;
+import joshua.decoder.ff.tm.format.MosesFormatReader;
 import joshua.util.FormatUtils;
 
 /**
@@ -130,10 +129,8 @@ public class MemoryBasedBatchGrammar extends AbstractGrammar {
     if (grammarFile != null) {
       if ("hiero".equals(format) || "thrax".equals(format) || "regexp".equals(format)) {
         return new HieroFormatReader(grammarFile);
-      } else if ("samt".equals(format)) {
-        return new SamtFormatReader(grammarFile);
-      } else if ("phrase".equals(format) || "moses".equals(format)) {
-        return new PhraseFormatReader(grammarFile, format.equals("moses"));
+      } else if ("moses".equals(format)) {
+        return new MosesFormatReader(grammarFile);
       } else {
         throw new RuntimeException(String.format("* FATAL: unknown grammar format '%s'", format));
       }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/76eb9587/src/joshua/decoder/ff/tm/packed/PackedGrammar.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/packed/PackedGrammar.java b/src/joshua/decoder/ff/tm/packed/PackedGrammar.java
index fb38cf0..cc58578 100644
--- a/src/joshua/decoder/ff/tm/packed/PackedGrammar.java
+++ b/src/joshua/decoder/ff/tm/packed/PackedGrammar.java
@@ -652,10 +652,7 @@ public class PackedGrammar extends AbstractGrammar {
 
         rules = new ArrayList<Rule>(num_rules);
         for (int i = 0; i < num_rules; i++) {
-          if (type.equals("moses") || type.equals("phrase"))
-            rules.add(new PackedPhrasePair(rule_position + 3 * i));
-          else
-            rules.add(new PackedRule(rule_position + 3 * i));
+          rules.add(new PackedRule(rule_position + 3 * i));
         }
 
         cached_rules.put(this, rules);
@@ -794,105 +791,6 @@ public class PackedGrammar extends AbstractGrammar {
         }
       }
       
-      /**
-       * A packed phrase pair represents a rule of the form of a phrase pair, packed with the
-       * grammar-packer.pl script, which simply adds a nonterminal [X] to the left-hand side of
-       * all phrase pairs (and converts the Moses features). The packer then packs these. We have
-       * to then put a nonterminal on the source and target sides to treat the phrase pairs like
-       * left-branching rules, which is how Joshua deals with phrase decoding. 
-       * 
-       * @author Matt Post <po...@cs.jhu.edu>
-       *
-       */
-      public final class PackedPhrasePair extends PackedRule {
-
-        private final Supplier<int[]> englishSupplier;
-        private final Supplier<byte[]> alignmentSupplier;
-
-        public PackedPhrasePair(int address) {
-          super(address);
-          englishSupplier = initializeEnglishSupplier();
-          alignmentSupplier = initializeAlignmentSupplier();
-        }
-
-        @Override
-        public int getArity() {
-          return PackedTrie.this.getArity() + 1;
-        }
-
-        /**
-         * Initialize a number of suppliers which get evaluated when their respective getters
-         * are called.
-         * Inner lambda functions are guaranteed to only be called once, because of this underlying
-         * structures are accessed in a threadsafe way.
-         * Guava's implementation makes sure only one read of a volatile variable occurs per get.
-         * This means this implementation should be as thread-safe and performant as possible.
-         */
-
-        private Supplier<int[]> initializeEnglishSupplier(){
-          Supplier<int[]> result = Suppliers.memoize(() ->{
-            int[] phrase = getTarget(source[address + 1]);
-            int[] tgt = new int[phrase.length + 1];
-            tgt[0] = -1;
-            for (int i = 0; i < phrase.length; i++)
-              tgt[i+1] = phrase[i];
-            return tgt;
-          });
-          return result;
-        }
-
-        private Supplier<byte[]> initializeAlignmentSupplier(){
-          Supplier<byte[]> result = Suppliers.memoize(() ->{
-            byte[] raw_alignment = getAlignmentArray(source[address + 2]);
-            byte[] points = new byte[raw_alignment.length + 2];
-            points[0] = points[1] = 0;
-            for (int i = 0; i < raw_alignment.length; i++)
-              points[i + 2] = (byte) (raw_alignment[i] + 1);
-            return points;
-          });
-          return result;
-        }
-
-        /**
-         * Take the English phrase of the underlying rule and prepend an [X].
-         * 
-         * @return
-         */
-        @Override
-        public int[] getEnglish() {
-          return this.englishSupplier.get();
-        }
-        
-        /**
-         * Take the French phrase of the underlying rule and prepend an [X].
-         * 
-         * @return
-         */
-        @Override
-        public int[] getFrench() {
-          int phrase[] = new int[src.length + 1];
-          int ntid = Vocabulary.id(PackedGrammar.this.joshuaConfiguration.default_non_terminal);
-          phrase[0] = ntid;
-          System.arraycopy(src,  0, phrase, 1, src.length);
-          return phrase;
-        }
-        
-        /**
-         * Similarly the alignment array needs to be shifted over by one.
-         * 
-         * @return
-         */
-        @Override
-        public byte[] getAlignment() {
-          // if no alignments in grammar do not fail
-          if (alignments == null) {
-            return null;
-          }
-
-          return this.alignmentSupplier.get();
-        }
-      }
-
       public class PackedRule extends Rule {
         protected final int address;
         private final Supplier<int[]> englishSupplier;

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/76eb9587/src/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java b/src/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java
index 12e79c5..a4df7e5 100644
--- a/src/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java
+++ b/src/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java
@@ -27,6 +27,7 @@ import joshua.decoder.ff.tm.Grammar;
 import joshua.decoder.ff.tm.Rule;
 import joshua.decoder.ff.tm.format.HieroFormatReader;
 import joshua.decoder.ff.tm.hash_based.MemoryBasedBatchGrammar;
+import joshua.util.FormatUtils;
 
 /**
  * This walker function builds up a new context-free grammar by visiting each node in a hypergraph.
@@ -79,8 +80,7 @@ public class GrammarBuilderWalkerFunction implements WalkerFunction {
 
   private static String getLabelWithSpanAsString(HGNode node) {
     String label = Vocabulary.word(node.lhs);
-    String cleanLabel = HieroFormatReader.cleanNonTerminal(label);
-    String unBracketedCleanLabel = cleanLabel.substring(1, cleanLabel.length() - 1);
+    String unBracketedCleanLabel = label.substring(1, label.length() - 1);
     return String.format("[%d-%s-%d]", node.i, unBracketedCleanLabel, node.j);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/76eb9587/src/joshua/util/CompareGrammars.java
----------------------------------------------------------------------
diff --git a/src/joshua/util/CompareGrammars.java b/src/joshua/util/CompareGrammars.java
deleted file mode 100644
index 109d7a1..0000000
--- a/src/joshua/util/CompareGrammars.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package joshua.util;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.util.HashSet;
-import java.util.Scanner;
-import java.util.Set;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import joshua.decoder.ff.tm.format.HieroFormatReader;
-
-/**
- * This class allows two grammars (loaded from disk) to be compared.
- * 
- * @author Lane Schwartz
- */
-public class CompareGrammars {
-
-  /** Logger for this class. */
-  private static final Logger logger = Logger.getLogger(CompareGrammars.class.getName());
-
-  /**
-   * Gets a set containing all unique instances of the specified field.
-   * 
-   * @param grammarFile File containing a grammar.
-   * @param fieldDelimiter Regular expression to split each line
-   * @param fieldNumber Field from each rule to extract
-   * @return set containing all unique instances of the specified field
-   * @throws FileNotFoundException
-   */
-  public static Set<String> getFields(File grammarFile, String fieldDelimiter, int fieldNumber)
-      throws FileNotFoundException {
-
-    Scanner grammarScanner = new Scanner(grammarFile);
-
-    Set<String> set = new HashSet<String>();
-
-    while (grammarScanner.hasNextLine()) {
-
-      String line = grammarScanner.nextLine();
-
-      String[] fields = line.split(fieldDelimiter);
-
-      set.add(fields[fieldNumber]);
-    }
-    
-    grammarScanner.close();
-
-    return set;
-  }
-
-  public static void compareValues(File grammarFile1, File grammarFile2, String fieldDelimiter,
-      int fieldNumber, String scoresDelimiter, int scoresFieldNumber, float delta)
-      throws FileNotFoundException {
-
-    Scanner grammarScanner1 = new Scanner(grammarFile1);
-    Scanner grammarScanner2 = new Scanner(grammarFile2);
-
-    Set<String> set = new HashSet<String>();
-
-    int counter = 0;
-    float totalOverDiffs = 0.0f;
-    while (grammarScanner1.hasNextLine() && grammarScanner2.hasNextLine()) {
-
-      counter++;
-
-      String line1 = grammarScanner1.nextLine();
-      String[] fields1 = line1.split(fieldDelimiter);
-      String[] scores1 = fields1[fieldNumber].split(scoresDelimiter);
-      float score1 = Float.valueOf(scores1[scoresFieldNumber]);
-
-      String line2 = grammarScanner2.nextLine();
-      String[] fields2 = line2.split(fieldDelimiter);
-      String[] scores2 = fields2[fieldNumber].split(scoresDelimiter);
-      float score2 = Float.valueOf(scores2[scoresFieldNumber]);
-
-      if (fields1[0].endsWith(fields2[0]) && fields1[1].endsWith(fields2[1])
-          && fields1[1].endsWith(fields2[1])) {
-
-        float diff1 = Math.abs(score1 - score2);
-        float diff2 = Math.abs(score2 - score1);
-        float diff = (diff1 < diff2) ? diff1 : diff2;
-
-        if (diff > delta) {
-          logger.fine("Line " + counter + ":  Score mismatch: " + score1 + " vs " + score2);
-          set.add(line1);
-          totalOverDiffs += diff;
-        } else if (logger.isLoggable(Level.FINEST)) {
-          logger.finest("Line " + counter + ": Scores MATCH: " + score1 + " vs " + score2);
-        }
-
-      } else {
-        throw new RuntimeException("Lines don't match: " + line1 + " and " + line2);
-      }
-    }
-    
-    grammarScanner1.close();
-    grammarScanner2.close();
-    
-    if (set.isEmpty()) {
-      logger.info("No score mismatches");
-    } else {
-      logger.warning("Number of mismatches: " + set.size() + " out of " + counter);
-      logger.warning("Total mismatch logProb mass: " + totalOverDiffs + " (" + totalOverDiffs
-          / set.size() + ") (" + totalOverDiffs / counter + ")");
-    }
-  }
-
-  /**
-   * Main method.
-   * 
-   * @param args names of the two grammars to be compared
-   * @throws FileNotFoundException
-   */
-  public static void main(String[] args) throws FileNotFoundException {
-
-    if (args.length != 2) {
-      logger.severe("Usage: " + CompareGrammars.class.toString() + " grammarFile1 grammarFile2");
-      System.exit(-1);
-    }
-
-    // Tell standard in and out to use UTF-8
-    FormatUtils.useUTF8();
-    logger.finest("Using UTF-8");
-
-    logger.info("Comparing grammar files " + args[0] + " and " + args[1]);
-
-    File grammarFile1 = new File(args[0]);
-    File grammarFile2 = new File(args[1]);
-
-    String fieldDelimiter = HieroFormatReader.getFieldDelimiter();
-
-    boolean compareScores = true;
-
-    // Compare left-hand sides
-    {
-      Set<String> leftHandSides1 = getFields(grammarFile1, fieldDelimiter, 0);
-      Set<String> leftHandSides2 = getFields(grammarFile2, fieldDelimiter, 0);
-
-      if (leftHandSides1.equals(leftHandSides2)) {
-        logger.info("Grammar files have the same set of left-hand sides");
-      } else {
-        logger.warning("Grammar files have differing sets of left-hand sides");
-        compareScores = false;
-      }
-    }
-
-    // Compare source right-hand sides
-    {
-      Set<String> sourceRHSs1 = getFields(grammarFile1, fieldDelimiter, 1);
-      Set<String> sourceRHSs2 = getFields(grammarFile2, fieldDelimiter, 1);
-
-      if (sourceRHSs1.equals(sourceRHSs2)) {
-        logger.info("Grammar files have the same set of source right-hand sides");
-      } else {
-        logger.warning("Grammar files have differing sets of source right-hand sides");
-        compareScores = false;
-      }
-    }
-
-
-    // Compare target right-hand sides
-    {
-      Set<String> targetRHSs1 = getFields(grammarFile1, fieldDelimiter, 2);
-      Set<String> targetRHSs2 = getFields(grammarFile2, fieldDelimiter, 2);
-
-      if (targetRHSs1.equals(targetRHSs2)) {
-        logger.info("Grammar files have the same set of target right-hand sides");
-      } else {
-        logger.warning("Grammar files have differing sets of target right-hand sides");
-        compareScores = false;
-      }
-    }
-
-    // Compare translation probs
-    if (compareScores) {
-      float delta = 0.001f;
-      compareValues(grammarFile1, grammarFile2, fieldDelimiter, 3, "\\s+", 0, delta);
-      compareValues(grammarFile1, grammarFile2, fieldDelimiter, 3, "\\s+", 1, delta);
-      compareValues(grammarFile1, grammarFile2, fieldDelimiter, 3, "\\s+", 2, delta);
-
-    }
-
-  }
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/76eb9587/src/joshua/util/FormatUtils.java
----------------------------------------------------------------------
diff --git a/src/joshua/util/FormatUtils.java b/src/joshua/util/FormatUtils.java
index 67b2bf3..c925cba 100644
--- a/src/joshua/util/FormatUtils.java
+++ b/src/joshua/util/FormatUtils.java
@@ -78,6 +78,15 @@ public class FormatUtils {
     return markup(cleanNonTerminal(nt));
   }
 
+  /**
+   * Nonterminals on source and target sides are represented as [X,1], where 1 is an integer
+   * that links the two sides. This function extracts the index, e.g.,
+   * 
+   * getNonterminalIndex("[X,7]") -> 7
+   * 
+   * @param the nonterminal index
+   * @return
+   */
   public static int getNonterminalIndex(String nt) {
     return Integer.parseInt(nt.substring(nt.indexOf(INDEX_SEPARATOR) + 1, nt.length() - 1));
   }


[03/15] incubator-joshua git commit: added license header

Posted by mj...@apache.org.
added license header


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/b11b2514
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/b11b2514
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/b11b2514

Branch: refs/heads/master
Commit: b11b2514f845657ea2815d0c0ab569d6fafdec0f
Parents: afa66ef
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 00:19:21 2016 +0200
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 00:19:21 2016 +0200

----------------------------------------------------------------------
 test/decoder/phrase/decode/test.sh | 17 ++++++++++++++++-
 1 file changed, 16 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/b11b2514/test/decoder/phrase/decode/test.sh
----------------------------------------------------------------------
diff --git a/test/decoder/phrase/decode/test.sh b/test/decoder/phrase/decode/test.sh
index 4732f73..7703aa4 100755
--- a/test/decoder/phrase/decode/test.sh
+++ b/test/decoder/phrase/decode/test.sh
@@ -1,5 +1,20 @@
 #!/bin/bash
-
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
 set -u
 
 cat corpus.es | $JOSHUA/bin/joshua-decoder -threads 1 -c config > output 2> log


[13/15] incubator-joshua git commit: Merge branch 'JOSHUA-272'

Posted by mj...@apache.org.
Merge branch 'JOSHUA-272'


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/aef0b2db
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/aef0b2db
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/aef0b2db

Branch: refs/heads/master
Commit: aef0b2dbe4555070aec9f15bb2c8d9dcb5671dcd
Parents: d0f7b53 868b340
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 05:52:11 2016 -0400
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 05:52:11 2016 -0400

----------------------------------------------------------------------
 scripts/support/grammar-packer.pl               |   4 +-
 scripts/support/moses2joshua_grammar.pl         |  13 +-
 scripts/support/moses_phrase_to_joshua.pl       |  23 ---
 src/joshua/corpus/TerminalIterator.java         |   4 +-
 src/joshua/corpus/Vocabulary.java               |  16 +-
 src/joshua/decoder/BLEU.java                    |   1 -
 src/joshua/decoder/Decoder.java                 |   1 -
 src/joshua/decoder/JoshuaConfiguration.java     |  14 +-
 src/joshua/decoder/ff/TargetBigram.java         |   3 +-
 src/joshua/decoder/ff/lm/LanguageModelFF.java   |   5 +-
 .../ff/lm/StateMinimizingLanguageModel.java     |   5 +-
 src/joshua/decoder/ff/tm/GrammarReader.java     |  55 +----
 src/joshua/decoder/ff/tm/PhraseRule.java        |  94 ---------
 src/joshua/decoder/ff/tm/Rule.java              |  30 +--
 .../decoder/ff/tm/format/HieroFormatReader.java |  85 ++++----
 .../decoder/ff/tm/format/MosesFormatReader.java | 106 ++++++++++
 .../ff/tm/format/PhraseFormatReader.java        | 128 ------------
 .../decoder/ff/tm/format/SamtFormatReader.java  | 136 ------------
 .../tm/hash_based/MemoryBasedBatchGrammar.java  |  12 +-
 .../decoder/ff/tm/packed/PackedGrammar.java     | 137 +++---------
 .../GrammarBuilderWalkerFunction.java           |  26 +--
 .../hypergraph/OutputStringExtractor.java       |   8 +-
 src/joshua/decoder/phrase/PhraseTable.java      |  13 +-
 src/joshua/decoder/phrase/Stacks.java           |   2 +-
 src/joshua/decoder/segment_file/Sentence.java   |   1 -
 src/joshua/metrics/MinimumChangeBLEU.java       |   1 -
 src/joshua/metrics/Precis.java                  |   1 -
 src/joshua/oracle/OracleExtractionHG.java       |   3 +-
 src/joshua/tools/GrammarPacker.java             | 161 ++++++---------
 src/joshua/util/CompareGrammars.java            | 207 -------------------
 src/joshua/util/FormatUtils.java                |  28 ++-
 test/decoder/phrase/decode/config               |   2 +-
 test/decoder/phrase/decode/rules.packed/config  |   3 +-
 .../decode/rules.packed/slice_00000.features    | Bin 4128858 -> 4128858 bytes
 .../decode/rules.packed/slice_00000.source      | Bin 1982228 -> 1982244 bytes
 .../decode/rules.packed/slice_00000.target      | Bin 1463856 -> 2652936 bytes
 .../rules.packed/slice_00000.target.lookup      | Bin 28 -> 32 bytes
 .../phrase/decode/rules.packed/vocabulary       | Bin 169225 -> 169236 bytes
 test/decoder/phrase/decode/test.sh              |  17 +-
 39 files changed, 350 insertions(+), 995 deletions(-)
----------------------------------------------------------------------



[07/15] incubator-joshua git commit: Removed call to Moses phrase table conversion script; phrase decoding now works (not backwards combat!)

Posted by mj...@apache.org.
Removed call to Moses phrase table conversion script; phrase decoding now works (not backwards combat!)


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/a43f7353
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/a43f7353
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/a43f7353

Branch: refs/heads/master
Commit: a43f7353e78ccbedabcf144dad33c57e864bbf84
Parents: f5adcde
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 09:14:23 2016 +0200
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 09:14:23 2016 +0200

----------------------------------------------------------------------
 scripts/support/grammar-packer.pl         |  2 +-
 scripts/support/moses_phrase_to_joshua.pl | 23 -----------------------
 2 files changed, 1 insertion(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/a43f7353/scripts/support/grammar-packer.pl
----------------------------------------------------------------------
diff --git a/scripts/support/grammar-packer.pl b/scripts/support/grammar-packer.pl
index d674d75..7cd3153 100755
--- a/scripts/support/grammar-packer.pl
+++ b/scripts/support/grammar-packer.pl
@@ -76,7 +76,7 @@ foreach my $grammar (@grammars) {
     }
   } else {
     # Moses phrase-based grammar -- prepend nonterminal symbol and -log() the weights
-    if (system("$CAT $grammar | $JOSHUA/scripts/support/moses_phrase_to_joshua.pl | sed 's/ ||| /\t/g' | LC_ALL=C sort -t'\t' -k2,2 -k3,3 --buffer-size=$opts{m} -T $opts{T} | sed 's/\t/ ||| /g' | gzip -9n > $sorted_grammar")) {
+    if (system("$CAT $grammar | sed 's/ ||| /\t/g' | LC_ALL=C sort -t'\t' -k1,1 -k2,2 --buffer-size=$opts{m} -T $opts{T} | sed 's/\t/ ||| /g' | gzip -9n > $sorted_grammar")) {
       print STDERR "* FATAL: Couldn't sort the grammar (not enough memory? short on tmp space?)\n";
       exit 2;
     }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/a43f7353/scripts/support/moses_phrase_to_joshua.pl
----------------------------------------------------------------------
diff --git a/scripts/support/moses_phrase_to_joshua.pl b/scripts/support/moses_phrase_to_joshua.pl
deleted file mode 100755
index 9fe727e..0000000
--- a/scripts/support/moses_phrase_to_joshua.pl
+++ /dev/null
@@ -1,23 +0,0 @@
-#!/usr/bin/perl
-
-# Converts a Moses phrase table to a Joshua grammar, suitable for packing.
-# (Joshua can read in Moses phrase tables directly when using the in-memory
-# representation, so in that case there is no need to do the conversion).
-
-use strict;
-use warnings;
-
-while (my $line = <>) {
-  chomp($line);
-  my @tokens = split(/ \|\|\| /, $line);
-
-  unshift(@tokens, "[X]");
-  $tokens[3] = join(" ", map { -mylog($_) } split(' ', $tokens[3]));
-
-  print join(" ||| ", @tokens) . $/;
-}
-
-sub mylog {
-  my ($num) = @_;
-  return ($num == 0) ? -100 : log($num);
-}


[05/15] incubator-joshua git commit: GrammarReader iterator now initialized in constructor

Posted by mj...@apache.org.
GrammarReader iterator now initialized in constructor


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/210573bb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/210573bb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/210573bb

Branch: refs/heads/master
Commit: 210573bbdf8ca6c1633e2f11f21fd03cf33473ca
Parents: 4781405
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 01:08:44 2016 +0200
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 01:08:44 2016 +0200

----------------------------------------------------------------------
 src/joshua/decoder/ff/tm/GrammarReader.java            | 13 ++-----------
 .../ff/tm/hash_based/MemoryBasedBatchGrammar.java      |  1 -
 2 files changed, 2 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/210573bb/src/joshua/decoder/ff/tm/GrammarReader.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/GrammarReader.java b/src/joshua/decoder/ff/tm/GrammarReader.java
index 7edab7c..3b973a2 100644
--- a/src/joshua/decoder/ff/tm/GrammarReader.java
+++ b/src/joshua/decoder/ff/tm/GrammarReader.java
@@ -51,18 +51,9 @@ public abstract class GrammarReader<R extends Rule> implements Iterable<R>, Iter
     this.fileName = null;
   }
 
-  public GrammarReader(String fileName) {
+  public GrammarReader(String fileName) throws IOException {
     this.fileName = fileName;
-  }
-
-  public void initialize() {
-    try {
-      this.reader = new LineReader(fileName);
-    } catch (IOException e) {
-      throw new RuntimeException("Error opening translation model file: " + fileName + "\n"
-          + (null != e.getMessage() ? e.getMessage() : "No details available. Sorry."), e);
-    }
-
+    this.reader = new LineReader(fileName);
     Decoder.LOG(1, String.format("Reading grammar from file %s...", fileName));
     numRulesRead = 0;
     advanceReader();

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/210573bb/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java b/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
index e85ce09..f0c9ac8 100644
--- a/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
+++ b/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
@@ -111,7 +111,6 @@ public class MemoryBasedBatchGrammar extends AbstractGrammar {
     // ==== loading grammar
     this.modelReader = createReader(formatKeyword, grammarFile);
     if (modelReader != null) {
-      modelReader.initialize();
       for (Rule rule : modelReader)
         if (rule != null) {
           addRule(rule);


[06/15] incubator-joshua git commit: Fixed grammar packing, big change

Posted by mj...@apache.org.
Fixed grammar packing, big change

- GrammarPacker now uses the appropriate {Moses,Hiero}FormatReader objects in explore() and binarize() passes, instead of doing its own parsing
- MosesFormatReader chains to HieroFormatReader after munging input, removes some redundancy
- Updated test case


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/f5adcdef
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/f5adcdef
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/f5adcdef

Branch: refs/heads/master
Commit: f5adcdefd3e52c58b36713798c0830d1c42099e3
Parents: 210573b
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 01:11:30 2016 +0200
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 01:11:30 2016 +0200

----------------------------------------------------------------------
 scripts/support/grammar-packer.pl               |   2 +-
 .../decoder/ff/tm/format/HieroFormatReader.java |  13 +-
 .../decoder/ff/tm/format/MosesFormatReader.java |  43 ++----
 .../tm/hash_based/MemoryBasedBatchGrammar.java  |   2 +-
 src/joshua/tools/GrammarPacker.java             | 131 +++++++------------
 test/decoder/phrase/decode/rules.packed/config  |   2 +-
 .../decode/rules.packed/slice_00000.features    | Bin 4128858 -> 4128858 bytes
 .../decode/rules.packed/slice_00000.source      | Bin 1982228 -> 1982228 bytes
 .../decode/rules.packed/slice_00000.target      | Bin 1463856 -> 1463856 bytes
 .../phrase/decode/rules.packed/vocabulary       | Bin 169225 -> 169225 bytes
 10 files changed, 66 insertions(+), 127 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f5adcdef/scripts/support/grammar-packer.pl
----------------------------------------------------------------------
diff --git a/scripts/support/grammar-packer.pl b/scripts/support/grammar-packer.pl
index e485513..d674d75 100755
--- a/scripts/support/grammar-packer.pl
+++ b/scripts/support/grammar-packer.pl
@@ -90,7 +90,7 @@ foreach my $grammar (@grammars) {
 my $grammars = join(" ", @sorted_grammars);
 my $outputs  = join(" ", @outputs);
 my $alignments = $opts{a} ? "--ga" : "";
-my $cmd = "java -Xmx$opts{m} -cp $JOSHUA/lib/args4j-2.0.29.jar:$JOSHUA/class joshua.tools.GrammarPackerCli -g $grammars --outputs $outputs $alignments";
+my $cmd = "java -Xmx$opts{m} -cp $JOSHUA/lib/args4j-2.0.29.jar:$JOSHUA/lib/guava-19.0.jar:$JOSHUA/class joshua.tools.GrammarPackerCli -g $grammars --outputs $outputs $alignments";
 print STDERR "Packing with $cmd...\n" if $opts{v};
 
 my $retval = system($cmd);

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f5adcdef/src/joshua/decoder/ff/tm/format/HieroFormatReader.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/format/HieroFormatReader.java b/src/joshua/decoder/ff/tm/format/HieroFormatReader.java
index d2a01eb..92d7ddb 100644
--- a/src/joshua/decoder/ff/tm/format/HieroFormatReader.java
+++ b/src/joshua/decoder/ff/tm/format/HieroFormatReader.java
@@ -18,6 +18,8 @@
  */
 package joshua.decoder.ff.tm.format;
 
+import java.io.IOException;
+
 import joshua.corpus.Vocabulary;
 import joshua.decoder.ff.tm.GrammarReader;
 import joshua.decoder.ff.tm.Rule;
@@ -41,7 +43,7 @@ public class HieroFormatReader extends GrammarReader<Rule> {
     super();
   }
 
-  public HieroFormatReader(String grammarFile) {
+  public HieroFormatReader(String grammarFile) throws IOException {
     super(grammarFile);
   }
 
@@ -52,7 +54,7 @@ public class HieroFormatReader extends GrammarReader<Rule> {
       throw new RuntimeException(String.format("Rule '%s' does not have four fields", line));
     }
 
-    int lhs = Vocabulary.id(FormatUtils.stripNonTerminalIndex(fields[0]));
+    int lhs = Vocabulary.id(fields[0]);
 
     /**
      * On the foreign side, we map nonterminals to negative IDs, and terminals to positive IDs.
@@ -61,16 +63,14 @@ public class HieroFormatReader extends GrammarReader<Rule> {
     String[] sourceWords = fields[1].split("\\s+");
     int[] sourceIDs = new int[sourceWords.length];
     for (int i = 0; i < sourceWords.length; i++) {
+      sourceIDs[i] = Vocabulary.id(sourceWords[i]);
       if (FormatUtils.isNonterminal(sourceWords[i])) {
-        Vocabulary.id(sourceWords[i]);
         sourceIDs[i] = Vocabulary.id(FormatUtils.stripNonTerminalIndex(sourceWords[i]));
         arity++;
         
         // TODO: the arity here (after incrementing) should match the rule index. Should
         // check that arity == FormatUtils.getNonterminalIndex(foreignWords[i]), throw runtime
         // error if not
-      } else {
-        sourceIDs[i] = Vocabulary.id(sourceWords[i]);
       }
     }
 
@@ -86,10 +86,9 @@ public class HieroFormatReader extends GrammarReader<Rule> {
     String[] targetWords = fields[2].split("\\s+");
     int[] targetIDs = new int[targetWords.length];
     for (int i = 0; i < targetWords.length; i++) {
+      targetIDs[i] = Vocabulary.id(targetWords[i]);
       if (FormatUtils.isNonterminal(targetWords[i])) {
         targetIDs[i] = -FormatUtils.getNonterminalIndex(targetWords[i]);
-      } else {
-        targetIDs[i] = Vocabulary.id(targetWords[i]);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f5adcdef/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/format/MosesFormatReader.java b/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
index 0b33ba1..d7710f2 100644
--- a/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
+++ b/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
@@ -18,6 +18,8 @@
  */
 package joshua.decoder.ff.tm.format;
 
+import java.io.IOException;
+
 import joshua.corpus.Vocabulary;
 import joshua.decoder.ff.tm.Rule;
 import joshua.util.io.LineReader;
@@ -44,7 +46,7 @@ public class MosesFormatReader extends HieroFormatReader {
 
   private int lhs;
   
-  public MosesFormatReader(String grammarFile) {
+  public MosesFormatReader(String grammarFile) throws IOException {
     super(grammarFile);
     this.lhs = Vocabulary.id("[X]");
   }
@@ -76,44 +78,23 @@ public class MosesFormatReader extends HieroFormatReader {
     String[] fields = line.split(fieldDelimiter);
 
     int arity = 1;
-    int fieldIndex = 0;
-    
-    // foreign side
-    String[] foreignWords = fields[fieldIndex].split("\\s+");
-    int[] french = new int[foreignWords.length + 1];
-    french[0] = lhs; 
-    for (int i = 0; i < foreignWords.length; i++) {
-      french[i+1] = Vocabulary.id(foreignWords[i]);
-    }
 
-    // English side
-    fieldIndex++;
-    String[] englishWords = fields[fieldIndex].split("\\s+");
-    int[] english = new int[englishWords.length + 1];
-    english[0] = -1;
-    for (int i = 0; i < englishWords.length; i++) {
-      english[i+1] = Vocabulary.id(englishWords[i]);
-    }
+    StringBuffer hieroLine = new StringBuffer();
+    hieroLine.append("[X] ||| [X,1] " + fields[0] + " ||| [X,1] " + fields[1] + " |||");
 
-    // transform feature values
-    fieldIndex++;
-    
-    String mosesFeatureString = fields[fieldIndex];
-    StringBuffer values = new StringBuffer();
+    String mosesFeatureString = fields[2];
     for (String value: mosesFeatureString.split(" ")) {
       float f = Float.parseFloat(value);
-      values.append(String.format("%f ", f <= 0.0 ? -100 : -Math.log(f)));
+      hieroLine.append(String.format(" %f", f <= 0.0 ? -100 : -Math.log(f)));
     }
 
-    String sparse_features = values.toString().trim();
-
-//    System.out.println(String.format("parseLine: %s\n  ->%s", line, sparse_features));
-
     // alignments
-    fieldIndex++;
-    String alignment = (fields.length > fieldIndex) ? fields[fieldIndex] : null;
+    if (fields.length >= 4)
+      hieroLine.append(" ||| " + fields[3]);
 
-    return new Rule(lhs, french, english, sparse_features, arity, alignment);
+    System.err.println(String.format("LINE: %s -> %s", line, hieroLine.toString()));
+    
+    return super.parseLine(hieroLine.toString());
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f5adcdef/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java b/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
index f0c9ac8..6740c26 100644
--- a/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
+++ b/src/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
@@ -123,7 +123,7 @@ public class MemoryBasedBatchGrammar extends AbstractGrammar {
     this.printGrammar();
   }
 
-  protected GrammarReader<Rule> createReader(String format, String grammarFile) {
+  protected GrammarReader<Rule> createReader(String format, String grammarFile) throws IOException {
 
     if (grammarFile != null) {
       if ("hiero".equals(format) || "thrax".equals(format) || "regexp".equals(format)) {

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f5adcdef/src/joshua/tools/GrammarPacker.java
----------------------------------------------------------------------
diff --git a/src/joshua/tools/GrammarPacker.java b/src/joshua/tools/GrammarPacker.java
index 33d3391..21072c6 100644
--- a/src/joshua/tools/GrammarPacker.java
+++ b/src/joshua/tools/GrammarPacker.java
@@ -37,6 +37,9 @@ import java.util.TreeMap;
 import java.util.logging.Logger;
 
 import joshua.corpus.Vocabulary;
+import joshua.decoder.ff.tm.Rule;
+import joshua.decoder.ff.tm.format.HieroFormatReader;
+import joshua.decoder.ff.tm.format.MosesFormatReader;
 import joshua.util.FormatUtils;
 import joshua.util.encoding.EncoderConfiguration;
 import joshua.util.encoding.FeatureTypeAnalyzer;
@@ -154,13 +157,12 @@ public class GrammarPacker {
    */
   public void pack() throws IOException {
     logger.info("Beginning exploration pass.");
-    LineReader grammar_reader = null;
-    LineReader alignment_reader = null;
 
     // Explore pass. Learn vocabulary and feature value histograms.
     logger.info("Exploring: " + grammar);
-    grammar_reader = new LineReader(grammar);
-    explore(grammar_reader);
+    
+    HieroFormatReader grammarReader = getGrammarReader();
+    explore(grammarReader);
 
     logger.info("Exploration pass complete. Freezing vocabulary and finalizing encoders.");
     if (dump != null) {
@@ -194,78 +196,56 @@ public class GrammarPacker {
 
     logger.info("Beginning packing pass.");
     // Actual binarization pass. Slice and pack source, target and data.
-    grammar_reader = new LineReader(grammar);
-
+    grammarReader = getGrammarReader();
+    LineReader alignment_reader = null;
     if (packAlignments && !grammarAlignments)
       alignment_reader = new LineReader(alignments);
-    binarize(grammar_reader, alignment_reader);
+    binarize(grammarReader, alignment_reader);
     logger.info("Packing complete.");
 
     logger.info("Packed grammar in: " + output);
     logger.info("Done.");
   }
 
-  private void explore(LineReader grammar) {
+  /**
+   * Returns a reader that turns whatever file format is found into Hiero grammar rules.
+   * 
+   * @param grammarFile
+   * @return
+   * @throws IOException 
+   */
+  private HieroFormatReader getGrammarReader() throws IOException {
+    LineReader reader = new LineReader(grammar);
+    String line = reader.next();
+    if (line.startsWith("[")) {
+      return new HieroFormatReader(grammar);
+    } else {
+      return new MosesFormatReader(grammar);
+    }
+  }
+
+  private void explore(HieroFormatReader reader) {
     int counter = 0;
     // We always assume a labeled grammar. Unlabeled features are assumed to be dense and to always
     // appear in the same order. They are assigned numeric names in order of appearance.
     this.types.setLabeled(true);
 
-    while (grammar.hasNext()) {
-      String line = grammar.next().trim();
+    for (Rule rule: reader) {
       counter++;
-      ArrayList<String> fields = new ArrayList<String>(Arrays.asList(line.split("\\s\\|{3}\\s")));
-
-      String lhs = null;
-      if (line.startsWith("[")) {
-        // hierarchical model
-        if (fields.size() < 4) {
-          logger.warning(String.format("Incomplete grammar line at line %d: '%s'", counter, line));
-          continue;
-        }
-        lhs = fields.remove(0);
-      } else {
-        // phrase-based model
-        if (fields.size() < 3) {
-          logger.warning("Incomplete phrase line at line " + counter);
-          logger.warning(line);
-          continue;
-        }
-        lhs = "[X]";
-      }
 
-      String[] source = fields.get(0).split("\\s");
-      String[] target = fields.get(1).split("\\s");
-      String[] features = fields.get(2).split("\\s");
-      
-      max_source_len = Math.max(max_source_len, source.length);
-
-      Vocabulary.id(lhs);
-      try {
-        /* Add symbols to vocabulary.
-         * NOTE: In case of nonterminals, we add both stripped versions ("[X]")
-         * and "[X,1]" to the vocabulary.
-         */
-        for (String source_word : source) {
-          Vocabulary.id(source_word);
-          if (FormatUtils.isNonterminal(source_word)) {
-            Vocabulary.id(FormatUtils.stripNonTerminalIndex(source_word));
-          }
-        }
-        for (String target_word : target) {
-          Vocabulary.id(target_word);
-          if (FormatUtils.isNonterminal(target_word)) {
-            Vocabulary.id(FormatUtils.stripNonTerminalIndex(target_word));
-          }
-        }
-      } catch (java.lang.StringIndexOutOfBoundsException e) {
-        System.err.println(String.format("* Skipping bad grammar line '%s'", line));
-        continue;
-      }
+      max_source_len = Math.max(max_source_len, rule.getFrench().length);
+
+      /* Add symbols to vocabulary.
+       * NOTE: In case of nonterminals, we add both stripped versions ("[X]")
+       * and "[X,1]" to the vocabulary.
+       * 
+       * TODO: MJP May 2016: do we need to add [X,1]? If so, should be done in FormatReaders.
+       */
 
       // Add feature names to vocabulary and pass the value through the
       // appropriate encoder.
       int feature_counter = 0;
+      String[] features = rule.getFeatureString().split("\\s+");
       for (int f = 0; f < features.length; ++f) {
         if (features[f].contains("=")) {
           String[] fe = features[f].split("=");
@@ -278,6 +258,7 @@ public class GrammarPacker {
         }
       }
     }
+    System.err.println("COUNTER: " + counter);
   }
 
   /**
@@ -288,7 +269,7 @@ public class GrammarPacker {
     return source_words[0] + SOURCE_WORDS_SEPARATOR + ((source_words.length > 1) ? source_words[1] : "");
   }
 
-  private void binarize(LineReader grammar_reader, LineReader alignment_reader) throws IOException {
+  private void binarize(HieroFormatReader grammarReader, LineReader alignment_reader) throws IOException {
     int counter = 0;
     int slice_counter = 0;
     int num_slices = 0;
@@ -306,36 +287,14 @@ public class GrammarPacker {
       alignment_buffer = new AlignmentBuffer();
 
     TreeMap<Integer, Float> features = new TreeMap<Integer, Float>();
-    while (grammar_reader.hasNext()) {
-      String grammar_line = grammar_reader.next().trim();
+    for (Rule rule: grammarReader) {
       counter++;
       slice_counter++;
 
-      ArrayList<String> fields = new ArrayList<String>(Arrays.asList(grammar_line.split("\\s\\|{3}\\s")));
-      String lhs_word;
-      String[] source_words;
-      String[] target_words;
-      String[] feature_entries;
-      if (grammar_line.startsWith("[")) {
-        if (fields.size() < 4)
-          continue;
-
-        lhs_word = fields.remove(0);
-        source_words = fields.get(0).split("\\s");
-        target_words = fields.get(1).split("\\s");
-        feature_entries = fields.get(2).split("\\s");
-
-      } else {
-        if (fields.size() < 3)
-          continue;
-        
-        lhs_word = "[X]";
-        String tmp = "[X,1] " + fields.get(0);
-        source_words = tmp.split("\\s");
-        tmp = "[X,1] " + fields.get(1);
-        target_words = tmp.split("\\s");
-        feature_entries = fields.get(2).split("\\s");
-      }
+      String lhs_word = Vocabulary.word(rule.getLHS());
+      String[] source_words = rule.getFrenchWords().split("\\s+");
+      String[] target_words = rule.getEnglishWords().split("\\s+");
+      String[] feature_entries = rule.getFeatureString().split("\\s+");
 
       // Reached slice limit size, indicate that we're closing up.
       if (!ready_to_flush
@@ -373,7 +332,7 @@ public class GrammarPacker {
       if (packAlignments) {
         String alignment_line;
         if (grammarAlignments) {
-          alignment_line = fields.get(3);
+          alignment_line = rule.getAlignmentString();
         } else {
           if (!alignment_reader.hasNext()) {
             logger.severe("No more alignments starting in line " + counter);

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f5adcdef/test/decoder/phrase/decode/rules.packed/config
----------------------------------------------------------------------
diff --git a/test/decoder/phrase/decode/rules.packed/config b/test/decoder/phrase/decode/rules.packed/config
index 287da2d..4c0c6b5 100644
--- a/test/decoder/phrase/decode/rules.packed/config
+++ b/test/decoder/phrase/decode/rules.packed/config
@@ -1 +1 @@
-max-source-len = 5
+max-source-len = 3

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f5adcdef/test/decoder/phrase/decode/rules.packed/slice_00000.features
----------------------------------------------------------------------
diff --git a/test/decoder/phrase/decode/rules.packed/slice_00000.features b/test/decoder/phrase/decode/rules.packed/slice_00000.features
index b67c809..ebefdd5 100644
Binary files a/test/decoder/phrase/decode/rules.packed/slice_00000.features and b/test/decoder/phrase/decode/rules.packed/slice_00000.features differ

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f5adcdef/test/decoder/phrase/decode/rules.packed/slice_00000.source
----------------------------------------------------------------------
diff --git a/test/decoder/phrase/decode/rules.packed/slice_00000.source b/test/decoder/phrase/decode/rules.packed/slice_00000.source
index 8679998..cdc98f6 100644
Binary files a/test/decoder/phrase/decode/rules.packed/slice_00000.source and b/test/decoder/phrase/decode/rules.packed/slice_00000.source differ

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f5adcdef/test/decoder/phrase/decode/rules.packed/slice_00000.target
----------------------------------------------------------------------
diff --git a/test/decoder/phrase/decode/rules.packed/slice_00000.target b/test/decoder/phrase/decode/rules.packed/slice_00000.target
index 07aefbe..fa82c0d 100644
Binary files a/test/decoder/phrase/decode/rules.packed/slice_00000.target and b/test/decoder/phrase/decode/rules.packed/slice_00000.target differ

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f5adcdef/test/decoder/phrase/decode/rules.packed/vocabulary
----------------------------------------------------------------------
diff --git a/test/decoder/phrase/decode/rules.packed/vocabulary b/test/decoder/phrase/decode/rules.packed/vocabulary
index 8b5e4d5..ff62042 100644
Binary files a/test/decoder/phrase/decode/rules.packed/vocabulary and b/test/decoder/phrase/decode/rules.packed/vocabulary differ


[14/15] incubator-joshua git commit: pipeline no longer depends on $HADOOP, also will not setup a cluster for you

Posted by mj...@apache.org.
pipeline no longer depends on $HADOOP, also will not setup a cluster for you

The pipeline used to have the option of setting up a Hadoop cluster for you, rolling it out in standalone mode. This is not a good solution to do here; it's much better to have the user do it, wrapping Joshua within the many tools available to do so. Therefore, this functionality is removed. The developer is now responsible for setting up her own Hadoop cluster and ensuring that the "hadoop" command is in her path.


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/78f727f3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/78f727f3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/78f727f3

Branch: refs/heads/master
Commit: 78f727f3a2fd6616b3f77ca50655c464a91671b4
Parents: aef0b2d
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 06:01:43 2016 -0400
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 06:01:43 2016 -0400

----------------------------------------------------------------------
 scripts/training/pipeline.pl | 89 ++++-----------------------------------
 1 file changed, 8 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/78f727f3/scripts/training/pipeline.pl
----------------------------------------------------------------------
diff --git a/scripts/training/pipeline.pl b/scripts/training/pipeline.pl
index ff9d762..88d641c 100755
--- a/scripts/training/pipeline.pl
+++ b/scripts/training/pipeline.pl
@@ -40,7 +40,6 @@ use v5.12;
 # Thus we undefine CDPATH to ensure this doesn't happen.
 delete $ENV{CDPATH};
 
-my $HADOOP = $ENV{HADOOP};
 my $MOSES = $ENV{MOSES};
 my $METEOR = $ENV{METEOR};
 my $THRAX = "$JOSHUA/thrax";
@@ -127,9 +126,6 @@ my $JOSHUA_MEM = "4g";
 # Hadoop via -Dmapred.child.java.opts
 my $HADOOP_MEM = "4g";
 
-# The location of a custom core-site.xml file, if desired (optional).
-my $HADOOP_CONF = undef;
-
 # memory available to the parser
 my $PARSER_MEM = "2g";
 
@@ -291,8 +287,6 @@ my $retval = GetOptions(
   "first-step=s"     => \$FIRST_STEP,
   "last-step=s"      => \$LAST_STEP,
   "aligner-chunk-size=s" => \$ALIGNER_BLOCKSIZE,
-  "hadoop=s"          => \$HADOOP,
-  "hadoop-conf=s"          => \$HADOOP_CONF,
   "tmp=s"             => \$TMPDIR,
   "nbest=i"           => \$NBEST,
   "reordering-limit=i" => \$REORDERING_LIMIT,
@@ -504,7 +498,6 @@ $_TUNE_GRAMMAR_FILE = get_absolute_path($_TUNE_GRAMMAR_FILE);
 $_TEST_GRAMMAR_FILE = get_absolute_path($_TEST_GRAMMAR_FILE);
 $THRAX_CONF_FILE = get_absolute_path($THRAX_CONF_FILE);
 $ALIGNMENT = get_absolute_path($ALIGNMENT);
-$HADOOP_CONF = get_absolute_path($HADOOP_CONF);
 
 foreach my $corpus (@CORPORA) {
   foreach my $ext ($TARGET,$SOURCE) {
@@ -561,11 +554,6 @@ if ($FILTERING eq "fast") {
   exit 1;
 }
 
-if (defined $HADOOP_CONF && ! -e $HADOOP_CONF) {
-  print STDERR "* FATAL: Couldn't find \$HADOOP_CONF file '$HADOOP_CONF'\n";
-  exit 1;
-}
-
 ## END SANITY CHECKS
 
 ####################################################################################################
@@ -1125,29 +1113,17 @@ if (! defined $GRAMMAR_FILE) {
                     $TRAIN{source}, $target_file, $ALIGNMENT,
                     "$DATA_DIRS{train}/thrax-input-file");
 
-    # Rollout the hadoop cluster if needed.  This causes $HADOOP to be defined (pointing to the
-    # unrolled directory).
-    start_hadoop_cluster() unless defined $HADOOP;
-
     # put the hadoop files in place
-    my $THRAXDIR;
     my $thrax_input;
-    if (! defined $HADOOP or $HADOOP eq "") {
-      $THRAXDIR = "thrax";
-
-      $thrax_input = "$DATA_DIRS{train}/thrax-input-file"
-
-    } else {
-      $THRAXDIR = "pipeline-$SOURCE-$TARGET-$GRAMMAR_TYPE-$RUNDIR";
-      $THRAXDIR =~ s#/#_#g;
+    my $THRAXDIR = "pipeline-$SOURCE-$TARGET-$GRAMMAR_TYPE-$RUNDIR";
+    $THRAXDIR =~ s#/#_#g;
 
-      $cachepipe->cmd("thrax-prep",
-                      "$HADOOP/bin/hadoop fs -rm -r $THRAXDIR; $HADOOP/bin/hadoop fs -mkdir $THRAXDIR; $HADOOP/bin/hadoop fs -put $DATA_DIRS{train}/thrax-input-file $THRAXDIR/input-file",
-                      "$DATA_DIRS{train}/thrax-input-file", 
-                      "grammar.gz");
+    $cachepipe->cmd("thrax-prep",
+                    "hadoop fs -rm -r $THRAXDIR; hadoop fs -mkdir $THRAXDIR; hadoop fs -put $DATA_DIRS{train}/thrax-input-file $THRAXDIR/input-file",
+                    "$DATA_DIRS{train}/thrax-input-file", 
+                    "grammar.gz");
 
-      $thrax_input = "$THRAXDIR/input-file";
-    }
+    $thrax_input = "$THRAXDIR/input-file";
 
     # copy the thrax config file
     my $thrax_file = "thrax-$GRAMMAR_TYPE.conf";
@@ -1156,25 +1132,12 @@ if (! defined $GRAMMAR_FILE) {
     system("mv $thrax_file.tmp $thrax_file");
 
     $cachepipe->cmd("thrax-run",
-                    "$HADOOP/bin/hadoop jar $THRAX/bin/thrax.jar -D mapreduce.task.timeout=0 -D mapreduce.map.java.opts='-Xmx$HADOOP_MEM' -D mapreduce.reduce.java.opts='-Xmx$HADOOP_MEM' -D hadoop.tmp.dir=$TMPDIR $thrax_file $THRAXDIR > thrax.log 2>&1; rm -f grammar grammar.gz; $HADOOP/bin/hadoop fs -getmerge $THRAXDIR/final/ grammar.gz", #; $HADOOP/bin/hadoop fs -rm -r $THRAXDIR",
+                    "hadoop jar $THRAX/bin/thrax.jar -D mapreduce.task.timeout=0 -D mapreduce.map.java.opts='-Xmx$HADOOP_MEM' -D mapreduce.reduce.java.opts='-Xmx$HADOOP_MEM' -D hadoop.tmp.dir=$TMPDIR $thrax_file $THRAXDIR > thrax.log 2>&1; rm -f grammar grammar.gz; hadoop fs -getmerge $THRAXDIR/final/ grammar.gz", #; hadoop fs -rm -r $THRAXDIR",
                     "$DATA_DIRS{train}/thrax-input-file",
                     $thrax_file,
                     "grammar.gz");
 #perl -pi -e 's/\.?0+\b//g' grammar; 
 
-    stop_hadoop_cluster() if $HADOOP eq "hadoop";
-
-    # cache the thrax-prep step, which depends on grammar.gz
-#    if ($HADOOP ne "hadoop") {
-#      $cachepipe->cmd("thrax-prep", "--cache-only");
-#    }
-
-    # clean up
-    # TODO: clean up real hadoop clusters too
-    # if ($HADOOP eq "hadoop") {
-    #   system("rm -rf $THRAXDIR hadoop hadoop-2.5.2");
-    # }
-
     $GRAMMAR_FILE = "grammar.gz";
   } else {
 
@@ -1973,42 +1936,6 @@ sub get_numrefs {
   }
 }
 
-sub start_hadoop_cluster {
-  rollout_hadoop_cluster();
-
-  # start the cluster
-  # system("./hadoop/bin/start-all.sh");
-  # sleep(120);
-}
-
-sub rollout_hadoop_cluster {
-  # if it's not already unpacked, unpack it
-  if (! -d "hadoop") {
-
-    my $hadoop_tmp_dir = tempdir("hadoop-XXXX", DIR => $TMPDIR, CLEANUP => 0);
-		system("tar xzf $JOSHUA/lib/hadoop-2.5.2.tar.gz -C $hadoop_tmp_dir");
-		system("ln -sf $hadoop_tmp_dir/hadoop-2.5.2 hadoop");
-    if (defined $HADOOP_CONF) {
-      print STDERR "Copying HADOOP_CONF($HADOOP_CONF) to hadoop/conf/core-site.xml\n";
-      system("cp $HADOOP_CONF hadoop/conf/core-site.xml");
-    }
-  }
-  
-  $ENV{HADOOP} = $HADOOP = "hadoop";
-  $ENV{HADOOP_CONF_DIR} = "";
-}
-
-sub stop_hadoop_cluster {
-  if ($HADOOP ne "hadoop") {
-		system("hadoop/bin/stop-all.sh");
-  }
-}
-
-#sub teardown_hadoop_cluster {
-#  stop_hadoop_cluster();
-#  system("rm -f hadoop");
-#}
-
 sub is_lattice {
   my $file = shift;
   open READ, "$CAT $file|" or die "can't read from potential lattice '$file'";


[15/15] incubator-joshua git commit: Merge branch 'JOSHUA-271'

Posted by mj...@apache.org.
Merge branch 'JOSHUA-271'


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/4d73c178
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/4d73c178
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/4d73c178

Branch: refs/heads/master
Commit: 4d73c17842e64f7f14df449dec0f9ba2acafe7b2
Parents: aef0b2d 78f727f
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 06:01:56 2016 -0400
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 06:01:56 2016 -0400

----------------------------------------------------------------------
 scripts/training/pipeline.pl | 89 ++++-----------------------------------
 1 file changed, 8 insertions(+), 81 deletions(-)
----------------------------------------------------------------------



[12/15] incubator-joshua git commit: Replaced redundant Vocabulary.nt() to FormatUtils.isNonterminal(), for clarity and consolidation

Posted by mj...@apache.org.
Replaced redundant Vocabulary.nt() to FormatUtils.isNonterminal(), for clarity and consolidation


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/868b3409
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/868b3409
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/868b3409

Branch: refs/heads/master
Commit: 868b340949f324b12d810d03c09c25fd5877d3dc
Parents: 366f408
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 05:50:16 2016 -0400
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 05:50:16 2016 -0400

----------------------------------------------------------------------
 src/joshua/corpus/TerminalIterator.java         |  4 +++-
 src/joshua/corpus/Vocabulary.java               | 10 ---------
 src/joshua/decoder/BLEU.java                    |  1 -
 src/joshua/decoder/Decoder.java                 |  1 -
 src/joshua/decoder/ff/TargetBigram.java         |  3 ++-
 src/joshua/decoder/ff/lm/LanguageModelFF.java   |  5 +++--
 .../ff/lm/StateMinimizingLanguageModel.java     |  5 +++--
 .../GrammarBuilderWalkerFunction.java           | 22 ++++++++++----------
 .../hypergraph/OutputStringExtractor.java       |  8 +++----
 src/joshua/decoder/segment_file/Sentence.java   |  1 -
 src/joshua/metrics/MinimumChangeBLEU.java       |  1 -
 src/joshua/metrics/Precis.java                  |  1 -
 src/joshua/oracle/OracleExtractionHG.java       |  3 ++-
 src/joshua/util/FormatUtils.java                |  8 +++++++
 14 files changed, 36 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/868b3409/src/joshua/corpus/TerminalIterator.java
----------------------------------------------------------------------
diff --git a/src/joshua/corpus/TerminalIterator.java b/src/joshua/corpus/TerminalIterator.java
index 29544fb..12cb16c 100644
--- a/src/joshua/corpus/TerminalIterator.java
+++ b/src/joshua/corpus/TerminalIterator.java
@@ -21,6 +21,8 @@ package joshua.corpus;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 
+import joshua.util.FormatUtils;
+
 /**
  * Iterator capable of iterating over those word identifiers in a phrase which represent terminals.
  * <p>
@@ -49,7 +51,7 @@ public class TerminalIterator implements Iterator<Integer> {
   /* See Javadoc for java.util.Iterator#next(). */
   public boolean hasNext() {
 
-    while (dirty || Vocabulary.nt(next)) {
+    while (dirty || FormatUtils.isNonterminal(next)) {
       nextIndex++;
       if (nextIndex < words.length) {
         next = words[nextIndex];

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/868b3409/src/joshua/corpus/Vocabulary.java
----------------------------------------------------------------------
diff --git a/src/joshua/corpus/Vocabulary.java b/src/joshua/corpus/Vocabulary.java
index 6f72ad8..74f6a47 100644
--- a/src/joshua/corpus/Vocabulary.java
+++ b/src/joshua/corpus/Vocabulary.java
@@ -227,16 +227,6 @@ public class Vocabulary {
     return UNKNOWN_WORD;
   }
 
-  /**
-   * Returns true if the Vocabulary ID represents a nonterminal.
-   *
-   * @param id
-   * @return
-   */
-  public static boolean nt(int id) {
-    return (id < 0);
-  }
-
   public static int size() {
     long lock_stamp = lock.readLock();
     try {

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/868b3409/src/joshua/decoder/BLEU.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/BLEU.java b/src/joshua/decoder/BLEU.java
index 1b3e3f8..2fd3287 100644
--- a/src/joshua/decoder/BLEU.java
+++ b/src/joshua/decoder/BLEU.java
@@ -20,7 +20,6 @@ package joshua.decoder;
 
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/868b3409/src/joshua/decoder/Decoder.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/Decoder.java b/src/joshua/decoder/Decoder.java
index 0057f87..22ed8b9 100644
--- a/src/joshua/decoder/Decoder.java
+++ b/src/joshua/decoder/Decoder.java
@@ -28,7 +28,6 @@ import java.io.FileNotFoundException;
 import java.io.FileWriter;
 import java.lang.reflect.Constructor;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/868b3409/src/joshua/decoder/ff/TargetBigram.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/TargetBigram.java b/src/joshua/decoder/ff/TargetBigram.java
index 846273d..defaaf1 100644
--- a/src/joshua/decoder/ff/TargetBigram.java
+++ b/src/joshua/decoder/ff/TargetBigram.java
@@ -31,6 +31,7 @@ import joshua.decoder.ff.state_maintenance.NgramDPState;
 import joshua.decoder.ff.tm.Rule;
 import joshua.decoder.hypergraph.HGNode;
 import joshua.decoder.segment_file.Sentence;
+import joshua.util.FormatUtils;
 import joshua.util.io.LineReader;
 
 /***
@@ -112,7 +113,7 @@ public class TargetBigram extends StatefulFF {
     for (int c = 0; c < enWords.length; c++) {
       int curID = enWords[c];
 
-      if (Vocabulary.nt(curID)) {
+      if (FormatUtils.isNonterminal(curID)) {
         int index = -(curID + 1);
         NgramDPState state = (NgramDPState) tailNodes.get(index).getDPState(stateIndex);
         int[] leftContext = state.getLeftLMStateWords();

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/868b3409/src/joshua/decoder/ff/lm/LanguageModelFF.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/lm/LanguageModelFF.java b/src/joshua/decoder/ff/lm/LanguageModelFF.java
index a002de7..984ce97 100644
--- a/src/joshua/decoder/ff/lm/LanguageModelFF.java
+++ b/src/joshua/decoder/ff/lm/LanguageModelFF.java
@@ -40,6 +40,7 @@ import joshua.decoder.ff.state_maintenance.NgramDPState;
 import joshua.decoder.ff.tm.Rule;
 import joshua.decoder.hypergraph.HGNode;
 import joshua.decoder.segment_file.Sentence;
+import joshua.util.FormatUtils;
 
 /**
  * This class performs the following:
@@ -318,7 +319,7 @@ public class LanguageModelFF extends StatefulFF {
      */
     for (int c = 0; c < enWords.length; c++) {
       int currentWord = enWords[c];
-      if (Vocabulary.nt(currentWord)) {
+      if (FormatUtils.isNonterminal(currentWord)) {
         estimate += scoreChunkLogP(words, considerIncompleteNgrams, skipStart);
         words.clear();
         skipStart = false;
@@ -376,7 +377,7 @@ public class LanguageModelFF extends StatefulFF {
     for (int c = 0; c < enWords.length; c++) {
       int curID = enWords[c];
 
-      if (Vocabulary.nt(curID)) {
+      if (FormatUtils.isNonterminal(curID)) {
         int index = -(curID + 1);
 
         NgramDPState state = (NgramDPState) tailNodes.get(index).getDPState(stateIndex);

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/868b3409/src/joshua/decoder/ff/lm/StateMinimizingLanguageModel.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/lm/StateMinimizingLanguageModel.java b/src/joshua/decoder/ff/lm/StateMinimizingLanguageModel.java
index f07b668..1f7e818 100644
--- a/src/joshua/decoder/ff/lm/StateMinimizingLanguageModel.java
+++ b/src/joshua/decoder/ff/lm/StateMinimizingLanguageModel.java
@@ -33,6 +33,7 @@ import joshua.decoder.ff.state_maintenance.KenLMState;
 import joshua.decoder.ff.tm.Rule;
 import joshua.decoder.hypergraph.HGNode;
 import joshua.decoder.segment_file.Sentence;
+import joshua.util.FormatUtils;
 
 /**
  * Wrapper for KenLM LMs with left-state minimization. We inherit from the regular
@@ -99,7 +100,7 @@ public class StateMinimizingLanguageModel extends LanguageModelFF {
     for (int x = 0; x < ruleWords.length; x++) {
       int id = ruleWords[x];
 
-      if (Vocabulary.nt(id)) {
+      if (FormatUtils.isNonterminal(id)) {
         // For the estimate, we can just mark negative values
         words[x] = -1;
 
@@ -131,7 +132,7 @@ public class StateMinimizingLanguageModel extends LanguageModelFF {
     for (int x = 0; x < ruleWords.length; x++) {
       int id = ruleWords[x];
 
-      if (Vocabulary.nt(id)) {
+      if (FormatUtils.isNonterminal(id)) {
         // Nonterminal: retrieve the KenLM long that records the state
         int index = -(id + 1);
         KenLMState state = (KenLMState) tailNodes.get(index).getDPState(stateIndex);

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/868b3409/src/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java b/src/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java
index a4df7e5..d4b11df 100644
--- a/src/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java
+++ b/src/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java
@@ -115,16 +115,16 @@ public class GrammarBuilderWalkerFunction implements WalkerFunction {
     int[] result = new int[english.length];
     for (int i = 0; i < english.length; i++) {
       int curr = english[i];
-      if (!Vocabulary.nt(curr)) {
-				// If it's a terminal symbol, we just copy it into the new rule.
+      if (! FormatUtils.isNonterminal(curr)) {
+        // If it's a terminal symbol, we just copy it into the new rule.
         result[i] = curr;
       } else {
-				// If it's a nonterminal, its value is -N, where N is the index
-				// of the nonterminal on the source side.
-				//
-				// That is, if we would call a nonterminal "[X,2]", the value of
-				// curr at this point is -2. And the tail node that it points at
-				// is #1 (since getTailNodes() is 0-indexed).
+        // If it's a nonterminal, its value is -N, where N is the index
+        // of the nonterminal on the source side.
+        //
+        // That is, if we would call a nonterminal "[X,2]", the value of
+        // curr at this point is -2. And the tail node that it points at
+        // is #1 (since getTailNodes() is 0-indexed).
         int index = -curr - 1;
         result[i] = getLabelWithSpan(edge.getTailNodes().get(index));
       }
@@ -135,12 +135,12 @@ public class GrammarBuilderWalkerFunction implements WalkerFunction {
 
   private static int[] getNewTargetFromSource(int[] source) {
     int[] result = new int[source.length];
-		int currNT = -1; // value to stick into NT slots
+    int currNT = -1; // value to stick into NT slots
     for (int i = 0; i < source.length; i++) {
       result[i] = source[i];
-      if (Vocabulary.nt(result[i])) {
+      if (FormatUtils.isNonterminal(result[i])) {
         result[i] = currNT;
-				currNT--;
+        currNT--;
       }
     }
     // System.err.printf("target: %s\n", result);

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/868b3409/src/joshua/decoder/hypergraph/OutputStringExtractor.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/hypergraph/OutputStringExtractor.java b/src/joshua/decoder/hypergraph/OutputStringExtractor.java
index acb2e17..f67a9df 100644
--- a/src/joshua/decoder/hypergraph/OutputStringExtractor.java
+++ b/src/joshua/decoder/hypergraph/OutputStringExtractor.java
@@ -20,13 +20,13 @@ package joshua.decoder.hypergraph;
 
 import static java.lang.Math.min;
 import static joshua.corpus.Vocabulary.getWords;
-import static joshua.corpus.Vocabulary.nt;
 
 import java.util.Stack;
 
 import joshua.decoder.ff.tm.Rule;
 import joshua.decoder.hypergraph.KBestExtractor.DerivationState;
 import joshua.decoder.hypergraph.KBestExtractor.DerivationVisitor;
+import joshua.util.FormatUtils;
 
 public class OutputStringExtractor implements WalkerFunction, DerivationVisitor {
   
@@ -66,7 +66,7 @@ public class OutputStringExtractor implements WalkerFunction, DerivationVisitor
   private static int getSourceNonTerminalPosition(final int[] words, int nonTerminalIndex) {
     int nonTerminalsSeen = 0;
     for (int i = 0; i < words.length; i++) {
-      if (nt(words[i])) {
+      if (FormatUtils.isNonterminal(words[i])) {
         nonTerminalsSeen++;
         if (nonTerminalsSeen == nonTerminalIndex) {
           return i;
@@ -89,7 +89,7 @@ public class OutputStringExtractor implements WalkerFunction, DerivationVisitor
    */
   private static int getTargetNonTerminalPosition(int[] words, int nonTerminalIndex) {
     for (int pos = 0; pos < words.length; pos++) {
-      if (nt(words[pos]) && -(words[pos] + 1) == nonTerminalIndex) {
+      if (FormatUtils.isNonterminal(words[pos]) && -(words[pos] + 1) == nonTerminalIndex) {
         return pos;
       }
     }
@@ -174,7 +174,7 @@ public class OutputStringExtractor implements WalkerFunction, DerivationVisitor
      * of child and the arity of this.
      */
     private void substituteNonTerminalAtPosition(final int[] words, final int position) {
-      assert(nt(this.words[position]));
+      assert(FormatUtils.isNonterminal(this.words[position]));
       final int[] result = new int[words.length + this.words.length - 1];
       int resultIndex = 0;
       for (int i = 0; i < position; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/868b3409/src/joshua/decoder/segment_file/Sentence.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/segment_file/Sentence.java b/src/joshua/decoder/segment_file/Sentence.java
index 588850b..c77b51c 100644
--- a/src/joshua/decoder/segment_file/Sentence.java
+++ b/src/joshua/decoder/segment_file/Sentence.java
@@ -19,7 +19,6 @@
 package joshua.decoder.segment_file;
 
 import static joshua.util.FormatUtils.addSentenceMarkers;
-import static joshua.util.FormatUtils.escapeSpecialSymbols;
 
 import java.util.ArrayList;
 import java.util.HashSet;

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/868b3409/src/joshua/metrics/MinimumChangeBLEU.java
----------------------------------------------------------------------
diff --git a/src/joshua/metrics/MinimumChangeBLEU.java b/src/joshua/metrics/MinimumChangeBLEU.java
index fa764c3..91ee81c 100644
--- a/src/joshua/metrics/MinimumChangeBLEU.java
+++ b/src/joshua/metrics/MinimumChangeBLEU.java
@@ -19,7 +19,6 @@
 package joshua.metrics;
 
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.logging.Logger;
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/868b3409/src/joshua/metrics/Precis.java
----------------------------------------------------------------------
diff --git a/src/joshua/metrics/Precis.java b/src/joshua/metrics/Precis.java
index 82f4106..84279ac 100644
--- a/src/joshua/metrics/Precis.java
+++ b/src/joshua/metrics/Precis.java
@@ -19,7 +19,6 @@
 package joshua.metrics;
 
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.logging.Logger;
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/868b3409/src/joshua/oracle/OracleExtractionHG.java
----------------------------------------------------------------------
diff --git a/src/joshua/oracle/OracleExtractionHG.java b/src/joshua/oracle/OracleExtractionHG.java
index 7e7fcb8..8a688e3 100644
--- a/src/joshua/oracle/OracleExtractionHG.java
+++ b/src/joshua/oracle/OracleExtractionHG.java
@@ -35,6 +35,7 @@ import joshua.decoder.hypergraph.HyperEdge;
 import joshua.decoder.hypergraph.HyperGraph;
 import joshua.decoder.hypergraph.KBestExtractor;
 import joshua.util.FileUtility;
+import joshua.util.FormatUtils;
 import joshua.util.io.LineReader;
 
 /**
@@ -384,7 +385,7 @@ public class OracleExtractionHG extends SplitHg {
     // #### get left_state_sequence, right_state_sequence, total_hyp_len, num_ngram_match
     for (int c = 0; c < en_words.length; c++) {
       int c_id = en_words[c];
-      if (Vocabulary.nt(c_id)) {
+      if (FormatUtils.isNonterminal(c_id)) {
         int index = -(c_id + 1);
         DPStateOracle ant_state = (DPStateOracle) l_ant_virtual_item.get(index).dp_state;
         total_hyp_len += ant_state.best_len;

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/868b3409/src/joshua/util/FormatUtils.java
----------------------------------------------------------------------
diff --git a/src/joshua/util/FormatUtils.java b/src/joshua/util/FormatUtils.java
index 568d3ca..f6cfcd7 100644
--- a/src/joshua/util/FormatUtils.java
+++ b/src/joshua/util/FormatUtils.java
@@ -44,6 +44,14 @@ public class FormatUtils {
   public static boolean isNonterminal(String token) {
     return (token.length() >=3 && token.charAt(0) == '[') && (token.charAt(token.length() - 1) == ']');
   }
+  
+  /**
+   * Determines whether the ID represents a nonterminal. This is a trivial check, since nonterminal
+   * IDs are simply negative ones.
+   */
+  public static boolean isNonterminal(int id) {
+    return id < 0;
+  }
 
   /**
    * Nonterminals are stored in the vocabulary in square brackets. This removes them when you 


[09/15] incubator-joshua git commit: escaping left curly braces

Posted by mj...@apache.org.
escaping left curly braces


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/f85b68ea
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/f85b68ea
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/f85b68ea

Branch: refs/heads/master
Commit: f85b68ea30d5d7a809168c137d751a9698bd575c
Parents: 9b39660
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 09:15:28 2016 +0200
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 09:15:28 2016 +0200

----------------------------------------------------------------------
 scripts/support/moses2joshua_grammar.pl | 13 ++++++++-----
 1 file changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f85b68ea/scripts/support/moses2joshua_grammar.pl
----------------------------------------------------------------------
diff --git a/scripts/support/moses2joshua_grammar.pl b/scripts/support/moses2joshua_grammar.pl
index ab10a23..4ab2f33 100755
--- a/scripts/support/moses2joshua_grammar.pl
+++ b/scripts/support/moses2joshua_grammar.pl
@@ -1,7 +1,9 @@
 #!/usr/bin/env perl
 # Matt Post <po...@cs.jhu.edu>
 
-# Converts a Moses grammars and phrase tables to a Joshua grammar.
+# Converts Moses grammars to a Joshua grammar. This script does not work for phrase tables;
+# Joshua can read Moses phrase tables directly, both via the grammar packer or the memory
+# based grammar loader.
 #
 # Usage: cat grammar.moses | moses2joshua_grammar.pl > grammar.joshua
 #
@@ -17,9 +19,10 @@
 #
 # (This doesn't apply to phrase tables, of course).
 #
-# (2) Phrase table values. Moses takes the log of each feature, whereas Joshua takes just
-#     negates the values when it reads them in. To make the conversion correct, this script
-#     computes the negative log of each of the feature values.
+# (2) Feautre values. The Moses decoder expects these features to be probabilities, whereas
+#     Joshua expects them to be negative logprobs (which it then negates when reading in;
+#     I know, this is crazy). To compute the conversion, this script computes the negative 
+#     log of each of the feature values.
 
 use strict;
 use warnings;
@@ -156,7 +159,7 @@ while (my $rule = <>) {
   print "$new_rule\n";
 
   if ($opts{m} and defined $tree) {
-    $tree =~ s/.*{{Tree\s+(.*)}}.*/$1/;
+    $tree =~ s/.*\{\{Tree\s+(.*)}}.*/$1/;
     # Remove brackets around substitution points
     $tree =~ s/\[([^\[\]\s]+)\]/$1/g;
     # Add quotes around terminals


[02/15] incubator-joshua git commit: test was using wrong config

Posted by mj...@apache.org.
test was using wrong config


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/afa66ef6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/afa66ef6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/afa66ef6

Branch: refs/heads/master
Commit: afa66ef65112dc5805e5b652f3a4a735e430e8a5
Parents: 76eb958
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 00:18:57 2016 +0200
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 00:18:57 2016 +0200

----------------------------------------------------------------------
 test/decoder/phrase/decode/config | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/afa66ef6/test/decoder/phrase/decode/config
----------------------------------------------------------------------
diff --git a/test/decoder/phrase/decode/config b/test/decoder/phrase/decode/config
index 9987b1a..7e6f2b8 100644
--- a/test/decoder/phrase/decode/config
+++ b/test/decoder/phrase/decode/config
@@ -1,4 +1,4 @@
-tm = moses -owner pt -maxspan 0 -path rules.packed -max-source-len 5
+tm = moses -owner pt -maxspan 0 -path rules.1.gz -max-source-len 5
 feature-function = StateMinimizingLanguageModel -lm_order 5 -lm_file lm.1.gz
 
 search = stack


[11/15] incubator-joshua git commit: packer now writes out a version, removed custom backend phrase-based checks

Posted by mj...@apache.org.
packer now writes out a version, removed custom backend phrase-based checks

-  The packed grammar now writes out a version number. The current version is 3. PackedGrammar will throw a runtime exception if no version information is found. However, the format has only changed for phrase-based grammars; any currently packed grammar can be made compatible by adding the line

       version = 3

to the packed grammar "config" file

- Removed checks for which grammar backend is being used for phrase-based decoding. It's now totally generic! Very nice.

- Updated test cases to match.


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/366f4086
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/366f4086
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/366f4086

Branch: refs/heads/master
Commit: 366f408672e2d29b69a78531b57056649629e978
Parents: 53a0fcf
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 05:47:48 2016 -0400
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 05:47:48 2016 -0400

----------------------------------------------------------------------
 src/joshua/decoder/ff/tm/GrammarReader.java     |  1 -
 .../decoder/ff/tm/format/MosesFormatReader.java |  8 ++---
 .../decoder/ff/tm/packed/PackedGrammar.java     | 33 ++++++++++++++---
 src/joshua/decoder/phrase/PhraseTable.java      | 13 +++----
 src/joshua/decoder/phrase/Stacks.java           |  2 +-
 src/joshua/tools/GrammarPacker.java             | 37 ++++++++++++++------
 test/decoder/phrase/decode/rules.packed/config  |  1 +
 7 files changed, 63 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/366f4086/src/joshua/decoder/ff/tm/GrammarReader.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/GrammarReader.java b/src/joshua/decoder/ff/tm/GrammarReader.java
index 3b973a2..3432e53 100644
--- a/src/joshua/decoder/ff/tm/GrammarReader.java
+++ b/src/joshua/decoder/ff/tm/GrammarReader.java
@@ -23,7 +23,6 @@ import java.util.Iterator;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import joshua.corpus.Vocabulary;
 import joshua.decoder.Decoder;
 import joshua.util.io.LineReader;
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/366f4086/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/format/MosesFormatReader.java b/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
index 7e43075..47a3e46 100644
--- a/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
+++ b/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
@@ -44,16 +44,14 @@ import joshua.util.io.LineReader;
 
 public class MosesFormatReader extends HieroFormatReader {
 
-  private int lhs;
-  
   public MosesFormatReader(String grammarFile) throws IOException {
     super(grammarFile);
-    this.lhs = Vocabulary.id("[X]");
+    Vocabulary.id("[X]");
   }
   
   public MosesFormatReader() {
     super();
-    this.lhs = Vocabulary.id("[X]");
+    Vocabulary.id("[X]");
   }
   
   /**
@@ -77,8 +75,6 @@ public class MosesFormatReader extends HieroFormatReader {
   public Rule parseLine(String line) {
     String[] fields = line.split(fieldDelimiter);
     
-    int arity = 1;
-
     StringBuffer hieroLine = new StringBuffer();
     hieroLine.append("[X] ||| [X,1] " + fields[0] + " ||| [X,1] " + fields[1] + " |||");
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/366f4086/src/joshua/decoder/ff/tm/packed/PackedGrammar.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/packed/PackedGrammar.java b/src/joshua/decoder/ff/tm/packed/PackedGrammar.java
index cc58578..e9f1a5c 100644
--- a/src/joshua/decoder/ff/tm/packed/PackedGrammar.java
+++ b/src/joshua/decoder/ff/tm/packed/PackedGrammar.java
@@ -91,6 +91,7 @@ import joshua.decoder.ff.tm.Rule;
 import joshua.decoder.ff.tm.RuleCollection;
 import joshua.decoder.ff.tm.Trie;
 import joshua.decoder.ff.tm.hash_based.ExtensionIterator;
+import joshua.util.FormatUtils;
 import joshua.util.encoding.EncoderConfiguration;
 import joshua.util.encoding.FloatEncoder;
 import joshua.util.io.LineReader;
@@ -109,19 +110,22 @@ public class PackedGrammar extends AbstractGrammar {
   private final File vocabFile; // store path to vocabulary file
 
   public static final String VOCABULARY_FILENAME = "vocabulary";
-
-  // The grammar specification keyword (e.g., "thrax" or "moses")
-  private String type;
+  
+  // The version number of the earliest supported grammar packer
+  public static final int SUPPORTED_VERSION = 3;
 
   // A rule cache for commonly used tries to avoid excess object allocations
   // Testing shows there's up to ~95% hit rate when cache size is 5000 Trie nodes.
   private final Cache<Trie, List<Rule>> cached_rules;
 
+  private String grammarDir;
+
   public PackedGrammar(String grammar_dir, int span_limit, String owner, String type,
       JoshuaConfiguration joshuaConfiguration) throws FileNotFoundException, IOException {
     super(joshuaConfiguration);
+
+    this.grammarDir = grammar_dir;
     this.spanLimit = span_limit;
-    this.type = type;
 
     // Read the vocabulary.
     vocabFile = new File(grammar_dir + File.separator + VOCABULARY_FILENAME);
@@ -566,7 +570,7 @@ public class PackedGrammar extends AbstractGrammar {
         System.arraycopy(parent_src, 0, src, 0, parent_src.length);
         src[src.length - 1] = symbol;
         arity = parent_arity;
-        if (Vocabulary.nt(symbol))
+        if (FormatUtils.isNonterminal(symbol))
           arity++;
       }
 
@@ -941,11 +945,30 @@ public class PackedGrammar extends AbstractGrammar {
     throw new RuntimeException("PackedGrammar.addRule(): I can't add rules");
   }
   
+  /** 
+   * Read the config file
+   * 
+   * TODO: this should be rewritten using typeconfig.
+   * 
+   * @param config
+   * @throws IOException
+   */
   private void readConfig(String config) throws IOException {
+    int version = 0;
+    
     for (String line: new LineReader(config)) {
       String[] tokens = line.split(" = ");
       if (tokens[0].equals("max-source-len"))
         this.maxSourcePhraseLength = Integer.parseInt(tokens[1]);
+      else if (tokens[0].equals("version")) {
+        version = Integer.parseInt(tokens[1]);
+      }
+    }
+    
+    if (version != 3) {
+      String message = String.format("The grammar at %s was packed with packer version %d, but the earliest supported version is %d",
+          this.grammarDir, version, SUPPORTED_VERSION);
+      throw new RuntimeException(message);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/366f4086/src/joshua/decoder/phrase/PhraseTable.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/phrase/PhraseTable.java b/src/joshua/decoder/phrase/PhraseTable.java
index bcf7135..38b7ef4 100644
--- a/src/joshua/decoder/phrase/PhraseTable.java
+++ b/src/joshua/decoder/phrase/PhraseTable.java
@@ -77,18 +77,14 @@ public class PhraseTable implements Grammar {
   }
       
   /**
-   * Returns the longest source phrase read. For {@link MemoryBasedBatchGrammar}s, we subtract 1
-   * since the grammar includes the nonterminal. For {@link PackedGrammar}s, the value was either
-   * in the packed config file (Joshua 6.0.2+) or was passed in via the TM config line.
+   * Returns the longest source phrase read. Because phrases have a dummy nonterminal prepended to
+   * them, we need to subtract 1.
    * 
    * @return
    */
   @Override
   public int getMaxSourcePhraseLength() {
-    if (backend instanceof MemoryBasedBatchGrammar)
-      return this.backend.getMaxSourcePhraseLength() - 1;
-    else
-      return this.backend.getMaxSourcePhraseLength();
+    return this.backend.getMaxSourcePhraseLength() - 1;
   }
 
   /**
@@ -100,8 +96,7 @@ public class PhraseTable implements Grammar {
   public RuleCollection getPhrases(int[] sourceWords) {
     if (sourceWords.length != 0) {
       Trie pointer = getTrieRoot();
-      if (! (backend instanceof PackedGrammar))
-        pointer = pointer.match(Vocabulary.id("[X]"));
+      pointer = pointer.match(Vocabulary.id("[X]"));
       int i = 0;
       while (pointer != null && i < sourceWords.length)
         pointer = pointer.match(sourceWords[i++]);

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/366f4086/src/joshua/decoder/phrase/Stacks.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/phrase/Stacks.java b/src/joshua/decoder/phrase/Stacks.java
index eda7d8b..f81ede9 100644
--- a/src/joshua/decoder/phrase/Stacks.java
+++ b/src/joshua/decoder/phrase/Stacks.java
@@ -137,7 +137,7 @@ public class Stacks {
         Stack tailStack = stacks.get(from_stack);
         
         if (Decoder.VERBOSE >= 3)
-          System.err.println(String.format("\n  WORDS %d MAX %d (STACK %d phrase_length %d)", source_words,
+          Decoder.LOG(3, String.format("\n  WORDS %d MAX %d (STACK %d phrase_length %d)", source_words,
               chart.MaxSourcePhraseLength(), from_stack, phrase_length));
         
         // Iterate over antecedents in this stack.

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/366f4086/src/joshua/tools/GrammarPacker.java
----------------------------------------------------------------------
diff --git a/src/joshua/tools/GrammarPacker.java b/src/joshua/tools/GrammarPacker.java
index df8383b..8c39582 100644
--- a/src/joshua/tools/GrammarPacker.java
+++ b/src/joshua/tools/GrammarPacker.java
@@ -1,4 +1,4 @@
-/*
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -28,7 +28,6 @@ import java.io.FileWriter;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.nio.ByteBuffer;
-import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
@@ -50,6 +49,20 @@ public class GrammarPacker {
 
   private static final Logger logger = Logger.getLogger(GrammarPacker.class.getName());
 
+  /**
+   * The packed grammar version number. Increment this any time you add new features, and update
+   * the documentation.
+   * 
+   * Version history:
+   * 
+   * - 3 (May 2016). This was the first version that was marked. It removed the special phrase-
+   * table packing that packed phrases without the [X,1] on the source and target sides, which
+   * then required special handling in the decoder to use for phrase-based decoding.
+   * 
+   * 
+   */
+  public static final int VERSION = 3;
+  
   // Size limit for slice in bytes.
   private static int DATA_SIZE_LIMIT = (int) (Integer.MAX_VALUE * 0.8);
   // Estimated average number of feature entries for one rule.
@@ -66,7 +79,7 @@ public class GrammarPacker {
   public String getGrammar() {
     return grammar;
   }
-  
+
   public String getOutputDirectory() {
     return output;
   }
@@ -160,7 +173,7 @@ public class GrammarPacker {
 
     // Explore pass. Learn vocabulary and feature value histograms.
     logger.info("Exploring: " + grammar);
-    
+
     HieroFormatReader grammarReader = getGrammarReader();
     explore(grammarReader);
 
@@ -185,9 +198,10 @@ public class GrammarPacker {
     logger.info(String.format("Writing config to '%s'", configFile));
     // Write config options
     FileWriter config = new FileWriter(configFile);
+    config.write(String.format("version = %d\n", VERSION));
     config.write(String.format("max-source-len = %d\n", max_source_len));
     config.close();
-    
+
     // Read previously written encoder configuration to match up to changed
     // vocabulary id's.
     logger.info("Reading encoding.");
@@ -212,7 +226,7 @@ public class GrammarPacker {
    * 
    * @param grammarFile
    * @return
-   * @throws IOException 
+   * @throws IOException
    */
   private HieroFormatReader getGrammarReader() throws IOException {
     LineReader reader = new LineReader(grammar);
@@ -224,14 +238,17 @@ public class GrammarPacker {
     }
   }
 
+  /**
+   * This first pass over the grammar 
+   * @param reader
+   */
   private void explore(HieroFormatReader reader) {
-    int counter = 0;
+
     // We always assume a labeled grammar. Unlabeled features are assumed to be dense and to always
     // appear in the same order. They are assigned numeric names in order of appearance.
     this.types.setLabeled(true);
 
     for (Rule rule: reader) {
-      counter++;
 
       max_source_len = Math.max(max_source_len, rule.getFrench().length);
 
@@ -239,7 +256,7 @@ public class GrammarPacker {
        * NOTE: In case of nonterminals, we add both stripped versions ("[X]")
        * and "[X,1]" to the vocabulary.
        * 
-       * TODO: MJP May 2016: do we need to add [X,1]? If so, should be done in FormatReaders.
+       * TODO: MJP May 2016: Is it necessary to add [X,1]?
        */
 
       // Add feature names to vocabulary and pass the value through the
@@ -359,7 +376,7 @@ public class GrammarPacker {
       for (int f = 0; f < feature_entries.length; ++f) {
         String feature_entry = feature_entries[f];
         int feature_id;
-        float feature_value; 
+        float feature_value;
         if (feature_entry.contains("=")) {
           String[] parts = feature_entry.split("=");
           if (parts[0].equals("Alignment"))

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/366f4086/test/decoder/phrase/decode/rules.packed/config
----------------------------------------------------------------------
diff --git a/test/decoder/phrase/decode/rules.packed/config b/test/decoder/phrase/decode/rules.packed/config
index 9c2b25e..7bdb804 100644
--- a/test/decoder/phrase/decode/rules.packed/config
+++ b/test/decoder/phrase/decode/rules.packed/config
@@ -1 +1,2 @@
+version = 3
 max-source-len = 4


[04/15] incubator-joshua git commit: renaming for readability

Posted by mj...@apache.org.
renaming for readability


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/47814050
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/47814050
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/47814050

Branch: refs/heads/master
Commit: 4781405080908edf86045dd2809cc2bff25d315f
Parents: b11b251
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 01:07:30 2016 +0200
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 01:07:30 2016 +0200

----------------------------------------------------------------------
 src/joshua/decoder/JoshuaConfiguration.java | 14 +++++++-------
 src/joshua/util/FormatUtils.java            | 11 ++---------
 2 files changed, 9 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/47814050/src/joshua/decoder/JoshuaConfiguration.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/JoshuaConfiguration.java b/src/joshua/decoder/JoshuaConfiguration.java
index 7a3de23..c874534 100644
--- a/src/joshua/decoder/JoshuaConfiguration.java
+++ b/src/joshua/decoder/JoshuaConfiguration.java
@@ -19,7 +19,7 @@
 package joshua.decoder;
 
 import static joshua.util.FormatUtils.cleanNonTerminal;
-import static joshua.util.FormatUtils.markup;
+import static joshua.util.FormatUtils.ensureNonTerminalBrackets;
 
 import java.io.File;
 import java.io.FileWriter;
@@ -74,8 +74,8 @@ public class JoshuaConfiguration {
   public String weights_file = "";
 
   // Default symbols. The symbol here should be enclosed in square brackets.
-  public String default_non_terminal = FormatUtils.markup("X");
-  public String goal_symbol = FormatUtils.markup("GOAL");
+  public String default_non_terminal = FormatUtils.ensureNonTerminalBrackets("X");
+  public String goal_symbol = FormatUtils.ensureNonTerminalBrackets("GOAL");
 
   /*
    * A list of OOV symbols in the form
@@ -445,7 +445,7 @@ public class JoshuaConfiguration {
                   while (str != null) {
                     String[] tokens = str.trim().split("\\s+");
 
-                    oovList.add(new OOVItem(FormatUtils.markup(tokens[0]),
+                    oovList.add(new OOVItem(FormatUtils.ensureNonTerminalBrackets(tokens[0]),
                             (float) Math.log(Float.parseFloat(tokens[1]))));
 
                     str = br.readLine();
@@ -469,7 +469,7 @@ public class JoshuaConfiguration {
               oovList = new ArrayList<OOVItem>();
 
               for (int i = 0; i < tokens.length; i += 2)
-                oovList.add(new OOVItem(FormatUtils.markup(tokens[i]),
+                oovList.add(new OOVItem(FormatUtils.ensureNonTerminalBrackets(tokens[i]),
                     (float) Math.log(Float.parseFloat(tokens[i + 1]))));
 
               Collections.sort(oovList);
@@ -483,11 +483,11 @@ public class JoshuaConfiguration {
             lattice_decoding = true;
 
           } else if (parameter.equals(normalize_key("default-non-terminal"))) {
-            default_non_terminal = markup(cleanNonTerminal(fds[1].trim()));
+            default_non_terminal = ensureNonTerminalBrackets(cleanNonTerminal(fds[1].trim()));
             logger.finest(String.format("default_non_terminal: %s", default_non_terminal));
 
           } else if (parameter.equals(normalize_key("goal-symbol"))) {
-            goal_symbol = markup(cleanNonTerminal(fds[1].trim()));
+            goal_symbol = ensureNonTerminalBrackets(cleanNonTerminal(fds[1].trim()));
             logger.finest("goalSymbol: " + goal_symbol);
 
           } else if (parameter.equals(normalize_key("weights-file"))) {

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/47814050/src/joshua/util/FormatUtils.java
----------------------------------------------------------------------
diff --git a/src/joshua/util/FormatUtils.java b/src/joshua/util/FormatUtils.java
index c925cba..568d3ca 100644
--- a/src/joshua/util/FormatUtils.java
+++ b/src/joshua/util/FormatUtils.java
@@ -75,7 +75,7 @@ public class FormatUtils {
    * Removes the index from a nonTerminal: [X,1] -> [X].
    */
   public static String stripNonTerminalIndex(String nt) {
-    return markup(cleanNonTerminal(nt));
+    return ensureNonTerminalBrackets(cleanNonTerminal(nt));
   }
 
   /**
@@ -97,19 +97,12 @@ public class FormatUtils {
    * @param nt the nonterminal string
    * @return the nonterminal string surrounded in square brackets (if not already)
    */
-  public static String markup(String nt) {
+  public static String ensureNonTerminalBrackets(String nt) {
     if (isNonterminal(nt)) 
       return nt;
     else 
       return "[" + nt + "]";
   }
-
-  public static String markup(String nt, int index) {
-    if (isNonterminal(nt)) {
-      return markup(cleanNonTerminal(nt), index);
-    }
-    return "[" + nt + INDEX_SEPARATOR + index + "]";
-  }
   
   public static String escapeSpecialSymbols(String s) {
     return s.replaceAll("\\[",  "-lsb-")


[10/15] incubator-joshua git commit: updated test case

Posted by mj...@apache.org.
updated test case


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/53a0fcf5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/53a0fcf5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/53a0fcf5

Branch: refs/heads/master
Commit: 53a0fcf531b637ec289964a788f8d8da9d36f283
Parents: f85b68e
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 09:15:45 2016 +0200
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 09:15:45 2016 +0200

----------------------------------------------------------------------
 test/decoder/phrase/decode/rules.packed/config  |   2 +-
 .../decode/rules.packed/slice_00000.features    | Bin 4128858 -> 4128858 bytes
 .../decode/rules.packed/slice_00000.source      | Bin 1982228 -> 1982244 bytes
 .../decode/rules.packed/slice_00000.target      | Bin 1463856 -> 2652936 bytes
 .../rules.packed/slice_00000.target.lookup      | Bin 28 -> 32 bytes
 .../phrase/decode/rules.packed/vocabulary       | Bin 169225 -> 169236 bytes
 6 files changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/53a0fcf5/test/decoder/phrase/decode/rules.packed/config
----------------------------------------------------------------------
diff --git a/test/decoder/phrase/decode/rules.packed/config b/test/decoder/phrase/decode/rules.packed/config
index 4c0c6b5..9c2b25e 100644
--- a/test/decoder/phrase/decode/rules.packed/config
+++ b/test/decoder/phrase/decode/rules.packed/config
@@ -1 +1 @@
-max-source-len = 3
+max-source-len = 4

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/53a0fcf5/test/decoder/phrase/decode/rules.packed/slice_00000.features
----------------------------------------------------------------------
diff --git a/test/decoder/phrase/decode/rules.packed/slice_00000.features b/test/decoder/phrase/decode/rules.packed/slice_00000.features
index ebefdd5..c4127ff 100644
Binary files a/test/decoder/phrase/decode/rules.packed/slice_00000.features and b/test/decoder/phrase/decode/rules.packed/slice_00000.features differ

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/53a0fcf5/test/decoder/phrase/decode/rules.packed/slice_00000.source
----------------------------------------------------------------------
diff --git a/test/decoder/phrase/decode/rules.packed/slice_00000.source b/test/decoder/phrase/decode/rules.packed/slice_00000.source
index cdc98f6..83d47dc 100644
Binary files a/test/decoder/phrase/decode/rules.packed/slice_00000.source and b/test/decoder/phrase/decode/rules.packed/slice_00000.source differ

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/53a0fcf5/test/decoder/phrase/decode/rules.packed/slice_00000.target
----------------------------------------------------------------------
diff --git a/test/decoder/phrase/decode/rules.packed/slice_00000.target b/test/decoder/phrase/decode/rules.packed/slice_00000.target
index fa82c0d..8094eef 100644
Binary files a/test/decoder/phrase/decode/rules.packed/slice_00000.target and b/test/decoder/phrase/decode/rules.packed/slice_00000.target differ

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/53a0fcf5/test/decoder/phrase/decode/rules.packed/slice_00000.target.lookup
----------------------------------------------------------------------
diff --git a/test/decoder/phrase/decode/rules.packed/slice_00000.target.lookup b/test/decoder/phrase/decode/rules.packed/slice_00000.target.lookup
index 3e8c294..1c6db18 100644
Binary files a/test/decoder/phrase/decode/rules.packed/slice_00000.target.lookup and b/test/decoder/phrase/decode/rules.packed/slice_00000.target.lookup differ

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/53a0fcf5/test/decoder/phrase/decode/rules.packed/vocabulary
----------------------------------------------------------------------
diff --git a/test/decoder/phrase/decode/rules.packed/vocabulary b/test/decoder/phrase/decode/rules.packed/vocabulary
index ff62042..e9b0900 100644
Binary files a/test/decoder/phrase/decode/rules.packed/vocabulary and b/test/decoder/phrase/decode/rules.packed/vocabulary differ


[08/15] incubator-joshua git commit: removed debugging lines

Posted by mj...@apache.org.
removed debugging lines


Project: http://git-wip-us.apache.org/repos/asf/incubator-joshua/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-joshua/commit/9b396606
Tree: http://git-wip-us.apache.org/repos/asf/incubator-joshua/tree/9b396606
Diff: http://git-wip-us.apache.org/repos/asf/incubator-joshua/diff/9b396606

Branch: refs/heads/master
Commit: 9b396606249aaa1c972d3a63eff68324a9758a5c
Parents: a43f735
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 09:14:55 2016 +0200
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 09:15:08 2016 +0200

----------------------------------------------------------------------
 src/joshua/decoder/ff/tm/format/MosesFormatReader.java | 4 +---
 src/joshua/tools/GrammarPacker.java                    | 1 -
 2 files changed, 1 insertion(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/9b396606/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/tm/format/MosesFormatReader.java b/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
index d7710f2..7e43075 100644
--- a/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
+++ b/src/joshua/decoder/ff/tm/format/MosesFormatReader.java
@@ -76,7 +76,7 @@ public class MosesFormatReader extends HieroFormatReader {
   @Override
   public Rule parseLine(String line) {
     String[] fields = line.split(fieldDelimiter);
-
+    
     int arity = 1;
 
     StringBuffer hieroLine = new StringBuffer();
@@ -92,8 +92,6 @@ public class MosesFormatReader extends HieroFormatReader {
     if (fields.length >= 4)
       hieroLine.append(" ||| " + fields[3]);
 
-    System.err.println(String.format("LINE: %s -> %s", line, hieroLine.toString()));
-    
     return super.parseLine(hieroLine.toString());
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/9b396606/src/joshua/tools/GrammarPacker.java
----------------------------------------------------------------------
diff --git a/src/joshua/tools/GrammarPacker.java b/src/joshua/tools/GrammarPacker.java
index 21072c6..df8383b 100644
--- a/src/joshua/tools/GrammarPacker.java
+++ b/src/joshua/tools/GrammarPacker.java
@@ -258,7 +258,6 @@ public class GrammarPacker {
         }
       }
     }
-    System.err.println("COUNTER: " + counter);
   }
 
   /**