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/06/03 21:01:35 UTC

[04/14] incubator-joshua git commit: Large refactor of the Translation output interface

Large refactor of the Translation output interface

Translation outputs were a bit of a mess, so a group of us sat down and came up with a plan to fix it. This should helpful in refining the API.

This commit fixes many of these problems.

-  Instead of returning Translation objects, the calls to Decoder.translate() now return HyperGraph objects. As before, a HyperGraph represents the complete (pruned) search space the decoder explored. A HyperGraph can then be operated on by KBestExtractors and by the new TranslationFactory object.
-  KBestExtractors is now an iterator that takes a HyperGraph object and returns DerivationState objects, each representing a single derivation tree
-  Translation and StructuredTranslation are now combined. Translation is effectively a dummy object with a number of fields of interest that get populated by TranslationFactory, per explicit requests. Each request returns the TranslationFactory object, so you can easily chain calls, and then retrieve the Translation object at the end. e.g.,
-  Neither KBestExtractors nor Translation objects do any printing. This improved encapsulation is a big improvement over the past. After building your Translation objects, they will contain only small objects such as strings, feature vectors, and alignments, that can be safely passed downstream while the HyperGraph gets destroyed. Also, code for processing and formatting is all now in one place, the TranslationFactory.

Other, unrelated edits:

-  Removed the forest rescoring and OracleExtraction classes. These are useful but not used, and are hard to read and should therefore be rewritten


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

Branch: refs/heads/joshua_api
Commit: f2f82c38af9aebd28f9d27f685a2e99767a2e575
Parents: fe88c68
Author: Matt Post <po...@cs.jhu.edu>
Authored: Wed May 25 11:06:13 2016 -0400
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Wed May 25 11:06:13 2016 -0400

----------------------------------------------------------------------
 src/joshua/decoder/Decoder.java                 | 147 ++--
 src/joshua/decoder/DecoderThread.java           |  21 +-
 src/joshua/decoder/JoshuaConfiguration.java     |  20 -
 src/joshua/decoder/StructuredTranslation.java   | 125 ---
 src/joshua/decoder/Translation.java             | 215 ++---
 src/joshua/decoder/TranslationFactory.java      | 127 +++
 src/joshua/decoder/Translations.java            |  16 +-
 src/joshua/decoder/ff/fragmentlm/Tree.java      |   2 +-
 .../hypergraph/FeatureVectorExtractor.java      |   1 -
 .../decoder/hypergraph/KBestExtractor.java      | 460 ++---------
 .../hypergraph/OutputStringExtractor.java       |   1 -
 .../decoder/hypergraph/ViterbiExtractor.java    |   4 +-
 .../hypergraph/WordAlignmentExtractor.java      |   7 +-
 src/joshua/decoder/io/JSONMessage.java          |  31 +-
 src/joshua/oracle/OracleExtractionHG.java       | 794 -------------------
 src/joshua/oracle/OracleExtractor.java          |  58 --
 16 files changed, 397 insertions(+), 1632 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2f82c38/src/joshua/decoder/Decoder.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/Decoder.java b/src/joshua/decoder/Decoder.java
index 22ed8b9..fc4ba89 100644
--- a/src/joshua/decoder/Decoder.java
+++ b/src/joshua/decoder/Decoder.java
@@ -36,6 +36,7 @@ import java.util.concurrent.BlockingQueue;
 
 import com.google.common.base.Strings;
 
+import hep.aida.ref.Test;
 import joshua.corpus.Vocabulary;
 import joshua.decoder.ff.FeatureVector;
 import joshua.decoder.JoshuaConfiguration.INPUT_TYPE;
@@ -50,6 +51,9 @@ import joshua.decoder.ff.tm.Trie;
 import joshua.decoder.ff.tm.format.HieroFormatReader;
 import joshua.decoder.ff.tm.hash_based.MemoryBasedBatchGrammar;
 import joshua.decoder.ff.tm.packed.PackedGrammar;
+import joshua.decoder.hypergraph.DerivationState;
+import joshua.decoder.hypergraph.HyperGraph;
+import joshua.decoder.hypergraph.KBestExtractor;
 import joshua.decoder.io.JSONMessage;
 import joshua.decoder.io.TranslationRequestStream;
 import joshua.decoder.phrase.PhraseTable;
@@ -86,10 +90,10 @@ import joshua.util.io.LineReader;
  */
 public class Decoder {
 
-  private final JoshuaConfiguration joshuaConfiguration;
+  private final JoshuaConfiguration config;
 
   public JoshuaConfiguration getJoshuaConfiguration() {
-    return joshuaConfiguration;
+    return config;
   }
 
   /*
@@ -141,10 +145,10 @@ public class Decoder {
    * testing.
    */
   private Decoder(JoshuaConfiguration joshuaConfiguration) {
-    this.joshuaConfiguration = joshuaConfiguration;
+    this.config = joshuaConfiguration;
     this.grammars = new ArrayList<Grammar>();
     this.threadPool = new ArrayBlockingQueue<DecoderThread>(
-        this.joshuaConfiguration.num_parallel_decoders, true);
+        this.config.num_parallel_decoders, true);
     this.customPhraseTable = null;
   }
 
@@ -183,7 +187,7 @@ public class Decoder {
      */
     private OutputStream out;
     
-    RequestParallelizer(TranslationRequestStream request, Translations response, OutputStream out) {
+    RequestParallelizer(TranslationRequestStream request, Translations response) {
       this.request = request;
       this.response = response;
       this.out = out;
@@ -309,7 +313,7 @@ public class Decoder {
         }
 
         // Search for the rule in the trie
-        int nt_i = Vocabulary.id(joshuaConfiguration.default_non_terminal);
+        int nt_i = Vocabulary.id(config.default_non_terminal);
         Trie trie = customPhraseTable.getTrieRoot().match(nt_i);
 
         for (String word: tokens[0].split("\\s+")) {
@@ -414,8 +418,8 @@ public class Decoder {
        * corresponding Translations object, and return the thread to the pool.
        */
       try {
-        Translation translation = decoderThread.translate(this.sentence);
-        translations.record(translation);
+        HyperGraph hg = decoderThread.translate(this.sentence);
+        translations.record(hg);
 
         /*
          * This is crucial! It's what makes the thread available for the next sentence to be
@@ -444,21 +448,27 @@ public class Decoder {
   public void decodeAll(TranslationRequestStream request, OutputStream out) throws IOException {
     Translations translations = new Translations(request);
 
-    /* Start a thread to handle requests on the input stream */
-    new RequestParallelizer(request, translations, out).start();
+    /* Start a thread to handle requests on the input stream. This thread will continually
+     * request individual DecoderThreads from the pool until all of the input segments have been
+     * translated. It returns them *in order* through an iterator interface as they become available.
+     */ 
+    new RequestParallelizer(request, translations).start();
     
     // Create the n-best output stream
     FileWriter nbest_out = null;
-    if (joshuaConfiguration.n_best_file != null)
-      nbest_out = new FileWriter(joshuaConfiguration.n_best_file);
+    if (config.n_best_file != null)
+      nbest_out = new FileWriter(config.n_best_file);
     
     for (;;) {
-      Translation translation = translations.next();
-      if (translation == null)
+      HyperGraph hg = translations.next();
+      if (hg == null)
         break;
 
-      if (joshuaConfiguration.input_type == INPUT_TYPE.json || joshuaConfiguration.server_type == SERVER_TYPE.HTTP) {
-        JSONMessage message = JSONMessage.buildMessage(translation);
+      Sentence sentence = hg.sentence;
+      
+      if (config.input_type == INPUT_TYPE.json || config.server_type == SERVER_TYPE.HTTP) {
+        KBestExtractor extractor = new KBestExtractor(sentence, hg, featureFunctions, weights, false, config);
+        JSONMessage message = JSONMessage.buildMessage(sentence, extractor, config);
         out.write(message.toString().getBytes());
         
       } else {
@@ -469,27 +479,42 @@ public class Decoder {
          * format.
          */
         String text;
-        if (joshuaConfiguration.moses) {
-          text = translation.toString().replaceAll("=", "= ");
-          // Write the complete formatted string to STDOUT
-          if (joshuaConfiguration.n_best_file != null)
-            nbest_out.write(text);
-          
-          // Extract just the translation and output that to STDOUT
-          text = text.substring(0,  text.indexOf('\n'));
-          String[] fields = text.split(" \\|\\|\\| ");
-          text = fields[1] + "\n";
+        if (config.moses) {
+          KBestExtractor extractor = new KBestExtractor(sentence, hg, featureFunctions, weights, false, config);
+          final String mosesFormat = "%i ||| %s ||| %f ||| %c"; 
           
-        } else {
-          text = translation.toString();
+          int k = 1;
+          for (DerivationState derivation: extractor) {
+            if (k > config.topN)
+              break;
+            
+            TranslationFactory factory = new TranslationFactory(sentence, derivation, config);
+            Translation translation = factory.formattedTranslation(mosesFormat).translation();
+            text = translation.getFormattedTranslation().replaceAll("=",  "= ");
+            // Write the complete formatted string to STDOUT
+            if (config.n_best_file != null)
+              nbest_out.write(text + "\n");
+            
+            k++;
+          }
         }
 
-        out.write(text.getBytes());
+        KBestExtractor extractor = new KBestExtractor(sentence, hg, featureFunctions, weights, false, config);
+        DerivationState viterbi = extractor.getViterbiDerivation();
+        Translation best = new TranslationFactory(sentence, viterbi, config)
+            .formattedTranslation(config.outputFormat)
+              .translation();
+        
+        Decoder.LOG(1, String.format("Translation %d: %.3f %s", sentence.id(), best.score(), best.toString()));
+
+        String bestString = best.getFormattedTranslation();
+        out.write(bestString.getBytes());
+        out.write("\n".getBytes());
       }
       out.flush();
     }
     
-    if (joshuaConfiguration.n_best_file != null)
+    if (config.n_best_file != null)
       nbest_out.close();
   }
 
@@ -500,16 +525,16 @@ public class Decoder {
    * @param sentence
    * @return The translated sentence
    */
-  public Translation decode(Sentence sentence) {
+  public HyperGraph decode(Sentence sentence) {
     // Get a thread.
 
     try {
       DecoderThread thread = threadPool.take();
-      Translation translation = thread.translate(sentence);
+      HyperGraph translation = thread.translate(sentence);
       threadPool.put(thread);
-
       return translation;
 
+
     } catch (InterruptedException e) {
       e.printStackTrace();
     }
@@ -614,7 +639,7 @@ public class Decoder {
    * @return the feature in Moses format
    */
   private String mosesize(String feature) {
-    if (joshuaConfiguration.moses) {
+    if (config.moses) {
       if (feature.startsWith("tm_") || feature.startsWith("lm_"))
         return feature.replace("_", "-");
     }
@@ -636,26 +661,26 @@ public class Decoder {
       /* Weights can be listed in a separate file (denoted by parameter "weights-file") or directly
        * in the Joshua config file. Config file values take precedent.
        */
-      this.readWeights(joshuaConfiguration.weights_file);
+      this.readWeights(config.weights_file);
       
       
       /* Add command-line-passed weights to the weights array for processing below */
-      if (!Strings.isNullOrEmpty(joshuaConfiguration.weight_overwrite)) {
-        String[] tokens = joshuaConfiguration.weight_overwrite.split("\\s+");
+      if (!Strings.isNullOrEmpty(config.weight_overwrite)) {
+        String[] tokens = config.weight_overwrite.split("\\s+");
         for (int i = 0; i < tokens.length; i += 2) {
           String feature = tokens[i];
           float value = Float.parseFloat(tokens[i+1]);
           
-          if (joshuaConfiguration.moses)
+          if (config.moses)
             feature = demoses(feature);
           
-          joshuaConfiguration.weights.add(String.format("%s %s", feature, tokens[i+1]));
+          config.weights.add(String.format("%s %s", feature, tokens[i+1]));
           Decoder.LOG(1, String.format("COMMAND LINE WEIGHT: %s -> %.3f", feature, value));
         }
       }
 
       /* Read the weights found in the config file */
-      for (String pairStr: joshuaConfiguration.weights) {
+      for (String pairStr: config.weights) {
         String pair[] = pairStr.split("\\s+");
 
         /* Sanity check for old-style unsupported feature invocations. */
@@ -690,10 +715,10 @@ public class Decoder {
       this.initializeFeatureFunctions();
 
       // This is mostly for compatibility with the Moses tuning script
-      if (joshuaConfiguration.show_weights_and_quit) {
+      if (config.show_weights_and_quit) {
         for (int i = 0; i < DENSE_FEATURE_NAMES.size(); i++) {
           String name = DENSE_FEATURE_NAMES.get(i);
-          if (joshuaConfiguration.moses) 
+          if (config.moses) 
             System.out.println(String.format("%s= %.5f", mosesize(name), weights.getDense(i)));
           else
             System.out.println(String.format("%s %.5f", name, weights.getDense(i)));
@@ -702,7 +727,7 @@ public class Decoder {
       }
       
       // Sort the TM grammars (needed to do cube pruning)
-      if (joshuaConfiguration.amortized_sorting) {
+      if (config.amortized_sorting) {
         Decoder.LOG(1, "Grammar sorting happening lazily on-demand.");
       } else {
         long pre_sort_time = System.currentTimeMillis();
@@ -714,9 +739,9 @@ public class Decoder {
       }
 
       // Create the threads
-      for (int i = 0; i < joshuaConfiguration.num_parallel_decoders; i++) {
+      for (int i = 0; i < config.num_parallel_decoders; i++) {
         this.threadPool.put(new DecoderThread(this.grammars, Decoder.weights,
-            this.featureFunctions, joshuaConfiguration));
+            this.featureFunctions, config));
       }
 
     } catch (IOException e) {
@@ -738,13 +763,13 @@ public class Decoder {
    */
   private void initializeTranslationGrammars() throws IOException {
 
-    if (joshuaConfiguration.tms.size() > 0) {
+    if (config.tms.size() > 0) {
 
       // collect packedGrammars to check if they use a shared vocabulary
       final List<PackedGrammar> packed_grammars = new ArrayList<>();
 
       // tm = {thrax/hiero,packed,samt,moses} OWNER LIMIT FILE
-      for (String tmLine : joshuaConfiguration.tms) {
+      for (String tmLine : config.tms) {
 
         String type = tmLine.substring(0,  tmLine.indexOf(' '));
         String[] args = tmLine.substring(tmLine.indexOf(' ')).trim().split("\\s+");
@@ -758,7 +783,7 @@ public class Decoder {
         if (! type.equals("moses") && ! type.equals("phrase")) {
           if (new File(path).isDirectory()) {
             try {
-              PackedGrammar packed_grammar = new PackedGrammar(path, span_limit, owner, type, joshuaConfiguration);
+              PackedGrammar packed_grammar = new PackedGrammar(path, span_limit, owner, type, config);
               packed_grammars.add(packed_grammar);
               grammar = packed_grammar;
             } catch (FileNotFoundException e) {
@@ -769,7 +794,7 @@ public class Decoder {
           } else {
             // thrax, hiero, samt
             grammar = new MemoryBasedBatchGrammar(type, path, owner,
-                joshuaConfiguration.default_non_terminal, span_limit, joshuaConfiguration);
+                config.default_non_terminal, span_limit, config);
           }
           
         } else {
@@ -778,8 +803,8 @@ public class Decoder {
               ? Integer.parseInt(parsedArgs.get("max-source-len"))
               : -1;
 
-          joshuaConfiguration.search_algorithm = "stack";
-          grammar = new PhraseTable(path, owner, type, joshuaConfiguration, maxSourceLen);
+          config.search_algorithm = "stack";
+          grammar = new PhraseTable(path, owner, type, config, maxSourceLen);
         }
 
         this.grammars.add(grammar);
@@ -789,25 +814,25 @@ public class Decoder {
 
     } else {
       Decoder.LOG(1, "* WARNING: no grammars supplied!  Supplying dummy glue grammar.");
-      MemoryBasedBatchGrammar glueGrammar = new MemoryBasedBatchGrammar("glue", joshuaConfiguration);
+      MemoryBasedBatchGrammar glueGrammar = new MemoryBasedBatchGrammar("glue", config);
       glueGrammar.setSpanLimit(-1);
       glueGrammar.addGlueRules(featureFunctions);
       this.grammars.add(glueGrammar);
     }
     
     /* Add the grammar for custom entries */
-    this.customPhraseTable = new PhraseTable(null, "custom", "phrase", joshuaConfiguration, 0);
+    this.customPhraseTable = new PhraseTable(null, "custom", "phrase", config, 0);
     this.grammars.add(this.customPhraseTable);
     
     /* Create an epsilon-deleting grammar */
-    if (joshuaConfiguration.lattice_decoding) {
+    if (config.lattice_decoding) {
       Decoder.LOG(1, "Creating an epsilon-deleting grammar");
-      MemoryBasedBatchGrammar latticeGrammar = new MemoryBasedBatchGrammar("lattice", joshuaConfiguration);
+      MemoryBasedBatchGrammar latticeGrammar = new MemoryBasedBatchGrammar("lattice", config);
       latticeGrammar.setSpanLimit(-1);
       HieroFormatReader reader = new HieroFormatReader();
 
-      String goalNT = FormatUtils.cleanNonTerminal(joshuaConfiguration.goal_symbol);
-      String defaultNT = FormatUtils.cleanNonTerminal(joshuaConfiguration.default_non_terminal);
+      String goalNT = FormatUtils.cleanNonTerminal(config.goal_symbol);
+      String defaultNT = FormatUtils.cleanNonTerminal(config.default_non_terminal);
 
       String ruleString = String.format("[%s] ||| [%s,1] <eps> ||| [%s,1] ||| ", goalNT, goalNT, defaultNT,
           goalNT, defaultNT);
@@ -826,7 +851,7 @@ public class Decoder {
       String owner = Vocabulary.word(grammar.getOwner());
       if (! ownersSeen.contains(owner)) {
         this.featureFunctions.add(new PhraseModel(weights, new String[] { "tm", "-owner", owner },
-            joshuaConfiguration, grammar));
+            config, grammar));
         ownersSeen.add(owner);
       }
     }
@@ -882,7 +907,7 @@ public class Decoder {
         Float value = Float.parseFloat(tokens[1]);
         
         // Kludge for compatibility with Moses tuners
-        if (joshuaConfiguration.moses) {
+        if (config.moses) {
           feature = demoses(feature);
         }
 
@@ -925,7 +950,7 @@ public class Decoder {
    */
   private void initializeFeatureFunctions() throws IOException {
 
-    for (String featureLine : joshuaConfiguration.features) {
+    for (String featureLine : config.features) {
       // feature-function = NAME args
       // 1. create new class named NAME, pass it config, weights, and the args
 
@@ -938,7 +963,7 @@ public class Decoder {
         Class<?> clas = getClass(featureName);
         Constructor<?> constructor = clas.getConstructor(FeatureVector.class,
             String[].class, JoshuaConfiguration.class);
-        this.featureFunctions.add((FeatureFunction) constructor.newInstance(weights, fields, joshuaConfiguration));
+        this.featureFunctions.add((FeatureFunction) constructor.newInstance(weights, fields, config));
       } catch (Exception e) {
         e.printStackTrace();
         System.err.println("* FATAL: could not find a feature '" + featureName + "'");

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2f82c38/src/joshua/decoder/DecoderThread.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/DecoderThread.java b/src/joshua/decoder/DecoderThread.java
index b1857cd..cf78420 100644
--- a/src/joshua/decoder/DecoderThread.java
+++ b/src/joshua/decoder/DecoderThread.java
@@ -27,6 +27,7 @@ import joshua.decoder.chart_parser.Chart;
 import joshua.decoder.ff.FeatureFunction;
 import joshua.decoder.ff.FeatureVector;
 import joshua.decoder.ff.SourceDependentFF;
+import joshua.decoder.ff.lm.StateMinimizingLanguageModel;
 import joshua.decoder.ff.tm.Grammar;
 import joshua.decoder.hypergraph.ForestWalker;
 import joshua.decoder.hypergraph.GrammarBuilderWalkerFunction;
@@ -92,7 +93,7 @@ public class DecoderThread extends Thread {
    * 
    * @param sentence The sentence to be translated.
    */
-  public Translation translate(Sentence sentence) {
+  public HyperGraph translate(Sentence sentence) {
 
     Decoder.LOG(1, "Input " + sentence.id() + ", " + sentence.fullSource());
 
@@ -102,7 +103,7 @@ public class DecoderThread extends Thread {
     // skip blank sentences
     if (sentence.isEmpty()) {
       Decoder.LOG(1, "Translation " + sentence.id() + ": Translation took 0 seconds");
-      return new Translation(sentence, null, featureFunctions, joshuaConfiguration);
+      return null;
     }
     
     long startTime = System.currentTimeMillis();
@@ -148,9 +149,21 @@ public class DecoderThread extends Thread {
     Decoder.LOG(1, String.format("Input %d: Memory used is %.1f MB", sentence.id(), (Runtime
         .getRuntime().totalMemory() - Runtime.getRuntime().freeMemory()) / 1000000.0));
 
+     /*
+     * KenLM hack. If using KenLMFF, we need to tell KenLM to delete the pool used to create chart
+     * objects for this sentence.
+     */
+    // TODO: make sure this works here
+    for (FeatureFunction feature : featureFunctions) {
+      if (feature instanceof StateMinimizingLanguageModel) {
+        ((StateMinimizingLanguageModel) feature).destroyPool(sentence.id());
+        break;
+      }
+    }
+    
     /* Return the translation unless we're doing synchronous parsing. */
     if (!joshuaConfiguration.parse || hypergraph == null) {
-      return new Translation(sentence, hypergraph, featureFunctions, joshuaConfiguration);
+      return hypergraph;
     }
 
     /*****************************************************************************************/
@@ -186,7 +199,7 @@ public class DecoderThread extends Thread {
     logger.info(String.format("Memory used after sentence %d is %.1f MB", sentence.id(), (Runtime
         .getRuntime().totalMemory() - Runtime.getRuntime().freeMemory()) / 1000000.0));
 
-    return new Translation(sentence, englishParse, featureFunctions, joshuaConfiguration); // or do something else
+    return englishParse;
   }
 
   private Grammar getGrammarFromHyperGraph(String goal, HyperGraph hg) {

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2f82c38/src/joshua/decoder/JoshuaConfiguration.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/JoshuaConfiguration.java b/src/joshua/decoder/JoshuaConfiguration.java
index c874534..3f20f46 100644
--- a/src/joshua/decoder/JoshuaConfiguration.java
+++ b/src/joshua/decoder/JoshuaConfiguration.java
@@ -209,18 +209,6 @@ public class JoshuaConfiguration {
   public int server_port = 0;
 
   /*
-   * Whether to do forest rescoring. If set to true, the references are expected on STDIN along with
-   * the input sentences in the following format:
-   * 
-   * input sentence ||| ||| reference1 ||| reference2 ...
-   * 
-   * (The second field is reserved for the output sentence for alignment and forced decoding).
-   */
-
-  public boolean rescoreForest = false;
-  public float rescoreForestWeight = 10.0f;
-
-  /*
    * Location of fragment mapping file, which maps flattened SCFG rules to their internal
    * representation.
    */
@@ -564,14 +552,6 @@ public class JoshuaConfiguration {
             server_port = Integer.parseInt(fds[1]);
             logger.info(String.format("    server-port: %d", server_port));
 
-          } else if (parameter.equals(normalize_key("rescore-forest"))) {
-            rescoreForest = true;
-            logger.info(String.format("    rescore-forest: %s", rescoreForest));
-
-          } else if (parameter.equals(normalize_key("rescore-forest-weight"))) {
-            rescoreForestWeight = Float.parseFloat(fds[1]);
-            logger.info(String.format("    rescore-forest-weight: %f", rescoreForestWeight));
-
           } else if (parameter.equals(normalize_key("maxlen"))) {
             // reset the maximum length
             maxlen = Integer.parseInt(fds[1]);

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2f82c38/src/joshua/decoder/StructuredTranslation.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/StructuredTranslation.java b/src/joshua/decoder/StructuredTranslation.java
deleted file mode 100644
index 7b2185f..0000000
--- a/src/joshua/decoder/StructuredTranslation.java
+++ /dev/null
@@ -1,125 +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;
-
-import static java.util.Arrays.asList;
-import static java.util.Collections.emptyList;
-import static joshua.decoder.hypergraph.ViterbiExtractor.getViterbiFeatures;
-import static joshua.decoder.hypergraph.ViterbiExtractor.getViterbiString;
-import static joshua.decoder.hypergraph.ViterbiExtractor.getViterbiWordAlignmentList;
-import static joshua.util.FormatUtils.removeSentenceMarkers;
-
-import java.util.List;
-import java.util.Map;
-
-import joshua.decoder.ff.FeatureFunction;
-import joshua.decoder.hypergraph.HyperGraph;
-import joshua.decoder.segment_file.Sentence;
-
-/**
- * structuredTranslation provides a more structured access to translation
- * results than the Translation class.
- * Members of instances of this class can be used upstream.
- * <br/>
- * TODO:
- * Enable K-Best extraction.
- * 
- * @author fhieber
- */
-public class StructuredTranslation {
-  
-  private final Sentence sourceSentence;
-  private final String translationString;
-  private final List<String> translationTokens;
-  private final float translationScore;
-  private final List<List<Integer>> translationWordAlignments;
-  private final Map<String,Float> translationFeatures;
-  private final float extractionTime;
-  
-  public StructuredTranslation(final Sentence sourceSentence,
-      final HyperGraph hypergraph,
-      final List<FeatureFunction> featureFunctions) {
-    
-      final long startTime = System.currentTimeMillis();
-      
-      this.sourceSentence = sourceSentence;
-      this.translationString = removeSentenceMarkers(getViterbiString(hypergraph));
-      this.translationTokens = extractTranslationTokens();
-      this.translationScore = extractTranslationScore(hypergraph);
-      this.translationFeatures = getViterbiFeatures(hypergraph, featureFunctions, sourceSentence).getMap();
-      this.translationWordAlignments = getViterbiWordAlignmentList(hypergraph);
-      this.extractionTime = (System.currentTimeMillis() - startTime) / 1000.0f;
-  }
-  
-  private float extractTranslationScore(final HyperGraph hypergraph) {
-    if (hypergraph == null) {
-      return 0;
-    } else {
-      return hypergraph.goalNode.getScore();
-    }
-  }
-  
-  private List<String> extractTranslationTokens() {
-    if (translationString.isEmpty()) {
-      return emptyList();
-    } else {
-      return asList(translationString.split("\\s+"));
-    }
-  }
-  
-  // Getters to use upstream
-  
-  public Sentence getSourceSentence() {
-    return sourceSentence;
-  }
-
-  public int getSentenceId() {
-    return sourceSentence.id();
-  }
-
-  public String getTranslationString() {
-    return translationString;
-  }
-
-  public List<String> getTranslationTokens() {
-    return translationTokens;
-  }
-
-  public float getTranslationScore() {
-    return translationScore;
-  }
-
-  /**
-   * Returns a list of target to source alignments.
-   */
-  public List<List<Integer>> getTranslationWordAlignments() {
-    return translationWordAlignments;
-  }
-  
-  public Map<String,Float> getTranslationFeatures() {
-    return translationFeatures;
-  }
-  
-  /**
-   * Time taken to build output information from the hypergraph.
-   */
-  public Float getExtractionTime() {
-    return extractionTime;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2f82c38/src/joshua/decoder/Translation.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/Translation.java b/src/joshua/decoder/Translation.java
index 8004d9f..2af065c 100644
--- a/src/joshua/decoder/Translation.java
+++ b/src/joshua/decoder/Translation.java
@@ -18,24 +18,16 @@
  */
 package joshua.decoder;
 
-import static joshua.decoder.hypergraph.ViterbiExtractor.getViterbiFeatures;
-import static joshua.decoder.hypergraph.ViterbiExtractor.getViterbiString;
-import static joshua.decoder.hypergraph.ViterbiExtractor.getViterbiWordAlignments;
 import static joshua.util.FormatUtils.removeSentenceMarkers;
+import static joshua.util.FormatUtils.unescapeSpecialSymbols;
 
-import java.io.BufferedWriter;
-import java.io.IOException;
-import java.io.StringWriter;
 import java.util.List;
 
-import joshua.decoder.ff.FeatureFunction;
 import joshua.decoder.ff.FeatureVector;
-import joshua.decoder.ff.lm.StateMinimizingLanguageModel;
-import joshua.decoder.hypergraph.HyperGraph;
-import joshua.decoder.hypergraph.KBestExtractor;
-import joshua.decoder.io.DeNormalize;
+import joshua.decoder.hypergraph.WordAlignmentState;
 import joshua.decoder.segment_file.Sentence;
 
+
 /**
  * This class represents translated input objects (sentences or lattices). It is aware of the source
  * sentence and id and contains the decoded hypergraph. Translation objects are returned by
@@ -45,158 +37,83 @@ import joshua.decoder.segment_file.Sentence;
  */
 
 public class Translation {
-  private Sentence source;
-
-  /**
-   * This stores the output of the translation so we don't have to hold onto the hypergraph while we
-   * wait for the outputs to be assembled.
-   */
-  private String output = null;
-
-  private StructuredTranslation structuredTranslation = null;
-  
-  public Translation(Sentence source, HyperGraph hypergraph, 
-      List<FeatureFunction> featureFunctions, JoshuaConfiguration joshuaConfiguration) {
-    this.source = source;
+  private final Sentence sourceSentence;
+  private final String rawTranslation;
+  private final String translation;
+  private final float translationScore;
+  private String formattedTranslation;
+  private List<List<Integer>> translationWordAlignments;
+  private float extractionTime;
+  private FeatureVector features;
+  private WordAlignmentState wordAlignment;
+
+  public Translation(final Sentence source, final String output, final float cost) {
+    this.sourceSentence = source;
+    this.rawTranslation = output;
+    this.translationScore = cost;
     
-    if (joshuaConfiguration.use_structured_output) {
-      
-      structuredTranslation = new StructuredTranslation(
-          source, hypergraph, featureFunctions);
-      this.output = structuredTranslation.getTranslationString();
-      
-    } else {
-
-      StringWriter sw = new StringWriter();
-      BufferedWriter out = new BufferedWriter(sw);
-
-      try {
-        if (hypergraph != null) {
-          if (!joshuaConfiguration.hypergraphFilePattern.equals("")) {
-            hypergraph.dump(String.format(joshuaConfiguration.hypergraphFilePattern, source.id()), featureFunctions);
-          }
-
-          long startTime = System.currentTimeMillis();
-
-          // We must put this weight as zero, otherwise we get an error when we try to retrieve it
-          // without checking
-          Decoder.weights.increment("BLEU", 0);
-          
-          if (joshuaConfiguration.topN == 0) {
-            
-            /* construct Viterbi output */
-            final String best = getViterbiString(hypergraph);
-            
-            Decoder.LOG(1, String.format("Translation %d: %.3f %s", source.id(), hypergraph.goalNode.getScore(),
-                best));
-            
-            /*
-             * Setting topN to 0 turns off k-best extraction, in which case we need to parse through
-             * the output-string, with the understanding that we can only substitute variables for the
-             * output string, sentence number, and model score.
-             */
-            String translation = joshuaConfiguration.outputFormat
-                .replace("%s", removeSentenceMarkers(best))
-                .replace("%S", DeNormalize.processSingleLine(best))
-                .replace("%c", String.format("%.3f", hypergraph.goalNode.getScore()))
-                .replace("%i", String.format("%d", source.id()));
-            
-            if (joshuaConfiguration.outputFormat.contains("%a")) {
-              translation = translation.replace("%a", getViterbiWordAlignments(hypergraph));
-            }
-            
-            if (joshuaConfiguration.outputFormat.contains("%f")) {
-              final FeatureVector features = getViterbiFeatures(hypergraph, featureFunctions, source);
-              translation = translation.replace("%f", joshuaConfiguration.moses ? features.mosesString() : features.toString());
-            }
-            
-            out.write(translation);
-            out.newLine();
-            
-          } else {
-            
-            final KBestExtractor kBestExtractor = new KBestExtractor(
-                source, featureFunctions, Decoder.weights, false, joshuaConfiguration);
-            kBestExtractor.lazyKBestExtractOnHG(hypergraph, joshuaConfiguration.topN, out);
-
-            if (joshuaConfiguration.rescoreForest) {
-              Decoder.weights.increment("BLEU", joshuaConfiguration.rescoreForestWeight);
-              kBestExtractor.lazyKBestExtractOnHG(hypergraph, joshuaConfiguration.topN, out);
-
-              Decoder.weights.increment("BLEU", -joshuaConfiguration.rescoreForestWeight);
-              kBestExtractor.lazyKBestExtractOnHG(hypergraph, joshuaConfiguration.topN, out);
-            }
-          }
-
-          float seconds = (float) (System.currentTimeMillis() - startTime) / 1000.0f;
-          Decoder.LOG(1, String.format("Input %d: %d-best extraction took %.3f seconds", id(),
-              joshuaConfiguration.topN, seconds));
-
-      } else {
-        
-        // Failed translations and blank lines get empty formatted outputs
-        // @formatter:off
-        String outputString = joshuaConfiguration.outputFormat
-            .replace("%s", source.source())
-            .replace("%e", "")
-            .replace("%S", "")
-            .replace("%t", "()")
-            .replace("%i", Integer.toString(source.id()))
-            .replace("%f", "")
-            .replace("%c", "0.000");
-        // @formatter:on
-
-        out.write(outputString);
-        out.newLine();
-      }
-
-        out.flush();
-      } catch (IOException e) {
-        e.printStackTrace();
-        System.exit(1);
-      }
-      
-      this.output = sw.toString();
-      
-    }
-
-    /*
-     * KenLM hack. If using KenLMFF, we need to tell KenLM to delete the pool used to create chart
-     * objects for this sentence.
-     */
-    for (FeatureFunction feature : featureFunctions) {
-      if (feature instanceof StateMinimizingLanguageModel) {
-        ((StateMinimizingLanguageModel) feature).destroyPool(getSourceSentence().id());
-        break;
-      }
-    }
+    this.translation = unescapeSpecialSymbols(removeSentenceMarkers(rawTranslation));
     
+//    final long startTime = System.currentTimeMillis();
+//    this.extractionTime = (System.currentTimeMillis() - startTime) / 1000.0f;
   }
 
   public Sentence getSourceSentence() {
-    return this.source;
+    return this.sourceSentence;
   }
 
+  public float score() {
+    return translationScore;
+  }
+
+  /**
+   * Returns a list of target to source alignments.
+   */
+  public List<List<Integer>> getTranslationWordAlignments() {
+    return translationWordAlignments;
+  }
+  
+  /**
+   * Time taken to build output information from the hypergraph.
+   */
+  public Float getExtractionTime() {
+    return extractionTime;
+  }
+  
   public int id() {
-    return source.id();
+    return sourceSentence.id();
   }
 
   @Override
   public String toString() {
-    return output;
+    return this.translation;
   }
   
-  /**
-   * Returns the StructuredTranslation object
-   * if JoshuaConfiguration.construct_structured_output == True.
-   * @throws RuntimeException if StructuredTranslation object not set.
-   * @return
-   */
-  public StructuredTranslation getStructuredTranslation() {
-    if (structuredTranslation == null) {
-      throw new RuntimeException("No StructuredTranslation object created. You should set JoshuaConfigration.construct_structured_output = true");
-    }
-    return structuredTranslation;
+  public String rawTranslation() {
+    return this.rawTranslation;
+  }
+
+  public void setFormattedTranslation(String formattedTranslation) {
+    this.formattedTranslation = formattedTranslation;
+  }
+  
+  public String getFormattedTranslation() {
+    return this.formattedTranslation;
+  }
+
+  public void setFeatures(FeatureVector features) {
+    this.features = features;
   }
   
+  public FeatureVector getFeatures() {
+    return this.features;
+  }
+
+  public void setWordAlignment(WordAlignmentState wordAlignment) {
+    this.wordAlignment = wordAlignment;
+  }
+
+  public Object getWordAlignment() {
+    return this.wordAlignment;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2f82c38/src/joshua/decoder/TranslationFactory.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/TranslationFactory.java b/src/joshua/decoder/TranslationFactory.java
new file mode 100644
index 0000000..9d1953e
--- /dev/null
+++ b/src/joshua/decoder/TranslationFactory.java
@@ -0,0 +1,127 @@
+package joshua.decoder;
+
+import joshua.decoder.ff.FeatureVector;
+import joshua.decoder.hypergraph.DerivationState;
+import joshua.decoder.hypergraph.KBestExtractor.Side;
+import joshua.decoder.io.DeNormalize;
+import joshua.decoder.segment_file.Sentence;
+import joshua.decoder.segment_file.Token;
+import joshua.util.FormatUtils;
+
+import java.util.List;
+
+public class TranslationFactory {
+
+  private final Sentence sentence;
+  private final JoshuaConfiguration config;
+
+  private DerivationState derivation;
+  private Translation translation;
+
+  public TranslationFactory(Sentence sentence, DerivationState derivation, JoshuaConfiguration config) {
+    this.sentence = sentence;
+    this.derivation = derivation;
+    this.config = config;
+    
+    if (this.derivation != null) {
+      this.translation = new Translation(sentence, derivation.getHypothesis(), derivation.getCost());
+    } else {
+      this.translation = new Translation(sentence, null, 0.0f);
+    }
+  }
+  
+  public Translation translation() {
+    return this.translation;
+  }
+
+  public TranslationFactory formattedTranslation(String format) {
+
+    // TODO: instead of calling replace() a million times, walk through yourself and find the
+    // special characters, and then replace them.  If you do this from the right side the index
+    // replacement should be a lot more efficient than what we're doing here, particularly since
+    // all these arguments get evaluated whether they're used or not
+
+    String output = format
+        .replace("%s", translation.toString())
+        .replace("%e", derivation.getHypothesis(Side.SOURCE))
+        .replace("%S", DeNormalize.processSingleLine(translation.toString()))
+        .replace("%c", String.format("%.3f", translation.score()))
+        .replace("%i", Integer.toString(sentence.id()));
+
+    if (output.contains("%a")) {
+      this.alignments().translation();
+      output = output.replace("%a", translation.getWordAlignment().toString());
+    }
+
+    if (config.outputFormat.contains("%f")) {
+      this.features();
+      final FeatureVector features = translation.getFeatures();
+      output = output.replace("%f", config.moses ? features.mosesString() : features.toString());
+    }
+    
+    if (output.contains("%t")) {
+      // TODO: also store in Translation objection
+      output = output.replace("%t", derivation.getTree());
+    }
+
+    /* %d causes a derivation with rules one per line to be output */
+    if (output.contains("%d")) {
+      // TODO: also store in Translation objection
+      output = output.replace("%d", derivation.getDerivation());
+    }
+
+    translation.setFormattedTranslation(maybeProjectCase(derivation, output));
+    return this;
+  }
+
+  /** 
+   * Stores the features
+   * 
+   * @return
+   */
+  public TranslationFactory features() {
+    translation.setFeatures(derivation.getFeatures());
+    return this;
+  }
+  
+  public TranslationFactory alignments() {
+    // TODO: write this
+    //    this.translation.setAlignments(getViterbiWordAlignmentList(derivation);
+    translation.setWordAlignment(derivation.getWordAlignment());
+    return this;
+  }
+  
+  /**
+   * If requested, projects source-side lettercase to target, and appends the alignment from
+   * to the source-side sentence in ||s.
+   * 
+   * @param hypothesis
+   * @param state
+   * @return
+   */
+  private String maybeProjectCase(DerivationState derivation, String hypothesis) {
+    String output = hypothesis;
+
+    if (config.project_case) {
+      String[] tokens = hypothesis.split("\\s+");
+      List<List<Integer>> points = derivation.getWordAlignment().toFinalList();
+      for (int i = 0; i < points.size(); i++) {
+        List<Integer> target = points.get(i);
+        for (int source: target) {
+          Token token = sentence.getTokens().get(source + 1); // skip <s>
+          String annotation = "";
+          if (token != null && token.getAnnotation("lettercase") != null)
+            annotation = token.getAnnotation("lettercase");
+          if (source != 0 && annotation.equals("upper"))
+            tokens[i] = FormatUtils.capitalize(tokens[i]);
+          else if (annotation.equals("all-upper"))
+            tokens[i] = tokens[i].toUpperCase();
+        }
+      }
+
+      output = String.join(" ",  tokens);
+    }
+
+    return output;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2f82c38/src/joshua/decoder/Translations.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/Translations.java b/src/joshua/decoder/Translations.java
index e6ba9e6..5bb0456 100644
--- a/src/joshua/decoder/Translations.java
+++ b/src/joshua/decoder/Translations.java
@@ -19,6 +19,8 @@
 package joshua.decoder;
 
 import java.util.LinkedList;
+
+import joshua.decoder.hypergraph.HyperGraph;
 import joshua.decoder.io.TranslationRequestStream;
 
 /**
@@ -44,13 +46,13 @@ public class Translations {
   private int currentID = 0;
 
   /* The set of translated sentences. */
-  private LinkedList<Translation> translations = null;
+  private LinkedList<HyperGraph> translations = null;
 
   private boolean spent = false;
 
   public Translations(TranslationRequestStream request) {
     this.request = request;
-    this.translations = new LinkedList<Translation>();
+    this.translations = new LinkedList<HyperGraph>();
   }
 
   /**
@@ -75,21 +77,21 @@ public class Translations {
    * 
    * @param translation
    */
-  public void record(Translation translation) {
+  public void record(HyperGraph hyperGraph) {
     synchronized (this) {
 
       /* Pad the set of translations with nulls to accommodate the new translation. */
-      int offset = translation.id() - currentID;
+      int offset = hyperGraph.sentID() - currentID;
       while (offset >= translations.size())
         translations.add(null);
-      translations.set(offset, translation);
+      translations.set(offset, hyperGraph);
 
       /*
        * If the id of the current translation is at the head of the list (first element), then we
        * have the next Translation to be return, and we should notify anyone waiting on next(),
        * which will then remove the item and increment the currentID.
        */
-      if (translation.id() == currentID) {
+      if (hyperGraph.sentID() == currentID) {
         this.notify();
       }
     }
@@ -99,7 +101,7 @@ public class Translations {
    * Returns the next Translation, blocking if necessary until it's available, since the next
    * Translation might not have been produced yet.
    */
-  public Translation next() {
+  public HyperGraph next() {
     synchronized (this) {
 
       /*

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2f82c38/src/joshua/decoder/ff/fragmentlm/Tree.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/ff/fragmentlm/Tree.java b/src/joshua/decoder/ff/fragmentlm/Tree.java
index b52ccce..5051c1b 100644
--- a/src/joshua/decoder/ff/fragmentlm/Tree.java
+++ b/src/joshua/decoder/ff/fragmentlm/Tree.java
@@ -26,9 +26,9 @@ import java.util.*;
 import joshua.corpus.Vocabulary;
 import joshua.decoder.ff.fragmentlm.Trees.PennTreeReader;
 import joshua.decoder.ff.tm.Rule;
+import joshua.decoder.hypergraph.DerivationState;
 import joshua.decoder.hypergraph.HGNode;
 import joshua.decoder.hypergraph.HyperEdge;
-import joshua.decoder.hypergraph.KBestExtractor.DerivationState;
 import joshua.util.io.LineReader;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2f82c38/src/joshua/decoder/hypergraph/FeatureVectorExtractor.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/hypergraph/FeatureVectorExtractor.java b/src/joshua/decoder/hypergraph/FeatureVectorExtractor.java
index dbe4f4b..a6e30a0 100644
--- a/src/joshua/decoder/hypergraph/FeatureVectorExtractor.java
+++ b/src/joshua/decoder/hypergraph/FeatureVectorExtractor.java
@@ -24,7 +24,6 @@ import java.util.List;
 
 import joshua.decoder.ff.FeatureFunction;
 import joshua.decoder.ff.FeatureVector;
-import joshua.decoder.hypergraph.KBestExtractor.DerivationState;
 import joshua.decoder.hypergraph.KBestExtractor.DerivationVisitor;
 import joshua.decoder.segment_file.Sentence;
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2f82c38/src/joshua/decoder/hypergraph/KBestExtractor.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/hypergraph/KBestExtractor.java b/src/joshua/decoder/hypergraph/KBestExtractor.java
index 6dd3207..b8f167c 100644
--- a/src/joshua/decoder/hypergraph/KBestExtractor.java
+++ b/src/joshua/decoder/hypergraph/KBestExtractor.java
@@ -18,9 +18,6 @@
  */
 package joshua.decoder.hypergraph;
 
-import static joshua.util.FormatUtils.unescapeSpecialSymbols;
-import static joshua.util.FormatUtils.removeSentenceMarkers;
-
 import java.io.BufferedWriter;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
@@ -29,18 +26,17 @@ import java.util.Comparator;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.PriorityQueue;
 
 import joshua.corpus.Vocabulary;
-import joshua.decoder.BLEU;
 import joshua.decoder.JoshuaConfiguration;
 import joshua.decoder.ff.FeatureFunction;
 import joshua.decoder.ff.FeatureVector;
 import joshua.decoder.ff.fragmentlm.Tree;
 import joshua.decoder.ff.state_maintenance.DPState;
 import joshua.decoder.ff.tm.Rule;
-import joshua.decoder.io.DeNormalize;
 import joshua.decoder.segment_file.Sentence;
 import joshua.decoder.segment_file.Token;
 import joshua.util.FormatUtils;
@@ -87,19 +83,22 @@ import joshua.util.FormatUtils;
  * The configuration parameter `top-n` controls how many items are returned. If this is set to 0,
  * k-best extraction should be turned off entirely.
  * 
+ * You can call getViterbiDerivation() essentially for free. But as soon as you call hasNext()
+ * (or next(), e.g., via the iterator), you're going to trigger some relatively expensive
+ * k-best computation.
+ * 
  * @author Zhifei Li, <zh...@gmail.com>
  * @author Matt Post <po...@cs.jhu.edu>
  */
-public class KBestExtractor {
+public class KBestExtractor implements Iterator<DerivationState>, Iterable<DerivationState> { 
   private final JoshuaConfiguration joshuaConfiguration;
-  private final String outputFormat;
   private final HashMap<HGNode, VirtualNode> virtualNodesTable = new HashMap<HGNode, VirtualNode>();
 
   // static final String rootSym = JoshuaConfiguration.goal_symbol;
   static final String rootSym = "ROOT";
   static final int rootID = Vocabulary.id(rootSym);
 
-  private enum Side {
+  public enum Side {
     SOURCE, TARGET
   };
 
@@ -107,55 +106,59 @@ public class KBestExtractor {
   private final boolean extractUniqueNbest;
 
   /* Which side to output (source or target) */
-  private final Side defaultSide;
+  final Side defaultSide;
 
   /* The input sentence */
-  private final Sentence sentence;
+  final Sentence sentence;
 
   /* The weights being used to score the forest */
-  private final FeatureVector weights;
+  final FeatureVector weights;
 
   /* The feature functions */
-  private final List<FeatureFunction> featureFunctions;
-
-  /* BLEU statistics of the references */
-  private BLEU.References references = null;
+  final List<FeatureFunction> featureFunctions;
+  private HyperGraph hyperGraph;
+  private DerivationState nextDerivation = null;
+  private int derivationCounter;
 
   public KBestExtractor(
       Sentence sentence,
+      HyperGraph hyperGraph,
       List<FeatureFunction> featureFunctions,
       FeatureVector weights,
       boolean isMonolingual,
       JoshuaConfiguration joshuaConfiguration) {
 
     this.featureFunctions = featureFunctions;
-
+    this.hyperGraph = hyperGraph;
     this.joshuaConfiguration = joshuaConfiguration;
-    this.outputFormat = this.joshuaConfiguration.outputFormat;
     this.extractUniqueNbest = joshuaConfiguration.use_unique_nbest;
 
     this.weights = weights;
     this.defaultSide = (isMonolingual ? Side.SOURCE : Side.TARGET);
     this.sentence = sentence;
-
-    if (joshuaConfiguration.rescoreForest) {
-      references = new BLEU.References(sentence.references());
-    }
+    
+    // initialize the iterator
+    this.derivationCounter = 0;
+    this.nextDerivation = getViterbiDerivation();
   }
 
   /**
-   * Returns the kth derivation.
+   * Returns the Viterbi derivation. You don't want to use the general k-best extraction code because
+   * (a) the Viterbi derivation is always needed and (b) k-best extraction is slow. So this is basically
+   * a convenience function that by-passes the expensive k-best extraction for a common use-case.
    * 
-   * You may need to reset_state() before you call this function for the first time.
-   * 
-   * @param node the node to start at
-   * @param k the kth best derivation (indexed from 1)
-   * @return the derivation object
+   * @return the Viterib derivation
    */
-  public DerivationState getKthDerivation(HGNode node, int k) {
-    VirtualNode virtualNode = getVirtualNode(node);
-    return virtualNode.lazyKBestExtractOnNode(this, k);
+  public DerivationState getViterbiDerivation() {
+    
+    /* TODO: this is just a short-cut to get this working. Instead of triggering the k-best extraction,
+     * it would be better to have a shortcut function that can construction a {@link DerivationState object}
+     * from the hypergraph directly, which would be a lot cheaper.
+     */
+    hasNext();
+    return this.nextDerivation;
   }
+
   
   /**
    * Compute the string that is output from the decoder, using the "output-format" config file
@@ -163,142 +166,18 @@ public class KBestExtractor {
    * 
    * You may need to reset_state() before you call this function for the first time.
    */
-  public String getKthHyp(HGNode node, int k) {
+  public DerivationState getKthHyp(HGNode node, int k) {
 
-    String outputString = null;
-    
     // Determine the k-best hypotheses at each HGNode
     VirtualNode virtualNode = getVirtualNode(node);
     DerivationState derivationState = virtualNode.lazyKBestExtractOnNode(this, k);
-//    DerivationState derivationState = getKthDerivation(node, k);
-    if (derivationState != null) {
-      // ==== read the kbest from each hgnode and convert to output format
-      String hypothesis = maybeProjectCase(
-                            unescapeSpecialSymbols(
-                              removeSentenceMarkers(
-                                derivationState.getHypothesis())), derivationState);
-      
-      
-      /*
-       * To save space, the decoder only stores the model cost,
-       * no the individual feature values.
-       * If you want to output them, you have to replay them.
-       */
-
-      FeatureVector features = new FeatureVector();
-      if (outputFormat.contains("%f") || outputFormat.contains("%d"))
-        features = derivationState.getFeatures();
-
-      outputString = outputFormat
-          .replace("%k", Integer.toString(k))
-          .replace("%s", hypothesis)
-          .replace("%S", DeNormalize.processSingleLine(hypothesis))
-          // TODO (kellens): Fix the recapitalization here
-          .replace("%i", Integer.toString(sentence.id()))
-          .replace("%f", joshuaConfiguration.moses ? features.mosesString() : features.toString())
-          .replace("%c", String.format("%.3f", derivationState.cost));
-
-      if (outputFormat.contains("%t")) {
-        outputString = outputString.replace("%t", derivationState.getTree());
-      }
 
-      if (outputFormat.contains("%e")) {
-        outputString = outputString.replace("%e", removeSentenceMarkers(derivationState.getHypothesis(Side.SOURCE)));
-      }
-
-      /* %d causes a derivation with rules one per line to be output */
-      if (outputFormat.contains("%d")) {
-        outputString = outputString.replace("%d", derivationState.getDerivation());
-      }
-      
-      /* %a causes output of word level alignments between input and output hypothesis */
-      if (outputFormat.contains("%a")) {
-        outputString = outputString.replace("%a",  derivationState.getWordAlignmentString());
-      }
-      
-    }
-
-    return outputString;
+    return derivationState;
   }
 
   // =========================== end kbestHypergraph
 
   /**
-   * If requested, projects source-side lettercase to target, and appends the alignment from
-   * to the source-side sentence in ||s.
-   * 
-   * @param hypothesis
-   * @param state
-   * @return
-   */
-  private String maybeProjectCase(String hypothesis, DerivationState state) {
-    String output = hypothesis;
-
-    if (joshuaConfiguration.project_case) {
-      String[] tokens = hypothesis.split("\\s+");
-      List<List<Integer>> points = state.getWordAlignment();
-      for (int i = 0; i < points.size(); i++) {
-        List<Integer> target = points.get(i);
-        for (int source: target) {
-          Token token = sentence.getTokens().get(source + 1); // skip <s>
-          String annotation = "";
-          if (token != null && token.getAnnotation("lettercase") != null)
-            annotation = token.getAnnotation("lettercase");
-          if (source != 0 && annotation.equals("upper"))
-            tokens[i] = FormatUtils.capitalize(tokens[i]);
-          else if (annotation.equals("all-upper"))
-            tokens[i] = tokens[i].toUpperCase();
-        }
-      }
-
-      output = String.join(" ",  tokens);
-    }
-
-    return output;
-  }
-
-  /**
-   * Convenience function for k-best extraction that prints to STDOUT.
-   */
-  public void lazyKBestExtractOnHG(HyperGraph hg, int topN) throws IOException {
-    lazyKBestExtractOnHG(hg, topN, new BufferedWriter(new OutputStreamWriter(System.out)));
-  }
-
-  /**
-   * This is the entry point for extracting k-best hypotheses. It computes all of them, writing
-   * the results to the BufferedWriter passed in. If you want intermediate access to the k-best
-   * derivations, you'll want to call getKthHyp() or getKthDerivation() directly.
-   * 
-   * The number of derivations that are looked for is controlled by the `top-n` parameter.
-   * Note that when `top-n` is set to 0, k-best extraction is disabled entirely, and only things 
-   * like the viterbi string and the model score are available to the decoder. Since k-best
-   * extraction involves the recomputation of features to get the component values, turning off
-   * that extraction saves a lot of time when only the 1-best string is desired.
-   * 
-   * @param hg the hypergraph to extract from
-   * @param topN how many to extract
-   * @param out object to write to
-   * @throws IOException
-   */
-  public void lazyKBestExtractOnHG(HyperGraph hg, int topN, BufferedWriter out) throws IOException {
-
-    resetState();
-
-    if (null == hg.goalNode)
-      return;
-
-    for (int k = 1; k <= topN; k++) {
-      String hypStr = getKthHyp(hg.goalNode, k);
-      if (null == hypStr)
-        break;
-
-      out.write(hypStr);
-      out.write("\n");
-      out.flush();
-    }
-  }
-
-  /**
    * This clears the virtualNodesTable, which maintains a list of virtual nodes. This should be
    * called in between forest rescorings.
    */
@@ -313,7 +192,7 @@ public class KBestExtractor {
    * @param hgnode
    * @return the corresponding VirtualNode
    */
-  private VirtualNode getVirtualNode(HGNode hgnode) {
+  VirtualNode getVirtualNode(HGNode hgnode) {
     VirtualNode virtualNode = virtualNodesTable.get(hgnode);
     if (null == virtualNode) {
       virtualNode = new VirtualNode(hgnode);
@@ -330,7 +209,7 @@ public class KBestExtractor {
    * queue of candidates.
    */
 
-  private class VirtualNode {
+  class VirtualNode {
 
     // The node being annotated.
     HGNode node = null;
@@ -452,7 +331,7 @@ public class KBestExtractor {
         newRanks[i] = previousState.ranks[i] + 1;
 
         // Create a new state so we can see if it's new. The cost will be set below if it is.
-        DerivationState nextState = new DerivationState(previousState.parentNode,
+        DerivationState nextState = new DerivationState(KBestExtractor.this, previousState.parentNode,
             previousState.edge, newRanks, 0.0f, previousState.edgePos);
 
         // Don't add the state to the list of candidates if it's already been added.
@@ -469,9 +348,6 @@ public class KBestExtractor {
                 + virtualTailNode.nbests.get(newRanks[i] - 1).getModelCost();
             nextState.setCost(cost);
 
-            if (joshuaConfiguration.rescoreForest)
-              nextState.bleu = nextState.computeBLEU();
-
             candHeap.add(nextState);
             derivationTable.add(nextState);
 
@@ -582,243 +458,12 @@ public class KBestExtractor {
       }
       cost = (float) hyperEdge.getBestDerivationScore();
 
-      DerivationState state = new DerivationState(parentNode, hyperEdge, ranks, cost, edgePos);
-      if (joshuaConfiguration.rescoreForest)
-        state.bleu = state.computeBLEU();
+      DerivationState state = new DerivationState(KBestExtractor.this, parentNode, hyperEdge, ranks, cost, edgePos);
 
       return state;
     }
   };
 
-  /**
-   * A DerivationState describes which path to follow through the hypergraph. For example, it
-   * might say to use the 1-best from the first tail node, the 9th-best from the second tail node,
-   * and so on. This information is represented recursively through a chain of DerivationState
-   * objects. This function follows that chain, extracting the information according to a number
-   * of parameters, and returning results to a string, and also (optionally) accumulating the
-   * feature values into the passed-in FeatureVector.
-   */
-
-  // each DerivationState roughly corresponds to a hypothesis
-  public class DerivationState {
-    /* The edge ("e" in the paper) */
-    public HyperEdge edge;
-
-    /* The edge's parent node */
-    public HGNode parentNode;
-
-    /*
-     * This state's position in its parent node's list of incoming hyperedges (used in signature
-     * calculation)
-     */
-    public int edgePos;
-
-    /*
-     * The rank item to select from each of the incoming tail nodes ("j" in the paper, an ArrayList
-     * of size |e|)
-     */
-    public int[] ranks;
-
-    /*
-     * The cost of the hypothesis, including a weighted BLEU score, if any.
-     */
-    private float cost;
-
-    private float bleu = 0.0f;
-
-    /*
-     * The BLEU sufficient statistics associated with the edge's derivation. Note that this is a
-     * function of the complete derivation headed by the edge, i.e., all the particular
-     * subderivations of edges beneath it. That is why it must be contained in DerivationState
-     * instead of in the HyperEdge itself.
-     */
-    BLEU.Stats stats = null;
-
-    public DerivationState(HGNode pa, HyperEdge e, int[] r, float c, int pos) {
-      parentNode = pa;
-      edge = e;
-      ranks = r;
-      cost = c;
-      edgePos = pos;
-      bleu = 0.0f;
-    }
-
-    /**
-     * Computes a scaled approximate BLEU from the accumulated statistics. We know the number of
-     * words; to compute the effective reference length, we take the real reference length statistic
-     * and scale it by the percentage of the input sentence that is consumed, based on the
-     * assumption that the total number of words in the hypothesis scales linearly with the input
-     * sentence span.
-     * 
-     * @return
-     */
-    public float computeBLEU() {
-      if (stats == null) {
-        float percentage = 1.0f * (parentNode.j - parentNode.i) / (sentence.length());
-        // System.err.println(String.format("computeBLEU: (%d - %d) / %d = %f", parentNode.j,
-        // parentNode.i, sentence.length(), percentage));
-        stats = BLEU.compute(edge, percentage, references);
-
-        if (edge.getTailNodes() != null) {
-          for (int id = 0; id < edge.getTailNodes().size(); id++) {
-            stats.add(getChildDerivationState(edge, id).stats);
-          }
-        }
-      }
-
-      return BLEU.score(stats);
-    }
-
-    public void setCost(float cost2) {
-      this.cost = cost2;
-    }
-
-    /**
-     * Returns the model cost. This is obtained by subtracting off the incorporated BLEU score (if
-     * used).
-     * 
-     * @return
-     */
-    public float getModelCost() {
-      return this.cost;
-    }
-
-    /**
-     * Returns the model cost plus the BLEU score.
-     * 
-     * @return
-     */
-    public float getCost() {
-      return cost - weights.getSparse("BLEU") * bleu;
-    }
-
-    public String toString() {
-      StringBuilder sb = new StringBuilder(String.format("DS[[ %s (%d,%d)/%d ||| ",
-          Vocabulary.word(parentNode.lhs), parentNode.i, parentNode.j, edgePos));
-      sb.append("ranks=[ ");
-      if (ranks != null)
-        for (int i = 0; i < ranks.length; i++)
-          sb.append(ranks[i] + " ");
-      sb.append("] ||| " + String.format("%.5f ]]", cost));
-      return sb.toString();
-    }
-
-    public boolean equals(Object other) {
-      if (other instanceof DerivationState) {
-        DerivationState that = (DerivationState) other;
-        if (edgePos == that.edgePos) {
-          if (ranks != null && that.ranks != null) {
-            if (ranks.length == that.ranks.length) {
-              for (int i = 0; i < ranks.length; i++)
-                if (ranks[i] != that.ranks[i])
-                  return false;
-              return true;
-            }
-          }
-        }
-      }
-
-      return false;
-    }
-
-    /**
-     * DerivationState objects are unique to each VirtualNode, so the unique identifying information
-     * only need contain the edge position and the ranks.
-     */
-    public int hashCode() {
-      int hash = edgePos;
-      if (ranks != null) {
-        for (int i = 0; i < ranks.length; i++)
-          hash = hash * 53 + i;
-      }
-
-      return hash;
-    }
-
-    /**
-     * Visits every state in the derivation in a depth-first order.
-     */
-    private DerivationVisitor visit(DerivationVisitor visitor) {
-      return visit(visitor, 0, 0);
-    }
-
-    private DerivationVisitor visit(DerivationVisitor visitor, int indent, int tailNodeIndex) {
-
-      visitor.before(this, indent, tailNodeIndex);
-
-      final Rule rule = edge.getRule();
-      final List<HGNode> tailNodes = edge.getTailNodes();
-
-      if (rule == null) {
-        getChildDerivationState(edge, 0).visit(visitor, indent + 1, 0);
-      } else {
-        if (tailNodes != null) {
-          for (int index = 0; index < tailNodes.size(); index++) {
-            getChildDerivationState(edge, index).visit(visitor, indent + 1, index);
-          }
-        }
-      }
-
-      visitor.after(this, indent, tailNodeIndex);
-
-      return visitor;
-    }
-
-    private String getWordAlignmentString() {
-      return visit(new WordAlignmentExtractor()).toString();
-    }
-    
-    private List<List<Integer>> getWordAlignment() {
-      WordAlignmentExtractor extractor = new WordAlignmentExtractor();
-      visit(extractor);
-      return extractor.getFinalWordAlignments();
-    }
-
-    private String getTree() {
-      return visit(new TreeExtractor()).toString();
-    }
-    
-    private String getHypothesis() {
-      return getHypothesis(defaultSide);
-    }
-
-    /**
-     * For stack decoding we keep using the old string-based
-     * HypothesisExtractor.
-     * For Hiero, we use a faster, int-based hypothesis extraction
-     * that is correct also for Side.SOURCE cases.
-     */
-    private String getHypothesis(final Side side) {
-      return visit(new OutputStringExtractor(side.equals(Side.SOURCE))).toString();
-    }
-
-    private FeatureVector getFeatures() {
-      final FeatureVectorExtractor extractor = new FeatureVectorExtractor(featureFunctions, sentence);
-      visit(extractor);
-      return extractor.getFeatures();
-    }
-
-    private String getDerivation() {
-      return visit(new DerivationExtractor()).toString();
-    }
-
-    /**
-     * Helper function for navigating the hierarchical list of DerivationState objects. This
-     * function looks up the VirtualNode corresponding to the HGNode pointed to by the edge's
-     * {tailNodeIndex}th tail node.
-     * 
-     * @param edge
-     * @param tailNodeIndex
-     * @return
-     */
-    public DerivationState getChildDerivationState(HyperEdge edge, int tailNodeIndex) {
-      HGNode child = edge.getTailNodes().get(tailNodeIndex);
-      VirtualNode virtualChild = getVirtualNode(child);
-      return virtualChild.nbests.get(ranks[tailNodeIndex] - 1);
-    }
-
-  } // end of Class DerivationState
-
   public static class DerivationStateComparator implements Comparator<DerivationState> {
     // natural order by cost
     public int compare(DerivationState one, DerivationState another) {
@@ -1001,6 +646,31 @@ public class KBestExtractor {
     @Override
     public void after(DerivationState state, int level, int tailNodeIndex) {}
   }
-  
 
+  @Override
+  public Iterator<DerivationState> iterator() {
+    return this;
+  }
+
+  @Override
+  public boolean hasNext() {
+    if (this.nextDerivation != null)
+      return true;
+
+    derivationCounter++;
+    
+    VirtualNode virtualNode = getVirtualNode(hyperGraph.goalNode);
+    this.nextDerivation = virtualNode.lazyKBestExtractOnNode(this, derivationCounter);
+    return this.nextDerivation != null;
+  }
+
+  @Override
+  public DerivationState next() {
+    if (this.hasNext()) {
+      DerivationState returnDerivation = this.nextDerivation;
+      this.nextDerivation = null;
+      return returnDerivation;
+    }
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2f82c38/src/joshua/decoder/hypergraph/OutputStringExtractor.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/hypergraph/OutputStringExtractor.java b/src/joshua/decoder/hypergraph/OutputStringExtractor.java
index f67a9df..8fc10ce 100644
--- a/src/joshua/decoder/hypergraph/OutputStringExtractor.java
+++ b/src/joshua/decoder/hypergraph/OutputStringExtractor.java
@@ -24,7 +24,6 @@ import static joshua.corpus.Vocabulary.getWords;
 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;
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2f82c38/src/joshua/decoder/hypergraph/ViterbiExtractor.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/hypergraph/ViterbiExtractor.java b/src/joshua/decoder/hypergraph/ViterbiExtractor.java
index 31c8dc0..b70446a 100644
--- a/src/joshua/decoder/hypergraph/ViterbiExtractor.java
+++ b/src/joshua/decoder/hypergraph/ViterbiExtractor.java
@@ -109,9 +109,9 @@ public class ViterbiExtractor {
   /**
    * Returns the Viterbi Word Alignments as list of lists (target-side).
    */
-  public static List<List<Integer>> getViterbiWordAlignmentList(final HyperGraph hg) {
+  public static WordAlignmentState getViterbiWordAlignmentList(final HyperGraph hg) {
     if (hg == null)
-      return emptyList();
+      return null;
     
     final WordAlignmentExtractor wordAlignmentWalker = new WordAlignmentExtractor();
     viterbiWalk(hg.goalNode, wordAlignmentWalker);

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2f82c38/src/joshua/decoder/hypergraph/WordAlignmentExtractor.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/hypergraph/WordAlignmentExtractor.java b/src/joshua/decoder/hypergraph/WordAlignmentExtractor.java
index 837c69f..cb2f059 100644
--- a/src/joshua/decoder/hypergraph/WordAlignmentExtractor.java
+++ b/src/joshua/decoder/hypergraph/WordAlignmentExtractor.java
@@ -24,7 +24,6 @@ import java.util.List;
 import java.util.Stack;
 
 import joshua.decoder.ff.tm.Rule;
-import joshua.decoder.hypergraph.KBestExtractor.DerivationState;
 import joshua.decoder.hypergraph.KBestExtractor.DerivationVisitor;
 
 /**
@@ -97,9 +96,9 @@ public class WordAlignmentExtractor implements WalkerFunction, DerivationVisitor
    * Final word alignment without sentence markers
    * or empty list if stack is empty.
    */
-  public List<List<Integer>> getFinalWordAlignments() {
+  public WordAlignmentState getFinalWordAlignments() {
     if (stack.isEmpty()) {
-      return emptyList();
+      return null;
     }
     
     if (stack.size() != 1) {
@@ -108,7 +107,7 @@ public class WordAlignmentExtractor implements WalkerFunction, DerivationVisitor
               "Stack of WordAlignmentExtractor should contain only a single (last) element, but was size %d", stack.size()));
     }
     
-    return stack.peek().toFinalList();
+    return stack.peek();
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2f82c38/src/joshua/decoder/io/JSONMessage.java
----------------------------------------------------------------------
diff --git a/src/joshua/decoder/io/JSONMessage.java b/src/joshua/decoder/io/JSONMessage.java
index 2733db4..bf75133 100644
--- a/src/joshua/decoder/io/JSONMessage.java
+++ b/src/joshua/decoder/io/JSONMessage.java
@@ -24,7 +24,12 @@ import java.util.List;
 import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
 
+import joshua.decoder.JoshuaConfiguration;
 import joshua.decoder.Translation;
+import joshua.decoder.TranslationFactory;
+import joshua.decoder.hypergraph.DerivationState;
+import joshua.decoder.hypergraph.KBestExtractor;
+import joshua.decoder.segment_file.Sentence;
 
 public class JSONMessage {
   public Data data = null;
@@ -86,19 +91,25 @@ public class JSONMessage {
     }
   }
 
-  public static JSONMessage buildMessage(Translation translation) {
+  public static JSONMessage buildMessage(Sentence sentence, KBestExtractor extractor, JoshuaConfiguration config) {
     JSONMessage message = new JSONMessage();
-    String[] results = translation.toString().split("\\n");
-    if (results.length > 0) {
-      JSONMessage.TranslationItem item = message.addTranslation(translation.getStructuredTranslation().getTranslationString());
+    
+    final String mosesFormat = "%i ||| %s ||| %f ||| %c"; 
+    
+    int k = 1;
+    for (DerivationState derivation: extractor) {
+      if (k > config.topN)
+        break;
+      
+      TranslationFactory factory = new TranslationFactory(sentence, derivation, config);
+      Translation translation = factory.formattedTranslation(mosesFormat).translation();
 
-      for (String result: results) {
-        String[] tokens = result.split(" \\|\\|\\| ");
-        String rawResult = tokens[1];
-        float score = Float.parseFloat(tokens[3]);
-        item.addHypothesis(rawResult, score);
-      }
+      JSONMessage.TranslationItem item = message.addTranslation(translation.toString());
+      item.addHypothesis(translation.toString(), translation.score());
+      
+      k++;
     }
+
     return message;
   }