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/08/24 19:45:45 UTC

[24/41] incubator-joshua git commit: Now building HGNode over the phrase when it's added

Now building HGNode over the phrase when it's added

This should be much quicker because the HGNode gets built only once, when the target phrases are added, instead of building it many times, each time they are used


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

Branch: refs/heads/JOSHUA-304
Commit: 12b834e271a361417cbdabf79036538493cdb122
Parents: d28b4f3
Author: Matt Post <po...@cs.jhu.edu>
Authored: Mon Aug 22 15:59:43 2016 -0500
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Mon Aug 22 15:59:43 2016 -0500

----------------------------------------------------------------------
 .../joshua/decoder/hypergraph/HGNode.java       | 18 +---
 .../apache/joshua/decoder/phrase/Candidate.java | 27 ++----
 .../apache/joshua/decoder/phrase/Future.java    |  4 +-
 .../joshua/decoder/phrase/PhraseChart.java      | 59 +++++++++----
 .../joshua/decoder/phrase/PhraseNodes.java      | 63 ++++++++++++++
 .../apache/joshua/decoder/phrase/Stacks.java    |  2 +-
 .../joshua/decoder/phrase/TargetPhrases.java    | 87 --------------------
 .../phrase/decode/PhraseDecodingTest.java       |  2 -
 8 files changed, 116 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/12b834e2/src/main/java/org/apache/joshua/decoder/hypergraph/HGNode.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/joshua/decoder/hypergraph/HGNode.java b/src/main/java/org/apache/joshua/decoder/hypergraph/HGNode.java
index 23f4247..c353a36 100644
--- a/src/main/java/org/apache/joshua/decoder/hypergraph/HGNode.java
+++ b/src/main/java/org/apache/joshua/decoder/hypergraph/HGNode.java
@@ -246,12 +246,11 @@ public class HGNode {
    */
   // sort by estTotalLogP: for pruning purpose
   public int compareTo(HGNode anotherItem) {
-    throw new RuntimeException("HGNode, compare functiuon should never be called");
+    throw new RuntimeException("HGNode.compareTo(HGNode) is not implemented");
     /*
      * if (this.estTotalLogP > anotherItem.estTotalLogP) { return -1; } else if (this.estTotalLogP
      * == anotherItem.estTotalLogP) { return 0; } else { return 1; }
      */
-
   }
 
   /**
@@ -285,21 +284,6 @@ public class HGNode {
     }
   };
 
-  /**
-   * natural order
-   * */
-  public static Comparator<HGNode> logPComparator = (item1, item2) -> {
-    float logp1 = item1.score;
-    float logp2 = item2.score;
-    if (logp1 > logp2) {
-      return 1;
-    } else if (logp1 == logp2) {
-      return 0;
-    } else {
-      return -1;
-    }
-  };
-
   public String toString() {
     StringBuilder sb = new StringBuilder();
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/12b834e2/src/main/java/org/apache/joshua/decoder/phrase/Candidate.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/joshua/decoder/phrase/Candidate.java b/src/main/java/org/apache/joshua/decoder/phrase/Candidate.java
index 2a5dc03..bc770f4 100644
--- a/src/main/java/org/apache/joshua/decoder/phrase/Candidate.java
+++ b/src/main/java/org/apache/joshua/decoder/phrase/Candidate.java
@@ -42,7 +42,6 @@ import org.apache.joshua.decoder.ff.FeatureFunction;
 import org.apache.joshua.decoder.ff.state_maintenance.DPState;
 import org.apache.joshua.decoder.ff.tm.Rule;
 import org.apache.joshua.decoder.hypergraph.HGNode;
-import org.apache.joshua.decoder.hypergraph.HyperEdge;
 import org.apache.joshua.decoder.segment_file.Sentence;
 
 public class Candidate implements Comparable<Candidate> {
@@ -54,7 +53,7 @@ public class Candidate implements Comparable<Candidate> {
   private final List<Hypothesis> hypotheses;
 
   // the list of target phrases gathered from a span of the input
-  private TargetPhrases phrases;
+  private PhraseNodes phrases;
   
   // future cost of applying phrases to hypotheses
   private float future_delta;
@@ -70,14 +69,7 @@ public class Candidate implements Comparable<Candidate> {
    * state. Expensive to compute so there is an option of delaying it.
    */
   private ComputeNodeResult computedResult;
-  
-  /*
-   * This is the HGNode built over the current target side phrase. It requires the computed results
-   * as part of its constructor, so we delay computing it unless needed.
-   */
-  private HGNode phraseNode;
-  private ComputeNodeResult phraseResult;
-  
+
   /**
    * When candidate objects are extended, the new one is initialized with the same underlying
    * "phrases" and "hypotheses" and "span" objects. So these all have to be equal, as well as
@@ -121,7 +113,7 @@ public class Candidate implements Comparable<Candidate> {
   }
 
   public Candidate(List<FeatureFunction> featureFunctions, Sentence sentence, 
-      List<Hypothesis> hypotheses, TargetPhrases phrases, float delta, int[] ranks) {
+      List<Hypothesis> hypotheses, PhraseNodes phrases, float delta, int[] ranks) {
     this.featureFunctions = featureFunctions;
     this.sentence = sentence;
     this.hypotheses = hypotheses;
@@ -131,7 +123,6 @@ public class Candidate implements Comparable<Candidate> {
     this.rule = isMonotonic() ? Hypothesis.MONO_RULE : Hypothesis.SWAP_RULE;
 //    this.score = hypotheses.get(ranks[0]).score + phrases.get(ranks[1]).getEstimatedCost();
 
-    this.phraseNode = null;
     this.computedResult = null;
     
     // TODO: compute this proactively or lazily according to a parameter
@@ -213,7 +204,8 @@ public class Candidate implements Comparable<Candidate> {
    * @return the phrase rule at position ranks[1]
    */
   public Rule getPhraseRule() {
-    return this.phrases.get(ranks[1]);
+    Rule rule = getPhraseNode().bestHyperedge.getRule();
+    return rule;
   }
   
   /**
@@ -223,7 +215,7 @@ public class Candidate implements Comparable<Candidate> {
    * @return a new hypergraph node representing the phrase translation
    */
   public HGNode getPhraseNode() {
-    return phraseNode;
+    return this.phrases.get(ranks[1]);
   }
   
   /**
@@ -234,11 +226,6 @@ public class Candidate implements Comparable<Candidate> {
    */
   public ComputeNodeResult computeResult() {
     if (computedResult == null) {
-      // add the phrase node
-      phraseResult = new ComputeNodeResult(featureFunctions, getPhraseRule(), null, phrases.i, phrases.j, null, sentence);
-      HyperEdge edge = new HyperEdge(getPhraseRule(), phraseResult.getViterbiCost(), phraseResult.getTransitionCost(), null, null);
-      phraseNode = new HGNode(phrases.i, phrases.j, getPhraseRule().getLHS(), phraseResult.getDPStates(), edge, phraseResult.getPruningEstimate());
-
       // add the rule
       // TODO: sourcepath
       computedResult = new ComputeNodeResult(featureFunctions, getRule(), getTailNodes(), getLastCovered(), getPhraseEnd(), null, sentence);
@@ -299,7 +286,7 @@ public class Candidate implements Comparable<Candidate> {
    */
   public float score() {
 //    float score = computedResult.getViterbiCost() + future_delta;
-    float score = getHypothesis().getScore() + future_delta + phraseResult.getTransitionCost() + computedResult.getTransitionCost();
+    float score = getHypothesis().getScore() + getPhraseNode().getScore() + future_delta + computedResult.getTransitionCost();
     return score;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/12b834e2/src/main/java/org/apache/joshua/decoder/phrase/Future.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/joshua/decoder/phrase/Future.java b/src/main/java/org/apache/joshua/decoder/phrase/Future.java
index 005c776..a81fff5 100644
--- a/src/main/java/org/apache/joshua/decoder/phrase/Future.java
+++ b/src/main/java/org/apache/joshua/decoder/phrase/Future.java
@@ -59,9 +59,9 @@ public class Future {
         if (begin == sentlen - 1 && end == sentlen) 
           setEntry(begin, end, 0.0f);
         else {
-          TargetPhrases phrases = chart.getRange(begin, end);
+          PhraseNodes phrases = chart.getRange(begin, end);
           if (phrases != null)
-            setEntry(begin, end, phrases.get(0).getEstimatedCost());
+            setEntry(begin, end, phrases.get(0).bestHyperedge.getRule().getEstimatedCost());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/12b834e2/src/main/java/org/apache/joshua/decoder/phrase/PhraseChart.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/joshua/decoder/phrase/PhraseChart.java b/src/main/java/org/apache/joshua/decoder/phrase/PhraseChart.java
index c0dbfbc..5607949 100644
--- a/src/main/java/org/apache/joshua/decoder/phrase/PhraseChart.java
+++ b/src/main/java/org/apache/joshua/decoder/phrase/PhraseChart.java
@@ -22,10 +22,12 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-import org.apache.joshua.decoder.Decoder;
+import org.apache.joshua.decoder.chart_parser.ComputeNodeResult;
 import org.apache.joshua.decoder.ff.FeatureFunction;
 import org.apache.joshua.decoder.ff.tm.Rule;
 import org.apache.joshua.decoder.ff.tm.RuleCollection;
+import org.apache.joshua.decoder.hypergraph.HGNode;
+import org.apache.joshua.decoder.hypergraph.HyperEdge;
 import org.apache.joshua.decoder.segment_file.Sentence;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -42,11 +44,16 @@ public class PhraseChart {
   private int max_source_phrase_length;
 
   // Banded array: different source lengths are next to each other.
-  private final List<TargetPhrases> entries;
+  private final List<PhraseNodes> entries;
 
   // number of translation options
-  int numOptions = 20;
+  private int numOptions = 20;
+  
+  // The feature functions
   private final List<FeatureFunction> features;
+  
+  // The input sentence
+  private Sentence sentence;
 
   /**
    * Create a new PhraseChart object, which represents all phrases that are
@@ -65,6 +72,7 @@ public class PhraseChart {
 
     this.numOptions = num_options;
     this.features = features;
+    this.sentence = source;
 
     max_source_phrase_length = 0;
     for (PhraseTable table1 : tables)
@@ -93,8 +101,10 @@ public class PhraseChart {
       }
     }
 
-    entries.stream().filter(phrases -> phrases != null)
-        .forEach(phrases -> phrases.finish(features, Decoder.weights, num_options));
+    /* 
+     * Sort all of the HGNodes that were added.
+     */
+    entries.stream().filter(phrases -> phrases != null).forEach(phrases -> phrases.finish());
 
     LOG.info("Input {}: Collecting options took {} seconds", source.id(),
         (System.currentTimeMillis() - startTime) / 1000.0f);
@@ -103,11 +113,13 @@ public class PhraseChart {
       for (int i = 1; i < sentence_length - 1; i++) {
         for (int j = i + 1; j < sentence_length && j <= i + max_source_phrase_length; j++) {
           if (source.hasPath(i, j)) {
-            TargetPhrases phrases = getRange(i, j);
+            PhraseNodes phrases = getRange(i, j);
             if (phrases != null) {
               LOG.debug("{} ({}-{})", source.source(i,j), i, j);
-              for (Rule rule: phrases)
+              for (HGNode node: phrases) {
+                Rule rule = node.bestHyperedge.getRule();
                 LOG.debug("    {} :: est={}", rule.getEnglishWords(), rule.getEstimatedCost());
+              }
             }
           }
         }
@@ -140,9 +152,9 @@ public class PhraseChart {
    * 
    * @param begin beginning of span
    * @param end end of span
-   * @return the {@link org.apache.joshua.decoder.phrase.TargetPhrases} at the specified position in this list.
+   * @return the {@link org.apache.joshua.decoder.phrase.PhraseNodes} at the specified position in this list.
    */
-  public TargetPhrases getRange(int begin, int end) {
+  public PhraseNodes getRange(int begin, int end) {
     int index = offset(begin, end);
     // System.err.println(String.format("PhraseChart::Range(%d,%d): found %d entries",
     // begin, end,
@@ -153,6 +165,8 @@ public class PhraseChart {
 
     if (index < 0 || index >= entries.size() || entries.get(index) == null)
       return null;
+    
+    // Produce the nodes for each of the features
 
     return entries.get(index);
   }
@@ -160,11 +174,11 @@ public class PhraseChart {
   /**
    * Add a set of phrases from a grammar to the current span.
    * 
-   * @param begin beginning of span
-   * @param end end of span
+   * @param i beginning of span
+   * @param j end of span
    * @param to a {@link org.apache.joshua.decoder.ff.tm.RuleCollection} to be used in scoring and sorting.
    */
-  private void addToRange(int begin, int end, RuleCollection to) {
+  private void addToRange(int i, int j, RuleCollection to) {
     if (to != null) {
       /*
        * This first call to getSortedRules() is important, because it is what
@@ -176,17 +190,28 @@ public class PhraseChart {
        * likely to have (often into the tens of thousands).
        */
       List<Rule> rules = to.getSortedRules(features);
+      
+      // TODO: I think this is a race condition
       if (numOptions > 0 && rules.size() > numOptions)
-        rules = rules.subList(0,  numOptions);
+        rules = rules.subList(0,  numOptions - 1);
 //        to.getRules().subList(numOptions, to.getRules().size()).clear();
 
       try {
-        int offset = offset(begin, end);
+        int offset = offset(i, j);
         if (entries.get(offset) == null)
-          entries.set(offset, new TargetPhrases(begin, end));
-        entries.get(offset).addAll(rules);
+          entries.set(offset, new PhraseNodes(i, j, numOptions));
+        PhraseNodes nodes = entries.get(offset);
+
+        // Turn each rule into an HGNode, add them one by one 
+        for (Rule rule: rules) {
+          ComputeNodeResult result = new ComputeNodeResult(features, rule, null, i, j, null, sentence);
+          HyperEdge edge = new HyperEdge(rule, result.getViterbiCost(), result.getTransitionCost(), null, null);
+          HGNode phraseNode = new HGNode(i, j, rule.getLHS(), result.getDPStates(), edge, result.getPruningEstimate());
+          nodes.add(phraseNode);
+        }
+//        entries.get(offset).addAll(rules);
       } catch (java.lang.IndexOutOfBoundsException e) {
-        LOG.error("Whoops! {} [{}-{}] too long ({})", to, begin, end, entries.size());
+        LOG.error("Whoops! {} [{}-{}] too long ({})", to, i, j, entries.size());
         LOG.error(e.getMessage(), e);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/12b834e2/src/main/java/org/apache/joshua/decoder/phrase/PhraseNodes.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/joshua/decoder/phrase/PhraseNodes.java b/src/main/java/org/apache/joshua/decoder/phrase/PhraseNodes.java
new file mode 100644
index 0000000..c20c05a
--- /dev/null
+++ b/src/main/java/org/apache/joshua/decoder/phrase/PhraseNodes.java
@@ -0,0 +1,63 @@
+/*
+ * 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 org.apache.joshua.decoder.phrase;
+
+import java.util.ArrayList;
+import java.util.Collections;
+
+import org.apache.joshua.decoder.hypergraph.HGNode;
+
+/**
+ * Represents a sorted collection of target-side phrases. Typically, these are phrases
+ * generated from the same source word sequence. The list of options is reduced to the number
+ * of translation options.
+ * 
+ * @author Matt Post
+ */
+
+public class PhraseNodes extends ArrayList<HGNode> {
+
+  private static final long serialVersionUID = 1L;
+  
+  public int i = -2;
+  public int j = -2;
+
+  public PhraseNodes(int i, int j, int initialSize) {
+    super(initialSize);
+    this.i = i;
+    this.j = j;
+  }
+  
+  /**
+   * Score the rules and sort them. Scoring is necessary because rules are only scored if they
+   * are used, in an effort to make reading in rules more efficient. This is starting to create
+   * some trouble and should probably be reworked.
+   * 
+   * @param features a {@link java.util.List} of {@link org.apache.joshua.decoder.ff.FeatureFunction}'s
+   * @param weights a populated {@link org.apache.joshua.decoder.ff.FeatureVector}
+   * @param num_options the number of options
+   */
+  public void finish() {
+    Collections.sort(this, HGNode.inverseLogPComparator);    
+//    System.err.println("TargetPhrases::finish()");
+//    for (Rule rule: this) 
+//      System.err.println("  " + rule);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/12b834e2/src/main/java/org/apache/joshua/decoder/phrase/Stacks.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/joshua/decoder/phrase/Stacks.java b/src/main/java/org/apache/joshua/decoder/phrase/Stacks.java
index 095289f..cfeaea2 100644
--- a/src/main/java/org/apache/joshua/decoder/phrase/Stacks.java
+++ b/src/main/java/org/apache/joshua/decoder/phrase/Stacks.java
@@ -174,7 +174,7 @@ public class Stacks {
             /* We have found a permissible phrase start point and length for the current coverage
              * vector. Find all the phrases over that span.
              */
-            TargetPhrases phrases = chart.getRange(begin, begin + phrase_length);
+            PhraseNodes phrases = chart.getRange(begin, begin + phrase_length);
             if (phrases == null)
               continue;
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/12b834e2/src/main/java/org/apache/joshua/decoder/phrase/TargetPhrases.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/joshua/decoder/phrase/TargetPhrases.java b/src/main/java/org/apache/joshua/decoder/phrase/TargetPhrases.java
deleted file mode 100644
index 5692d1a..0000000
--- a/src/main/java/org/apache/joshua/decoder/phrase/TargetPhrases.java
+++ /dev/null
@@ -1,87 +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 org.apache.joshua.decoder.phrase;
-
-import java.util.ArrayList;	
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.joshua.decoder.ff.FeatureFunction;
-import org.apache.joshua.decoder.ff.FeatureVector;
-import org.apache.joshua.decoder.ff.tm.Rule;
-
-/**
- * Represents a sorted collection of target-side phrases. Typically, these are phrases
- * generated from the same source word sequence. The list of options is reduced to the number
- * of translation options.
- * 
- * @author Matt Post
- */
-
-public class TargetPhrases extends ArrayList<Rule> {
-
-  private static final long serialVersionUID = 1L;
-  
-  public int i = -2;
-  public int j = -2;
-
-  public TargetPhrases(int i, int j) {
-    super();
-    
-    this.i = i;
-    this.j = j;
-  }
-  
-  /**
-   * Initialize with a collection of rules.
-   * 
-   * @param list a {@link java.util.List} of {@link org.apache.joshua.decoder.ff.tm.Rule}'s
-   */
-  public TargetPhrases(List<Rule> list) {
-    super();
-    
-    for (Rule rule: list) {
-      add(rule);
-    }
-  }
-  
-  /**
-   * Score the rules and sort them. Scoring is necessary because rules are only scored if they
-   * are used, in an effort to make reading in rules more efficient. This is starting to create
-   * some trouble and should probably be reworked.
-   * 
-   * @param features a {@link java.util.List} of {@link org.apache.joshua.decoder.ff.FeatureFunction}'s
-   * @param weights a populated {@link org.apache.joshua.decoder.ff.FeatureVector}
-   * @param num_options the number of options
-   */
-  public void finish(List<FeatureFunction> features, FeatureVector weights, int num_options) {
-    for (Rule rule: this) { 
-      rule.estimateRuleCost(features);
-//      System.err.println("TargetPhrases:finish(): " + rule);
-    }
-    Collections.sort(this, Rule.EstimatedCostComparator);
-    
-    if (this.size() > num_options)
-      this.removeRange(num_options, this.size());
-    
-//    System.err.println("TargetPhrases::finish()");
-//    for (Rule rule: this) 
-//      System.err.println("  " + rule);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/12b834e2/src/test/java/org/apache/joshua/decoder/phrase/decode/PhraseDecodingTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/joshua/decoder/phrase/decode/PhraseDecodingTest.java b/src/test/java/org/apache/joshua/decoder/phrase/decode/PhraseDecodingTest.java
index 5e878cb..8be3c36 100644
--- a/src/test/java/org/apache/joshua/decoder/phrase/decode/PhraseDecodingTest.java
+++ b/src/test/java/org/apache/joshua/decoder/phrase/decode/PhraseDecodingTest.java
@@ -21,8 +21,6 @@
 import static org.testng.Assert.assertEquals;
 
 import java.io.IOException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
 
 import org.apache.joshua.decoder.Decoder;
 import org.apache.joshua.decoder.JoshuaConfiguration;